From d687f514aedaca9821d5baa05079830c6d5e476d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 2 Jun 2017 06:25:54 -0400 Subject: [PATCH 01/54] Introduce primary context The target of a primary relocation is not aware of the state of the replication group. In particular, it is not tracking in-sync and initializing shards and their checkpoints. This means that after the target shard is started, its knowledge of the replication group could differ from that of the relocation source. In particular, this differing view can lead to it computing a global checkpoint that moves backwards after it becomes aware of the state of the entire replication group. This commit addresses this issue by transferring a primary context during relocation handoff. --- .../index/seqno/GlobalCheckpointTracker.java | 20 +++++ .../index/seqno/SeqNoPrimaryContext.java | 80 +++++++++++++++++ .../index/seqno/SequenceNumbersService.java | 25 ++++++ .../elasticsearch/index/shard/IndexShard.java | 35 +++++++- .../index/shard/PrimaryContext.java | 64 ++++++++++++++ .../recovery/PeerRecoveryTargetService.java | 18 ++++ .../RecoveryHandoffPrimaryContextRequest.java | 87 +++++++++++++++++++ .../recovery/RecoverySourceHandler.java | 5 +- .../indices/recovery/RecoveryTarget.java | 8 +- .../recovery/RecoveryTargetHandler.java | 8 ++ .../recovery/RemoteRecoveryTargetHandler.java | 15 +++- .../IndexLevelReplicationTests.java | 1 + .../index/shard/IndexShardTests.java | 16 ++-- .../recovery/RecoverySourceHandlerTests.java | 2 +- .../elasticsearch/recovery/RelocationIT.java | 8 -- 15 files changed, 366 insertions(+), 26 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java create mode 100644 core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java create mode 100644 core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index ea6edef7a12fa..3ee309c940a57 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -27,6 +27,7 @@ import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.ShardId; +import java.util.Arrays; import java.util.HashSet; import java.util.Locale; import java.util.Set; @@ -248,6 +249,25 @@ public synchronized void updateAllocationIdsFromMaster( updateGlobalCheckpointOnPrimary(); } + /** + * Updates the known allocation IDs and the local checkpoints for the corresponding allocations from a primary relocation source. + * + * @param seqNoPrimaryContext the sequence number context + */ + synchronized void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContext seqNoPrimaryContext) { + final Set inSyncAllocationIds = + new HashSet<>(Arrays.asList(seqNoPrimaryContext.inSyncLocalCheckpoints().keys().toArray(String.class))); + final Set trackingAllocationIds = + new HashSet<>(Arrays.asList(seqNoPrimaryContext.trackingLocalCheckpoints().keys().toArray(String.class))); + updateAllocationIdsFromMaster(inSyncAllocationIds, trackingAllocationIds); + for (final ObjectLongCursor cursor : seqNoPrimaryContext.inSyncLocalCheckpoints()) { + updateLocalCheckpoint(cursor.key, cursor.value); + } + for (final ObjectLongCursor cursor : seqNoPrimaryContext.trackingLocalCheckpoints()) { + updateLocalCheckpoint(cursor.key, cursor.value); + } + } + /** * Marks the shard with the provided allocation ID as in-sync with the primary shard. This method will block until the local checkpoint * on the specified shard advances above the current global checkpoint. diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java new file mode 100644 index 0000000000000..fa75ffdd51676 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.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 com.carrotsearch.hppc.ObjectLongHashMap; +import com.carrotsearch.hppc.ObjectLongMap; +import com.carrotsearch.hppc.cursors.ObjectLongCursor; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; + +import java.io.IOException; + +public class SeqNoPrimaryContext implements Writeable { + + private ObjectLongMap inSyncLocalCheckpoints; + + public ObjectLongMap inSyncLocalCheckpoints() { + return inSyncLocalCheckpoints; + } + + private ObjectLongMap trackingLocalCheckpoints; + + public ObjectLongMap trackingLocalCheckpoints() { + return trackingLocalCheckpoints; + } + + public SeqNoPrimaryContext(final ObjectLongMap inSyncLocalCheckpoints, final ObjectLongMap trackingLocalCheckpoints) { + this.inSyncLocalCheckpoints = inSyncLocalCheckpoints; + this.trackingLocalCheckpoints = trackingLocalCheckpoints; + } + + public SeqNoPrimaryContext(StreamInput in) throws IOException { + inSyncLocalCheckpoints = readMap(in); + trackingLocalCheckpoints = readMap(in); + } + + private static ObjectLongMap readMap(final StreamInput in) throws IOException { + final int length = in.readInt(); + final ObjectLongMap map = new ObjectLongHashMap<>(length); + for (int i = 0; i < length; i++) { + final String key = in.readString(); + final long value = in.readZLong(); + map.addTo(key, value); + } + return map; + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + writeMap(out, inSyncLocalCheckpoints); + writeMap(out, trackingLocalCheckpoints); + } + + private static void writeMap(final StreamOutput out, final ObjectLongMap map) throws IOException { + out.writeInt(map.size()); + for (ObjectLongCursor cursor : map) { + out.writeString(cursor.key); + out.writeZLong(cursor.value); + } + } + +} diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java index 4180c7e0f7d92..f8d7615c8edec 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.seqno; +import com.carrotsearch.hppc.ObjectLongHashMap; +import com.carrotsearch.hppc.ObjectLongMap; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.ShardId; @@ -174,6 +176,15 @@ public void updateAllocationIdsFromMaster(final Set activeAllocationIds, globalCheckpointTracker.updateAllocationIdsFromMaster(activeAllocationIds, initializingAllocationIds); } + /** + * Updates the known allocation IDs and the local checkpoints for the corresponding allocations from a primary relocation source. + * + * @param seqNoPrimaryContext the sequence number context + */ + public void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContext seqNoPrimaryContext) { + globalCheckpointTracker.updateAllocationIdsFromPrimaryContext(seqNoPrimaryContext); + } + /** * Check if there are any recoveries pending in-sync. * @@ -183,4 +194,18 @@ public boolean pendingInSync() { return globalCheckpointTracker.pendingInSync(); } + /** + * Get the sequence number primary context for the shard. This includes the state of the global checkpoint tracker. + * + * @return the sequence number primary context + */ + public SeqNoPrimaryContext seqNoPrimaryContext() { + synchronized (globalCheckpointTracker) { + final ObjectLongMap inSyncLocalCheckpoints = new ObjectLongHashMap<>(globalCheckpointTracker.inSyncLocalCheckpoints); + final ObjectLongMap trackingLocalCheckpoints = + new ObjectLongHashMap<>(globalCheckpointTracker.trackingLocalCheckpoints); + return new SeqNoPrimaryContext(inSyncLocalCheckpoints, trackingLocalCheckpoints); + } + } + } 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 83edd73350b10..26f435d411eed 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -95,6 +95,7 @@ import org.elasticsearch.index.refresh.RefreshStats; import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.ShardSearchStats; +import org.elasticsearch.index.seqno.SeqNoPrimaryContext; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.similarity.SimilarityService; @@ -478,7 +479,7 @@ public IndexShardState markAsRecovering(String reason, RecoveryState recoverySta } } - public void relocated(String reason) throws IllegalIndexShardStateException, InterruptedException { + public void relocated(final String reason, final Runnable onBlocked) throws IllegalIndexShardStateException, InterruptedException { assert shardRouting.primary() : "only primaries can be marked as relocated: " + shardRouting; try { indexShardOperationPermits.blockOperations(30, TimeUnit.MINUTES, () -> { @@ -498,6 +499,7 @@ public void relocated(String reason) throws IllegalIndexShardStateException, Int throw new IllegalIndexShardStateException(shardId, IndexShardState.STARTED, ": shard is no longer relocating " + shardRouting); } + onBlocked.run(); changeState(IndexShardState.RELOCATED, reason); } }); @@ -510,6 +512,17 @@ public void relocated(String reason) throws IllegalIndexShardStateException, Int } } + /** + * Obtain the primary context for the shard. The shard must be serving as the relocation source for a primary shard. + * + * @return the primary for the shard + */ + public PrimaryContext primaryContext() { + verifyPrimary(); + assert shardRouting.relocating(); + assert !shardRouting.isRelocationTarget(); + return new PrimaryContext(getEngine().seqNoService().seqNoPrimaryContext()); + } public IndexShardState state() { return state; @@ -1248,7 +1261,7 @@ private void ensureWriteAllowed(Engine.Operation op) throws IllegalIndexShardSta private void verifyPrimary() { if (shardRouting.primary() == false) { - throw new IllegalStateException("shard is not a primary " + shardRouting); + throw new IllegalStateException("shard " + shardRouting + " is not a primary"); } } @@ -1256,8 +1269,8 @@ private void verifyReplicationTarget() { final IndexShardState state = state(); if (shardRouting.primary() && shardRouting.active() && state != IndexShardState.RELOCATED) { // must use exception that is not ignored by replication logic. See TransportActions.isShardNotAvailableException - throw new IllegalStateException("active primary shard cannot be a replication target before " + - " relocation hand off " + shardRouting + ", state is [" + state + "]"); + throw new IllegalStateException("active primary shard " + shardRouting + " cannot be a replication target before " + + "relocation hand off, state is [" + state + "]"); } } @@ -1598,6 +1611,20 @@ public void updateAllocationIdsFromMaster(final Set activeAllocationIds, } } + /** + * Updates the known allocation IDs and the local checkpoints for the corresponding allocations from a primary relocation source. + * + * @param seqNoPrimaryContext the sequence number context + */ + public void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContext seqNoPrimaryContext) { + verifyPrimary(); + assert shardRouting.isRelocationTarget(); + final Engine engine = getEngineOrNull(); + if (engine != null) { + engine.seqNoService().updateAllocationIdsFromPrimaryContext(seqNoPrimaryContext); + } + } + /** * Check if there are any recoveries pending in-sync. * diff --git a/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java new file mode 100644 index 0000000000000..d17b563af2f54 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java @@ -0,0 +1,64 @@ +/* + * 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.shard; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.index.seqno.SeqNoPrimaryContext; + +import java.io.IOException; + +/** + * Represents the primary context which encapsulates the view the primary shard has of its replication group. + */ +public class PrimaryContext implements Writeable { + + private final SeqNoPrimaryContext seqNoPrimaryContext; + + public SeqNoPrimaryContext seqNoPrimaryContext() { + return seqNoPrimaryContext; + } + + /** + * Construct a primary context with the sequence number primary context. + * + * @param seqNoPrimaryContext the sequence number primary context + */ + public PrimaryContext(final SeqNoPrimaryContext seqNoPrimaryContext) { + this.seqNoPrimaryContext = seqNoPrimaryContext; + } + + /** + * Construct a primary context from a stream. + * + * @param in the stream + * @throws IOException if an I/O exception occurs reading from the stream + */ + public PrimaryContext(final StreamInput in) throws IOException { + seqNoPrimaryContext = new SeqNoPrimaryContext(in); + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + seqNoPrimaryContext.writeTo(out); + } + +} diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 4823edcc2f119..37ab2798b1f4b 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -82,6 +82,7 @@ public static class Actions { public static final String PREPARE_TRANSLOG = "internal:index/shard/recovery/prepare_translog"; public static final String FINALIZE = "internal:index/shard/recovery/finalize"; public static final String WAIT_CLUSTERSTATE = "internal:index/shard/recovery/wait_clusterstate"; + public static final String HANDOFF_PRIMARY_CONTEXT = "internal:index/shard/recovery/hand_off_primary_context"; } private final ThreadPool threadPool; @@ -116,6 +117,11 @@ public PeerRecoveryTargetService(Settings settings, ThreadPool threadPool, Trans FinalizeRecoveryRequestHandler()); transportService.registerRequestHandler(Actions.WAIT_CLUSTERSTATE, RecoveryWaitForClusterStateRequest::new, ThreadPool.Names.GENERIC, new WaitForClusterStateRequestHandler()); + transportService.registerRequestHandler( + Actions.HANDOFF_PRIMARY_CONTEXT, + RecoveryHandoffPrimaryContextRequest::new, + ThreadPool.Names.GENERIC, + new HandoffPrimaryContextRequestHandler()); } @Override @@ -411,6 +417,18 @@ public void messageReceived(RecoveryWaitForClusterStateRequest request, Transpor } } + class HandoffPrimaryContextRequestHandler implements TransportRequestHandler { + + @Override + public void messageReceived(final RecoveryHandoffPrimaryContextRequest request, final TransportChannel channel) throws Exception { + try (RecoveryRef recoveryRef = onGoingRecoveries.getRecoverySafe(request.recoveryId(), request.shardId())) { + recoveryRef.target().handoffPrimaryContext(request.primaryContext()); + } + channel.sendResponse(TransportResponse.Empty.INSTANCE); + } + + } + class TranslogOperationsRequestHandler implements TransportRequestHandler { @Override diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java new file mode 100644 index 0000000000000..95601061514c9 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java @@ -0,0 +1,87 @@ +/* + * 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.indices.recovery; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.shard.PrimaryContext; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.transport.TransportRequest; + +import java.io.IOException; + +/** + * The request object to handoff the primary context to the relocation target. + */ +public class RecoveryHandoffPrimaryContextRequest extends TransportRequest { + + private long recoveryId; + private ShardId shardId; + private PrimaryContext primaryContext; + + /** + * Initialize an empty request (used to serialize into when reading from a stream). + */ + @SuppressWarnings("WeakerAccess") + public RecoveryHandoffPrimaryContextRequest() { + } + + /** + * Initialize a request for the specified relocation. + * + * @param recoveryId the recovery ID of the relocation + * @param shardId the shard ID of the relocation + * @param primaryContext the primary context + */ + RecoveryHandoffPrimaryContextRequest(final long recoveryId, final ShardId shardId, final PrimaryContext primaryContext) { + this.recoveryId = recoveryId; + this.shardId = shardId; + this.primaryContext = primaryContext; + } + + long recoveryId() { + return this.recoveryId; + } + + ShardId shardId() { + return shardId; + } + + PrimaryContext primaryContext() { + return primaryContext; + } + + @Override + public void readFrom(final StreamInput in) throws IOException { + super.readFrom(in); + recoveryId = in.readLong(); + shardId = ShardId.readShardId(in); + primaryContext = new PrimaryContext(in); + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + super.writeTo(out); + out.writeLong(recoveryId); + shardId.writeTo(out); + primaryContext.writeTo(out); + } + +} 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 8abd3a05d8e68..49273d1182ebf 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -463,7 +463,10 @@ public void finalizeRecovery(final long targetLocalCheckpoint) { cancellableThreads.execute(() -> recoveryTarget.ensureClusterStateVersion(currentClusterStateVersion)); logger.trace("performing relocation hand-off"); - cancellableThreads.execute(() -> shard.relocated("to " + request.targetNode())); + cancellableThreads.execute( + () -> shard.relocated( + "to " + request.targetNode(), + () -> recoveryTarget.handoffPrimaryContext(shard.primaryContext()))); } /* * if the recovery process fails after setting the shard state to RELOCATED, both relocation source and diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 6a465f111150c..0f1ecd346ecd2 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -44,6 +44,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardNotRecoveringException; import org.elasticsearch.index.shard.IndexShardState; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; @@ -61,8 +62,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; import java.util.function.LongConsumer; +import java.util.stream.Collectors; /** * Represents a recovery where the current node is the target node of the recovery. To track recoveries in a central place, instances of @@ -378,6 +379,11 @@ public void ensureClusterStateVersion(long clusterStateVersion) { ensureClusterStateVersionCallback.accept(clusterStateVersion); } + @Override + public void handoffPrimaryContext(final PrimaryContext primaryContext) { + indexShard.updateAllocationIdsFromPrimaryContext(primaryContext.seqNoPrimaryContext()); + } + @Override public long indexTranslogOperations(List operations, int totalTranslogOps) throws MapperException, IOException { final RecoveryState.Translog translog = state().getTranslog(); diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java index 42cf1bc1ce19d..34b0df2293f3f 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java @@ -19,6 +19,7 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; @@ -49,6 +50,13 @@ public interface RecoveryTargetHandler { */ void ensureClusterStateVersion(long clusterStateVersion); + /** + * Handoff the primary context between the relocation source and the relocation target. + * + * @param primaryContext the primary context from the relocation source + */ + void handoffPrimaryContext(PrimaryContext primaryContext); + /** * Index a set of translog operations on the target * @param operations operations to index diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index a4f24b710b2fa..1b88613da1d44 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -23,15 +23,14 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.transport.EmptyTransportResponseHandler; -import org.elasticsearch.transport.FutureTransportResponseHandler; import org.elasticsearch.transport.TransportFuture; import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.io.IOException; @@ -97,7 +96,17 @@ public void ensureClusterStateVersion(long clusterStateVersion) { transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.WAIT_CLUSTERSTATE, new RecoveryWaitForClusterStateRequest(recoveryId, shardId, clusterStateVersion), TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionLongTimeout()).build(), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + } + + @Override + public void handoffPrimaryContext(final PrimaryContext primaryContext) { + transportService.submitRequest( + targetNode, + PeerRecoveryTargetService.Actions.HANDOFF_PRIMARY_CONTEXT, + new RecoveryHandoffPrimaryContextRequest(recoveryId, shardId, primaryContext), + TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionTimeout()).build(), + EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); } @Override diff --git a/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index c13177a62505d..92837f25871ee 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentHelper; 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 b299168ce622f..2b94c0af2739e 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -540,7 +540,7 @@ public void testOperationPermitOnReplicaShards() throws InterruptedException, Ex routing = TestShardRouting.newShardRouting(routing.shardId(), routing.currentNodeId(), "otherNode", true, ShardRoutingState.RELOCATING, AllocationId.newRelocation(routing.allocationId())); indexShard.updateRoutingEntry(routing); - indexShard.relocated("test"); + indexShard.relocated("test", () -> {}); engineClosed = false; break; } @@ -1050,7 +1050,7 @@ public void testLockingBeforeAndAfterRelocated() throws Exception { Thread recoveryThread = new Thread(() -> { latch.countDown(); try { - shard.relocated("simulated recovery"); + shard.relocated("simulated recovery", () -> {}); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -1079,7 +1079,7 @@ public void testDelayedOperationsBeforeAndAfterRelocated() throws Exception { shard.updateRoutingEntry(ShardRoutingHelper.relocate(shard.routingEntry(), "other_node")); Thread recoveryThread = new Thread(() -> { try { - shard.relocated("simulated recovery"); + shard.relocated("simulated recovery", () -> {}); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -1132,7 +1132,7 @@ public void run() { AtomicBoolean relocated = new AtomicBoolean(); final Thread recoveryThread = new Thread(() -> { try { - shard.relocated("simulated recovery"); + shard.relocated("simulated recovery", () -> {}); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -1166,7 +1166,7 @@ public void testRelocatedShardCanNotBeRevived() throws IOException, InterruptedE final IndexShard shard = newStartedShard(true); final ShardRouting originalRouting = shard.routingEntry(); shard.updateRoutingEntry(ShardRoutingHelper.relocate(originalRouting, "other_node")); - shard.relocated("test"); + shard.relocated("test", () -> {}); expectThrows(IllegalIndexShardStateException.class, () -> shard.updateRoutingEntry(originalRouting)); closeShards(shard); } @@ -1176,7 +1176,7 @@ public void testShardCanNotBeMarkedAsRelocatedIfRelocationCancelled() throws IOE final ShardRouting originalRouting = shard.routingEntry(); shard.updateRoutingEntry(ShardRoutingHelper.relocate(originalRouting, "other_node")); shard.updateRoutingEntry(originalRouting); - expectThrows(IllegalIndexShardStateException.class, () -> shard.relocated("test")); + expectThrows(IllegalIndexShardStateException.class, () -> shard.relocated("test", () -> {})); closeShards(shard); } @@ -1195,7 +1195,7 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { cyclicBarrier.await(); - shard.relocated("test"); + shard.relocated("test", () -> {}); } }); relocationThread.start(); @@ -1370,7 +1370,7 @@ public void testRecoveryFailsAfterMovingToRelocatedState() throws InterruptedExc assertThat(shard.state(), equalTo(IndexShardState.STARTED)); ShardRouting inRecoveryRouting = ShardRoutingHelper.relocate(origRouting, "some_node"); shard.updateRoutingEntry(inRecoveryRouting); - shard.relocated("simulate mark as relocated"); + shard.relocated("simulate mark as relocated", () -> {}); assertThat(shard.state(), equalTo(IndexShardState.RELOCATED)); try { shard.updateRoutingEntry(origRouting); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 09a787ce0d36f..e2f1bababbb67 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -448,7 +448,7 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int relocated.set(true); assertTrue(recoveriesDelayed.get()); return null; - }).when(shard).relocated(any(String.class)); + }).when(shard).relocated(any(String.class), any(Runnable.class)); when(shard.acquireIndexCommit(anyBoolean())).thenReturn(mock(Engine.IndexCommitRef.class)); final Supplier currentClusterStateVersionSupplier = () -> { diff --git a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java index fe83847bff218..48f6fdeaedbbd 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -514,14 +514,6 @@ public void testIndexAndRelocateConcurrently() throws ExecutionException, Interr // refresh is a replication action so this forces a global checkpoint sync which is needed as these are asserted on in tear down client().admin().indices().prepareRefresh("test").get(); - /* - * We have to execute a second refresh as in the face of relocations, the relocation target is not aware of the in-sync set and so - * the first refresh would bring back the local checkpoint for any shards added to the in-sync set that the relocation target was - * not tracking. - */ - // TODO: remove this after a primary context is transferred during relocation handoff - client().admin().indices().prepareRefresh("test").get(); - } class RecoveryCorruption extends MockTransportService.DelegateTransport { From 8d871cfafd3510fa98f5174c496a5afca0b29be4 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 8 Jun 2017 09:26:40 -0400 Subject: [PATCH 02/54] Fix test --- .../java/org/elasticsearch/index/shard/IndexShardTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 2b94c0af2739e..83b948fb4f0ab 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -555,7 +555,7 @@ public void testOperationPermitOnReplicaShards() throws InterruptedException, Ex if (shardRouting.primary() == false) { final IllegalStateException e = expectThrows(IllegalStateException.class, () -> indexShard.acquirePrimaryOperationPermit(null, ThreadPool.Names.INDEX)); - assertThat(e, hasToString(containsString("shard is not a primary"))); + assertThat(e, hasToString(containsString("shard " + shardRouting + " is not a primary"))); } final long primaryTerm = indexShard.getPrimaryTerm(); From a197a64ef6886d04c727b41afc49e0d6320f66b9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 8 Jun 2017 09:45:47 -0400 Subject: [PATCH 03/54] Add assertion messages --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 26f435d411eed..9a64e3f5e2b50 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -519,8 +519,8 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille */ public PrimaryContext primaryContext() { verifyPrimary(); - assert shardRouting.relocating(); - assert !shardRouting.isRelocationTarget(); + assert shardRouting.relocating() : "primary context can only be obtained from a relocating primary but was " + shardRouting; + assert !shardRouting.isRelocationTarget() : "primary context can only be obtained from relocation source but was " + shardRouting; return new PrimaryContext(getEngine().seqNoService().seqNoPrimaryContext()); } From 1e4255c102ae8aeeeb3523732e18ec511fd69af0 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 8 Jun 2017 09:53:30 -0400 Subject: [PATCH 04/54] Javadocs --- .../org/elasticsearch/index/seqno/SeqNoPrimaryContext.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java index fa75ffdd51676..538df6c691c24 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java @@ -28,6 +28,10 @@ import java.io.IOException; +/** + * Represents the sequence number component of the primary context. This is the knowledge on the primary of the in-sync and initializing + * shards and their local checkpoints. + */ public class SeqNoPrimaryContext implements Writeable { private ObjectLongMap inSyncLocalCheckpoints; From c1588bbd0dada210d6f97e46a69cede1f61c9603 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 9 Jun 2017 14:28:05 -0400 Subject: [PATCH 05/54] Barrier between marking a shard in sync and relocating --- .../elasticsearch/index/shard/IndexShard.java | 55 ++++++++++++++++--- .../recovery/RecoverySourceHandler.java | 21 ++++++- 2 files changed, 66 insertions(+), 10 deletions(-) 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 9a64e3f5e2b50..4e248d99a5a63 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -57,6 +57,7 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; +import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexNotFoundException; @@ -1536,19 +1537,55 @@ public void waitForOpsToComplete(final long seqNo) throws InterruptedException { /** * Marks the shard with the provided allocation ID as in-sync with the primary shard. See * {@link org.elasticsearch.index.seqno.GlobalCheckpointTracker#markAllocationIdAsInSync(String, long)} - * for additional details. + * for additional details. Because this operation could be completed asynchronously on another thread, the caller must provide a latch; + * this latch will be counted down after the operation completes. * - * @param allocationId the allocation ID of the shard to mark as in-sync - * @param localCheckpoint the current local checkpoint on the shard + * @param allocationId the allocation ID of the shard to mark as in-sync + * @param localCheckpoint the current local checkpoint on the shard + * @param latch a latch that is counted down after the operation completes + * @param onFailureConsumer a callback if an exception occurs completing the operation */ - public void markAllocationIdAsInSync(final String allocationId, final long localCheckpoint) throws InterruptedException { - verifyPrimary(); - getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint); + public void markAllocationIdAsInSync( + final String allocationId, + final long localCheckpoint, + final CountDownLatch latch, + final Consumer onFailureConsumer) { /* - * We could have blocked waiting for the replica to catch up that we fell idle and there will not be a background sync to the - * replica; mark our self as active to force a future background sync. + * Before marking the shard as in-sync we acquire an operation permit. We do this so that there is a barrier between marking a shard + * as in-sync and relocating a shard. If we acquire the permit then no relocation handoff can complete before we are done marking + * the shard as in-sync. If the relocation handoff holds all the permits then after the handoff completes and we acquire the permit + * then the state of the shard will be relocated and this recovery will fail. */ - active.compareAndSet(false, true); + acquirePrimaryOperationPermit( + new ActionListener() { + @Override + public void onResponse(final Releasable releasable) { + // we could have been relocated while waiting for a permit + if (state() == IndexShardState.RELOCATED) { + onFailure(new IndexShardRelocatedException(shardId)); + } + try { + getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint); + /* + * We could have blocked waiting for the replica to catch up that we fell idle and there will not be a + * background sync to the replica; mark our self as active to force a future background sync. + */ + active.compareAndSet(false, true); + latch.countDown(); + } catch (final InterruptedException e) { + onFailure(e); + onFailureConsumer.accept(e); + } finally { + releasable.close(); + } + } + + @Override + public void onFailure(final Exception e) { + latch.countDown(); + } + }, + ThreadPool.Names.GENERIC); } /** 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 49273d1182ebf..ca173949b7ac3 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -41,6 +41,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.CancellableThreads; +import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.seqno.LocalCheckpointTracker; @@ -59,7 +60,9 @@ import java.io.OutputStream; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.StreamSupport; @@ -448,7 +451,23 @@ public void finalizeRecovery(final long targetLocalCheckpoint) { StopWatch stopWatch = new StopWatch().start(); logger.trace("finalizing recovery"); cancellableThreads.execute(() -> { - shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint); + final CountDownLatch latch = new CountDownLatch(1); + final AtomicReference reference = new AtomicReference<>(); + shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint, latch, reference::set); + /* + * We wait for the shard to be marked as in-sync. However, our latch could also be counted down if marking the shard in-sync + * was interrupted or for any other exception that is thrown marking the shard as in-sync. Therefore, after we are unlatched we + * check if an exception was thrown. Interrupted exceptions must be handled specially due to their meaning to cancellable + * threads. All other exceptions are wrapped as runtime exceptions and rethrown. + */ + latch.await(); + if (reference.get() != null) { + if (reference.get() instanceof InterruptedException) { + throw (InterruptedException) reference.get(); + } else { + throw new RuntimeException(reference.get()); + } + } recoveryTarget.finalizeRecovery(shard.getGlobalCheckpoint()); }); From 0b473863ff297ef0f1083002edd0d65cabaffeb3 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 9 Jun 2017 17:46:56 -0400 Subject: [PATCH 06/54] Fix misplaced call --- .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 4e248d99a5a63..494df3fb781db 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1574,7 +1574,6 @@ public void onResponse(final Releasable releasable) { latch.countDown(); } catch (final InterruptedException e) { onFailure(e); - onFailureConsumer.accept(e); } finally { releasable.close(); } @@ -1582,6 +1581,7 @@ public void onResponse(final Releasable releasable) { @Override public void onFailure(final Exception e) { + onFailureConsumer.accept(e); latch.countDown(); } }, From 73bc0d14f8a7dc30b604a038791ca2c5d715300f Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 9 Jun 2017 17:47:34 -0400 Subject: [PATCH 07/54] Paranoia --- .../java/org/elasticsearch/index/shard/IndexShard.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 494df3fb781db..e482541864b0b 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1581,8 +1581,11 @@ public void onResponse(final Releasable releasable) { @Override public void onFailure(final Exception e) { - onFailureConsumer.accept(e); - latch.countDown(); + try { + onFailureConsumer.accept(e); + } finally { + latch.countDown(); + } } }, ThreadPool.Names.GENERIC); From 76162e4cda6231b51ad912503a753bc18c52c76b Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 9 Jun 2017 17:48:16 -0400 Subject: [PATCH 08/54] Better latch countdown --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 e482541864b0b..e287278e1c519 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1564,6 +1564,7 @@ public void onResponse(final Releasable releasable) { if (state() == IndexShardState.RELOCATED) { onFailure(new IndexShardRelocatedException(shardId)); } + boolean success = false; try { getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint); /* @@ -1571,11 +1572,14 @@ public void onResponse(final Releasable releasable) { * background sync to the replica; mark our self as active to force a future background sync. */ active.compareAndSet(false, true); - latch.countDown(); + success = true; } catch (final InterruptedException e) { onFailure(e); } finally { releasable.close(); + if (success) { + latch.countDown(); + } } } From a799e69a7159442f3f9690e45ef607cb1d6b28b9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 9 Jun 2017 22:17:40 -0400 Subject: [PATCH 09/54] Catch any exception --- .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 90d127c4a5cb3..e4f9c25cd78fc 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1574,7 +1574,7 @@ public void onResponse(final Releasable releasable) { */ active.compareAndSet(false, true); success = true; - } catch (final InterruptedException e) { + } catch (final Exception e) { onFailure(e); } finally { releasable.close(); From 58adfa0d2101e1832bc364a2abc8f480c03ac74a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sat, 10 Jun 2017 12:59:32 -0400 Subject: [PATCH 10/54] Fix comment --- .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e4f9c25cd78fc..666e7564ea9c0 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1569,7 +1569,7 @@ public void onResponse(final Releasable releasable) { try { getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint); /* - * We could have blocked waiting for the replica to catch up that we fell idle and there will not be a + * We could have blocked so long waiting for the replica to catch up that we fell idle and there will not be a * background sync to the replica; mark our self as active to force a future background sync. */ active.compareAndSet(false, true); From 03b863b067f97508a919baa50c7b786e7776f6d9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sat, 10 Jun 2017 14:05:59 -0400 Subject: [PATCH 11/54] Fix wait for cluster state relocation test --- .../indices/recovery/RecoverySourceHandlerTests.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index e2f1bababbb67..6e00299d2aeaa 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -75,7 +75,9 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -84,6 +86,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -450,6 +453,10 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int return null; }).when(shard).relocated(any(String.class), any(Runnable.class)); when(shard.acquireIndexCommit(anyBoolean())).thenReturn(mock(Engine.IndexCommitRef.class)); + doAnswer(invocationOrMock -> { + ((CountDownLatch)invocationOrMock.getArguments()[2]).countDown(); + return null; + }).when(shard).markAllocationIdAsInSync(any(String.class), anyLong(), any(CountDownLatch.class), any(Consumer.class)); final Supplier currentClusterStateVersionSupplier = () -> { assertFalse(ensureClusterStateVersionCalled.get()); From 8504b8201d99052e50415e555cc8b81ba60846c8 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 12 Jun 2017 21:31:04 -0400 Subject: [PATCH 12/54] Update knowledge via upate local checkpoint API --- .../index/seqno/GlobalCheckpointTracker.java | 65 +++++++++++++++++-- 1 file changed, 60 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 3ee309c940a57..d4b36dac8c7ec 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -23,6 +23,7 @@ import com.carrotsearch.hppc.ObjectLongMap; import com.carrotsearch.hppc.cursors.ObjectLongCursor; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.collect.HppcMaps; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.ShardId; @@ -255,14 +256,68 @@ public synchronized void updateAllocationIdsFromMaster( * @param seqNoPrimaryContext the sequence number context */ synchronized void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContext seqNoPrimaryContext) { - final Set inSyncAllocationIds = - new HashSet<>(Arrays.asList(seqNoPrimaryContext.inSyncLocalCheckpoints().keys().toArray(String.class))); - final Set trackingAllocationIds = - new HashSet<>(Arrays.asList(seqNoPrimaryContext.trackingLocalCheckpoints().keys().toArray(String.class))); - updateAllocationIdsFromMaster(inSyncAllocationIds, trackingAllocationIds); + /* + * We are gathered here today to witness the relocation handoff transferring knowledge from the relocation source to the relocation + * target. We need to consider the possibility that the version of the cluster state on the relocation source when the primary + * context was sampled is different than the version of the cluster state on the relocation target at this exact moment. We define + * the following values: + * - version(source) = the cluster state version on the relocation source used to ensure a minimum cluster state version on the + * relocation target + * - version(context) = the cluster state version on the relocation source when the primary context was sampled + * - version(target) = the current cluster state version on the relocation target + * + * We know that version(source) <= version(target) and version(context) < version(target), version(context) = version(target), and + * version(target) < version(context) are all possibilities. + * + * The case of version(context) = version(target) causes no issues as in this case the knowledge of the in-sync and initializing + * shards the target receives from the master will be equal to the knowledge of the in-sync and initializing shards the target + * receives from the relocation source via the primary context. + * + * In the case when version(context) < version(target) or version(target) < version(context), we first consider shards that could be + * contained in the primary context but not contained in the cluster state applied on the target. + * + * Suppose there is such a shard and that it is an in-sync shard. However, marking a shard as in-sync requires an operation permit + * on the primary shard. Such a permit can not be obtained after the relocation handoff has started as the relocation handoff blocks + * all operations. Therefore, there can not be such a shard that is marked in-sync. + * + * Now consider the case of an initializing shard that is contained in the primary context but not contained in the cluster state + * applied on the target. + * + * If version(context) < version(target) it means that the shard has been removed by a later cluster state update that is already + * applied on the target and we only need to ensure that we do not add it to the tracking map on the target. The call to + * GlobalCheckpointTracker#updateLocalCheckpoint(String, long) is a no-op for such shards and this is safe. + * + * If version(target) < version(context) it means that the shard has started initializing by a later cluster state update has not + * yet arrived on the target. However, there is a delay on recoveries before we ensure that version(source) <= version(target). + * Therefore, such a shard can never initialize from the relocation source and will have to await the handoff completing. As such, + * these shards are not problematic. + * + * Now we consider shards that are contained in the cluster state applied on the target but not contained in the primary context. + * + * If version(context) < version(target) it means that the target has learned of an initializing shard that the source is not aware + * of. As explained above, this initialization can only succeed after the relocation is complete, and only with the target as the + * source of the recovery. + * + * Otherwise, if version(target) < version(context) it only means that the global checkpoint on the target will be held back until a + * later cluster state update arrives because the target will not learn of the removal until later. + * + * In both cases, no calls to update the local checkpoint for such shards will be made. This case is safe too. + */ for (final ObjectLongCursor cursor : seqNoPrimaryContext.inSyncLocalCheckpoints()) { updateLocalCheckpoint(cursor.key, cursor.value); + assert cursor.value >= globalCheckpoint + : "local checkpoint [" + cursor.value + "] violates being at least the global checkpoint [" + globalCheckpoint + "]"; + try { + markAllocationIdAsInSync(cursor.key, cursor.value); + } catch (final InterruptedException e) { + /* + * Since the local checkpoint already exceeds the global checkpoint here, we never blocking waiting for advancement. This + * means that we can never be interrupted. If we are bail, something is catastrophically wrong. + */ + throw new AssertionError(e); + } } + for (final ObjectLongCursor cursor : seqNoPrimaryContext.trackingLocalCheckpoints()) { updateLocalCheckpoint(cursor.key, cursor.value); } From 5f05d92ef3850c483b3f802a0a3037ee03e78e78 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 14 Jun 2017 11:58:31 -0400 Subject: [PATCH 13/54] toString --- .../elasticsearch/index/seqno/SeqNoPrimaryContext.java | 8 ++++++++ .../org/elasticsearch/index/shard/PrimaryContext.java | 6 ++++++ .../recovery/RecoveryHandoffPrimaryContextRequest.java | 8 ++++++++ 3 files changed, 22 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java index 538df6c691c24..e95f387e3edd5 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java @@ -81,4 +81,12 @@ private static void writeMap(final StreamOutput out, final ObjectLongMap } } + @Override + public String toString() { + return "SeqNoPrimaryContext{" + + "inSyncLocalCheckpoints=" + inSyncLocalCheckpoints + + ", trackingLocalCheckpoints=" + trackingLocalCheckpoints + + '}'; + } + } diff --git a/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java index d17b563af2f54..dfd58962c1ef9 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java +++ b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java @@ -61,4 +61,10 @@ public void writeTo(final StreamOutput out) throws IOException { seqNoPrimaryContext.writeTo(out); } + @Override + public String toString() { + return "PrimaryContext{" + + "seqNoPrimaryContext=" + seqNoPrimaryContext + + '}'; + } } diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java index 95601061514c9..00b37be6c935b 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java @@ -84,4 +84,12 @@ public void writeTo(final StreamOutput out) throws IOException { primaryContext.writeTo(out); } + @Override + public String toString() { + return "RecoveryHandoffPrimaryContextRequest{" + + "recoveryId=" + recoveryId + + ", shardId=" + shardId + + ", primaryContext=" + primaryContext + + '}'; + } } From a35e497170696cd00dcea62871a50c6f2894465a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 14 Jun 2017 11:58:39 -0400 Subject: [PATCH 14/54] Visibility --- .../indices/recovery/RecoveryHandoffPrimaryContextRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java index 00b37be6c935b..ac6c9d7b3416a 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java @@ -30,7 +30,7 @@ /** * The request object to handoff the primary context to the relocation target. */ -public class RecoveryHandoffPrimaryContextRequest extends TransportRequest { +class RecoveryHandoffPrimaryContextRequest extends TransportRequest { private long recoveryId; private ShardId shardId; From b54a8d62e0779c76700d59255378c17f0ec946fd Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 14 Jun 2017 12:15:57 -0400 Subject: [PATCH 15/54] Refactor permit --- .../elasticsearch/index/shard/IndexShard.java | 58 +++---------------- .../recovery/RecoverySourceHandler.java | 26 ++++----- 2 files changed, 21 insertions(+), 63 deletions(-) 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 666e7564ea9c0..4a888e3dd96a6 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1541,59 +1541,17 @@ public void waitForOpsToComplete(final long seqNo) throws InterruptedException { * for additional details. Because this operation could be completed asynchronously on another thread, the caller must provide a latch; * this latch will be counted down after the operation completes. * - * @param allocationId the allocation ID of the shard to mark as in-sync - * @param localCheckpoint the current local checkpoint on the shard - * @param latch a latch that is counted down after the operation completes - * @param onFailureConsumer a callback if an exception occurs completing the operation + * @param allocationId the allocation ID of the shard to mark as in-sync + * @param localCheckpoint the current local checkpoint on the shard */ - public void markAllocationIdAsInSync( - final String allocationId, - final long localCheckpoint, - final CountDownLatch latch, - final Consumer onFailureConsumer) { + public void markAllocationIdAsInSync(final String allocationId, final long localCheckpoint) throws InterruptedException { + verifyPrimary(); + getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint); /* - * Before marking the shard as in-sync we acquire an operation permit. We do this so that there is a barrier between marking a shard - * as in-sync and relocating a shard. If we acquire the permit then no relocation handoff can complete before we are done marking - * the shard as in-sync. If the relocation handoff holds all the permits then after the handoff completes and we acquire the permit - * then the state of the shard will be relocated and this recovery will fail. + * We could have blocked so long waiting for the replica to catch up that we fell idle and there will not be a background sync to + * the replica; mark our self as active to force a future background sync. */ - acquirePrimaryOperationPermit( - new ActionListener() { - @Override - public void onResponse(final Releasable releasable) { - // we could have been relocated while waiting for a permit - if (state() == IndexShardState.RELOCATED) { - onFailure(new IndexShardRelocatedException(shardId)); - } - boolean success = false; - try { - getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint); - /* - * We could have blocked so long waiting for the replica to catch up that we fell idle and there will not be a - * background sync to the replica; mark our self as active to force a future background sync. - */ - active.compareAndSet(false, true); - success = true; - } catch (final Exception e) { - onFailure(e); - } finally { - releasable.close(); - if (success) { - latch.countDown(); - } - } - } - - @Override - public void onFailure(final Exception e) { - try { - onFailureConsumer.accept(e); - } finally { - latch.countDown(); - } - } - }, - ThreadPool.Names.GENERIC); + active.compareAndSet(false, true); } /** 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 ca173949b7ac3..42728b24d79db 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -31,6 +31,7 @@ import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.IOUtils; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.StopWatch; import org.elasticsearch.common.bytes.BytesArray; @@ -53,6 +54,7 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.RemoteTransportException; import java.io.BufferedOutputStream; @@ -451,23 +453,21 @@ public void finalizeRecovery(final long targetLocalCheckpoint) { StopWatch stopWatch = new StopWatch().start(); logger.trace("finalizing recovery"); cancellableThreads.execute(() -> { - final CountDownLatch latch = new CountDownLatch(1); - final AtomicReference reference = new AtomicReference<>(); - shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint, latch, reference::set); /* - * We wait for the shard to be marked as in-sync. However, our latch could also be counted down if marking the shard in-sync - * was interrupted or for any other exception that is thrown marking the shard as in-sync. Therefore, after we are unlatched we - * check if an exception was thrown. Interrupted exceptions must be handled specially due to their meaning to cancellable - * threads. All other exceptions are wrapped as runtime exceptions and rethrown. + * Before marking the shard as in-sync we acquire an operation permit. We do this so that there is a barrier between marking a + * shard as in-sync and relocating a shard. If we acquire the permit then no relocation handoff can complete before we are done + * marking the shard as in-sync. If the relocation handoff holds all the permits then after the handoff completes and we acquire + * the permit then the state of the shard will be relocated and this recovery will fail. */ - latch.await(); - if (reference.get() != null) { - if (reference.get() instanceof InterruptedException) { - throw (InterruptedException) reference.get(); - } else { - throw new RuntimeException(reference.get()); + final PlainActionFuture onAcquired = new PlainActionFuture<>(); + shard.acquirePrimaryOperationPermit(onAcquired, ThreadPool.Names.SAME); + try (Releasable ignored = onAcquired.actionGet()) { + if (shard.state() == IndexShardState.RELOCATED) { + throw new IndexShardRelocatedException(shard.shardId()); } + shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint); } + recoveryTarget.finalizeRecovery(shard.getGlobalCheckpoint()); }); From 48157cd2dce8fc99371a2de6c7c913d7b8b21c22 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 14 Jun 2017 12:18:44 -0400 Subject: [PATCH 16/54] Push down --- .../index/seqno/GlobalCheckpointTracker.java | 11 +++++++++++ .../index/seqno/SequenceNumbersService.java | 7 +------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index d4b36dac8c7ec..2c83f6cc61b34 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -250,6 +250,17 @@ public synchronized void updateAllocationIdsFromMaster( updateGlobalCheckpointOnPrimary(); } + /** + * Get the sequence number primary context for the shard. This includes the state of the global checkpoint tracker. + * + * @return the sequence number primary context + */ + synchronized SeqNoPrimaryContext seqNoPrimaryContext() { + final ObjectLongMap inSyncLocalCheckpoints = new ObjectLongHashMap<>(this.inSyncLocalCheckpoints); + final ObjectLongMap trackingLocalCheckpoints = new ObjectLongHashMap<>(this.trackingLocalCheckpoints); + return new SeqNoPrimaryContext(inSyncLocalCheckpoints, trackingLocalCheckpoints); + } + /** * Updates the known allocation IDs and the local checkpoints for the corresponding allocations from a primary relocation source. * diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java index f8d7615c8edec..10cc1b307bde9 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -200,12 +200,7 @@ public boolean pendingInSync() { * @return the sequence number primary context */ public SeqNoPrimaryContext seqNoPrimaryContext() { - synchronized (globalCheckpointTracker) { - final ObjectLongMap inSyncLocalCheckpoints = new ObjectLongHashMap<>(globalCheckpointTracker.inSyncLocalCheckpoints); - final ObjectLongMap trackingLocalCheckpoints = - new ObjectLongHashMap<>(globalCheckpointTracker.trackingLocalCheckpoints); - return new SeqNoPrimaryContext(inSyncLocalCheckpoints, trackingLocalCheckpoints); - } + return globalCheckpointTracker.seqNoPrimaryContext(); } } From 9a58ff416dae6289869f078317d388643d7413c0 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 14 Jun 2017 12:19:46 -0400 Subject: [PATCH 17/54] Imports --- .../org/elasticsearch/index/seqno/GlobalCheckpointTracker.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 2c83f6cc61b34..5defd34e0cdbb 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -23,12 +23,10 @@ import com.carrotsearch.hppc.ObjectLongMap; import com.carrotsearch.hppc.cursors.ObjectLongCursor; import org.elasticsearch.common.SuppressForbidden; -import org.elasticsearch.common.collect.HppcMaps; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.ShardId; -import java.util.Arrays; import java.util.HashSet; import java.util.Locale; import java.util.Set; From 62966c5c2d2beee72faffae28889452f6253aac1 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 14 Jun 2017 12:58:21 -0400 Subject: [PATCH 18/54] Docs --- .../index/seqno/GlobalCheckpointTracker.java | 1 + .../elasticsearch/index/shard/IndexShard.java | 19 ++++++++++++++----- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 5defd34e0cdbb..1ed372299f7d5 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -312,6 +312,7 @@ synchronized void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContex * * In both cases, no calls to update the local checkpoint for such shards will be made. This case is safe too. */ + assert inSyncLocalCheckpoints.isEmpty() : inSyncLocalCheckpoints; for (final ObjectLongCursor cursor : seqNoPrimaryContext.inSyncLocalCheckpoints()) { updateLocalCheckpoint(cursor.key, cursor.value); assert cursor.value >= globalCheckpoint 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 4a888e3dd96a6..6ba85c8c796ee 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -480,6 +480,15 @@ public IndexShardState markAsRecovering(String reason, RecoveryState recoverySta } } + /** + * Completes the relocation. Operations are blocked and current operations are drained before changing state to relocated. The provided + * {@link Runnable} is executed after all operations are successfully blocked. + * + * @param reason the reason for the relocation + * @param onBlocked a {@link Runnable} that is executed after operations are blocked + * @throws IllegalIndexShardStateException if the shard is not relocating due to concurrent cancellation + * @throws InterruptedException if blocking operations is interrupted + */ public void relocated(final String reason, final Runnable onBlocked) throws IllegalIndexShardStateException, InterruptedException { assert shardRouting.primary() : "only primaries can be marked as relocated: " + shardRouting; try { @@ -491,11 +500,11 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille if (state != IndexShardState.STARTED) { throw new IndexShardNotStartedException(shardId, state); } - // if the master cancelled the recovery, the target will be removed - // and the recovery will stopped. - // However, it is still possible that we concurrently end up here - // and therefore have to protect we don't mark the shard as relocated when - // its shard routing says otherwise. + /* + * If the master cancelled recovery, the target will be removed and the recovery will be cancelled. However, it is still + * possible that we concurrently end up here and therefore have to protect that we do not mark the shard as relocated + * when its shard routing says otherwise. + */ if (shardRouting.relocating() == false) { throw new IllegalIndexShardStateException(shardId, IndexShardState.STARTED, ": shard is no longer relocating " + shardRouting); From a4dda9307eb20fd6d74bae39b7613ffab4524546 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 14 Jun 2017 13:49:13 -0400 Subject: [PATCH 19/54] Fix compilation --- .../indices/recovery/RecoverySourceHandlerTests.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 6e00299d2aeaa..3e483bdf77a4f 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -453,10 +453,6 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int return null; }).when(shard).relocated(any(String.class), any(Runnable.class)); when(shard.acquireIndexCommit(anyBoolean())).thenReturn(mock(Engine.IndexCommitRef.class)); - doAnswer(invocationOrMock -> { - ((CountDownLatch)invocationOrMock.getArguments()[2]).countDown(); - return null; - }).when(shard).markAllocationIdAsInSync(any(String.class), anyLong(), any(CountDownLatch.class), any(Consumer.class)); final Supplier currentClusterStateVersionSupplier = () -> { assertFalse(ensureClusterStateVersionCalled.get()); From ef1d345fb90717c69983fac913140ba1252a245a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 14 Jun 2017 14:29:14 -0400 Subject: [PATCH 20/54] Remove assertion --- .../org/elasticsearch/index/seqno/GlobalCheckpointTracker.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 1ed372299f7d5..e67c11e18218b 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -312,10 +312,9 @@ synchronized void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContex * * In both cases, no calls to update the local checkpoint for such shards will be made. This case is safe too. */ - assert inSyncLocalCheckpoints.isEmpty() : inSyncLocalCheckpoints; for (final ObjectLongCursor cursor : seqNoPrimaryContext.inSyncLocalCheckpoints()) { updateLocalCheckpoint(cursor.key, cursor.value); - assert cursor.value >= globalCheckpoint + '' assert cursor.value >= globalCheckpoint : "local checkpoint [" + cursor.value + "] violates being at least the global checkpoint [" + globalCheckpoint + "]"; try { markAllocationIdAsInSync(cursor.key, cursor.value); From 51ad65b0ac2bbfe3e0f9e45e14c55d8097a4942d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 14 Jun 2017 14:32:41 -0400 Subject: [PATCH 21/54] Fix compilation --- .../org/elasticsearch/index/seqno/GlobalCheckpointTracker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index e67c11e18218b..5defd34e0cdbb 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -314,7 +314,7 @@ synchronized void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContex */ for (final ObjectLongCursor cursor : seqNoPrimaryContext.inSyncLocalCheckpoints()) { updateLocalCheckpoint(cursor.key, cursor.value); - '' assert cursor.value >= globalCheckpoint + assert cursor.value >= globalCheckpoint : "local checkpoint [" + cursor.value + "] violates being at least the global checkpoint [" + globalCheckpoint + "]"; try { markAllocationIdAsInSync(cursor.key, cursor.value); From 4ba8d5c9a2a9930463ddd9ecfbc40218f69e2a22 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 15 Jun 2017 08:31:23 -0400 Subject: [PATCH 22/54] Remove context wrapper --- .../index/seqno/GlobalCheckpointTracker.java | 16 ++--- ...rimaryContext.java => PrimaryContext.java} | 6 +- .../index/seqno/SequenceNumbersService.java | 16 ++--- .../elasticsearch/index/shard/IndexShard.java | 11 ++- .../index/shard/PrimaryContext.java | 70 ------------------- .../RecoveryHandoffPrimaryContextRequest.java | 2 +- .../indices/recovery/RecoveryTarget.java | 4 +- .../recovery/RecoveryTargetHandler.java | 2 +- .../recovery/RemoteRecoveryTargetHandler.java | 2 +- 9 files changed, 28 insertions(+), 101 deletions(-) rename core/src/main/java/org/elasticsearch/index/seqno/{SeqNoPrimaryContext.java => PrimaryContext.java} (92%) delete mode 100644 core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 5defd34e0cdbb..9a552a666ab1d 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -249,22 +249,22 @@ public synchronized void updateAllocationIdsFromMaster( } /** - * Get the sequence number primary context for the shard. This includes the state of the global checkpoint tracker. + * Get the primary context for the shard. This includes the state of the global checkpoint tracker. * - * @return the sequence number primary context + * @return the primary context */ - synchronized SeqNoPrimaryContext seqNoPrimaryContext() { + synchronized PrimaryContext primaryContext() { final ObjectLongMap inSyncLocalCheckpoints = new ObjectLongHashMap<>(this.inSyncLocalCheckpoints); final ObjectLongMap trackingLocalCheckpoints = new ObjectLongHashMap<>(this.trackingLocalCheckpoints); - return new SeqNoPrimaryContext(inSyncLocalCheckpoints, trackingLocalCheckpoints); + return new PrimaryContext(inSyncLocalCheckpoints, trackingLocalCheckpoints); } /** * Updates the known allocation IDs and the local checkpoints for the corresponding allocations from a primary relocation source. * - * @param seqNoPrimaryContext the sequence number context + * @param primaryContext the primary context */ - synchronized void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContext seqNoPrimaryContext) { + synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext primaryContext) { /* * We are gathered here today to witness the relocation handoff transferring knowledge from the relocation source to the relocation * target. We need to consider the possibility that the version of the cluster state on the relocation source when the primary @@ -312,7 +312,7 @@ synchronized void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContex * * In both cases, no calls to update the local checkpoint for such shards will be made. This case is safe too. */ - for (final ObjectLongCursor cursor : seqNoPrimaryContext.inSyncLocalCheckpoints()) { + for (final ObjectLongCursor cursor : primaryContext.inSyncLocalCheckpoints()) { updateLocalCheckpoint(cursor.key, cursor.value); assert cursor.value >= globalCheckpoint : "local checkpoint [" + cursor.value + "] violates being at least the global checkpoint [" + globalCheckpoint + "]"; @@ -327,7 +327,7 @@ synchronized void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContex } } - for (final ObjectLongCursor cursor : seqNoPrimaryContext.trackingLocalCheckpoints()) { + for (final ObjectLongCursor cursor : primaryContext.trackingLocalCheckpoints()) { updateLocalCheckpoint(cursor.key, cursor.value); } } diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java b/core/src/main/java/org/elasticsearch/index/seqno/PrimaryContext.java similarity index 92% rename from core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java rename to core/src/main/java/org/elasticsearch/index/seqno/PrimaryContext.java index e95f387e3edd5..95cbe18f98a9d 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoPrimaryContext.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/PrimaryContext.java @@ -32,7 +32,7 @@ * Represents the sequence number component of the primary context. This is the knowledge on the primary of the in-sync and initializing * shards and their local checkpoints. */ -public class SeqNoPrimaryContext implements Writeable { +public class PrimaryContext implements Writeable { private ObjectLongMap inSyncLocalCheckpoints; @@ -46,12 +46,12 @@ public ObjectLongMap trackingLocalCheckpoints() { return trackingLocalCheckpoints; } - public SeqNoPrimaryContext(final ObjectLongMap inSyncLocalCheckpoints, final ObjectLongMap trackingLocalCheckpoints) { + public PrimaryContext(final ObjectLongMap inSyncLocalCheckpoints, final ObjectLongMap trackingLocalCheckpoints) { this.inSyncLocalCheckpoints = inSyncLocalCheckpoints; this.trackingLocalCheckpoints = trackingLocalCheckpoints; } - public SeqNoPrimaryContext(StreamInput in) throws IOException { + public PrimaryContext(final StreamInput in) throws IOException { inSyncLocalCheckpoints = readMap(in); trackingLocalCheckpoints = readMap(in); } diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java index 10cc1b307bde9..9ce955c959353 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -19,8 +19,6 @@ package org.elasticsearch.index.seqno; -import com.carrotsearch.hppc.ObjectLongHashMap; -import com.carrotsearch.hppc.ObjectLongMap; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.ShardId; @@ -179,10 +177,10 @@ public void updateAllocationIdsFromMaster(final Set activeAllocationIds, /** * Updates the known allocation IDs and the local checkpoints for the corresponding allocations from a primary relocation source. * - * @param seqNoPrimaryContext the sequence number context + * @param primaryContext the sequence number context */ - public void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContext seqNoPrimaryContext) { - globalCheckpointTracker.updateAllocationIdsFromPrimaryContext(seqNoPrimaryContext); + public void updateAllocationIdsFromPrimaryContext(final PrimaryContext primaryContext) { + globalCheckpointTracker.updateAllocationIdsFromPrimaryContext(primaryContext); } /** @@ -195,12 +193,12 @@ public boolean pendingInSync() { } /** - * Get the sequence number primary context for the shard. This includes the state of the global checkpoint tracker. + * Get the primary context for the shard. This includes the state of the global checkpoint tracker. * - * @return the sequence number primary context + * @return the primary context */ - public SeqNoPrimaryContext seqNoPrimaryContext() { - return globalCheckpointTracker.seqNoPrimaryContext(); + public PrimaryContext primaryContext() { + return globalCheckpointTracker.primaryContext(); } } 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 6ba85c8c796ee..a6d6af77d4b5d 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -57,7 +57,6 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; -import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexNotFoundException; @@ -96,7 +95,7 @@ import org.elasticsearch.index.refresh.RefreshStats; import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.ShardSearchStats; -import org.elasticsearch.index.seqno.SeqNoPrimaryContext; +import org.elasticsearch.index.seqno.PrimaryContext; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.similarity.SimilarityService; @@ -531,7 +530,7 @@ public PrimaryContext primaryContext() { verifyPrimary(); assert shardRouting.relocating() : "primary context can only be obtained from a relocating primary but was " + shardRouting; assert !shardRouting.isRelocationTarget() : "primary context can only be obtained from relocation source but was " + shardRouting; - return new PrimaryContext(getEngine().seqNoService().seqNoPrimaryContext()); + return getEngine().seqNoService().primaryContext(); } public IndexShardState state() { @@ -1626,14 +1625,14 @@ public void updateAllocationIdsFromMaster(final Set activeAllocationIds, /** * Updates the known allocation IDs and the local checkpoints for the corresponding allocations from a primary relocation source. * - * @param seqNoPrimaryContext the sequence number context + * @param primaryContext the sequence number context */ - public void updateAllocationIdsFromPrimaryContext(final SeqNoPrimaryContext seqNoPrimaryContext) { + public void updateAllocationIdsFromPrimaryContext(final PrimaryContext primaryContext) { verifyPrimary(); assert shardRouting.isRelocationTarget(); final Engine engine = getEngineOrNull(); if (engine != null) { - engine.seqNoService().updateAllocationIdsFromPrimaryContext(seqNoPrimaryContext); + engine.seqNoService().updateAllocationIdsFromPrimaryContext(primaryContext); } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java deleted file mode 100644 index dfd58962c1ef9..0000000000000 --- a/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.shard; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.index.seqno.SeqNoPrimaryContext; - -import java.io.IOException; - -/** - * Represents the primary context which encapsulates the view the primary shard has of its replication group. - */ -public class PrimaryContext implements Writeable { - - private final SeqNoPrimaryContext seqNoPrimaryContext; - - public SeqNoPrimaryContext seqNoPrimaryContext() { - return seqNoPrimaryContext; - } - - /** - * Construct a primary context with the sequence number primary context. - * - * @param seqNoPrimaryContext the sequence number primary context - */ - public PrimaryContext(final SeqNoPrimaryContext seqNoPrimaryContext) { - this.seqNoPrimaryContext = seqNoPrimaryContext; - } - - /** - * Construct a primary context from a stream. - * - * @param in the stream - * @throws IOException if an I/O exception occurs reading from the stream - */ - public PrimaryContext(final StreamInput in) throws IOException { - seqNoPrimaryContext = new SeqNoPrimaryContext(in); - } - - @Override - public void writeTo(final StreamOutput out) throws IOException { - seqNoPrimaryContext.writeTo(out); - } - - @Override - public String toString() { - return "PrimaryContext{" + - "seqNoPrimaryContext=" + seqNoPrimaryContext + - '}'; - } -} diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java index ac6c9d7b3416a..61a66ea25d6df 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java @@ -21,7 +21,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.index.shard.PrimaryContext; +import org.elasticsearch.index.seqno.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.transport.TransportRequest; diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 0f1ecd346ecd2..4182b79647966 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -41,10 +41,10 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.MapperException; +import org.elasticsearch.index.seqno.PrimaryContext; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardNotRecoveringException; import org.elasticsearch.index.shard.IndexShardState; -import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; @@ -381,7 +381,7 @@ public void ensureClusterStateVersion(long clusterStateVersion) { @Override public void handoffPrimaryContext(final PrimaryContext primaryContext) { - indexShard.updateAllocationIdsFromPrimaryContext(primaryContext.seqNoPrimaryContext()); + indexShard.updateAllocationIdsFromPrimaryContext(primaryContext); } @Override diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java index 34b0df2293f3f..196fcc0613ec2 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java @@ -19,7 +19,7 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.index.shard.PrimaryContext; +import org.elasticsearch.index.seqno.PrimaryContext; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index 1b88613da1d44..6a0dd1b52ad84 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -23,7 +23,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.index.shard.PrimaryContext; +import org.elasticsearch.index.seqno.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; From 48572a67da40ebc28dee536b71acb2ab97928de5 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 15 Jun 2017 08:36:02 -0400 Subject: [PATCH 23/54] Move PrimaryContext to new package --- .../org/elasticsearch/index/seqno/GlobalCheckpointTracker.java | 1 + .../org/elasticsearch/index/seqno/SequenceNumbersService.java | 1 + .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 1 - .../elasticsearch/index/{seqno => shard}/PrimaryContext.java | 2 +- .../indices/recovery/RecoveryHandoffPrimaryContextRequest.java | 2 +- .../java/org/elasticsearch/indices/recovery/RecoveryTarget.java | 2 +- .../elasticsearch/indices/recovery/RecoveryTargetHandler.java | 2 +- .../indices/recovery/RemoteRecoveryTargetHandler.java | 2 +- 8 files changed, 7 insertions(+), 6 deletions(-) rename core/src/main/java/org/elasticsearch/index/{seqno => shard}/PrimaryContext.java (98%) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 9a552a666ab1d..0c15474124e43 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import java.util.HashSet; diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java index 9ce955c959353..b4ffc0bbffb1a 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -21,6 +21,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import java.util.Set; 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 a6d6af77d4b5d..db26fdf1a56b2 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -95,7 +95,6 @@ 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.PrimaryContext; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.similarity.SimilarityService; diff --git a/core/src/main/java/org/elasticsearch/index/seqno/PrimaryContext.java b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java similarity index 98% rename from core/src/main/java/org/elasticsearch/index/seqno/PrimaryContext.java rename to core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java index 95cbe18f98a9d..ff6f342fd0bec 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/PrimaryContext.java +++ b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.index.seqno; +package org.elasticsearch.index.shard; import com.carrotsearch.hppc.ObjectLongHashMap; import com.carrotsearch.hppc.ObjectLongMap; diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java index 61a66ea25d6df..ac6c9d7b3416a 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java @@ -21,7 +21,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.index.seqno.PrimaryContext; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.transport.TransportRequest; diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 4182b79647966..87dabc77f0f6a 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -41,7 +41,7 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.MapperException; -import org.elasticsearch.index.seqno.PrimaryContext; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardNotRecoveringException; import org.elasticsearch.index.shard.IndexShardState; diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java index 196fcc0613ec2..34b0df2293f3f 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java @@ -19,7 +19,7 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.index.seqno.PrimaryContext; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index 6a0dd1b52ad84..1b88613da1d44 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -23,7 +23,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.index.seqno.PrimaryContext; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; From c9bd0d72a4b3416ea101e54a887e23c3f7283cc0 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 15 Jun 2017 09:09:37 -0400 Subject: [PATCH 24/54] Piping for cluster state version This commit adds piping for the cluster state version to the global checkpoint tracker. We do not use it yet. --- .../index/seqno/GlobalCheckpointTracker.java | 13 ++++--- .../index/seqno/SequenceNumbersService.java | 12 ++++--- .../elasticsearch/index/shard/IndexShard.java | 12 ++++--- .../index/shard/PrimaryContext.java | 19 ++++++++-- .../cluster/IndicesClusterStateService.java | 12 ++++--- .../index/engine/InternalEngineTests.java | 5 ++- .../ESIndexLevelReplicationTestCase.java | 10 +++++- .../seqno/GlobalCheckpointTrackerTests.java | 36 +++++++++++-------- ...actIndicesClusterStateServiceTestCase.java | 5 ++- 9 files changed, 84 insertions(+), 40 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 0c15474124e43..132c2b61b2f8a 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -43,6 +43,8 @@ */ public class GlobalCheckpointTracker extends AbstractIndexShardComponent { + private long clusterStateVersion; + /* * This map holds the last known local checkpoint for every active shard and initializing shard copies that has been brought up to speed * through recovery. These shards are treated as valid copies and participate in determining the global checkpoint. This map is keyed by @@ -211,11 +213,14 @@ synchronized void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { /** * Notifies the service of the current allocation ids in the cluster state. This method trims any shards that have been removed. * - * @param activeAllocationIds the allocation IDs of the currently active shard copies - * @param initializingAllocationIds the allocation IDs of the currently initializing shard copies + * @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master + * @param activeAllocationIds the allocation IDs of the currently active shard copies + * @param initializingAllocationIds the allocation IDs of the currently initializing shard copies */ public synchronized void updateAllocationIdsFromMaster( - final Set activeAllocationIds, final Set initializingAllocationIds) { + final long applyingClusterStateVersion, final Set activeAllocationIds, final Set initializingAllocationIds) { + clusterStateVersion = applyingClusterStateVersion; + // remove shards whose allocation ID no longer exists inSyncLocalCheckpoints.removeAll(a -> !activeAllocationIds.contains(a) && !initializingAllocationIds.contains(a)); @@ -257,7 +262,7 @@ public synchronized void updateAllocationIdsFromMaster( synchronized PrimaryContext primaryContext() { final ObjectLongMap inSyncLocalCheckpoints = new ObjectLongHashMap<>(this.inSyncLocalCheckpoints); final ObjectLongMap trackingLocalCheckpoints = new ObjectLongHashMap<>(this.trackingLocalCheckpoints); - return new PrimaryContext(inSyncLocalCheckpoints, trackingLocalCheckpoints); + return new PrimaryContext(clusterStateVersion, inSyncLocalCheckpoints, trackingLocalCheckpoints); } /** diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java index b4ffc0bbffb1a..4d174744a0e2f 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -166,13 +166,15 @@ public void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { /** * Notifies the service of the current allocation IDs in the cluster state. See - * {@link GlobalCheckpointTracker#updateAllocationIdsFromMaster(Set, Set)} for details. + * {@link GlobalCheckpointTracker#updateAllocationIdsFromMaster(long, Set, Set)} for details. * - * @param activeAllocationIds the allocation IDs of the currently active shard copies - * @param initializingAllocationIds the allocation IDs of the currently initializing shard copies + * @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master + * @param activeAllocationIds the allocation IDs of the currently active shard copies + * @param initializingAllocationIds the allocation IDs of the currently initializing shard copies */ - public void updateAllocationIdsFromMaster(final Set activeAllocationIds, final Set initializingAllocationIds) { - globalCheckpointTracker.updateAllocationIdsFromMaster(activeAllocationIds, initializingAllocationIds); + public void updateAllocationIdsFromMaster( + final long applyingClusterStateVersion, final Set activeAllocationIds, final Set initializingAllocationIds) { + globalCheckpointTracker.updateAllocationIdsFromMaster(applyingClusterStateVersion, activeAllocationIds, initializingAllocationIds); } /** 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 db26fdf1a56b2..6cfcd044b1802 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1606,18 +1606,20 @@ public void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { /** * Notifies the service of the current allocation IDs in the cluster state. See - * {@link org.elasticsearch.index.seqno.GlobalCheckpointTracker#updateAllocationIdsFromMaster(Set, Set)} + * {@link org.elasticsearch.index.seqno.GlobalCheckpointTracker#updateAllocationIdsFromMaster(long, Set, Set)} * for details. * - * @param activeAllocationIds the allocation IDs of the currently active shard copies - * @param initializingAllocationIds the allocation IDs of the currently initializing shard copies + * @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master + * @param activeAllocationIds the allocation IDs of the currently active shard copies + * @param initializingAllocationIds the allocation IDs of the currently initializing shard copies */ - public void updateAllocationIdsFromMaster(final Set activeAllocationIds, final Set initializingAllocationIds) { + public void updateAllocationIdsFromMaster( + final long applyingClusterStateVersion, final Set activeAllocationIds, final Set initializingAllocationIds) { verifyPrimary(); final Engine engine = getEngineOrNull(); // if the engine is not yet started, we are not ready yet and can just ignore this if (engine != null) { - engine.seqNoService().updateAllocationIdsFromMaster(activeAllocationIds, initializingAllocationIds); + engine.seqNoService().updateAllocationIdsFromMaster(applyingClusterStateVersion, activeAllocationIds, initializingAllocationIds); } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java index ff6f342fd0bec..cacad083e725f 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java +++ b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java @@ -34,6 +34,12 @@ */ public class PrimaryContext implements Writeable { + private long clusterStateVersion; + + public long clusterStateVersion() { + return clusterStateVersion; + } + private ObjectLongMap inSyncLocalCheckpoints; public ObjectLongMap inSyncLocalCheckpoints() { @@ -46,12 +52,17 @@ public ObjectLongMap trackingLocalCheckpoints() { return trackingLocalCheckpoints; } - public PrimaryContext(final ObjectLongMap inSyncLocalCheckpoints, final ObjectLongMap trackingLocalCheckpoints) { + public PrimaryContext( + final long clusterStateVersion, + final ObjectLongMap inSyncLocalCheckpoints, + final ObjectLongMap trackingLocalCheckpoints) { + this.clusterStateVersion = clusterStateVersion; this.inSyncLocalCheckpoints = inSyncLocalCheckpoints; this.trackingLocalCheckpoints = trackingLocalCheckpoints; } public PrimaryContext(final StreamInput in) throws IOException { + clusterStateVersion = in.readVLong(); inSyncLocalCheckpoints = readMap(in); trackingLocalCheckpoints = readMap(in); } @@ -69,6 +80,7 @@ private static ObjectLongMap readMap(final StreamInput in) throws IOExce @Override public void writeTo(final StreamOutput out) throws IOException { + out.writeVLong(clusterStateVersion); writeMap(out, inSyncLocalCheckpoints); writeMap(out, trackingLocalCheckpoints); } @@ -83,8 +95,9 @@ private static void writeMap(final StreamOutput out, final ObjectLongMap @Override public String toString() { - return "SeqNoPrimaryContext{" + - "inSyncLocalCheckpoints=" + inSyncLocalCheckpoints + + return "PrimaryContext{" + + "clusterStateVersion=" + clusterStateVersion + + ", inSyncLocalCheckpoints=" + inSyncLocalCheckpoints + ", trackingLocalCheckpoints=" + trackingLocalCheckpoints + '}'; } diff --git a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 9d091429f2240..f07c97e0b1a12 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -561,7 +561,7 @@ private void updateShard(DiscoveryNodes nodes, ShardRouting shardRouting, Shard final Set initializingIds = allocationIdsForShardsOnNodesThatUnderstandSeqNos(indexShardRoutingTable.getAllInitializingShards(), nodes); shard.updatePrimaryTerm(clusterState.metaData().index(shard.shardId().getIndex()).primaryTerm(shard.shardId().id())); - shard.updateAllocationIdsFromMaster(activeIds, initializingIds); + shard.updateAllocationIdsFromMaster(clusterState.version(), activeIds, initializingIds); } } catch (Exception e) { failAndRemoveShard(shardRouting, true, "failed updating shard routing entry", e, clusterState); @@ -746,12 +746,14 @@ public interface Shard { /** * Notifies the service of the current allocation ids in the cluster state. - * See {@link GlobalCheckpointTracker#updateAllocationIdsFromMaster(Set, Set)} for details. + * See {@link GlobalCheckpointTracker#updateAllocationIdsFromMaster(long, Set, Set)} for details. * - * @param activeAllocationIds the allocation ids of the currently active shard copies - * @param initializingAllocationIds the allocation ids of the currently initializing shard copies + * @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master + * @param activeAllocationIds the allocation ids of the currently active shard copies + * @param initializingAllocationIds the allocation ids of the currently initializing shard copies */ - void updateAllocationIdsFromMaster(Set activeAllocationIds, Set initializingAllocationIds); + void updateAllocationIdsFromMaster( + long applyingClusterStateVersion, Set activeAllocationIds, Set initializingAllocationIds); } public interface AllocatedIndex extends Iterable, IndexComponent { 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 16e746a67f7bd..a6222ed706ce5 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -2019,7 +2019,10 @@ public void testSeqNoAndCheckpoints() throws IOException { initialEngine = engine; initialEngine .seqNoService() - .updateAllocationIdsFromMaster(new HashSet<>(Arrays.asList("primary", "replica")), Collections.emptySet()); + .updateAllocationIdsFromMaster( + randomNonNegativeLong(), + new HashSet<>(Arrays.asList("primary", "replica")), + Collections.emptySet()); for (int op = 0; op < opCount; op++) { final String id; // mostly index, sometimes delete diff --git a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 72ace394d0119..e33a255b88455 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -118,6 +118,7 @@ protected DiscoveryNode getDiscoveryNode(String id) { } protected class ReplicationGroup implements AutoCloseable, Iterable { + private long clusterStateVersion; private IndexShard primary; private IndexMetaData indexMetaData; private final List replicas; @@ -130,6 +131,7 @@ protected class ReplicationGroup implements AutoCloseable, Iterable primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting)); replicas = new ArrayList<>(); this.indexMetaData = indexMetaData; + clusterStateVersion = 1; updateAllocationIDsOnPrimary(); for (int i = 0; i < indexMetaData.getNumberOfReplicas(); i++) { addReplica(); @@ -210,6 +212,7 @@ public void startPrimary() throws IOException { primary.markAsRecovering("store", new RecoveryState(primary.routingEntry(), pNode, null)); primary.recoverFromStore(); primary.updateRoutingEntry(ShardRoutingHelper.moveToStarted(primary.routingEntry())); + clusterStateVersion++; updateAllocationIDsOnPrimary(); for (final IndexShard replica : replicas) { recoverReplica(replica); @@ -229,6 +232,7 @@ assert shardRoutings().stream() .filter(shardRouting -> shardRouting.isSameAllocation(replica.routingEntry())).findFirst().isPresent() == false : "replica with aId [" + replica.routingEntry().allocationId() + "] already exists"; replicas.add(replica); + clusterStateVersion++; updateAllocationIDsOnPrimary(); } @@ -243,6 +247,7 @@ public synchronized IndexShard addReplicaWithExistingPath(final ShardPath shardP final IndexShard newReplica = newShard(shardRouting, shardPath, indexMetaData, null, getEngineFactory(shardRouting)); replicas.add(newReplica); + clusterStateVersion++; updateAllocationIDsOnPrimary(); return newReplica; } @@ -263,12 +268,14 @@ public synchronized void promoteReplicaToPrimary(IndexShard replica) throws IOEx primary = replica; primary.updateRoutingEntry(replica.routingEntry().moveActiveReplicaToPrimary()); primary.updatePrimaryTerm(newTerm); + clusterStateVersion++; updateAllocationIDsOnPrimary(); } synchronized boolean removeReplica(IndexShard replica) { final boolean removed = replicas.remove(replica); if (removed) { + clusterStateVersion++; updateAllocationIDsOnPrimary(); } return removed; @@ -288,6 +295,7 @@ public void recoverReplica( BiFunction targetSupplier, boolean markAsRecovering) throws IOException { ESIndexLevelReplicationTestCase.this.recoverReplica(replica, primary, targetSupplier, markAsRecovering); + clusterStateVersion++; updateAllocationIDsOnPrimary(); } @@ -375,7 +383,7 @@ private void updateAllocationIDsOnPrimary() { initializing.add(shard.allocationId().getId()); } } - primary.updateAllocationIdsFromMaster(active, initializing); + primary.updateAllocationIdsFromMaster(clusterStateVersion, active, initializing); } } diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java index 61eb45813288b..05e402bb900b7 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java @@ -79,6 +79,7 @@ private Map randomAllocationsWithLocalCheckpoints(int min, int max } public void testGlobalCheckpointUpdate() { + final long initialClusterStateVersion = randomNonNegativeLong(); Map allocations = new HashMap<>(); Map activeWithCheckpoints = randomAllocationsWithLocalCheckpoints(0, 5); Set active = new HashSet<>(activeWithCheckpoints.keySet()); @@ -107,7 +108,7 @@ public void testGlobalCheckpointUpdate() { logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type); }); - tracker.updateAllocationIdsFromMaster(active, initializing); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, active, initializing); initializing.forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId, tracker.getGlobalCheckpoint())); allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId, allocations.get(aId))); @@ -130,7 +131,7 @@ public void testGlobalCheckpointUpdate() { Set newActive = new HashSet<>(active); newActive.add(extraId); - tracker.updateAllocationIdsFromMaster(newActive, initializing); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 1, newActive, initializing); // now notify for the new id tracker.updateLocalCheckpoint(extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4)); @@ -146,6 +147,7 @@ public void testMissingActiveIdsPreventAdvance() { assigned.putAll(active); assigned.putAll(initializing); tracker.updateAllocationIdsFromMaster( + randomNonNegativeLong(), active.keySet(), initializing.keySet()); randomSubsetOf(initializing.keySet()).forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint())); @@ -166,7 +168,7 @@ public void testMissingActiveIdsPreventAdvance() { public void testMissingInSyncIdsPreventAdvance() { final Map active = randomAllocationsWithLocalCheckpoints(0, 5); final Map initializing = randomAllocationsWithLocalCheckpoints(1, 5); - tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet()); + tracker.updateAllocationIdsFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet()); initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint())); randomSubsetOf(randomInt(initializing.size() - 1), initializing.keySet()).forEach(aId -> tracker.updateLocalCheckpoint(aId, initializing.get(aId))); @@ -184,7 +186,7 @@ public void testInSyncIdsAreIgnoredIfNotValidatedByMaster() { final Map active = randomAllocationsWithLocalCheckpoints(1, 5); final Map initializing = randomAllocationsWithLocalCheckpoints(1, 5); final Map nonApproved = randomAllocationsWithLocalCheckpoints(1, 5); - tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet()); + tracker.updateAllocationIdsFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet()); initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint())); nonApproved.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint())); @@ -196,6 +198,7 @@ public void testInSyncIdsAreIgnoredIfNotValidatedByMaster() { } public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { + final long initialClusterStateVersion = randomNonNegativeLong(); final Map activeToStay = randomAllocationsWithLocalCheckpoints(1, 5); final Map initializingToStay = randomAllocationsWithLocalCheckpoints(1, 5); final Map activeToBeRemoved = randomAllocationsWithLocalCheckpoints(1, 5); @@ -211,7 +214,7 @@ public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { if (randomBoolean()) { allocations.putAll(initializingToBeRemoved); } - tracker.updateAllocationIdsFromMaster(active, initializing); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, active, initializing); if (randomBoolean()) { initializingToStay.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, tracker.getGlobalCheckpoint())); } else { @@ -223,11 +226,11 @@ public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { // now remove shards if (randomBoolean()) { - tracker.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet()); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 1, activeToStay.keySet(), initializingToStay.keySet()); allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid, ckp + 10L)); } else { allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid, ckp + 10L)); - tracker.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet()); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 2, activeToStay.keySet(), initializingToStay.keySet()); } final long checkpoint = Stream.concat(activeToStay.values().stream(), initializingToStay.values().stream()) @@ -243,7 +246,8 @@ public void testWaitForAllocationIdToBeInSync() throws BrokenBarrierException, I final AtomicBoolean complete = new AtomicBoolean(); final String inSyncAllocationId =randomAlphaOfLength(16); final String trackingAllocationId = randomAlphaOfLength(16); - tracker.updateAllocationIdsFromMaster(Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId)); + tracker.updateAllocationIdsFromMaster( + randomNonNegativeLong(), Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId)); tracker.updateLocalCheckpoint(inSyncAllocationId, globalCheckpoint); final Thread thread = new Thread(() -> { try { @@ -291,7 +295,8 @@ public void testWaitForAllocationIdToBeInSyncCanBeInterrupted() throws BrokenBar final AtomicBoolean interrupted = new AtomicBoolean(); final String inSyncAllocationId = randomAlphaOfLength(16); final String trackingAllocationId = randomAlphaOfLength(32); - tracker.updateAllocationIdsFromMaster(Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId)); + tracker.updateAllocationIdsFromMaster( + randomNonNegativeLong(), Collections.singleton(inSyncAllocationId), Collections.singleton(trackingAllocationId)); tracker.updateLocalCheckpoint(inSyncAllocationId, globalCheckpoint); final Thread thread = new Thread(() -> { try { @@ -329,6 +334,7 @@ public void testWaitForAllocationIdToBeInSyncCanBeInterrupted() throws BrokenBar } public void testUpdateAllocationIdsFromMaster() throws Exception { + final long initialClusterStateVersion = randomNonNegativeLong(); final int numberOfActiveAllocationsIds = randomIntBetween(2, 16); final Set activeAllocationIds = IntStream.range(0, numberOfActiveAllocationsIds).mapToObj(i -> randomAlphaOfLength(16)).collect(Collectors.toSet()); @@ -343,7 +349,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { } } while (true); }).collect(Collectors.toSet()); - tracker.updateAllocationIdsFromMaster(activeAllocationIds, initializingIds); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, activeAllocationIds, initializingIds); // first we assert that the in-sync and tracking sets are set up correctly assertTrue(activeAllocationIds.stream().allMatch(a -> tracker.inSyncLocalCheckpoints.containsKey(a))); @@ -364,7 +370,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { final List removingInitializingAllocationIds = randomSubsetOf(initializingIds); final Set newInitializingAllocationIds = initializingIds.stream().filter(a -> !removingInitializingAllocationIds.contains(a)).collect(Collectors.toSet()); - tracker.updateAllocationIdsFromMaster(newActiveAllocationIds, newInitializingAllocationIds); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 1, newActiveAllocationIds, newInitializingAllocationIds); assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.inSyncLocalCheckpoints.containsKey(a))); assertTrue(removingActiveAllocationIds.stream().noneMatch(a -> tracker.inSyncLocalCheckpoints.containsKey(a))); assertTrue(newInitializingAllocationIds.stream().allMatch(a -> tracker.trackingLocalCheckpoints.containsKey(a))); @@ -376,7 +382,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { */ newActiveAllocationIds.add(randomAlphaOfLength(32)); newInitializingAllocationIds.add(randomAlphaOfLength(64)); - tracker.updateAllocationIdsFromMaster(newActiveAllocationIds, newInitializingAllocationIds); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 2, newActiveAllocationIds, newInitializingAllocationIds); assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.inSyncLocalCheckpoints.containsKey(a))); assertTrue( newActiveAllocationIds @@ -416,7 +422,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { // using a different length than we have been using above ensures that we can not collide with a previous allocation ID final String newSyncingAllocationId = randomAlphaOfLength(128); newInitializingAllocationIds.add(newSyncingAllocationId); - tracker.updateAllocationIdsFromMaster(newActiveAllocationIds, newInitializingAllocationIds); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 3, newActiveAllocationIds, newInitializingAllocationIds); final CyclicBarrier barrier = new CyclicBarrier(2); final Thread thread = new Thread(() -> { try { @@ -450,7 +456,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { * the in-sync set even if we receive a cluster state update that does not reflect this. * */ - tracker.updateAllocationIdsFromMaster(newActiveAllocationIds, newInitializingAllocationIds); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion + 4, newActiveAllocationIds, newInitializingAllocationIds); assertFalse(tracker.trackingLocalCheckpoints.containsKey(newSyncingAllocationId)); assertTrue(tracker.inSyncLocalCheckpoints.containsKey(newSyncingAllocationId)); } @@ -471,7 +477,7 @@ public void testRaceUpdatingGlobalCheckpoint() throws InterruptedException, Brok final String active = randomAlphaOfLength(16); final String initializing = randomAlphaOfLength(32); - tracker.updateAllocationIdsFromMaster(Collections.singleton(active), Collections.singleton(initializing)); + tracker.updateAllocationIdsFromMaster(randomNonNegativeLong(), Collections.singleton(active), Collections.singleton(initializing)); final CyclicBarrier barrier = new CyclicBarrier(4); diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index 4a3b94163962a..46341c23c022a 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -319,6 +319,7 @@ public Index index() { * Mock for {@link IndexShard} */ protected class MockIndexShard implements IndicesClusterStateService.Shard { + private volatile long clusterStateVersion; private volatile ShardRouting shardRouting; private volatile RecoveryState recoveryState; private volatile Set activeAllocationIds; @@ -368,7 +369,9 @@ public void updatePrimaryTerm(long primaryTerm) { } @Override - public void updateAllocationIdsFromMaster(Set activeAllocationIds, Set initializingAllocationIds) { + public void updateAllocationIdsFromMaster( + long applyingClusterStateVersion, Set activeAllocationIds, Set initializingAllocationIds) { + this.clusterStateVersion = applyingClusterStateVersion; this.activeAllocationIds = activeAllocationIds; this.initializingAllocationIds = initializingAllocationIds; } From acca222063a2013955b29701e751163c3d62c5d9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 15 Jun 2017 10:47:06 -0400 Subject: [PATCH 25/54] Remove unused import --- .../elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java index 05e402bb900b7..5b21aa809c16b 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java @@ -29,7 +29,6 @@ import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; From d459f166669c6327bc1e18a15871de79d67ba0bd Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 15 Jun 2017 11:06:58 -0400 Subject: [PATCH 26/54] Implement versioning in tracker --- .../index/seqno/GlobalCheckpointTracker.java | 14 ++++++++++++++ .../recovery/RecoveryWhileUnderLoadIT.java | 2 +- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 132c2b61b2f8a..a9ded1c55e6a7 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -28,6 +28,7 @@ import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; +import java.util.Arrays; import java.util.HashSet; import java.util.Locale; import java.util.Set; @@ -219,6 +220,10 @@ synchronized void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { */ public synchronized void updateAllocationIdsFromMaster( final long applyingClusterStateVersion, final Set activeAllocationIds, final Set initializingAllocationIds) { + if (applyingClusterStateVersion < clusterStateVersion) { + return; + } + clusterStateVersion = applyingClusterStateVersion; // remove shards whose allocation ID no longer exists @@ -318,6 +323,15 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri * * In both cases, no calls to update the local checkpoint for such shards will be made. This case is safe too. */ + + if (primaryContext.clusterStateVersion() > clusterStateVersion) { + final Set activeAllocationIds = + new HashSet<>(Arrays.asList(primaryContext.inSyncLocalCheckpoints().keys().toArray(String.class))); + final Set initializingAllocationIds = + new HashSet<>(Arrays.asList(primaryContext.trackingLocalCheckpoints().keys().toArray(String.class))); + updateAllocationIdsFromMaster(primaryContext.clusterStateVersion(), activeAllocationIds, initializingAllocationIds); + } + for (final ObjectLongCursor cursor : primaryContext.inSyncLocalCheckpoints()) { updateLocalCheckpoint(cursor.key, cursor.value); assert cursor.value >= globalCheckpoint diff --git a/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java b/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java index e1a7a07448f1b..b0d25f43bd694 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java @@ -53,7 +53,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; -@TestLogging("_root:DEBUG,org.elasticsearch.index.shard:TRACE") +@TestLogging("_root:DEBUG,org.elasticsearch.index.shard:TRACE,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.index.seqno:TRACE,org.elasticsearch.indices.recovery:TRACE") public class RecoveryWhileUnderLoadIT extends ESIntegTestCase { private final Logger logger = Loggers.getLogger(RecoveryWhileUnderLoadIT.class); From 4471dc22e1ed13c5141324acc08e4af841fd60c0 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 15 Jun 2017 11:47:49 -0400 Subject: [PATCH 27/54] Fix test --- .../indices/recovery/RecoverySourceHandlerTests.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 3e483bdf77a4f..f059f5b57b110 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -35,6 +35,7 @@ import org.apache.lucene.util.IOUtils; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.bytes.BytesArray; @@ -75,9 +76,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -86,7 +85,6 @@ import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -453,6 +451,12 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int return null; }).when(shard).relocated(any(String.class), any(Runnable.class)); when(shard.acquireIndexCommit(anyBoolean())).thenReturn(mock(Engine.IndexCommitRef.class)); + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + final ActionListener listener = (ActionListener)invocationOnMock.getArguments()[0]; + listener.onResponse(() -> {}); + return null; + }).when(shard).acquirePrimaryOperationPermit(any(ActionListener.class), any(String.class)); final Supplier currentClusterStateVersionSupplier = () -> { assertFalse(ensureClusterStateVersionCalled.get()); From 56f3c175d06aa3444a9806f8ead3bf0cab54816f Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 15 Jun 2017 12:38:44 -0400 Subject: [PATCH 28/54] Unneeded public --- .../indices/recovery/RecoveryHandoffPrimaryContextRequest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java index ac6c9d7b3416a..6646f6cea5d41 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryHandoffPrimaryContextRequest.java @@ -39,8 +39,7 @@ class RecoveryHandoffPrimaryContextRequest extends TransportRequest { /** * Initialize an empty request (used to serialize into when reading from a stream). */ - @SuppressWarnings("WeakerAccess") - public RecoveryHandoffPrimaryContextRequest() { + RecoveryHandoffPrimaryContextRequest() { } /** From f97fd920355a7a6c60a9e3efbe6297bcfd7f2a99 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 15 Jun 2017 13:41:05 -0400 Subject: [PATCH 29/54] Imports --- .../index/replication/IndexLevelReplicationTests.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index 198b02e17c36e..c9de12e2778f8 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -18,18 +18,15 @@ */ package org.elasticsearch.index.replication; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexableField; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; @@ -45,7 +42,6 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.recovery.RecoveryTarget; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.hamcrest.Matcher; import java.io.IOException; From 29ca82ac5a569e1c252d82f95925d03e72cf1c1c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 15 Jun 2017 15:32:33 -0400 Subject: [PATCH 30/54] Promote on our own --- .../index/seqno/GlobalCheckpointTracker.java | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index a9ded1c55e6a7..d01a535773c6f 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -336,14 +336,9 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri updateLocalCheckpoint(cursor.key, cursor.value); assert cursor.value >= globalCheckpoint : "local checkpoint [" + cursor.value + "] violates being at least the global checkpoint [" + globalCheckpoint + "]"; - try { - markAllocationIdAsInSync(cursor.key, cursor.value); - } catch (final InterruptedException e) { - /* - * Since the local checkpoint already exceeds the global checkpoint here, we never blocking waiting for advancement. This - * means that we can never be interrupted. If we are bail, something is catastrophically wrong. - */ - throw new AssertionError(e); + if (trackingLocalCheckpoints.containsKey(cursor.key)) { + final long current = trackingLocalCheckpoints.remove(cursor.key); + inSyncLocalCheckpoints.put(cursor.key, current); } } From d7a8021d524e79f6cea785c0b92f86dc03453bab Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 16 Jun 2017 11:19:29 -0400 Subject: [PATCH 31/54] Add tests --- .../common/collect/LongTuple.java | 66 ++++++ .../index/seqno/GlobalCheckpointTracker.java | 45 ++-- .../seqno/GlobalCheckpointTrackerTests.java | 199 ++++++++++++++++-- 3 files changed, 286 insertions(+), 24 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/common/collect/LongTuple.java diff --git a/core/src/main/java/org/elasticsearch/common/collect/LongTuple.java b/core/src/main/java/org/elasticsearch/common/collect/LongTuple.java new file mode 100644 index 0000000000000..1b580d3485583 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/common/collect/LongTuple.java @@ -0,0 +1,66 @@ +/* + * 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.common.collect; + +public class LongTuple { + + public static LongTuple tuple(final T v1, final long v2) { + return new LongTuple<>(v1, v2); + } + + private final T v1; + private final long v2; + + private LongTuple(final T v1, final long v2) { + this.v1 = v1; + this.v2 = v2; + } + + public T v1() { + return v1; + } + + public long v2() { + return v2; + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + LongTuple tuple = (LongTuple) o; + + return (v1 == null ? tuple.v1 == null : v1.equals(tuple.v1)) && (v2 == tuple.v2); + } + + @Override + public int hashCode() { + int result = v1 != null ? v1.hashCode() : 0; + result = 31 * result + Long.hashCode(v2); + return result; + } + + @Override + public String toString() { + return "Tuple [v1=" + v1 + ", v2=" + v2 + "]"; + } + +} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index d01a535773c6f..2a472ed77d25d 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -23,15 +23,22 @@ import com.carrotsearch.hppc.ObjectLongMap; import com.carrotsearch.hppc.cursors.ObjectLongCursor; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.collect.LongTuple; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import java.util.Arrays; +import java.util.Comparator; import java.util.HashSet; +import java.util.List; import java.util.Locale; import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** * This class is responsible of tracking the global checkpoint. The global checkpoint is the highest sequence number for which all lower (or @@ -44,7 +51,7 @@ */ public class GlobalCheckpointTracker extends AbstractIndexShardComponent { - private long clusterStateVersion; + long appliedClusterStateVersion; /* * This map holds the last known local checkpoint for every active shard and initializing shard copies that has been brought up to speed @@ -220,11 +227,11 @@ synchronized void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { */ public synchronized void updateAllocationIdsFromMaster( final long applyingClusterStateVersion, final Set activeAllocationIds, final Set initializingAllocationIds) { - if (applyingClusterStateVersion < clusterStateVersion) { + if (applyingClusterStateVersion < appliedClusterStateVersion) { return; } - clusterStateVersion = applyingClusterStateVersion; + appliedClusterStateVersion = applyingClusterStateVersion; // remove shards whose allocation ID no longer exists inSyncLocalCheckpoints.removeAll(a -> !activeAllocationIds.contains(a) && !initializingAllocationIds.contains(a)); @@ -267,7 +274,7 @@ public synchronized void updateAllocationIdsFromMaster( synchronized PrimaryContext primaryContext() { final ObjectLongMap inSyncLocalCheckpoints = new ObjectLongHashMap<>(this.inSyncLocalCheckpoints); final ObjectLongMap trackingLocalCheckpoints = new ObjectLongHashMap<>(this.trackingLocalCheckpoints); - return new PrimaryContext(clusterStateVersion, inSyncLocalCheckpoints, trackingLocalCheckpoints); + return new PrimaryContext(appliedClusterStateVersion, inSyncLocalCheckpoints, trackingLocalCheckpoints); } /** @@ -324,7 +331,7 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri * In both cases, no calls to update the local checkpoint for such shards will be made. This case is safe too. */ - if (primaryContext.clusterStateVersion() > clusterStateVersion) { + if (primaryContext.clusterStateVersion() > appliedClusterStateVersion) { final Set activeAllocationIds = new HashSet<>(Arrays.asList(primaryContext.inSyncLocalCheckpoints().keys().toArray(String.class))); final Set initializingAllocationIds = @@ -332,13 +339,27 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri updateAllocationIdsFromMaster(primaryContext.clusterStateVersion(), activeAllocationIds, initializingAllocationIds); } - for (final ObjectLongCursor cursor : primaryContext.inSyncLocalCheckpoints()) { - updateLocalCheckpoint(cursor.key, cursor.value); - assert cursor.value >= globalCheckpoint - : "local checkpoint [" + cursor.value + "] violates being at least the global checkpoint [" + globalCheckpoint + "]"; - if (trackingLocalCheckpoints.containsKey(cursor.key)) { - final long current = trackingLocalCheckpoints.remove(cursor.key); - inSyncLocalCheckpoints.put(cursor.key, current); + /* + * As we are updating the local checkpoints for the in-sync allocation IDs, the global checkpoint will advance in place; this means + * that we have to sort the incoming local checkpoints from smallest to largest lest we violate that the global checkpoint does not + * regress. + */ + final List> inSync = + StreamSupport + .stream(primaryContext.inSyncLocalCheckpoints().spliterator(), false) + .map(e -> LongTuple.tuple(e.key, e.value)) + .collect(Collectors.toList()); + + inSync.sort(Comparator.comparingLong(LongTuple::v2)); + + for (final LongTuple cursor : inSync) { + assert cursor.v2() >= globalCheckpoint + : "local checkpoint [" + cursor.v2() + "] violates being at least the global checkpoint [" + globalCheckpoint + "]"; + updateLocalCheckpoint(cursor.v1(), cursor.v2()); + if (trackingLocalCheckpoints.containsKey(cursor.v1())) { + final long current = trackingLocalCheckpoints.remove(cursor.v1()); + inSyncLocalCheckpoints.put(cursor.v1(), current); + updateGlobalCheckpointOnPrimary(); } } diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java index 5b21aa809c16b..a3ad3448fd6e7 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java @@ -19,9 +19,13 @@ package org.elasticsearch.index.seqno; +import com.carrotsearch.hppc.ObjectLongHashMap; +import com.carrotsearch.hppc.ObjectLongMap; import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; @@ -42,8 +46,10 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; +import java.util.stream.StreamSupport; import static org.elasticsearch.index.seqno.SequenceNumbersService.UNASSIGNED_SEQ_NO; +import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.not; @@ -335,19 +341,11 @@ public void testWaitForAllocationIdToBeInSyncCanBeInterrupted() throws BrokenBar public void testUpdateAllocationIdsFromMaster() throws Exception { final long initialClusterStateVersion = randomNonNegativeLong(); final int numberOfActiveAllocationsIds = randomIntBetween(2, 16); - final Set activeAllocationIds = - IntStream.range(0, numberOfActiveAllocationsIds).mapToObj(i -> randomAlphaOfLength(16)).collect(Collectors.toSet()); final int numberOfInitializingIds = randomIntBetween(2, 16); - final Set initializingIds = - IntStream.range(0, numberOfInitializingIds).mapToObj(i -> { - do { - final String initializingId = randomAlphaOfLength(16); - // ensure we do not duplicate an allocation ID in active and initializing sets - if (!activeAllocationIds.contains(initializingId)) { - return initializingId; - } - } while (true); - }).collect(Collectors.toSet()); + final Tuple, Set> activeAndInitializingAllocationIds = + randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds); + final Set activeAllocationIds = activeAndInitializingAllocationIds.v1(); + final Set initializingIds = activeAndInitializingAllocationIds.v2(); tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, activeAllocationIds, initializingIds); // first we assert that the in-sync and tracking sets are set up correctly @@ -521,7 +519,184 @@ public void testRaceUpdatingGlobalCheckpoint() throws InterruptedException, Brok markingThread.join(); assertThat(tracker.getGlobalCheckpoint(), equalTo((long) nextActiveLocalCheckpoint)); + } + + public void testPrimaryContextOlderThanAppliedClusterState() { + final long initialClusterStateVersion = randomIntBetween(0, Integer.MAX_VALUE - 1) + 1; + final int numberOfActiveAllocationsIds = randomIntBetween(0, 8); + final int numberOfInitializingIds = randomIntBetween(0, 8); + final Tuple, Set> activeAndInitializingAllocationIds = + randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds); + final Set activeAllocationIds = activeAndInitializingAllocationIds.v1(); + final Set initializingAllocationIds = activeAndInitializingAllocationIds.v2(); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, activeAllocationIds, initializingAllocationIds); + + /* + * We are going to establish a primary context from a cluster state version older than the applied cluster state version on the + * tracker. Because of recovery barriers established during relocation handoff, we know that the set of active allocation IDs in the + * newer cluster state is a superset of the allocation IDs in the applied cluster state with the caveat that an existing + * initializing allocation ID could have moved to an in-sync allocation ID within the tracker due to recovery finalization, and the + * set of initializing allocation IDs is otherwise arbitrary. + */ + final int numberOfAdditionalInitializingAllocationIds = randomIntBetween(0, 8); + final Set initializedAllocationIds = new HashSet<>(randomSubsetOf(initializingAllocationIds)); + final Set newInitializingAllocationIds = + randomAllocationIdsExcludingExistingIds( + Sets.union(activeAllocationIds, initializingAllocationIds), numberOfAdditionalInitializingAllocationIds); + final Set contextInitializingIds = Sets.union( + new HashSet<>(randomSubsetOf(Sets.difference(initializingAllocationIds, initializedAllocationIds))), + newInitializingAllocationIds); + + final int numberOfAdditionalActiveAllocationIds = randomIntBetween(0, 8); + final Set contextActiveAllocationIds = Sets.union( + Sets.union( + activeAllocationIds, + randomAllocationIdsExcludingExistingIds(activeAllocationIds, numberOfAdditionalActiveAllocationIds)), + initializedAllocationIds); + + final ObjectLongMap activeAllocationIdsLocalCheckpoints = new ObjectLongHashMap<>(); + for (final String allocationId : contextActiveAllocationIds) { + activeAllocationIdsLocalCheckpoints.put(allocationId, randomNonNegativeLong()); + } + final ObjectLongMap initializingAllocationIdsLocalCheckpoints = new ObjectLongHashMap<>(); + for (final String allocationId : contextInitializingIds) { + initializingAllocationIdsLocalCheckpoints.put(allocationId, randomNonNegativeLong()); + } + + final PrimaryContext primaryContext = new PrimaryContext( + initialClusterStateVersion - randomIntBetween(0, Math.toIntExact(initialClusterStateVersion) - 1), + activeAllocationIdsLocalCheckpoints, + initializingAllocationIdsLocalCheckpoints); + + tracker.updateAllocationIdsFromPrimaryContext(primaryContext); + + // the primary context carries an older cluster state version + assertThat(tracker.appliedClusterStateVersion, equalTo(initialClusterStateVersion)); + + // only existing active allocation IDs and initializing allocation IDs that moved to initialized should be in-sync + assertThat( + Sets.union(activeAllocationIds, initializedAllocationIds), + equalTo( + StreamSupport + .stream(tracker.inSyncLocalCheckpoints.keys().spliterator(), false) + .map(e -> e.value) + .collect(Collectors.toSet()))); + + // the local checkpoints known to the tracker for in-sync shards should match what is known in the primary context + for (final String allocationId : Sets.union(activeAllocationIds, initializedAllocationIds)) { + assertThat( + tracker.inSyncLocalCheckpoints.get(allocationId), equalTo(primaryContext.inSyncLocalCheckpoints().get(allocationId))); + } + + // only existing initializing allocation IDs that did not moved to initialized should be tracked + assertThat( + Sets.difference(initializingAllocationIds, initializedAllocationIds), + equalTo( + StreamSupport + .stream(tracker.trackingLocalCheckpoints.keys().spliterator(), false) + .map(e -> e.value) + .collect(Collectors.toSet()))); + + // the local checkpoints known to the tracker for initializing shards should match what is known in the primary context + for (final String allocationId : Sets.difference(initializingAllocationIds, initializedAllocationIds)) { + if (primaryContext.trackingLocalCheckpoints().containsKey(allocationId)) { + assertThat( + tracker.trackingLocalCheckpoints.get(allocationId), + equalTo(primaryContext.trackingLocalCheckpoints().get(allocationId))); + } else { + assertThat(tracker.trackingLocalCheckpoints.get(allocationId), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO)); + } + } + + // the global checkpoint can only be computed from active allocation IDs and initializing allocation IDs that moved to initializing + final long globalCheckpoint = + StreamSupport + .stream(activeAllocationIdsLocalCheckpoints.spliterator(), false) + .filter(e -> tracker.inSyncLocalCheckpoints.containsKey(e.key) || initializedAllocationIds.contains(e.key)) + .mapToLong(e -> e.value) + .min() + .orElse(SequenceNumbersService.UNASSIGNED_SEQ_NO); + assertThat(tracker.getGlobalCheckpoint(), equalTo(globalCheckpoint)); + } + + public void testPrimaryContextNewerThanAppliedClusterState() { + final long initialClusterStateVersion = randomIntBetween(0, Integer.MAX_VALUE); + final int numberOfActiveAllocationsIds = randomIntBetween(0, 8); + final int numberOfInitializingIds = randomIntBetween(0, 8); + final Tuple, Set> activeAndInitializingAllocationIds = + randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds); + final Set activeAllocationIds = activeAndInitializingAllocationIds.v1(); + final Set initializingAllocationIds = activeAndInitializingAllocationIds.v2(); + tracker.updateAllocationIdsFromMaster(initialClusterStateVersion, activeAllocationIds, initializingAllocationIds); + + /* + * We are going to establish a primary context from a cluster state version older than the applied cluster state version on the + * tracker. Because of recovery barriers established during relocation handoff, we know that the set of active allocation IDs in the + * newer cluster state is a subset of the allocation IDs in the applied cluster state with the caveat that an existing initializing + * allocation ID could have moved to an in-sync allocation ID within the tracker due to recovery finalization, and the set of + * initializing allocation IDs is otherwise arbitrary. + */ + final int numberOfNewInitializingAllocationIds = randomIntBetween(0, 8); + final Set initializedAllocationIds = new HashSet<>(randomSubsetOf(initializingAllocationIds)); + final Set newInitializingAllocationIds = + randomAllocationIdsExcludingExistingIds( + Sets.union(activeAllocationIds, initializingAllocationIds), numberOfNewInitializingAllocationIds); + + final ObjectLongMap activeAllocationIdsLocalCheckpoints = new ObjectLongHashMap<>(); + for (final String allocationId : Sets.union(new HashSet<>(randomSubsetOf(activeAllocationIds)), initializedAllocationIds)) { + activeAllocationIdsLocalCheckpoints.put(allocationId, randomNonNegativeLong()); + } + final ObjectLongMap initializingIdsLocalCheckpoints = new ObjectLongHashMap<>(); + final Set contextInitializingAllocationIds = Sets.union( + new HashSet<>(randomSubsetOf(Sets.difference(initializingAllocationIds, initializedAllocationIds))), + newInitializingAllocationIds); + for (final String allocationId : contextInitializingAllocationIds) { + initializingIdsLocalCheckpoints.put(allocationId, randomNonNegativeLong()); + } + + final PrimaryContext primaryContext = + new PrimaryContext( + initialClusterStateVersion + randomIntBetween(0, Integer.MAX_VALUE) + 1, + activeAllocationIdsLocalCheckpoints, + initializingIdsLocalCheckpoints); + + tracker.updateAllocationIdsFromPrimaryContext(primaryContext); + + final PrimaryContext trackerPrimaryContext = tracker.primaryContext(); + + final long globalCheckpoint = + StreamSupport + .stream(activeAllocationIdsLocalCheckpoints.values().spliterator(), false) + .mapToLong(e -> e.value) + .min() + .orElse(SequenceNumbersService.UNASSIGNED_SEQ_NO); + + // the primary context contains knowledge of the state of the entire universe + assertThat(primaryContext.clusterStateVersion(), equalTo(trackerPrimaryContext.clusterStateVersion())); + assertThat(primaryContext.inSyncLocalCheckpoints(), equalTo(trackerPrimaryContext.inSyncLocalCheckpoints())); + assertThat(primaryContext.trackingLocalCheckpoints(), equalTo(trackerPrimaryContext.trackingLocalCheckpoints())); + assertThat(tracker.getGlobalCheckpoint(), equalTo(globalCheckpoint)); + } + + private Tuple, Set> randomActiveAndInitializingAllocationIds( + final int numberOfActiveAllocationsIds, + final int numberOfInitializingIds) { + final Set activeAllocationIds = + IntStream.range(0, numberOfActiveAllocationsIds).mapToObj(i -> randomAlphaOfLength(16) + i).collect(Collectors.toSet()); + final Set initializingIds = randomAllocationIdsExcludingExistingIds(activeAllocationIds, numberOfInitializingIds); + return Tuple.tuple(activeAllocationIds, initializingIds); + } + private Set randomAllocationIdsExcludingExistingIds(final Set existingAllocationIds, final int numberOfAllocationIds) { + return IntStream.range(0, numberOfAllocationIds).mapToObj(i -> { + do { + final String newAllocationId = randomAlphaOfLength(16); + // ensure we do not duplicate an allocation ID + if (!existingAllocationIds.contains(newAllocationId)) { + return newAllocationId + i; + } + } while (true); + }).collect(Collectors.toSet()); } private void markAllocationIdAsInSyncQuietly( From 9683757dc16bfc40721dc674fea591246bba34a9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 16 Jun 2017 11:21:01 -0400 Subject: [PATCH 32/54] Import --- .../elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java index a3ad3448fd6e7..6c99cf4198a9a 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java @@ -49,7 +49,6 @@ import java.util.stream.StreamSupport; import static org.elasticsearch.index.seqno.SequenceNumbersService.UNASSIGNED_SEQ_NO; -import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.not; From 5422f6e6bc4063268ec656194ccde56f603b0098 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 16 Jun 2017 11:32:16 -0400 Subject: [PATCH 33/54] Newline --- .../main/java/org/elasticsearch/common/collect/LongTuple.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/common/collect/LongTuple.java b/core/src/main/java/org/elasticsearch/common/collect/LongTuple.java index 1b580d3485583..fab8850d162b9 100644 --- a/core/src/main/java/org/elasticsearch/common/collect/LongTuple.java +++ b/core/src/main/java/org/elasticsearch/common/collect/LongTuple.java @@ -63,4 +63,4 @@ public String toString() { return "Tuple [v1=" + v1 + ", v2=" + v2 + "]"; } -} \ No newline at end of file +} From d04f14a38b6377f328f4fd95040431c652442ea5 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 16 Jun 2017 11:44:54 -0400 Subject: [PATCH 34/54] Update comment --- .../index/seqno/GlobalCheckpointTracker.java | 33 ++++++++++--------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 2a472ed77d25d..e2d7d4cdb214f 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -300,15 +300,18 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri * shards the target receives from the master will be equal to the knowledge of the in-sync and initializing shards the target * receives from the relocation source via the primary context. * - * In the case when version(context) < version(target) or version(target) < version(context), we first consider shards that could be - * contained in the primary context but not contained in the cluster state applied on the target. + * Let us now consider the case that version(context) < version(target). In this case, the active allocation IDs in the primary + * context can be a superset of the active allocation IDs contained in the applied cluster state. This is because no new shards can + * have been started as marking a shard as in-sync is blocked during relocation handoff. Note however that the relocation target + * itself will have been marked in-sync during recovery and therefore is an active allocation ID from the perspective of the primary + * context. * - * Suppose there is such a shard and that it is an in-sync shard. However, marking a shard as in-sync requires an operation permit - * on the primary shard. Such a permit can not be obtained after the relocation handoff has started as the relocation handoff blocks - * all operations. Therefore, there can not be such a shard that is marked in-sync. + * Finally, we consider the case that version(target) < version(context). In this case, the active allocation IDs in the primary + * context can be a subset of the active allocation IDs contained the applied cluster state. This is again because no new shards can + * have been started. Moreover, existing active allocation IDs could have been removed from the cluster state. * - * Now consider the case of an initializing shard that is contained in the primary context but not contained in the cluster state - * applied on the target. + * In each of these latter two cases, consider initializing shards that are contained in the primary context but not contained in + * the cluster state applied on the target. * * If version(context) < version(target) it means that the shard has been removed by a later cluster state update that is already * applied on the target and we only need to ensure that we do not add it to the tracking map on the target. The call to @@ -319,16 +322,16 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri * Therefore, such a shard can never initialize from the relocation source and will have to await the handoff completing. As such, * these shards are not problematic. * - * Now we consider shards that are contained in the cluster state applied on the target but not contained in the primary context. + * Lastly, again in these two cases, what about initializing shards that are contained in cluster state applied on the target but + * not contained in the cluster state applied on the target. * - * If version(context) < version(target) it means that the target has learned of an initializing shard that the source is not aware - * of. As explained above, this initialization can only succeed after the relocation is complete, and only with the target as the - * source of the recovery. + * If version(context) < version(target) it means that a shard has started initializing by a later cluster state that is applied on + * the target but not yet known to what would be the relocation source. As recoveries are delayed at this time, these shards can not + * cause a problem and we do not mutate remove these shards from the tracking map, so we are safe here. * - * Otherwise, if version(target) < version(context) it only means that the global checkpoint on the target will be held back until a - * later cluster state update arrives because the target will not learn of the removal until later. - * - * In both cases, no calls to update the local checkpoint for such shards will be made. This case is safe too. + * If version(target) < version(context) it means that a shard has started initializing but was removed by a later cluster state. In + * this case, as the cluster state version on the primary context exceeds the applied cluster state version, we replace the tracking + * map and are safe here too. */ if (primaryContext.clusterStateVersion() > appliedClusterStateVersion) { From 52af334aeef2264060c334463be7a63b94f78c6a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 13:47:39 -0400 Subject: [PATCH 35/54] Serialization --- .../java/org/elasticsearch/index/shard/PrimaryContext.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java index cacad083e725f..8a067d3718159 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java +++ b/core/src/main/java/org/elasticsearch/index/shard/PrimaryContext.java @@ -68,7 +68,7 @@ public PrimaryContext(final StreamInput in) throws IOException { } private static ObjectLongMap readMap(final StreamInput in) throws IOException { - final int length = in.readInt(); + final int length = in.readVInt(); final ObjectLongMap map = new ObjectLongHashMap<>(length); for (int i = 0; i < length; i++) { final String key = in.readString(); @@ -86,7 +86,7 @@ public void writeTo(final StreamOutput out) throws IOException { } private static void writeMap(final StreamOutput out, final ObjectLongMap map) throws IOException { - out.writeInt(map.size()); + out.writeVInt(map.size()); for (ObjectLongCursor cursor : map) { out.writeString(cursor.key); out.writeZLong(cursor.value); From e1de29691627d380eb4d492014d69d388cb7b8ea Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 13:51:31 -0400 Subject: [PATCH 36/54] Assertion message --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 37daa8879bdc0..c41437f34ce50 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1704,7 +1704,8 @@ public void updateAllocationIdsFromMaster( */ public void updateAllocationIdsFromPrimaryContext(final PrimaryContext primaryContext) { verifyPrimary(); - assert shardRouting.isRelocationTarget(); + assert shardRouting.isRelocationTarget() + : "only relocation target can update allocation IDs from primary context but was: " + shardRouting; final Engine engine = getEngineOrNull(); if (engine != null) { engine.seqNoService().updateAllocationIdsFromPrimaryContext(primaryContext); From 643fa8a8e1a2be338e585a9697f198302553e61a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 13:51:55 -0400 Subject: [PATCH 37/54] Update stale comment --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 c41437f34ce50..17bcb2399f4c0 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1619,8 +1619,7 @@ public void waitForOpsToComplete(final long seqNo) throws InterruptedException { /** * Marks the shard with the provided allocation ID as in-sync with the primary shard. See * {@link org.elasticsearch.index.seqno.GlobalCheckpointTracker#markAllocationIdAsInSync(String, long)} - * for additional details. Because this operation could be completed asynchronously on another thread, the caller must provide a latch; - * this latch will be counted down after the operation completes. + * for additional details. * * @param allocationId the allocation ID of the shard to mark as in-sync * @param localCheckpoint the current local checkpoint on the shard From 4e1fa9c396176f4736e4851d7340eb69d2e8a72b Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 13:52:14 -0400 Subject: [PATCH 38/54] Remove newline --- core/src/main/java/org/elasticsearch/index/shard/IndexShard.java | 1 - 1 file changed, 1 deletion(-) 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 17bcb2399f4c0..a554834ef3c5d 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -525,7 +525,6 @@ public IndexShardState markAsRecovering(String reason, RecoveryState recoverySta * @throws InterruptedException if blocking operations is interrupted */ public void relocated(final String reason, final Runnable onBlocked) throws IllegalIndexShardStateException, InterruptedException { - assert shardRouting.primary() : "only primaries can be marked as relocated: " + shardRouting; try { indexShardOperationPermits.blockOperations(30, TimeUnit.MINUTES, () -> { From a275167ecaf1f54f3baae29c15682533a818f9fc Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 13:52:38 -0400 Subject: [PATCH 39/54] Less verbose --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 a554834ef3c5d..91c78a8f598f9 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1702,8 +1702,7 @@ public void updateAllocationIdsFromMaster( */ public void updateAllocationIdsFromPrimaryContext(final PrimaryContext primaryContext) { verifyPrimary(); - assert shardRouting.isRelocationTarget() - : "only relocation target can update allocation IDs from primary context but was: " + shardRouting; + assert shardRouting.isRelocationTarget() : "only relocation target can update allocation IDs from primary context: " + shardRouting; final Engine engine = getEngineOrNull(); if (engine != null) { engine.seqNoService().updateAllocationIdsFromPrimaryContext(primaryContext); From 00e4083ffc2bd0dcfef69be7dbf94d278ba2b251 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 13:53:15 -0400 Subject: [PATCH 40/54] Remove redundant assertion --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 91c78a8f598f9..1087f7af06ef0 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -571,8 +571,7 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille */ public PrimaryContext primaryContext() { verifyPrimary(); - assert shardRouting.relocating() : "primary context can only be obtained from a relocating primary but was " + shardRouting; - assert !shardRouting.isRelocationTarget() : "primary context can only be obtained from relocation source but was " + shardRouting; + assert shardRouting.relocating() : "primary context can only be obtained from a relocating primary: " + shardRouting; return getEngine().seqNoService().primaryContext(); } From 5f168848d8502dcb0d4e864f0c72a2fea6751eea Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 15:29:09 -0400 Subject: [PATCH 41/54] Tracking -> in-sync --- .../index/seqno/GlobalCheckpointTracker.java | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index e2d7d4cdb214f..0060be5d68230 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -360,8 +360,7 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri : "local checkpoint [" + cursor.v2() + "] violates being at least the global checkpoint [" + globalCheckpoint + "]"; updateLocalCheckpoint(cursor.v1(), cursor.v2()); if (trackingLocalCheckpoints.containsKey(cursor.v1())) { - final long current = trackingLocalCheckpoints.remove(cursor.v1()); - inSyncLocalCheckpoints.put(cursor.v1(), current); + moveAllocationIdFromTrackingToInSync(cursor.v1(), "relocation"); updateGlobalCheckpointOnPrimary(); } } @@ -418,15 +417,13 @@ private synchronized void waitForAllocationIdToBeInSync(final String allocationI */ final long current = trackingLocalCheckpoints.getOrDefault(allocationId, Long.MIN_VALUE); if (current >= globalCheckpoint) { - logger.trace("marked [{}] as in-sync with local checkpoint [{}]", allocationId, current); - trackingLocalCheckpoints.remove(allocationId); /* * This is prematurely adding the allocation ID to the in-sync map as at this point recovery is not yet finished and could * still abort. At this point we will end up with a shard in the in-sync map holding back the global checkpoint because the * shard never recovered and we would have to wait until either the recovery retries and completes successfully, or the * master fails the shard and issues a cluster state update that removes the shard from the set of active allocation IDs. */ - inSyncLocalCheckpoints.put(allocationId, current); + moveAllocationIdFromTrackingToInSync(allocationId, "recovery"); break; } else { waitForLocalCheckpointToAdvance(); @@ -434,6 +431,21 @@ private synchronized void waitForAllocationIdToBeInSync(final String allocationI } } + /** + * Moves a tracking allocation ID to be in-sync. This can occur when a shard is recovering from the primary and its local checkpoint has + * advanced past the global checkpoint, or during relocation hand-off when the relocation target learns of an in-sync shard from the + * relocation source. + * + * @param allocationId the allocation ID to move + * @param reason the reason for the transition + */ + private synchronized void moveAllocationIdFromTrackingToInSync(final String allocationId, final String reason) { + assert trackingLocalCheckpoints.containsKey(allocationId); + final long current = trackingLocalCheckpoints.remove(allocationId); + inSyncLocalCheckpoints.put(allocationId, current); + logger.trace("marked [{}] as in-sync with local checkpoint [{}] due to [{}]", allocationId, current, reason); + } + /** * Wait for the local checkpoint to advance to the global checkpoint. * From 3a53fd36b68ce20731bab08e7d966fd0d2ba64dc Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 16:04:58 -0400 Subject: [PATCH 42/54] Assertions --- .../index/seqno/GlobalCheckpointTracker.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 0060be5d68230..89a8c2c551ff4 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -30,6 +30,7 @@ import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; +import java.security.Security; import java.util.Arrays; import java.util.Comparator; import java.util.HashSet; @@ -334,6 +335,14 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri * map and are safe here too. */ + assert StreamSupport + .stream(inSyncLocalCheckpoints.spliterator(), false) + .allMatch(e -> e.value == SequenceNumbersService.UNASSIGNED_SEQ_NO) : inSyncLocalCheckpoints; + assert StreamSupport + .stream(trackingLocalCheckpoints.spliterator(), false) + .allMatch(e -> e.value == SequenceNumbersService.UNASSIGNED_SEQ_NO) : trackingLocalCheckpoints; + assert pendingInSync.isEmpty() : pendingInSync; + if (primaryContext.clusterStateVersion() > appliedClusterStateVersion) { final Set activeAllocationIds = new HashSet<>(Arrays.asList(primaryContext.inSyncLocalCheckpoints().keys().toArray(String.class))); From ccde79826cf09277235d6ef78ce7870beb6273d3 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 21:45:59 -0400 Subject: [PATCH 43/54] Just say no Friends do not let friends block the cluster state update thread on network operations. --- .../elasticsearch/index/shard/IndexShard.java | 49 +++++++++++-------- 1 file changed, 28 insertions(+), 21 deletions(-) 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 1087f7af06ef0..155aa863edcb8 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -530,28 +530,15 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille indexShardOperationPermits.blockOperations(30, TimeUnit.MINUTES, () -> { // no shard operation permits are being held here, move state from started to relocated assert indexShardOperationPermits.getActiveOperationsCount() == 0 : - "in-flight operations in progress while moving shard state to relocated"; + "in-flight operations in progress while moving shard state to relocated"; + /* + * We should not invoke the runnable under the mutex as the expected implementation is to handoff the primary context via a + * network operation. Doing this under the mutex can implicitly block the cluster state update thread on network operations. + */ + verifyRelocating(); + onBlocked.run(); synchronized (mutex) { - if (state != IndexShardState.STARTED) { - throw new IndexShardNotStartedException(shardId, state); - } - /* - * If the master cancelled recovery, the target will be removed and the recovery will be cancelled. However, it is still - * possible that we concurrently end up here and therefore have to protect that we do not mark the shard as relocated - * when its shard routing says otherwise. - */ - if (shardRouting.relocating() == false) { - throw new IllegalIndexShardStateException(shardId, IndexShardState.STARTED, - ": shard is no longer relocating " + shardRouting); - } - - if (primaryReplicaResyncInProgress.get()) { - throw new IllegalIndexShardStateException(shardId, IndexShardState.STARTED, - ": primary relocation is forbidden while primary-replica resync is in progress " + shardRouting); - } - - onBlocked.run(); - + verifyRelocating(); changeState(IndexShardState.RELOCATED, reason); } }); @@ -564,6 +551,26 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille } } + private void verifyRelocating() { + if (state != IndexShardState.STARTED) { + throw new IndexShardNotStartedException(shardId, state); + } + /* + * If the master cancelled recovery, the target will be removed and the recovery will be cancelled. However, it is still possible + * that we concurrently end up here and therefore have to protect that we do not mark the shard as relocated when its shard routing + * says otherwise. + */ + if (shardRouting.relocating() == false) { + throw new IllegalIndexShardStateException(shardId, IndexShardState.STARTED, + ": shard is no longer relocating " + shardRouting); + } + + if (primaryReplicaResyncInProgress.get()) { + throw new IllegalIndexShardStateException(shardId, IndexShardState.STARTED, + ": primary relocation is forbidden while primary-replica resync is in progress " + shardRouting); + } + } + /** * Obtain the primary context for the shard. The shard must be serving as the relocation source for a primary shard. * From e6bbe8badf1d235f1d084d0648e1953491fd9654 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 22 Jun 2017 21:48:18 -0400 Subject: [PATCH 44/54] Extra newline --- core/src/main/java/org/elasticsearch/index/shard/IndexShard.java | 1 + 1 file changed, 1 insertion(+) 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 155aa863edcb8..45e9051c492bd 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -560,6 +560,7 @@ private void verifyRelocating() { * that we concurrently end up here and therefore have to protect that we do not mark the shard as relocated when its shard routing * says otherwise. */ + if (shardRouting.relocating() == false) { throw new IllegalIndexShardStateException(shardId, IndexShardState.STARTED, ": shard is no longer relocating " + shardRouting); From ff54eec619295eb1b03a05e87bc9d645aa457065 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 23 Jun 2017 08:07:54 -0400 Subject: [PATCH 45/54] Add allocation ID to assertion --- .../elasticsearch/index/seqno/GlobalCheckpointTracker.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 89a8c2c551ff4..c2e2c21567376 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -366,7 +366,9 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri for (final LongTuple cursor : inSync) { assert cursor.v2() >= globalCheckpoint - : "local checkpoint [" + cursor.v2() + "] violates being at least the global checkpoint [" + globalCheckpoint + "]"; + : "local checkpoint [" + cursor.v2() + "] " + + "for allocation ID [" + cursor.v1() + "] " + + "violates being at least the global checkpoint [" + globalCheckpoint + "]"; updateLocalCheckpoint(cursor.v1(), cursor.v2()); if (trackingLocalCheckpoints.containsKey(cursor.v1())) { moveAllocationIdFromTrackingToInSync(cursor.v1(), "relocation"); From 6ea61eff31d7ddc8627e42ffc10a1a4cdbd4e555 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 23 Jun 2017 08:10:05 -0400 Subject: [PATCH 46/54] Rename method --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 45e9051c492bd..ef7289809dc0d 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -535,10 +535,10 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille * We should not invoke the runnable under the mutex as the expected implementation is to handoff the primary context via a * network operation. Doing this under the mutex can implicitly block the cluster state update thread on network operations. */ - verifyRelocating(); + verifyRelocationState(); onBlocked.run(); synchronized (mutex) { - verifyRelocating(); + verifyRelocationState(); changeState(IndexShardState.RELOCATED, reason); } }); @@ -551,7 +551,7 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille } } - private void verifyRelocating() { + private void verifyRelocationState() { if (state != IndexShardState.STARTED) { throw new IndexShardNotStartedException(shardId, state); } From f6f6acb407ade63b874dda779f14a269c5d0e7df Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 23 Jun 2017 08:10:29 -0400 Subject: [PATCH 47/54] Another rename --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 ef7289809dc0d..ec04552033d5d 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -535,10 +535,10 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille * We should not invoke the runnable under the mutex as the expected implementation is to handoff the primary context via a * network operation. Doing this under the mutex can implicitly block the cluster state update thread on network operations. */ - verifyRelocationState(); + verifyRelocatingState(); onBlocked.run(); synchronized (mutex) { - verifyRelocationState(); + verifyRelocatingState(); changeState(IndexShardState.RELOCATED, reason); } }); @@ -551,7 +551,7 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille } } - private void verifyRelocationState() { + private void verifyRelocatingState() { if (state != IndexShardState.STARTED) { throw new IndexShardNotStartedException(shardId, state); } From de862cd07272b99bbbd5a1f288d8c4fe551ac2bd Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 23 Jun 2017 08:43:41 -0400 Subject: [PATCH 48/54] Introduce sealing --- .../index/seqno/GlobalCheckpointTracker.java | 38 ++++++++++++++++++- .../index/seqno/SequenceNumbersService.java | 6 ++- .../elasticsearch/index/shard/IndexShard.java | 34 +++++++++-------- .../recovery/RecoverySourceHandler.java | 5 +-- .../seqno/GlobalCheckpointTrackerTests.java | 31 ++++++++------- .../index/shard/IndexShardTests.java | 16 ++++---- .../recovery/RecoverySourceHandlerTests.java | 3 +- 7 files changed, 86 insertions(+), 47 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index c2e2c21567376..382db8ee9491b 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.collect.LongTuple; import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.PrimaryContext; @@ -38,6 +39,7 @@ import java.util.Locale; import java.util.Set; import java.util.TreeMap; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -80,6 +82,12 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent { */ private long globalCheckpoint; + /* + * During relocation handoff, the state of the global checkpoint tracker is sampled. After sampling, there should be no additional + * mutations to this tracker until the handoff has completed. + */ + private boolean sealed = false; + /** * Initialize the global checkpoint service. The specified global checkpoint should be set to the last known global checkpoint, or * {@link SequenceNumbersService#UNASSIGNED_SEQ_NO}. @@ -106,6 +114,9 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent { * @param localCheckpoint the local checkpoint for the shard */ public synchronized void updateLocalCheckpoint(final String allocationId, final long localCheckpoint) { + if (sealed) { + throw new IllegalStateException("global checkpoint tracker is sealed"); + } final boolean updated; if (updateLocalCheckpoint(allocationId, localCheckpoint, inSyncLocalCheckpoints, "in-sync")) { updated = true; @@ -228,6 +239,9 @@ synchronized void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { */ public synchronized void updateAllocationIdsFromMaster( final long applyingClusterStateVersion, final Set activeAllocationIds, final Set initializingAllocationIds) { + if (sealed) { + throw new IllegalStateException("global checkpoint tracker is sealed"); + } if (applyingClusterStateVersion < appliedClusterStateVersion) { return; } @@ -272,10 +286,24 @@ public synchronized void updateAllocationIdsFromMaster( * * @return the primary context */ - synchronized PrimaryContext primaryContext() { + synchronized Releasable primaryContext(final Consumer consumer) { + if (sealed) { + throw new IllegalStateException("global checkpoint tracker is sealed"); + } + sealed = true; final ObjectLongMap inSyncLocalCheckpoints = new ObjectLongHashMap<>(this.inSyncLocalCheckpoints); final ObjectLongMap trackingLocalCheckpoints = new ObjectLongHashMap<>(this.trackingLocalCheckpoints); - return new PrimaryContext(appliedClusterStateVersion, inSyncLocalCheckpoints, trackingLocalCheckpoints); + try { + consumer.accept(new PrimaryContext(appliedClusterStateVersion, inSyncLocalCheckpoints, trackingLocalCheckpoints)); + } catch (final Exception e) { + sealed = false; + throw e; + } + return () -> { + synchronized (this) { + sealed = false; + } + }; } /** @@ -284,6 +312,9 @@ synchronized PrimaryContext primaryContext() { * @param primaryContext the primary context */ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext primaryContext) { + if (sealed) { + throw new IllegalStateException("global checkpoint tracker is sealed"); + } /* * We are gathered here today to witness the relocation handoff transferring knowledge from the relocation source to the relocation * target. We need to consider the possibility that the version of the cluster state on the relocation source when the primary @@ -391,6 +422,9 @@ synchronized void updateAllocationIdsFromPrimaryContext(final PrimaryContext pri * @throws InterruptedException if the thread is interrupted waiting for the local checkpoint on the shard to advance */ public synchronized void markAllocationIdAsInSync(final String allocationId, final long localCheckpoint) throws InterruptedException { + if (sealed) { + throw new IllegalStateException("global checkpoint tracker is sealed"); + } if (!trackingLocalCheckpoints.containsKey(allocationId)) { /* * This can happen if the recovery target has been failed and the cluster state update from the master has triggered removing diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java index 4d174744a0e2f..d352b9aa13779 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -19,12 +19,14 @@ package org.elasticsearch.index.seqno; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import java.util.Set; +import java.util.function.Consumer; /** * Encapsulates the local and global checkpoints into a single service for use as a shard component. @@ -200,8 +202,8 @@ public boolean pendingInSync() { * * @return the primary context */ - public PrimaryContext primaryContext() { - return globalCheckpointTracker.primaryContext(); + public Releasable primaryContext(final Consumer consumer) { + return globalCheckpointTracker.primaryContext(consumer); } } 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 ec04552033d5d..0175fd56c3679 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -515,16 +515,17 @@ public IndexShardState markAsRecovering(String reason, RecoveryState recoverySta private final AtomicBoolean primaryReplicaResyncInProgress = new AtomicBoolean(); - /** - * Completes the relocation. Operations are blocked and current operations are drained before changing state to relocated. The provided - * {@link Runnable} is executed after all operations are successfully blocked. - * - * @param reason the reason for the relocation - * @param onBlocked a {@link Runnable} that is executed after operations are blocked - * @throws IllegalIndexShardStateException if the shard is not relocating due to concurrent cancellation - * @throws InterruptedException if blocking operations is interrupted - */ - public void relocated(final String reason, final Runnable onBlocked) throws IllegalIndexShardStateException, InterruptedException { + /** + * Completes the relocation. Operations are blocked and current operations are drained before changing state to relocated. The provided + * {@link Runnable} is executed after all operations are successfully blocked. + * + * @param reason the reason for the relocation + * @param consumer a {@link Runnable} that is executed after operations are blocked + * @throws IllegalIndexShardStateException if the shard is not relocating due to concurrent cancellation + * @throws InterruptedException if blocking operations is interrupted + */ + public void relocated( + final String reason, final Consumer consumer) throws IllegalIndexShardStateException, InterruptedException { assert shardRouting.primary() : "only primaries can be marked as relocated: " + shardRouting; try { indexShardOperationPermits.blockOperations(30, TimeUnit.MINUTES, () -> { @@ -536,10 +537,11 @@ public void relocated(final String reason, final Runnable onBlocked) throws Ille * network operation. Doing this under the mutex can implicitly block the cluster state update thread on network operations. */ verifyRelocatingState(); - onBlocked.run(); - synchronized (mutex) { - verifyRelocatingState(); - changeState(IndexShardState.RELOCATED, reason); + try (Releasable ignored = primaryContext(consumer)) { + synchronized (mutex) { + verifyRelocatingState(); + changeState(IndexShardState.RELOCATED, reason); + } } }); } catch (TimeoutException e) { @@ -577,10 +579,10 @@ private void verifyRelocatingState() { * * @return the primary for the shard */ - public PrimaryContext primaryContext() { + public Releasable primaryContext(final Consumer consumer) { verifyPrimary(); assert shardRouting.relocating() : "primary context can only be obtained from a relocating primary: " + shardRouting; - return getEngine().seqNoService().primaryContext(); + return getEngine().seqNoService().primaryContext(consumer); } public IndexShardState state() { 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 b1933a6a77992..3097c8e668f58 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -484,10 +484,7 @@ public void finalizeRecovery(final long targetLocalCheckpoint) { cancellableThreads.execute(() -> recoveryTarget.ensureClusterStateVersion(currentClusterStateVersion)); logger.trace("performing relocation hand-off"); - cancellableThreads.execute( - () -> shard.relocated( - "to " + request.targetNode(), - () -> recoveryTarget.handoffPrimaryContext(shard.primaryContext()))); + cancellableThreads.execute(() -> shard.relocated("to " + request.targetNode(), recoveryTarget::handoffPrimaryContext)); } /* * if the recovery process fails after setting the shard state to RELOCATED, both relocation source and diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java index 6c99cf4198a9a..9b5da400f49e5 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java @@ -23,6 +23,7 @@ import com.carrotsearch.hppc.ObjectLongMap; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.PrimaryContext; @@ -42,6 +43,7 @@ import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -661,20 +663,21 @@ public void testPrimaryContextNewerThanAppliedClusterState() { tracker.updateAllocationIdsFromPrimaryContext(primaryContext); - final PrimaryContext trackerPrimaryContext = tracker.primaryContext(); - - final long globalCheckpoint = - StreamSupport - .stream(activeAllocationIdsLocalCheckpoints.values().spliterator(), false) - .mapToLong(e -> e.value) - .min() - .orElse(SequenceNumbersService.UNASSIGNED_SEQ_NO); - - // the primary context contains knowledge of the state of the entire universe - assertThat(primaryContext.clusterStateVersion(), equalTo(trackerPrimaryContext.clusterStateVersion())); - assertThat(primaryContext.inSyncLocalCheckpoints(), equalTo(trackerPrimaryContext.inSyncLocalCheckpoints())); - assertThat(primaryContext.trackingLocalCheckpoints(), equalTo(trackerPrimaryContext.trackingLocalCheckpoints())); - assertThat(tracker.getGlobalCheckpoint(), equalTo(globalCheckpoint)); + final AtomicReference trackerPrimaryContext = new AtomicReference<>(); + try (Releasable ignored = tracker.primaryContext(trackerPrimaryContext::set)) { + final long globalCheckpoint = + StreamSupport + .stream(activeAllocationIdsLocalCheckpoints.values().spliterator(), false) + .mapToLong(e -> e.value) + .min() + .orElse(SequenceNumbersService.UNASSIGNED_SEQ_NO); + + // the primary context contains knowledge of the state of the entire universe + assertThat(primaryContext.clusterStateVersion(), equalTo(trackerPrimaryContext.get().clusterStateVersion())); + assertThat(primaryContext.inSyncLocalCheckpoints(), equalTo(trackerPrimaryContext.get().inSyncLocalCheckpoints())); + assertThat(primaryContext.trackingLocalCheckpoints(), equalTo(trackerPrimaryContext.get().trackingLocalCheckpoints())); + assertThat(tracker.getGlobalCheckpoint(), equalTo(globalCheckpoint)); + } } private Tuple, Set> randomActiveAndInitializingAllocationIds( 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 516fcd274d736..a341c26898759 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -536,7 +536,7 @@ public void testOperationPermitOnReplicaShards() throws InterruptedException, Ex routing = TestShardRouting.newShardRouting(routing.shardId(), routing.currentNodeId(), "otherNode", true, ShardRoutingState.RELOCATING, AllocationId.newRelocation(routing.allocationId())); indexShard.updateRoutingEntry(routing); - indexShard.relocated("test", () -> {}); + indexShard.relocated("test", primaryContext -> {}); engineClosed = false; break; } @@ -1042,7 +1042,7 @@ public void testLockingBeforeAndAfterRelocated() throws Exception { Thread recoveryThread = new Thread(() -> { latch.countDown(); try { - shard.relocated("simulated recovery", () -> {}); + shard.relocated("simulated recovery", primaryContext -> {}); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -1071,7 +1071,7 @@ public void testDelayedOperationsBeforeAndAfterRelocated() throws Exception { shard.updateRoutingEntry(ShardRoutingHelper.relocate(shard.routingEntry(), "other_node")); Thread recoveryThread = new Thread(() -> { try { - shard.relocated("simulated recovery", () -> {}); + shard.relocated("simulated recovery", primaryContext -> {}); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -1124,7 +1124,7 @@ public void run() { AtomicBoolean relocated = new AtomicBoolean(); final Thread recoveryThread = new Thread(() -> { try { - shard.relocated("simulated recovery", () -> {}); + shard.relocated("simulated recovery", primaryContext -> {}); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -1158,7 +1158,7 @@ public void testRelocatedShardCanNotBeRevived() throws IOException, InterruptedE final IndexShard shard = newStartedShard(true); final ShardRouting originalRouting = shard.routingEntry(); shard.updateRoutingEntry(ShardRoutingHelper.relocate(originalRouting, "other_node")); - shard.relocated("test", () -> {}); + shard.relocated("test", primaryContext -> {}); expectThrows(IllegalIndexShardStateException.class, () -> shard.updateRoutingEntry(originalRouting)); closeShards(shard); } @@ -1168,7 +1168,7 @@ public void testShardCanNotBeMarkedAsRelocatedIfRelocationCancelled() throws IOE final ShardRouting originalRouting = shard.routingEntry(); shard.updateRoutingEntry(ShardRoutingHelper.relocate(originalRouting, "other_node")); shard.updateRoutingEntry(originalRouting); - expectThrows(IllegalIndexShardStateException.class, () -> shard.relocated("test", () -> {})); + expectThrows(IllegalIndexShardStateException.class, () -> shard.relocated("test", primaryContext -> {})); closeShards(shard); } @@ -1187,7 +1187,7 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { cyclicBarrier.await(); - shard.relocated("test", () -> {}); + shard.relocated("test", primaryContext -> {}); } }); relocationThread.start(); @@ -1362,7 +1362,7 @@ public void testRecoveryFailsAfterMovingToRelocatedState() throws InterruptedExc assertThat(shard.state(), equalTo(IndexShardState.STARTED)); ShardRouting inRecoveryRouting = ShardRoutingHelper.relocate(origRouting, "some_node"); shard.updateRoutingEntry(inRecoveryRouting); - shard.relocated("simulate mark as relocated", () -> {}); + shard.relocated("simulate mark as relocated", primaryContext -> {}); assertThat(shard.state(), equalTo(IndexShardState.RELOCATED)); try { shard.updateRoutingEntry(origRouting); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index add89768373de..5532ad040f26d 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -77,6 +77,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -454,7 +455,7 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int relocated.set(true); assertTrue(recoveriesDelayed.get()); return null; - }).when(shard).relocated(any(String.class), any(Runnable.class)); + }).when(shard).relocated(any(String.class), any(Consumer.class)); when(shard.acquireIndexCommit(anyBoolean())).thenReturn(mock(Engine.IndexCommitRef.class)); doAnswer(invocationOnMock -> { @SuppressWarnings("unchecked") From 077b3f4c7590383d239d4190a10dda3feaaea604 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 23 Jun 2017 09:37:53 -0400 Subject: [PATCH 49/54] Sealing tests --- .../index/seqno/GlobalCheckpointTracker.java | 16 +++++++- .../seqno/GlobalCheckpointTrackerTests.java | 37 +++++++++++++++++++ 2 files changed, 51 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 382db8ee9491b..b30cfdb3483b4 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -219,6 +219,9 @@ public synchronized long getGlobalCheckpoint() { * @param globalCheckpoint the global checkpoint */ synchronized void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { + if (sealed) { + throw new IllegalStateException("global checkpoint tracker is sealed"); + } /* * The global checkpoint here is a local knowledge which is updated under the mandate of the primary. It can happen that the primary * information is lagging compared to a replica (e.g., if a replica is promoted to primary but has stale info relative to other @@ -504,12 +507,21 @@ private synchronized void waitForLocalCheckpointToAdvance() throws InterruptedEx /** * Check if there are any recoveries pending in-sync. * - * @return {@code true} if there is at least one shard pending in-sync, otherwise false + * @return true if there is at least one shard pending in-sync, otherwise false */ - public boolean pendingInSync() { + boolean pendingInSync() { return !pendingInSync.isEmpty(); } + /** + * Check if the tracker is sealed. + * + * @return true if the tracker is sealed, otherwise false. + */ + boolean sealed() { + return sealed; + } + /** * Returns the local checkpoint for the shard with the specified allocation ID, or {@link SequenceNumbersService#UNASSIGNED_SEQ_NO} if * the shard is not in-sync. diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java index 9b5da400f49e5..ceeca5f417738 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java @@ -51,9 +51,12 @@ import java.util.stream.StreamSupport; import static org.elasticsearch.index.seqno.SequenceNumbersService.UNASSIGNED_SEQ_NO; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.not; +import static org.mockito.Mockito.mock; public class GlobalCheckpointTrackerTests extends ESTestCase { @@ -680,6 +683,40 @@ public void testPrimaryContextNewerThanAppliedClusterState() { } } + public void testPrimaryContextSealing() { + // the tracker should start in the state of not being sealed + assertFalse(tracker.sealed()); + + // sampling the primary context should seal the tracker + final Releasable releasable = tracker.primaryContext(primaryContext -> {}); + assertTrue(tracker.sealed()); + + // invoking any method that mutates the state of the tracker should fail + assertIllegalStateExceptionWhenSealed(() -> tracker.updateLocalCheckpoint(randomAlphaOfLength(16), randomNonNegativeLong())); + assertIllegalStateExceptionWhenSealed( + () -> tracker.updateAllocationIdsFromMaster(randomNonNegativeLong(), Collections.emptySet(), Collections.emptySet())); + assertIllegalStateExceptionWhenSealed(() -> tracker.updateAllocationIdsFromPrimaryContext(mock(PrimaryContext.class))); + assertIllegalStateExceptionWhenSealed(() -> tracker.primaryContext(primaryContext -> {})); + assertIllegalStateExceptionWhenSealed(() -> tracker.markAllocationIdAsInSync(randomAlphaOfLength(16), randomNonNegativeLong())); + + // closing the releasable should unseal the tracker + releasable.close(); + assertFalse(tracker.sealed()); + + // an exception thrown by the consumer should not leave the tracker in a sealed state + expectThrows( + RuntimeException.class, + () -> tracker.primaryContext(primaryContext -> { + throw new RuntimeException(); + })); + assertFalse(tracker.sealed()); + } + + private void assertIllegalStateExceptionWhenSealed(final ThrowingRunnable runnable) { + final IllegalStateException e = expectThrows(IllegalStateException.class, runnable); + assertThat(e, hasToString(containsString("global checkpoint tracker is sealed"))); + } + private Tuple, Set> randomActiveAndInitializingAllocationIds( final int numberOfActiveAllocationsIds, final int numberOfInitializingIds) { From 4e05714b1454d47d879d183dc06182621d856014 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 23 Jun 2017 09:39:08 -0400 Subject: [PATCH 50/54] One more assertion --- .../elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java index ceeca5f417738..4691ebd645d98 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java @@ -693,6 +693,7 @@ public void testPrimaryContextSealing() { // invoking any method that mutates the state of the tracker should fail assertIllegalStateExceptionWhenSealed(() -> tracker.updateLocalCheckpoint(randomAlphaOfLength(16), randomNonNegativeLong())); + assertIllegalStateExceptionWhenSealed(() -> tracker.updateGlobalCheckpointOnReplica(randomNonNegativeLong())); assertIllegalStateExceptionWhenSealed( () -> tracker.updateAllocationIdsFromMaster(randomNonNegativeLong(), Collections.emptySet(), Collections.emptySet())); assertIllegalStateExceptionWhenSealed(() -> tracker.updateAllocationIdsFromPrimaryContext(mock(PrimaryContext.class))); From 744b03a6fb0229feea6df7ce676702a34e7f4ace Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 26 Jun 2017 13:04:41 -0400 Subject: [PATCH 51/54] Fix imports --- .../org/elasticsearch/index/seqno/GlobalCheckpointTracker.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index b30cfdb3483b4..b3ecf41647a5d 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -24,21 +24,18 @@ import com.carrotsearch.hppc.cursors.ObjectLongCursor; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.collect.LongTuple; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; -import java.security.Security; import java.util.Arrays; import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Locale; import java.util.Set; -import java.util.TreeMap; import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.StreamSupport; From 3c2731f054cf4245c12ebc2c008afac49ee68c68 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 26 Jun 2017 13:33:17 -0400 Subject: [PATCH 52/54] Safer sealing --- .../index/seqno/GlobalCheckpointTracker.java | 24 +++++++-------- .../index/seqno/SequenceNumbersService.java | 13 +++++--- .../elasticsearch/index/shard/IndexShard.java | 17 ++++------- .../seqno/GlobalCheckpointTrackerTests.java | 30 ++++++++----------- 4 files changed, 36 insertions(+), 48 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index b3ecf41647a5d..af32aa771d8cd 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -24,7 +24,6 @@ import com.carrotsearch.hppc.cursors.ObjectLongCursor; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.collect.LongTuple; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.PrimaryContext; @@ -36,7 +35,6 @@ import java.util.List; import java.util.Locale; import java.util.Set; -import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -286,24 +284,22 @@ public synchronized void updateAllocationIdsFromMaster( * * @return the primary context */ - synchronized Releasable primaryContext(final Consumer consumer) { + synchronized PrimaryContext primaryContext() { if (sealed) { throw new IllegalStateException("global checkpoint tracker is sealed"); } sealed = true; final ObjectLongMap inSyncLocalCheckpoints = new ObjectLongHashMap<>(this.inSyncLocalCheckpoints); final ObjectLongMap trackingLocalCheckpoints = new ObjectLongHashMap<>(this.trackingLocalCheckpoints); - try { - consumer.accept(new PrimaryContext(appliedClusterStateVersion, inSyncLocalCheckpoints, trackingLocalCheckpoints)); - } catch (final Exception e) { - sealed = false; - throw e; - } - return () -> { - synchronized (this) { - sealed = false; - } - }; + return new PrimaryContext(appliedClusterStateVersion, inSyncLocalCheckpoints, trackingLocalCheckpoints); + } + + /** + * Releases a previously acquired primary context. + */ + synchronized void releasePrimaryContext() { + assert sealed; + sealed = false; } /** diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java index d352b9aa13779..6d8b87599a125 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -19,14 +19,12 @@ package org.elasticsearch.index.seqno; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.PrimaryContext; import org.elasticsearch.index.shard.ShardId; import java.util.Set; -import java.util.function.Consumer; /** * Encapsulates the local and global checkpoints into a single service for use as a shard component. @@ -202,8 +200,15 @@ public boolean pendingInSync() { * * @return the primary context */ - public Releasable primaryContext(final Consumer consumer) { - return globalCheckpointTracker.primaryContext(consumer); + public PrimaryContext primaryContext() { + return globalCheckpointTracker.primaryContext(); + } + + /** + * Releases a previously acquired primary context. + */ + public void releasePrimaryContext() { + globalCheckpointTracker.releasePrimaryContext(); } } 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 0175fd56c3679..13ced02f6b88a 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -537,11 +537,15 @@ public void relocated( * network operation. Doing this under the mutex can implicitly block the cluster state update thread on network operations. */ verifyRelocatingState(); - try (Releasable ignored = primaryContext(consumer)) { + final PrimaryContext primaryContext = getEngine().seqNoService().primaryContext(); + try { + consumer.accept(primaryContext); synchronized (mutex) { verifyRelocatingState(); changeState(IndexShardState.RELOCATED, reason); } + } catch (final Exception e) { + getEngine().seqNoService().releasePrimaryContext(); } }); } catch (TimeoutException e) { @@ -574,17 +578,6 @@ private void verifyRelocatingState() { } } - /** - * Obtain the primary context for the shard. The shard must be serving as the relocation source for a primary shard. - * - * @return the primary for the shard - */ - public Releasable primaryContext(final Consumer consumer) { - verifyPrimary(); - assert shardRouting.relocating() : "primary context can only be obtained from a relocating primary: " + shardRouting; - return getEngine().seqNoService().primaryContext(consumer); - } - public IndexShardState state() { return state; } diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java index 4691ebd645d98..ae4aab107f207 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTrackerTests.java @@ -23,7 +23,6 @@ import com.carrotsearch.hppc.ObjectLongMap; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.PrimaryContext; @@ -43,7 +42,6 @@ import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -666,8 +664,9 @@ public void testPrimaryContextNewerThanAppliedClusterState() { tracker.updateAllocationIdsFromPrimaryContext(primaryContext); - final AtomicReference trackerPrimaryContext = new AtomicReference<>(); - try (Releasable ignored = tracker.primaryContext(trackerPrimaryContext::set)) { + final PrimaryContext trackerPrimaryContext = tracker.primaryContext(); + try { + assertTrue(tracker.sealed()); final long globalCheckpoint = StreamSupport .stream(activeAllocationIdsLocalCheckpoints.values().spliterator(), false) @@ -676,10 +675,13 @@ public void testPrimaryContextNewerThanAppliedClusterState() { .orElse(SequenceNumbersService.UNASSIGNED_SEQ_NO); // the primary context contains knowledge of the state of the entire universe - assertThat(primaryContext.clusterStateVersion(), equalTo(trackerPrimaryContext.get().clusterStateVersion())); - assertThat(primaryContext.inSyncLocalCheckpoints(), equalTo(trackerPrimaryContext.get().inSyncLocalCheckpoints())); - assertThat(primaryContext.trackingLocalCheckpoints(), equalTo(trackerPrimaryContext.get().trackingLocalCheckpoints())); + assertThat(primaryContext.clusterStateVersion(), equalTo(trackerPrimaryContext.clusterStateVersion())); + assertThat(primaryContext.inSyncLocalCheckpoints(), equalTo(trackerPrimaryContext.inSyncLocalCheckpoints())); + assertThat(primaryContext.trackingLocalCheckpoints(), equalTo(trackerPrimaryContext.trackingLocalCheckpoints())); assertThat(tracker.getGlobalCheckpoint(), equalTo(globalCheckpoint)); + } finally { + tracker.releasePrimaryContext(); + assertFalse(tracker.sealed()); } } @@ -688,7 +690,7 @@ public void testPrimaryContextSealing() { assertFalse(tracker.sealed()); // sampling the primary context should seal the tracker - final Releasable releasable = tracker.primaryContext(primaryContext -> {}); + tracker.primaryContext(); assertTrue(tracker.sealed()); // invoking any method that mutates the state of the tracker should fail @@ -697,19 +699,11 @@ public void testPrimaryContextSealing() { assertIllegalStateExceptionWhenSealed( () -> tracker.updateAllocationIdsFromMaster(randomNonNegativeLong(), Collections.emptySet(), Collections.emptySet())); assertIllegalStateExceptionWhenSealed(() -> tracker.updateAllocationIdsFromPrimaryContext(mock(PrimaryContext.class))); - assertIllegalStateExceptionWhenSealed(() -> tracker.primaryContext(primaryContext -> {})); + assertIllegalStateExceptionWhenSealed(() -> tracker.primaryContext()); assertIllegalStateExceptionWhenSealed(() -> tracker.markAllocationIdAsInSync(randomAlphaOfLength(16), randomNonNegativeLong())); // closing the releasable should unseal the tracker - releasable.close(); - assertFalse(tracker.sealed()); - - // an exception thrown by the consumer should not leave the tracker in a sealed state - expectThrows( - RuntimeException.class, - () -> tracker.primaryContext(primaryContext -> { - throw new RuntimeException(); - })); + tracker.releasePrimaryContext(); assertFalse(tracker.sealed()); } From e19c8b3d7e2785ed698257805f70dfae0ac610d4 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 26 Jun 2017 13:41:42 -0400 Subject: [PATCH 53/54] Remove check --- .../org/elasticsearch/index/seqno/GlobalCheckpointTracker.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index af32aa771d8cd..d3071e5cf0073 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -237,9 +237,6 @@ synchronized void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { */ public synchronized void updateAllocationIdsFromMaster( final long applyingClusterStateVersion, final Set activeAllocationIds, final Set initializingAllocationIds) { - if (sealed) { - throw new IllegalStateException("global checkpoint tracker is sealed"); - } if (applyingClusterStateVersion < appliedClusterStateVersion) { return; } From 0d33a8884ea247c69921749f913ff3c55a14e9cd Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 26 Jun 2017 13:43:19 -0400 Subject: [PATCH 54/54] Remove another sealed check --- .../org/elasticsearch/index/seqno/GlobalCheckpointTracker.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index d3071e5cf0073..aeafbc1110850 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -214,9 +214,6 @@ public synchronized long getGlobalCheckpoint() { * @param globalCheckpoint the global checkpoint */ synchronized void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { - if (sealed) { - throw new IllegalStateException("global checkpoint tracker is sealed"); - } /* * The global checkpoint here is a local knowledge which is updated under the mandate of the primary. It can happen that the primary * information is lagging compared to a replica (e.g., if a replica is promoted to primary but has stale info relative to other