From c0e7443edf599ec93aff1562ff70e62abc14bfe3 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 28 Aug 2017 16:21:33 -0400 Subject: [PATCH 01/38] Introduce global checkpoint background sync It is the exciting return of the global checkpoint background sync. Long, long ago, in snapshot version far, far away we had and only had a global checkpoint background sync. This sync would fire periodically and send the global checkpoint from the primary shard to the replicas so that they could update their local knowledge of the global checkpoint. Later in time, as we sped ahead towards finalizing the initial version of sequence IDs, we realized that we need the global checkpoint updates to be inline. This means that on a replication operation, the primary shard would piggy back the global checkpoint with the replication operation to the replicas. The replicas would update their local knowledge of the global checkpoint and reply with their local checkpoint. However, this could allow the global checkpoint on the primary to advance again and the replicas would fall behind in their local knowledge of the global checkpoint. If another replication operation never fired, then the replicas would be permanently behind. To account for this, we added one more sync that would fire when the primary shard fell idle. However, this has problems: - the shard idle timer defaults to five minutes, a long time to wait for the replicas to learn of the new global checkpoint - if a replica missed the sync, there was no follow-up sync to catch them up - there is an inherent race condition where the primary shard could fall idle mid-operation (after having sent the replication request to the replicas); in this case, there would never be a background sync after the operation completes - tying the global checkpoint sync to the idle timer was never natural To fix this, we add back a global checkpoint background sync that fires on a timer. This timer fires every thirty seconds, and is not configurable (for simplicity). This background sync is smarter in the sense that it only sends a sync if the global checkpoint on at least one replica is lagging that of the primary. This necessitates adding the primary shard tracking its knowledge of the local knowledge of the global checkpoint on the replicas. When the timer fires, we can compare the global checkpoint on the primary to its knowledge of the global checkpoint on the replicas and only send a sync if there is a shard behind. During replication operations it can be the case that the timer fires and sends a sync that would be covered by an in-flight operation. This is okay, the extra sync does not hurt and we do not need the complexity of optimizing away this duplicate sync. --- .../replication/ReplicationOperation.java | 9 ++ .../TransportReplicationAction.java | 5 ++ .../metadata/MetaDataCreateIndexService.java | 2 +- .../metadata/MetaDataIndexAliasesService.java | 2 +- .../MetaDataIndexTemplateService.java | 2 +- .../metadata/MetaDataMappingService.java | 2 +- .../org/elasticsearch/index/IndexModule.java | 6 +- .../org/elasticsearch/index/IndexService.java | 85 +++++++++++++++++-- .../seqno/GlobalCheckpointSyncAction.java | 11 ++- .../index/seqno/GlobalCheckpointTracker.java | 35 ++++++++ .../index/seqno/SequenceNumbersService.java | 23 +++++ .../elasticsearch/index/shard/IndexShard.java | 68 ++++++++++++--- .../elasticsearch/indices/IndicesService.java | 37 +++++--- .../cluster/IndicesClusterStateService.java | 43 ++++++---- .../ReplicationOperationTests.java | 6 ++ .../metadata/IndexCreationTaskTests.java | 2 +- .../elasticsearch/index/IndexModuleTests.java | 2 +- .../elasticsearch/index/IndexServiceIT.java | 78 +++++++++++++++++ .../ESIndexLevelReplicationTestCase.java | 13 ++- .../index/shard/IndexShardIT.java | 2 +- .../index/shard/IndexShardTests.java | 66 +++++++++++++- ...dicesLifecycleListenerSingleNodeTests.java | 2 +- ...actIndicesClusterStateServiceTestCase.java | 3 +- .../indices/cluster/ClusterStateChanges.java | 3 +- ...ClusterStateServiceRandomUpdatesTests.java | 28 +++--- .../index/shard/IndexShardTestCase.java | 46 ++++++---- .../test/InternalSettingsPlugin.java | 17 +++- 27 files changed, 495 insertions(+), 103 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/index/IndexServiceIT.java diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index f28beed1d7fac..f62485740a731 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -173,6 +173,7 @@ public void onResponse(ReplicaResponse response) { successfulShards.incrementAndGet(); try { primary.updateLocalCheckpointForShard(shard.allocationId().getId(), response.localCheckpoint()); + primary.updateGlobalCheckpointForShard(shard.allocationId().getId(), globalCheckpoint); } catch (final AlreadyClosedException e) { // okay, the index was deleted or this shard was never activated after a relocation; fall through and finish normally } catch (final Exception e) { @@ -315,6 +316,14 @@ public interface Primary< */ void updateLocalCheckpointForShard(String allocationId, long checkpoint); + /** + * Update the local knowledge of the global checkpoint for the specified allocation ID. + * + * @param allocationId the allocation ID to update the global checkpoint for + * @param globalCheckpoint the global checkpoint + */ + void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint); + /** * Returns the local checkpoint on the primary shard. * diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 1a96a159179c4..885240d0f11eb 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -1006,6 +1006,11 @@ public void updateLocalCheckpointForShard(String allocationId, long checkpoint) indexShard.updateLocalCheckpointForShard(allocationId, checkpoint); } + @Override + public void updateGlobalCheckpointForShard(final String allocationId, final long globalCheckpoint) { + indexShard.updateGlobalCheckpointForShard(allocationId, globalCheckpoint); + } + @Override public long localCheckpoint() { return indexShard.getLocalCheckpoint(); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index 8b262a249c844..f2495edfc6b24 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -428,7 +428,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { (tmpImd.getNumberOfReplicas() + 1) + "]"); } // create the index here (on the master) to validate it can be created, as well as adding the mapping - final IndexService indexService = indicesService.createIndex(tmpImd, Collections.emptyList()); + final IndexService indexService = indicesService.createIndex(tmpImd, Collections.emptyList(), s -> {}); createdIndex = indexService.index(); // now add the mappings MapperService mapperService = indexService.mapperService(); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java index 0949032db1665..ea4bdc83b564c 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java @@ -139,7 +139,7 @@ ClusterState innerExecute(ClusterState currentState, Iterable actio if (indexService == null) { // temporarily create the index and add mappings so we can parse the filter try { - indexService = indicesService.createIndex(index, emptyList()); + indexService = indicesService.createIndex(index, emptyList(), s -> {}); indicesToClose.add(index.getIndex()); } catch (IOException e) { throw new ElasticsearchException("Failed to create temporary index for parsing the alias", e); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java index c96895b94e793..c404a7972a9ce 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java @@ -213,7 +213,7 @@ private static void validateAndAddTemplate(final PutRequest request, IndexTempla .build(); final IndexMetaData tmpIndexMetadata = IndexMetaData.builder(temporaryIndexName).settings(dummySettings).build(); - IndexService dummyIndexService = indicesService.createIndex(tmpIndexMetadata, Collections.emptyList()); + IndexService dummyIndexService = indicesService.createIndex(tmpIndexMetadata, Collections.emptyList(), s -> {}); createdIndex = dummyIndexService.index(); templateBuilder.order(request.order); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java index 865b58c468a52..c56a2c8716f65 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java @@ -145,7 +145,7 @@ ClusterState executeRefresh(final ClusterState currentState, final List {}); removeIndex = true; indexService.mapperService().merge(indexMetaData, MergeReason.MAPPING_RECOVERY, true); } diff --git a/core/src/main/java/org/elasticsearch/index/IndexModule.java b/core/src/main/java/org/elasticsearch/index/IndexModule.java index 630fe11e0a811..43c16673f02be 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexModule.java +++ b/core/src/main/java/org/elasticsearch/index/IndexModule.java @@ -40,6 +40,7 @@ import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.IndexingOperationListener; import org.elasticsearch.index.shard.SearchOperationListener; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.similarity.BM25SimilarityProvider; import org.elasticsearch.index.similarity.SimilarityProvider; import org.elasticsearch.index.similarity.SimilarityService; @@ -330,7 +331,8 @@ public IndexService newIndexService( IndicesQueryCache indicesQueryCache, MapperRegistry mapperRegistry, IndicesFieldDataCache indicesFieldDataCache, - NamedWriteableRegistry namedWriteableRegistry) + NamedWriteableRegistry namedWriteableRegistry, + Consumer globalCheckpointSyncer) throws IOException { final IndexEventListener eventListener = freeze(); IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null @@ -365,7 +367,7 @@ public IndexService newIndexService( new SimilarityService(indexSettings, scriptService, similarities), shardStoreDeleter, analysisRegistry, engineFactory.get(), circuitBreakerService, bigArrays, threadPool, scriptService, client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry, - indicesFieldDataCache, searchOperationListeners, indexOperationListeners, namedWriteableRegistry); + indicesFieldDataCache, searchOperationListeners, indexOperationListeners, namedWriteableRegistry, globalCheckpointSyncer); } /** diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index a4d03929cbb57..5cd21c2406db5 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -82,6 +82,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; import java.util.function.LongSupplier; import java.util.function.Supplier; @@ -109,10 +110,12 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final AtomicBoolean closed = new AtomicBoolean(false); private final AtomicBoolean deleted = new AtomicBoolean(false); private final IndexSettings indexSettings; - private final List indexingOperationListeners; private final List searchOperationListeners; + private final List indexingOperationListeners; + private final Consumer globalCheckpointSyncer; private volatile AsyncRefreshTask refreshTask; private volatile AsyncTranslogFSync fsyncTask; + private volatile AsyncGlobalCheckpointTask globalCheckpointTask; // don't convert to Setting<> and register... we only set this in tests and register via a plugin private final String INDEX_TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING = "index.translog.retention.check_interval"; @@ -145,7 +148,8 @@ public IndexService( IndicesFieldDataCache indicesFieldDataCache, List searchOperationListeners, List indexingOperationListeners, - NamedWriteableRegistry namedWriteableRegistry) throws IOException { + NamedWriteableRegistry namedWriteableRegistry, + Consumer globalCheckpointSyncer) throws IOException { super(indexSettings); this.indexSettings = indexSettings; this.xContentRegistry = xContentRegistry; @@ -182,11 +186,13 @@ public IndexService( this.engineFactory = engineFactory; // initialize this last -- otherwise if the wrapper requires any other member to be non-null we fail with an NPE this.searcherWrapper = wrapperFactory.newWrapper(this); - this.indexingOperationListeners = Collections.unmodifiableList(indexingOperationListeners); this.searchOperationListeners = Collections.unmodifiableList(searchOperationListeners); + this.indexingOperationListeners = Collections.unmodifiableList(indexingOperationListeners); + this.globalCheckpointSyncer = globalCheckpointSyncer; // kick off async ops for the first shard in this index this.refreshTask = new AsyncRefreshTask(this); this.trimTranslogTask = new AsyncTrimTranslogTask(this); + this.globalCheckpointTask = new AsyncGlobalCheckpointTask(this); rescheduleFsyncTask(indexSettings.getTranslogDurability()); } @@ -268,7 +274,15 @@ public synchronized void close(final String reason, boolean delete) throws IOExc } } } finally { - IOUtils.close(bitsetFilterCache, indexCache, indexFieldData, mapperService, refreshTask, fsyncTask, trimTranslogTask); + IOUtils.close( + bitsetFilterCache, + indexCache, + indexFieldData, + mapperService, + refreshTask, + fsyncTask, + trimTranslogTask, + globalCheckpointTask); } } } @@ -365,7 +379,7 @@ public synchronized IndexShard createShard(ShardRouting routing) throws IOExcept indexShard = new IndexShard(routing, this.indexSettings, path, store, indexSortSupplier, indexCache, mapperService, similarityService, engineFactory, eventListener, searcherWrapper, threadPool, bigArrays, engineWarmer, - searchOperationListeners, indexingOperationListeners); + searchOperationListeners, indexingOperationListeners, () -> globalCheckpointSyncer.accept(shardId)); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap(); @@ -710,6 +724,30 @@ private void maybeTrimTranslog() { } } + private void syncGlobalCheckpoints() { + for (final IndexShard shard : this.shards.values()) { + if (shard.routingEntry().active() && shard.routingEntry().primary()) { + switch (shard.state()) { + case CREATED: + case RECOVERING: + case CLOSED: + continue; + case POST_RECOVERY: + case STARTED: + case RELOCATED: + try { + shard.maybeSyncGlobalCheckpoint(); + } catch (final AlreadyClosedException | IndexShardClosedException e) { + // the shard was closed concurrently, continue + } + continue; + default: + throw new IllegalStateException("unknown state [" + shard.state() + "]"); + } + } + } + } + abstract static class BaseAsyncTask implements Runnable, Closeable { protected final IndexService indexService; protected final ThreadPool threadPool; @@ -877,6 +915,39 @@ public String toString() { } } + private static final TimeValue GLOBAL_CHECKPOINT_SYNC_INTERVAL = TimeValue.timeValueSeconds(30); + + /** + * Background task that syncs the global checkpoint to replicas. + */ + final class AsyncGlobalCheckpointTask extends BaseAsyncTask { + + AsyncGlobalCheckpointTask(final IndexService indexService) { + // index.global_checkpoint_sync_interval is not a real setting, it is only registered in tests + super( + indexService, + indexService + .getIndexSettings() + .getSettings() + .getAsTime("index.global_checkpoint_sync.interval", GLOBAL_CHECKPOINT_SYNC_INTERVAL)); + } + + @Override + protected void runInternal() { + indexService.syncGlobalCheckpoints(); + } + + @Override + protected String getThreadPool() { + return ThreadPool.Names.GENERIC; + } + + @Override + public String toString() { + return "global_checkpoint_sync"; + } + } + AsyncRefreshTask getRefreshTask() { // for tests return refreshTask; } @@ -885,6 +956,10 @@ AsyncTranslogFSync getFsyncTask() { // for tests return fsyncTask; } + AsyncGlobalCheckpointTask getGlobalCheckpointTask() { + return globalCheckpointTask; + } + /** * Clears the caches for the given shard id if the shard is still allocated on this node */ diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java index c7059d05919b1..8ca223f9084ba 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java @@ -47,7 +47,7 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction< GlobalCheckpointSyncAction.Request, GlobalCheckpointSyncAction.Request, - ReplicationResponse> implements IndexEventListener { + ReplicationResponse> { public static String ACTION_NAME = "indices:admin/seq_no/global_checkpoint_sync"; @@ -76,6 +76,10 @@ public GlobalCheckpointSyncAction( ThreadPool.Names.SAME); } + public void updateGlobalCheckpointForShard(final ShardId shardId) { + execute(new Request(shardId)); + } + @Override protected ReplicationResponse newResponseInstance() { return new ReplicationResponse(); @@ -93,11 +97,6 @@ protected void sendReplicaRequest( } } - @Override - public void onShardInactive(final IndexShard indexShard) { - execute(new Request(indexShard.shardId())); - } - @Override protected PrimaryResult shardOperationOnPrimary( final Request request, final IndexShard indexShard) throws Exception { 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 4df58bcab4459..4c810ec449bde 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.seqno; +import com.carrotsearch.hppc.ObjectLongHashMap; +import com.carrotsearch.hppc.ObjectLongMap; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -496,6 +498,38 @@ public synchronized void updateLocalCheckpoint(final String allocationId, final assert invariant(); } + private ObjectLongMap globalCheckpoints = new ObjectLongHashMap<>(); + + /** + * Update the local knowledge of the global checkpoint for the specified allocation ID. + * + * @param allocationId the allocation ID to update the global checkpoint for + * @param globalCheckpoint the global checkpoint + */ + synchronized void updateGlobalCheckpointForShard(final String allocationId, final long globalCheckpoint) { + assert primaryMode; + assert handoffInProgress == false; + final long current = globalCheckpoints.getOrDefault(allocationId, SequenceNumbers.UNASSIGNED_SEQ_NO); + if (globalCheckpoint > current) { + globalCheckpoints.put(allocationId, globalCheckpoint); + } + } + + /** + * Returns the global checkpoints for all shards. + * + * @param allocationId the allocationId to use for the global checkpoint on the primary + * + * @return the global checkpoints for all shards + */ + synchronized ObjectLongMap getGlobalCheckpoints(final String allocationId) { + assert primaryMode; + assert handoffInProgress == false; + final ObjectLongMap copy = new ObjectLongHashMap<>(globalCheckpoints); + copy.put(allocationId, globalCheckpoint); + return copy; + } + /** * Computes the global checkpoint based on the given local checkpoints. In case where there are entries preventing the * computation to happen (for example due to blocking), it returns the fallback value. @@ -585,6 +619,7 @@ public synchronized void completeRelocationHandoff() { lcps.localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; } }); + globalCheckpoints.clear(); assert invariant(); } 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 44ad8db39a2a6..4a16d82318181 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -19,7 +19,9 @@ package org.elasticsearch.index.seqno; +import com.carrotsearch.hppc.ObjectLongMap; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.ReplicationGroup; @@ -131,6 +133,16 @@ public void updateLocalCheckpointForShard(final String allocationId, final long globalCheckpointTracker.updateLocalCheckpoint(allocationId, checkpoint); } + /** + * Update the local knowledge of the global checkpoint for the specified allocation ID. + * + * @param allocationId the allocation ID to update the global checkpoint for + * @param globalCheckpoint the global checkpoint + */ + public void updateGlobalCheckpointForShard(final String allocationId, final long globalCheckpoint) { + globalCheckpointTracker.updateGlobalCheckpointForShard(allocationId, globalCheckpoint); + } + /** * Called when the recovery process for a shard is ready to open the engine on the target shard. * See {@link GlobalCheckpointTracker#initiateTracking(String)} for details. @@ -179,6 +191,17 @@ public long getGlobalCheckpoint() { return globalCheckpointTracker.getGlobalCheckpoint(); } + /** + * Returns the global checkpoints for all shards. + * + * @param allocationId the allocationId to use for the global checkpoint on the primary + * + * @return the global checkpoints for all shards + */ + public ObjectLongMap getGlobalCheckpoints(final String allocationId) { + return globalCheckpointTracker.getGlobalCheckpoints(allocationId); + } + /** * Updates the global checkpoint on a replica shard after it has been updated by the primary. * 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 22d6ba20be2cd..b6c68f2740d71 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.shard; +import com.carrotsearch.hppc.ObjectLongMap; +import com.carrotsearch.hppc.cursors.ObjectLongCursor; import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CheckIndex; import org.apache.lucene.index.IndexCommit; @@ -102,6 +104,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.GlobalCheckpointSyncAction; import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -155,6 +158,7 @@ import java.util.function.Consumer; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import static org.elasticsearch.index.mapper.SourceToParse.source; @@ -196,6 +200,11 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl protected final EngineFactory engineFactory; private final IndexingOperationListener indexingOperationListeners; + private final Runnable globalCheckpointSyncer; + + Runnable getGlobalCheckpointSyncer() { + return globalCheckpointSyncer; + } @Nullable private RecoveryState recoveryState; @@ -232,11 +241,24 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl */ private final RefreshListeners refreshListeners; - public IndexShard(ShardRouting shardRouting, IndexSettings indexSettings, ShardPath path, Store store, - Supplier indexSortSupplier, IndexCache indexCache, MapperService mapperService, SimilarityService similarityService, - @Nullable EngineFactory engineFactory, - IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, ThreadPool threadPool, BigArrays bigArrays, - Engine.Warmer warmer, List searchOperationListener, List listeners) throws IOException { + public IndexShard( + ShardRouting shardRouting, + IndexSettings indexSettings, + ShardPath path, + Store store, + Supplier indexSortSupplier, + IndexCache indexCache, + MapperService mapperService, + SimilarityService similarityService, + @Nullable EngineFactory engineFactory, + IndexEventListener indexEventListener, + IndexSearcherWrapper indexSearcherWrapper, + ThreadPool threadPool, + BigArrays bigArrays, + Engine.Warmer warmer, + List searchOperationListener, + List listeners, + Runnable globalCheckpointSyncer) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); this.shardRouting = shardRouting; @@ -256,6 +278,7 @@ public IndexShard(ShardRouting shardRouting, IndexSettings indexSettings, ShardP final List listenersList = new ArrayList<>(listeners); listenersList.add(internalIndexingStats); this.indexingOperationListeners = new IndexingOperationListener.CompositeListener(listenersList, logger); + this.globalCheckpointSyncer = globalCheckpointSyncer; final List searchListenersList = new ArrayList<>(searchOperationListener); searchListenersList.add(searchStats); this.searchOperationListener = new SearchOperationListener.CompositeListener(searchListenersList, logger); @@ -1667,6 +1690,18 @@ public void updateLocalCheckpointForShard(final String allocationId, final long getEngine().seqNoService().updateLocalCheckpointForShard(allocationId, checkpoint); } + /** + * Update the local knowledge of the global checkpoint for the specified allocation ID. + * + * @param allocationId the allocation ID to update the global checkpoint for + * @param globalCheckpoint the global checkpoint + */ + public void updateGlobalCheckpointForShard(final String allocationId, final long globalCheckpoint) { + verifyPrimary(); + verifyNotClosed(); + getEngine().seqNoService().updateGlobalCheckpointForShard(allocationId, globalCheckpoint); + } + /** * Waits for all operations up to the provided sequence number to complete. * @@ -1704,11 +1739,6 @@ public void initiateTracking(final String allocationId) { public void markAllocationIdAsInSync(final String allocationId, final long localCheckpoint) throws InterruptedException { verifyPrimary(); 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); } /** @@ -1729,6 +1759,24 @@ public long getGlobalCheckpoint() { return getEngine().seqNoService().getGlobalCheckpoint(); } + /** + * Syncs the global checkpoint to the replicas if the global checkpoint on at least one replica is behind the global checkpoint on the + * primary. + */ + public void maybeSyncGlobalCheckpoint() { + verifyPrimary(); + verifyNotClosed(); + final String allocationId = routingEntry().allocationId().getId(); + final ObjectLongMap globalCheckpoints = getEngine().seqNoService().getGlobalCheckpoints(allocationId); + assert globalCheckpoints.containsKey(allocationId); + final long globalCheckpoint = globalCheckpoints.get(allocationId); + final boolean syncNeeded = + StreamSupport.stream(globalCheckpoints.values().spliterator(), false).anyMatch(v -> v.value < globalCheckpoint); + if (syncNeeded) { + globalCheckpointSyncer.run(); + } + } + /** * Returns the current replication group for the shard. * diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index 3039305c42a3d..3c5fd2c8d21cc 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -43,7 +43,6 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractLifecycleComponent; @@ -375,12 +374,18 @@ public IndexService indexServiceSafe(Index index) { /** * Creates a new {@link IndexService} for the given metadata. - * @param indexMetaData the index metadata to create the index for - * @param builtInListeners a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with the per-index listeners + * + * @param indexMetaData the index metadata to create the index for + * @param builtInListeners a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with the + * per-index listeners + * @param globalCheckpointSyncer callback for syncing global checkpoints * @throws ResourceAlreadyExistsException if the index already exists. */ @Override - public synchronized IndexService createIndex(IndexMetaData indexMetaData, List builtInListeners) throws IOException { + public synchronized IndexService createIndex( + final IndexMetaData indexMetaData, + final List builtInListeners, + final Consumer globalCheckpointSyncer) throws IOException { ensureChangesAllowed(); if (indexMetaData.getIndexUUID().equals(IndexMetaData.INDEX_UUID_NA_VALUE)) { throw new IllegalArgumentException("index must have a real UUID found value: [" + indexMetaData.getIndexUUID() + "]"); @@ -399,13 +404,14 @@ public void onStoreClosed(ShardId shardId) { finalListeners.add(onStoreClose); finalListeners.add(oldShardsStats); final IndexService indexService = - createIndexService( - "create index", - indexMetaData, - indicesQueryCache, - indicesFieldDataCache, - finalListeners, - indexingMemoryController); + createIndexService( + "create index", + indexMetaData, + indicesQueryCache, + indicesFieldDataCache, + globalCheckpointSyncer, + finalListeners, + indexingMemoryController); boolean success = false; try { indexService.getIndexEventListener().afterIndexCreated(indexService); @@ -423,8 +429,10 @@ public void onStoreClosed(ShardId shardId) { * This creates a new IndexService without registering it */ private synchronized IndexService createIndexService(final String reason, - IndexMetaData indexMetaData, IndicesQueryCache indicesQueryCache, + IndexMetaData indexMetaData, + IndicesQueryCache indicesQueryCache, IndicesFieldDataCache indicesFieldDataCache, + Consumer globalCheckpointSyncer, List builtInListeners, IndexingOperationListener... indexingOperationListeners) throws IOException { final IndexSettings idxSettings = new IndexSettings(indexMetaData, this.settings, indexScopeSetting); @@ -454,7 +462,8 @@ private synchronized IndexService createIndexService(final String reason, indicesQueryCache, mapperRegistry, indicesFieldDataCache, - namedWriteableRegistry); + namedWriteableRegistry, + globalCheckpointSyncer); } /** @@ -485,7 +494,7 @@ public synchronized void verifyIndexMetadata(IndexMetaData metaData, IndexMetaDa closeables.add(indicesQueryCache); // this will also fail if some plugin fails etc. which is nice since we can verify that early final IndexService service = - createIndexService("metadata verification", metaData, indicesQueryCache, indicesFieldDataCache, emptyList()); + createIndexService("metadata verification", metaData, indicesQueryCache, indicesFieldDataCache, s -> {}, emptyList()); closeables.add(() -> service.close("metadata verification", false)); service.mapperService().merge(metaData, MapperService.MergeReason.MAPPING_RECOVERY, true); if (metaData.equals(metaDataUpdate) == false) { 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 3c1ee5b841293..81f8c16f4cf31 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -118,35 +118,44 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final boolean sendRefreshMapping; private final List buildInIndexListener; private final PrimaryReplicaSyncer primaryReplicaSyncer; + private final Consumer globalCheckpointSyncer; @Inject - public IndicesClusterStateService(Settings settings, IndicesService indicesService, ClusterService clusterService, - ThreadPool threadPool, PeerRecoveryTargetService recoveryTargetService, + public IndicesClusterStateService(Settings settings, + IndicesService indicesService, + ClusterService clusterService, + ThreadPool threadPool, + PeerRecoveryTargetService recoveryTargetService, ShardStateAction shardStateAction, NodeMappingRefreshAction nodeMappingRefreshAction, RepositoriesService repositoriesService, - SearchService searchService, SyncedFlushService syncedFlushService, - PeerRecoverySourceService peerRecoverySourceService, SnapshotShardsService snapshotShardsService, - GlobalCheckpointSyncAction globalCheckpointSyncAction, - PrimaryReplicaSyncer primaryReplicaSyncer) { + SearchService searchService, + SyncedFlushService syncedFlushService, + PeerRecoverySourceService peerRecoverySourceService, + SnapshotShardsService snapshotShardsService, + PrimaryReplicaSyncer primaryReplicaSyncer, + GlobalCheckpointSyncAction globalCheckpointSyncAction) { this(settings, (AllocatedIndices>) indicesService, clusterService, threadPool, recoveryTargetService, shardStateAction, nodeMappingRefreshAction, repositoriesService, searchService, syncedFlushService, peerRecoverySourceService, - snapshotShardsService, globalCheckpointSyncAction, primaryReplicaSyncer); + snapshotShardsService, primaryReplicaSyncer, globalCheckpointSyncAction::updateGlobalCheckpointForShard); } // for tests IndicesClusterStateService(Settings settings, AllocatedIndices> indicesService, ClusterService clusterService, - ThreadPool threadPool, PeerRecoveryTargetService recoveryTargetService, + ThreadPool threadPool, + PeerRecoveryTargetService recoveryTargetService, ShardStateAction shardStateAction, NodeMappingRefreshAction nodeMappingRefreshAction, RepositoriesService repositoriesService, - SearchService searchService, SyncedFlushService syncedFlushService, - PeerRecoverySourceService peerRecoverySourceService, SnapshotShardsService snapshotShardsService, - GlobalCheckpointSyncAction globalCheckpointSyncAction, - PrimaryReplicaSyncer primaryReplicaSyncer) { + SearchService searchService, + SyncedFlushService syncedFlushService, + PeerRecoverySourceService peerRecoverySourceService, + SnapshotShardsService snapshotShardsService, + PrimaryReplicaSyncer primaryReplicaSyncer, + Consumer globalCheckpointSyncer) { super(settings); this.buildInIndexListener = Arrays.asList( @@ -154,8 +163,7 @@ public IndicesClusterStateService(Settings settings, IndicesService indicesServi recoveryTargetService, searchService, syncedFlushService, - snapshotShardsService, - globalCheckpointSyncAction); + snapshotShardsService); this.indicesService = indicesService; this.clusterService = clusterService; this.threadPool = threadPool; @@ -164,6 +172,7 @@ public IndicesClusterStateService(Settings settings, IndicesService indicesServi this.nodeMappingRefreshAction = nodeMappingRefreshAction; this.repositoriesService = repositoriesService; this.primaryReplicaSyncer = primaryReplicaSyncer; + this.globalCheckpointSyncer = globalCheckpointSyncer; this.sendRefreshMapping = this.settings.getAsBoolean("indices.cluster.send_refresh_mapping", true); } @@ -442,7 +451,7 @@ private void createIndices(final ClusterState state) { AllocatedIndex indexService = null; try { - indexService = indicesService.createIndex(indexMetaData, buildInIndexListener); + indexService = indicesService.createIndex(indexMetaData, buildInIndexListener, globalCheckpointSyncer); if (indexService.updateMapping(indexMetaData) && sendRefreshMapping) { nodeMappingRefreshAction.nodeMappingRefresh(state.nodes().getMasterNode(), new NodeMappingRefreshAction.NodeMappingRefreshRequest(indexMetaData.getIndex().getName(), @@ -789,10 +798,12 @@ public interface AllocatedIndices> * @param indexMetaData the index metadata to create the index for * @param builtInIndexListener a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with * the per-index listeners + * @param globalCheckpointSyncer callback for syncing global checkpoints * @throws ResourceAlreadyExistsException if the index already exists. */ U createIndex(IndexMetaData indexMetaData, - List builtInIndexListener) throws IOException; + List builtInIndexListener, + Consumer globalCheckpointSyncer) throws IOException; /** * Verify that the contents on disk for the given index is deleted; if not, delete the contents. diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java index e590e12895591..15d23dfb130bf 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java @@ -380,6 +380,7 @@ static class TestPrimary implements ReplicationOperation.Primary clusterStateSupplier; final Map knownLocalCheckpoints = new HashMap<>(); + final Map knownGlobalCheckpoints = new HashMap<>(); TestPrimary(ShardRouting routing, Supplier clusterStateSupplier) { this.routing = routing; @@ -434,6 +435,11 @@ public void updateLocalCheckpointForShard(String allocationId, long checkpoint) knownLocalCheckpoints.put(allocationId, checkpoint); } + @Override + public void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint) { + knownGlobalCheckpoints.compute(allocationId, (k, v) -> v == null || globalCheckpoint > v ? globalCheckpoint : v); + } + @Override public long localCheckpoint() { return localCheckpoint; diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java index 9dc989961f32b..1db44840d11ba 100644 --- a/core/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java @@ -443,6 +443,6 @@ private void setupIndicesService() throws Exception { when(service.getIndexSortSupplier()).thenReturn(supplier); when(service.getIndexEventListener()).thenReturn(mock(IndexEventListener.class)); - when(indicesService.createIndex(anyObject(), anyObject())).thenReturn(service); + when(indicesService.createIndex(anyObject(), anyObject(), anyObject())).thenReturn(service); } } diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 4b9645a3af87d..69f8f4adc8d31 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -140,7 +140,7 @@ public void tearDown() throws Exception { private IndexService newIndexService(IndexModule module) throws IOException { return module.newIndexService(nodeEnvironment, xContentRegistry(), deleter, circuitBreakerService, bigArrays, threadPool, scriptService, null, indicesQueryCache, mapperRegistry, - new IndicesFieldDataCache(settings, listener), writableRegistry()); + new IndicesFieldDataCache(settings, listener), writableRegistry(), s -> {}); } public void testWrapperIsBound() throws IOException { diff --git a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java new file mode 100644 index 0000000000000..d227de5801230 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java @@ -0,0 +1,78 @@ +/* + * 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; + +import org.elasticsearch.action.admin.indices.stats.IndexShardStats; +import org.elasticsearch.action.admin.indices.stats.IndexStats; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.seqno.SeqNoStats; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.InternalSettingsPlugin; + +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; +import java.util.stream.Stream; + +import static org.elasticsearch.test.InternalSettingsPlugin.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING; +import static org.hamcrest.Matchers.equalTo; + +public class IndexServiceIT extends ESIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Collections.singletonList(InternalSettingsPlugin.class); + } + + public void testGlobalCheckpointSync() throws Exception { + internalCluster().startNode(); + prepareCreate("test", Settings.builder().put(GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms")).get(); + ensureGreen("test"); + final int numberOfDocuments = randomIntBetween(1, 128); + for (int i = 0; i < numberOfDocuments; i++) { + final String id = Integer.toString(i); + client().prepareIndex("test", "test", id).setSource("{\"foo\": " + id + "}", XContentType.JSON).get(); + } + final IndicesStatsResponse stats = client().admin().indices().prepareStats().clear().get(); + final IndexStats indexStats = stats.getIndex("test"); + assertBusy(() -> { + for (final IndexShardStats indexShardStats : indexStats.getIndexShards().values()) { + Optional maybePrimary = + Stream.of(indexShardStats.getShards()) + .filter(s -> s.getShardRouting().active() && s.getShardRouting().primary()) + .findFirst(); + if (!maybePrimary.isPresent()) { + continue; + } + final ShardStats primary = maybePrimary.get(); + final SeqNoStats primarySeqNoStats = primary.getSeqNoStats(); + for (final ShardStats shardStats : indexShardStats) { + final SeqNoStats seqNoStats = shardStats.getSeqNoStats(); + assertThat(seqNoStats.getGlobalCheckpoint(), equalTo(primarySeqNoStats.getGlobalCheckpoint())); + } + } + }); + } + +} 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 c38d3434c3b8f..691c157007463 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -141,7 +141,7 @@ protected class ReplicationGroup implements AutoCloseable, Iterable ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { final ShardRouting primaryRouting = this.createShardRouting("s0", true); - primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting)); + primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting), () -> {}); replicas = new ArrayList<>(); this.indexMetaData = indexMetaData; updateAllocationIDsOnPrimary(); @@ -238,7 +238,7 @@ public void startPrimary() throws IOException { public IndexShard addReplica() throws IOException { final ShardRouting replicaRouting = createShardRouting("s" + replicaId.incrementAndGet(), false); final IndexShard replica = - newShard(replicaRouting, indexMetaData, null, getEngineFactory(replicaRouting)); + newShard(replicaRouting, indexMetaData, null, getEngineFactory(replicaRouting), () -> {}); addReplica(replica); return replica; } @@ -259,8 +259,8 @@ public synchronized IndexShard addReplicaWithExistingPath(final ShardPath shardP false, ShardRoutingState.INITIALIZING, RecoverySource.PeerRecoverySource.INSTANCE); - final IndexShard newReplica = newShard(shardRouting, shardPath, indexMetaData, null, - getEngineFactory(shardRouting)); + final IndexShard newReplica = + newShard(shardRouting, shardPath, indexMetaData, null, getEngineFactory(shardRouting), () -> {}); replicas.add(newReplica); updateAllocationIDsOnPrimary(); return newReplica; @@ -481,6 +481,11 @@ public void updateLocalCheckpointForShard(String allocationId, long checkpoint) replicationGroup.getPrimary().updateLocalCheckpointForShard(allocationId, checkpoint); } + @Override + public void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint) { + replicationGroup.getPrimary().updateGlobalCheckpointForShard(allocationId, globalCheckpoint); + } + @Override public long localCheckpoint() { return replicationGroup.getPrimary().getLocalCheckpoint(); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index 2346ba290ae4e..ed5f31f4ff3da 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -539,7 +539,7 @@ public static final IndexShard newIndexShard(IndexService indexService, IndexSha IndexShard newShard = new IndexShard(initializingShardRouting, indexService.getIndexSettings(), shard.shardPath(), shard.store(), indexService.getIndexSortSupplier(), indexService.cache(), indexService.mapperService(), indexService.similarityService(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper, - indexService.getThreadPool(), indexService.getBigArrays(), null, Collections.emptyList(), Arrays.asList(listeners)); + indexService.getThreadPool(), indexService.getBigArrays(), null, Collections.emptyList(), Arrays.asList(listeners), () -> {}); return newShard; } 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 b275da702b083..1dc6090adc157 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -711,6 +711,68 @@ private void finish() { closeShards(indexShard); } + public void testGlobalCheckpointSync() throws IOException { + // create the primary shard with a callback that sets a boolean when the global checkpoint sync is invoked + final ShardId shardId = new ShardId("index", "_na_", 0); + final ShardRouting shardRouting = + TestShardRouting.newShardRouting( + shardId, + randomAlphaOfLength(8), + true, + ShardRoutingState.INITIALIZING, + RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE); + final Settings settings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 2) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + final IndexMetaData.Builder indexMetadata = IndexMetaData.builder(shardRouting.getIndexName()).settings(settings).primaryTerm(0, 1); + final AtomicBoolean synced = new AtomicBoolean(); + final IndexShard primaryShard = newShard(shardRouting, indexMetadata.build(), null, null, () -> { synced.set(true); }); + // add two replicas + recoverShardFromStore(primaryShard); + final IndexShard firstReplicaShard = newShard(shardId, false); + recoverReplica(firstReplicaShard, primaryShard); + final IndexShard secondReplicaShard = newShard(shardId, false); + recoverReplica(secondReplicaShard, primaryShard); + final long checkpoint = randomIntBetween(0, 128); + + // set up local checkpoints on the shard copies + primaryShard.updateLocalCheckpointForShard(shardRouting.allocationId().getId(), checkpoint); + final int firstReplicaLocalCheckpoint = randomIntBetween(0, Math.toIntExact(checkpoint)); + final String firstReplicaAllocationId = firstReplicaShard.routingEntry().allocationId().getId(); + primaryShard.updateLocalCheckpointForShard(firstReplicaAllocationId, firstReplicaLocalCheckpoint); + final int secondReplicaLocalCheckpoint = randomIntBetween(0, Math.toIntExact(checkpoint)); + final String secondReplicaAllocationId = secondReplicaShard.routingEntry().allocationId().getId(); + primaryShard.updateLocalCheckpointForShard(secondReplicaAllocationId, secondReplicaLocalCheckpoint); + + // initialize the local knowledge on the primary of the global checkpoint on the replica shards + final int firstReplicaGlobalCheckpoint = Math.toIntExact(primaryShard.getGlobalCheckpoint()); + primaryShard.updateGlobalCheckpointForShard( + firstReplicaAllocationId, + randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), firstReplicaGlobalCheckpoint)); + final int secondReplicaGlobalCheckpoint = Math.toIntExact(primaryShard.getGlobalCheckpoint()); + primaryShard.updateGlobalCheckpointForShard( + secondReplicaAllocationId, + randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), secondReplicaGlobalCheckpoint)); + + // simulate a background maybe sync; it should only run if one of the replica copies global checkpoint lags the primary + primaryShard.maybeSyncGlobalCheckpoint(); + assertThat(synced.get(), equalTo(firstReplicaGlobalCheckpoint < checkpoint || secondReplicaGlobalCheckpoint < checkpoint)); + + // simulate that the background sync advanced the global checkpoint on the replica copies + primaryShard.updateGlobalCheckpointForShard(firstReplicaAllocationId, primaryShard.getGlobalCheckpoint()); + primaryShard.updateGlobalCheckpointForShard(secondReplicaAllocationId, primaryShard.getGlobalCheckpoint()); + + // reset our boolean so that we can assert after another simulated maybe sync + synced.set(false); + + primaryShard.maybeSyncGlobalCheckpoint(); + + // this time there should not be a sync since all the replica copies are caught up with the primary + assertFalse(synced.get()); + } + public void testRestoreLocalCheckpointTrackerFromTranslogOnPromotion() throws IOException, InterruptedException { final IndexShard indexShard = newStartedShard(false); final int operations = 1024 - scaledRandomIntBetween(0, 1024); @@ -1678,7 +1740,7 @@ public IndexSearcher wrap(IndexSearcher searcher) throws EngineException { closeShards(shard); IndexShard newShard = newShard( ShardRoutingHelper.initWithSameId(shard.routingEntry(), RecoverySource.StoreRecoverySource.EXISTING_STORE_INSTANCE), - shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, null); + shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, null, () -> {}); recoverShardFromStore(newShard); @@ -1824,7 +1886,7 @@ public IndexSearcher wrap(IndexSearcher searcher) throws EngineException { closeShards(shard); IndexShard newShard = newShard( ShardRoutingHelper.initWithSameId(shard.routingEntry(), RecoverySource.StoreRecoverySource.EXISTING_STORE_INSTANCE), - shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, null); + shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, null, () -> {}); recoverShardFromStore(newShard); diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index 0dc760d63bfe7..db6cd91938d9a 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -121,7 +121,7 @@ public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRem }; indicesService.removeIndex(idx, DELETED, "simon says"); try { - IndexService index = indicesService.createIndex(metaData, Arrays.asList(countingListener)); + IndexService index = indicesService.createIndex(metaData, Arrays.asList(countingListener), s -> {}); assertEquals(3, counter.get()); idx = index.index(); ShardRouting newRouting = shardRouting; 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 208e7443c7daf..d5f860d05cdd6 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -190,7 +190,8 @@ protected class MockIndicesService implements AllocatedIndices buildInIndexListener) throws IOException { + List buildInIndexListener, + Consumer globalCheckpointSyncer) throws IOException { MockIndexService indexService = new MockIndexService(new IndexSettings(indexMetaData, Settings.EMPTY)); indices = newMapBuilder(indices).put(indexMetaData.getIndexUUID(), indexService).immutableMap(); return indexService; diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index b1725ead326d9..847e271925e23 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -95,6 +95,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyList; +import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -139,7 +140,7 @@ public ClusterStateChanges(NamedXContentRegistry xContentRegistry, ThreadPool th // MetaDataCreateIndexService creates indices using its IndicesService instance to check mappings -> fake it here try { @SuppressWarnings("unchecked") final List listeners = anyList(); - when(indicesService.createIndex(any(IndexMetaData.class), listeners)) + when(indicesService.createIndex(any(IndexMetaData.class), listeners, anyObject())) .then(invocationOnMock -> { IndexService indexService = mock(IndexService.class); IndexMetaData indexMetaData = (IndexMetaData)invocationOnMock.getArguments()[0]; diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index a356693213f35..bc5a5b95b958a 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -410,20 +410,20 @@ private IndicesClusterStateService createIndicesClusterStateService(DiscoveryNod final ShardStateAction shardStateAction = mock(ShardStateAction.class); final PrimaryReplicaSyncer primaryReplicaSyncer = mock(PrimaryReplicaSyncer.class); return new IndicesClusterStateService( - settings, - indicesService, - clusterService, - threadPool, - recoveryTargetService, - shardStateAction, - null, - repositoriesService, - null, - null, - null, - null, - null, - primaryReplicaSyncer); + settings, + indicesService, + clusterService, + threadPool, + recoveryTargetService, + shardStateAction, + null, + repositoriesService, + null, + null, + null, + null, + primaryReplicaSyncer, + s -> {}); } private class RecordingIndicesService extends MockIndicesService { diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 71fb310fa2cd5..6e9fec975292b 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -160,7 +160,9 @@ protected IndexShard newShard(boolean primary) throws IOException { * @param shardRouting the {@link ShardRouting} to use for this shard * @param listeners an optional set of listeners to add to the shard */ - protected IndexShard newShard(ShardRouting shardRouting, IndexingOperationListener... listeners) throws IOException { + protected IndexShard newShard( + final ShardRouting shardRouting, + final IndexingOperationListener... listeners) throws IOException { assert shardRouting.initializing() : shardRouting; Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) @@ -197,9 +199,7 @@ protected IndexShard newShard(ShardId shardId, boolean primary, IndexingOperatio */ protected IndexShard newShard(ShardId shardId, boolean primary, String nodeId, IndexMetaData indexMetaData, @Nullable IndexSearcherWrapper searcherWrapper) throws IOException { - ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING, - primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting, indexMetaData, searcherWrapper, null); + return newShard(shardId, primary, nodeId, indexMetaData, searcherWrapper, () -> {}); } /** @@ -211,11 +211,10 @@ protected IndexShard newShard(ShardId shardId, boolean primary, String nodeId, I * (ready to recover from another shard) */ protected IndexShard newShard(ShardId shardId, boolean primary, String nodeId, IndexMetaData indexMetaData, - Runnable globalCheckpointSyncer, - @Nullable IndexSearcherWrapper searcherWrapper) throws IOException { + @Nullable IndexSearcherWrapper searcherWrapper, Runnable globalCheckpointSyncer) throws IOException { ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING, primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting, indexMetaData, searcherWrapper, null); + return newShard(shardRouting, indexMetaData, searcherWrapper, null, globalCheckpointSyncer); } @@ -229,40 +228,45 @@ protected IndexShard newShard(ShardId shardId, boolean primary, String nodeId, I */ protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, IndexingOperationListener... listeners) throws IOException { - return newShard(routing, indexMetaData, null, null, listeners); + return newShard(routing, indexMetaData, null, null, () -> {}, listeners); } /** * creates a new initializing shard. The shard will will be put in its proper path under the * current node id the shard is assigned to. - * @param routing shard routing to use + * @param routing shard routing to use * @param indexMetaData indexMetaData for the shard, including any mapping * @param indexSearcherWrapper an optional wrapper to be used during searchers + * @param globalCheckpointSyncer callback for syncing global checkpoints * @param listeners an optional set of listeners to add to the shard */ protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, @Nullable IndexSearcherWrapper indexSearcherWrapper, @Nullable EngineFactory engineFactory, + Runnable globalCheckpointSyncer, IndexingOperationListener... listeners) throws IOException { // add node id as name to settings for proper logging final ShardId shardId = routing.shardId(); final NodeEnvironment.NodePath nodePath = new NodeEnvironment.NodePath(createTempDir()); ShardPath shardPath = new ShardPath(false, nodePath.resolve(shardId), nodePath.resolve(shardId), shardId); - return newShard(routing, shardPath, indexMetaData, indexSearcherWrapper, engineFactory, listeners); + return newShard(routing, shardPath, indexMetaData, indexSearcherWrapper, engineFactory, globalCheckpointSyncer, listeners); } /** * creates a new initializing shard. - * @param routing shard routing to use - * @param shardPath path to use for shard data - * @param indexMetaData indexMetaData for the shard, including any mapping - * @param indexSearcherWrapper an optional wrapper to be used during searchers - * @param listeners an optional set of listeners to add to the shard + * + * @param routing shard routing to use + * @param shardPath path to use for shard data + * @param indexMetaData indexMetaData for the shard, including any mapping + * @param indexSearcherWrapper an optional wrapper to be used during searchers + * @param globalCheckpointSyncer callback for syncing global checkpoints + * @param listeners an optional set of listeners to add to the shard */ protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMetaData indexMetaData, @Nullable IndexSearcherWrapper indexSearcherWrapper, @Nullable EngineFactory engineFactory, + Runnable globalCheckpointSyncer, IndexingOperationListener... listeners) throws IOException { final Settings nodeSettings = Settings.builder().put("node.name", routing.currentNodeId()).build(); final IndexSettings indexSettings = new IndexSettings(indexMetaData, nodeSettings); @@ -279,9 +283,9 @@ protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMe }; final Engine.Warmer warmer = searcher -> { }; - indexShard = new IndexShard(routing, indexSettings, shardPath, store, () ->null, indexCache, mapperService, similarityService, + indexShard = new IndexShard(routing, indexSettings, shardPath, store, () -> null, indexCache, mapperService, similarityService, engineFactory, indexEventListener, indexSearcherWrapper, threadPool, - BigArrays.NON_RECYCLING_INSTANCE, warmer, Collections.emptyList(), Arrays.asList(listeners)); + BigArrays.NON_RECYCLING_INSTANCE, warmer, Collections.emptyList(), Arrays.asList(listeners), globalCheckpointSyncer); success = true; } finally { if (success == false) { @@ -311,7 +315,13 @@ protected IndexShard reinitShard(IndexShard current, IndexingOperationListener.. */ protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexingOperationListener... listeners) throws IOException { closeShards(current); - return newShard(routing, current.shardPath(), current.indexSettings().getIndexMetaData(), null, current.engineFactory, listeners); + return newShard( + routing, + current.shardPath(), + current.indexSettings().getIndexMetaData(), + null, current.engineFactory, + current.getGlobalCheckpointSyncer(), + listeners); } /** diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java index 12920a5f1504e..545e86b8a8436 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java @@ -42,9 +42,22 @@ public final class InternalSettingsPlugin extends Plugin { Setting.timeSetting("index.translog.retention.check_interval", new TimeValue(10, TimeUnit.MINUTES), new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope); + public static final Setting GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING = + Setting.timeSetting( + "index.global_checkpoint_sync.interval", + new TimeValue(30, TimeUnit.SECONDS), + new TimeValue(0, TimeUnit.MILLISECONDS), + Property.Dynamic, + Property.IndexScope); + @Override public List> getSettings() { - return Arrays.asList(VERSION_CREATED, MERGE_ENABLED, - INDEX_CREATION_DATE_SETTING, PROVIDED_NAME_SETTING, TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING); + return Arrays.asList( + VERSION_CREATED, + MERGE_ENABLED, + INDEX_CREATION_DATE_SETTING, + PROVIDED_NAME_SETTING, + TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING, + GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING); } } From 511f96e39dea5ca97ac38ea301230afc6ed009cd Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 11 Sep 2017 18:57:18 -0400 Subject: [PATCH 02/38] Cleanup after test --- .../java/org/elasticsearch/index/shard/IndexShardTests.java | 2 ++ 1 file changed, 2 insertions(+) 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 1dc6090adc157..b0d0e470bc3b2 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -771,6 +771,8 @@ public void testGlobalCheckpointSync() throws IOException { // this time there should not be a sync since all the replica copies are caught up with the primary assertFalse(synced.get()); + + closeShards(secondReplicaShard, firstReplicaShard, primaryShard); } public void testRestoreLocalCheckpointTrackerFromTranslogOnPromotion() throws IOException, InterruptedException { From e1dc814a812c120d16c606eff2a7099f7d768ac2 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 12 Sep 2017 06:11:07 -0400 Subject: [PATCH 03/38] Recollect stats --- .../src/test/java/org/elasticsearch/index/IndexServiceIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java index d227de5801230..74ff005bfa4d8 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java @@ -54,9 +54,9 @@ public void testGlobalCheckpointSync() throws Exception { final String id = Integer.toString(i); client().prepareIndex("test", "test", id).setSource("{\"foo\": " + id + "}", XContentType.JSON).get(); } - final IndicesStatsResponse stats = client().admin().indices().prepareStats().clear().get(); - final IndexStats indexStats = stats.getIndex("test"); assertBusy(() -> { + final IndicesStatsResponse stats = client().admin().indices().prepareStats().clear().get(); + final IndexStats indexStats = stats.getIndex("test"); for (final IndexShardStats indexShardStats : indexStats.getIndexShards().values()) { Optional maybePrimary = Stream.of(indexShardStats.getShards()) From e0657a7e006eb84bde17047c6e01e8ac6074c0a6 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 19 Sep 2017 22:27:16 -0400 Subject: [PATCH 04/38] Add after-op sync --- .../TransportReplicationAction.java | 33 +++++++++++++++++-- .../replication/TransportWriteAction.java | 2 +- .../elasticsearch/index/shard/IndexShard.java | 20 ++++++----- 3 files changed, 44 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 516554d92e8cd..17d4def4f5148 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -108,12 +108,26 @@ public abstract class TransportReplicationAction< protected final String transportReplicaAction; protected final String transportPrimaryAction; + private final boolean syncGlobalCheckpointAfterOperation; + protected TransportReplicationAction(Settings settings, String actionName, TransportService transportService, ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, Supplier request, Supplier replicaRequest, String executor) { + this(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters, + indexNameExpressionResolver, request, replicaRequest, executor, false); + } + + + protected TransportReplicationAction(Settings settings, String actionName, TransportService transportService, + ClusterService clusterService, IndicesService indicesService, + ThreadPool threadPool, ShardStateAction shardStateAction, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, Supplier request, + Supplier replicaRequest, String executor, + boolean syncGlobalCheckpointAfterOperation) { super(settings, actionName, threadPool, actionFilters, indexNameExpressionResolver, transportService.getTaskManager()); this.transportService = transportService; this.clusterService = clusterService; @@ -126,6 +140,8 @@ protected TransportReplicationAction(Settings settings, String actionName, Trans registerRequestHandlers(actionName, transportService, request, replicaRequest, executor); this.transportOptions = transportOptions(); + + this.syncGlobalCheckpointAfterOperation = syncGlobalCheckpointAfterOperation; } protected void registerRequestHandlers(String actionName, TransportService transportService, Supplier request, @@ -150,7 +166,7 @@ protected void doExecute(Task task, Request request, ActionListener li new ReroutePhase((ReplicationTask) task, request, listener).run(); } - protected ReplicationOperation.Replicas newReplicasProxy(long primaryTerm) { + protected ReplicationOperation.Replicas newReplicasProxy(long primaryTerm) { return new ReplicasProxy(primaryTerm); } @@ -384,7 +400,20 @@ public void onFailure(Exception e) { protected ReplicationOperation> createReplicatedOperation( Request request, ActionListener> listener, PrimaryShardReference primaryShardReference) { - return new ReplicationOperation<>(request, primaryShardReference, listener, + return new ReplicationOperation<>( + request, + primaryShardReference, + ActionListener.wrap(response -> { + try { + if (syncGlobalCheckpointAfterOperation && + response.finalResponseIfSuccessful.getShardInfo().getFailed() == 0) { + threadPool.generic().execute(primaryShardReference.indexShard::maybeSyncGlobalCheckpoint); + } + } finally { + listener.onResponse(response); + } + }, + listener::onFailure), newReplicasProxy(primaryTerm), logger, actionName); } } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index 31c72108ecf65..ec3dcd94d3084 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -71,7 +71,7 @@ protected TransportWriteAction(Settings settings, String actionName, TransportSe ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, Supplier request, Supplier replicaRequest, String executor) { super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters, - indexNameExpressionResolver, request, replicaRequest, executor); + indexNameExpressionResolver, request, replicaRequest, executor, true); } /** Syncs operation result to the translog or throws a shard not available failure */ 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 abaa84dfeec79..ab28ec627bc80 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1777,14 +1777,18 @@ public ObjectLongMap getGlobalCheckpoints() { public void maybeSyncGlobalCheckpoint() { verifyPrimary(); verifyNotClosed(); - final String allocationId = routingEntry().allocationId().getId(); - final ObjectLongMap globalCheckpoints = getGlobalCheckpoints(); - assert globalCheckpoints.containsKey(allocationId); - final long globalCheckpoint = globalCheckpoints.get(allocationId); - final boolean syncNeeded = - StreamSupport.stream(globalCheckpoints.values().spliterator(), false).anyMatch(v -> v.value < globalCheckpoint); - if (syncNeeded) { - globalCheckpointSyncer.run(); + // only sync if there are not operations in flight + if (getEngine().seqNoService().getMaxSeqNo() == getEngine().seqNoService().getGlobalCheckpoint()) { + final ObjectLongMap globalCheckpoints = getGlobalCheckpoints(); + final String allocationId = routingEntry().allocationId().getId(); + assert globalCheckpoints.containsKey(allocationId); + final long globalCheckpoint = globalCheckpoints.get(allocationId); + final boolean syncNeeded = + StreamSupport.stream(globalCheckpoints.values().spliterator(), false).anyMatch(v -> v.value < globalCheckpoint); + // only sync if there is a shard lagging the primary + if (syncNeeded) { + globalCheckpointSyncer.run(); + } } } From f7a76cd81c9b03e529b58c920be33831dcf3fa47 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 19 Sep 2017 22:57:34 -0400 Subject: [PATCH 05/38] Iteration --- .../elasticsearch/index/shard/IndexShardTests.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) 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 b0d0e470bc3b2..7d3afad4dc5df 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -136,6 +136,7 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static java.util.Collections.max; import static org.elasticsearch.cluster.routing.TestShardRouting.newShardRouting; import static org.elasticsearch.common.lucene.Lucene.cleanLuceneIndex; import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; @@ -735,7 +736,11 @@ public void testGlobalCheckpointSync() throws IOException { recoverReplica(firstReplicaShard, primaryShard); final IndexShard secondReplicaShard = newShard(shardId, false); recoverReplica(secondReplicaShard, primaryShard); - final long checkpoint = randomIntBetween(0, 128); + final int maxSeqNo = randomIntBetween(0, 128); + for (int i = 0; i < maxSeqNo; i++) { + primaryShard.getEngine().seqNoService().generateSeqNo(); + } + final long checkpoint = rarely() ? maxSeqNo - scaledRandomIntBetween(0, maxSeqNo) : maxSeqNo; // set up local checkpoints on the shard copies primaryShard.updateLocalCheckpointForShard(shardRouting.allocationId().getId(), checkpoint); @@ -758,7 +763,10 @@ public void testGlobalCheckpointSync() throws IOException { // simulate a background maybe sync; it should only run if one of the replica copies global checkpoint lags the primary primaryShard.maybeSyncGlobalCheckpoint(); - assertThat(synced.get(), equalTo(firstReplicaGlobalCheckpoint < checkpoint || secondReplicaGlobalCheckpoint < checkpoint)); + assertThat( + synced.get(), + equalTo(maxSeqNo == primaryShard.getGlobalCheckpoint() && + (firstReplicaGlobalCheckpoint < checkpoint || secondReplicaGlobalCheckpoint < checkpoint))); // simulate that the background sync advanced the global checkpoint on the replica copies primaryShard.updateGlobalCheckpointForShard(firstReplicaAllocationId, primaryShard.getGlobalCheckpoint()); From 3881d766a241698949de3baff0a0a78faa4f9a7a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 16:58:21 -0400 Subject: [PATCH 06/38] Iteration --- .../TransportReplicationAction.java | 32 +++++++++----- .../org/elasticsearch/index/IndexService.java | 7 ++- .../seqno/GlobalCheckpointSyncAction.java | 2 +- .../elasticsearch/index/shard/IndexShard.java | 14 ++++-- .../index/shard/IndexShardTests.java | 43 +++++++------------ .../elasticsearch/recovery/RelocationIT.java | 5 --- .../elasticsearch/backwards/IndexingIT.java | 3 -- 7 files changed, 55 insertions(+), 51 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 17d4def4f5148..297ad82f17eb9 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -375,6 +375,13 @@ private ActionListener createResponseListener(final PrimaryShardRefere return new ActionListener() { @Override public void onResponse(Response response) { + if (syncGlobalCheckpointAfterOperation) { + try { + primaryShardReference.indexShard.maybeSyncGlobalCheckpoint("post-operation"); + } catch (final Exception e) { + logger.trace("ignored", e); + } + } primaryShardReference.close(); // release shard operation lock before responding to caller setPhase(replicationTask, "finished"); try { @@ -403,17 +410,20 @@ protected ReplicationOperation( request, primaryShardReference, - ActionListener.wrap(response -> { - try { - if (syncGlobalCheckpointAfterOperation && - response.finalResponseIfSuccessful.getShardInfo().getFailed() == 0) { - threadPool.generic().execute(primaryShardReference.indexShard::maybeSyncGlobalCheckpoint); - } - } finally { - listener.onResponse(response); - } - }, - listener::onFailure), + listener, +// ActionListener.wrap(response -> { +// try { +// System.out.println(Thread.currentThread().getName()); +// if (syncGlobalCheckpointAfterOperation && +// response.finalResponseIfSuccessful.getShardInfo().getFailed() == 0) { +// threadPool.generic().execute( +// () -> primaryShardReference.indexShard.maybeSyncGlobalCheckpoint("post-operation")); +// } +// } finally { +// listener.onResponse(response); +// } +// }, +// listener::onFailure), newReplicasProxy(primaryTerm), logger, actionName); } } diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 5cd21c2406db5..06d4b2082f900 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -25,6 +25,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.Accountable; import org.apache.lucene.util.IOUtils; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; @@ -736,7 +737,11 @@ private void syncGlobalCheckpoints() { case STARTED: case RELOCATED: try { - shard.maybeSyncGlobalCheckpoint(); + shard.acquirePrimaryOperationPermit( + ActionListener.wrap( + releasable -> shard.maybeSyncGlobalCheckpoint("background"), + e -> logger.info("failed to execute background global checkpoint sync", e)), + ThreadPool.Names.SAME); } catch (final AlreadyClosedException | IndexShardClosedException e) { // the shard was closed concurrently, continue } diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java index fc7fe4afbb110..0f3caa8a15b7f 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java @@ -73,7 +73,7 @@ public GlobalCheckpointSyncAction( indexNameExpressionResolver, Request::new, Request::new, - ThreadPool.Names.SAME); + ThreadPool.Names.MANAGEMENT); } public void updateGlobalCheckpointForShard(final ShardId shardId) { 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 ab28ec627bc80..18baa458435e3 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -21,6 +21,7 @@ import com.carrotsearch.hppc.ObjectLongMap; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CheckIndex; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexOptions; @@ -1774,19 +1775,26 @@ public ObjectLongMap getGlobalCheckpoints() { * Syncs the global checkpoint to the replicas if the global checkpoint on at least one replica is behind the global checkpoint on the * primary. */ - public void maybeSyncGlobalCheckpoint() { + public void maybeSyncGlobalCheckpoint(final String reason) { verifyPrimary(); verifyNotClosed(); + if (state == IndexShardState.RELOCATED) { + return; + } // only sync if there are not operations in flight - if (getEngine().seqNoService().getMaxSeqNo() == getEngine().seqNoService().getGlobalCheckpoint()) { + final SeqNoStats stats = getEngine().seqNoService().stats(); + if (stats.getMaxSeqNo() == stats.getGlobalCheckpoint()) { final ObjectLongMap globalCheckpoints = getGlobalCheckpoints(); final String allocationId = routingEntry().allocationId().getId(); assert globalCheckpoints.containsKey(allocationId); final long globalCheckpoint = globalCheckpoints.get(allocationId); final boolean syncNeeded = - StreamSupport.stream(globalCheckpoints.values().spliterator(), false).anyMatch(v -> v.value < globalCheckpoint); + StreamSupport + .stream(globalCheckpoints.values().spliterator(), false) + .anyMatch(v -> v.value < globalCheckpoint); // only sync if there is a shard lagging the primary if (syncNeeded) { + logger.trace("syncing global checkpoint for [{}]", reason); globalCheckpointSyncer.run(); } } 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 7d3afad4dc5df..c2864ea68b8e7 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -730,12 +730,10 @@ public void testGlobalCheckpointSync() throws IOException { final IndexMetaData.Builder indexMetadata = IndexMetaData.builder(shardRouting.getIndexName()).settings(settings).primaryTerm(0, 1); final AtomicBoolean synced = new AtomicBoolean(); final IndexShard primaryShard = newShard(shardRouting, indexMetadata.build(), null, null, () -> { synced.set(true); }); - // add two replicas + // add a replicas recoverShardFromStore(primaryShard); - final IndexShard firstReplicaShard = newShard(shardId, false); - recoverReplica(firstReplicaShard, primaryShard); - final IndexShard secondReplicaShard = newShard(shardId, false); - recoverReplica(secondReplicaShard, primaryShard); + final IndexShard replicaShard = newShard(shardId, false); + recoverReplica(replicaShard, primaryShard); final int maxSeqNo = randomIntBetween(0, 128); for (int i = 0; i < maxSeqNo; i++) { primaryShard.getEngine().seqNoService().generateSeqNo(); @@ -744,43 +742,34 @@ public void testGlobalCheckpointSync() throws IOException { // set up local checkpoints on the shard copies primaryShard.updateLocalCheckpointForShard(shardRouting.allocationId().getId(), checkpoint); - final int firstReplicaLocalCheckpoint = randomIntBetween(0, Math.toIntExact(checkpoint)); - final String firstReplicaAllocationId = firstReplicaShard.routingEntry().allocationId().getId(); - primaryShard.updateLocalCheckpointForShard(firstReplicaAllocationId, firstReplicaLocalCheckpoint); - final int secondReplicaLocalCheckpoint = randomIntBetween(0, Math.toIntExact(checkpoint)); - final String secondReplicaAllocationId = secondReplicaShard.routingEntry().allocationId().getId(); - primaryShard.updateLocalCheckpointForShard(secondReplicaAllocationId, secondReplicaLocalCheckpoint); + final int replicaLocalCheckpoint = randomIntBetween(0, Math.toIntExact(checkpoint)); + final String replicaAllocationId = replicaShard.routingEntry().allocationId().getId(); + primaryShard.updateLocalCheckpointForShard(replicaAllocationId, replicaLocalCheckpoint); // initialize the local knowledge on the primary of the global checkpoint on the replica shards - final int firstReplicaGlobalCheckpoint = Math.toIntExact(primaryShard.getGlobalCheckpoint()); + final int replicaGlobalCheckpoint = Math.toIntExact(primaryShard.getGlobalCheckpoint()); primaryShard.updateGlobalCheckpointForShard( - firstReplicaAllocationId, - randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), firstReplicaGlobalCheckpoint)); - final int secondReplicaGlobalCheckpoint = Math.toIntExact(primaryShard.getGlobalCheckpoint()); - primaryShard.updateGlobalCheckpointForShard( - secondReplicaAllocationId, - randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), secondReplicaGlobalCheckpoint)); + replicaAllocationId, + randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), replicaGlobalCheckpoint)); - // simulate a background maybe sync; it should only run if one of the replica copies global checkpoint lags the primary - primaryShard.maybeSyncGlobalCheckpoint(); + // simulate a background maybe sync; it should only run if the knowledge on the replica of the global checkpoint lags the primary + primaryShard.maybeSyncGlobalCheckpoint("test"); assertThat( synced.get(), - equalTo(maxSeqNo == primaryShard.getGlobalCheckpoint() && - (firstReplicaGlobalCheckpoint < checkpoint || secondReplicaGlobalCheckpoint < checkpoint))); + equalTo(maxSeqNo == primaryShard.getGlobalCheckpoint() && (replicaGlobalCheckpoint < checkpoint))); - // simulate that the background sync advanced the global checkpoint on the replica copies - primaryShard.updateGlobalCheckpointForShard(firstReplicaAllocationId, primaryShard.getGlobalCheckpoint()); - primaryShard.updateGlobalCheckpointForShard(secondReplicaAllocationId, primaryShard.getGlobalCheckpoint()); + // simulate that the background sync advanced the global checkpoint on the replica + primaryShard.updateGlobalCheckpointForShard(replicaAllocationId, primaryShard.getGlobalCheckpoint()); // reset our boolean so that we can assert after another simulated maybe sync synced.set(false); - primaryShard.maybeSyncGlobalCheckpoint(); + primaryShard.maybeSyncGlobalCheckpoint("test"); // this time there should not be a sync since all the replica copies are caught up with the primary assertFalse(synced.get()); - closeShards(secondReplicaShard, firstReplicaShard, primaryShard); + closeShards(replicaShard, primaryShard); } public void testRestoreLocalCheckpointTrackerFromTranslogOnPromotion() throws IOException, InterruptedException { diff --git a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java index 04e1a846bd64f..adb00d981bd8a 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -383,9 +383,6 @@ public void indexShardStateChanged(IndexShard indexShard, @Nullable IndexShardSt } - // 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(); - } public void testCancellationCleansTempFiles() throws Exception { @@ -526,8 +523,6 @@ public void testIndexAndRelocateConcurrently() throws ExecutionException, Interr assertSearchHits(afterRelocation, ids.toArray(new String[ids.size()])); } - // 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(); } class RecoveryCorruption extends MockTransportService.DelegateTransport { diff --git a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java index 28594301949df..c158423dfdf66 100644 --- a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java +++ b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java @@ -213,7 +213,6 @@ public void testSeqNoCheckpoints() throws Exception { final int numberOfInitialDocs = 1 + randomInt(5); logger.info("indexing [{}] docs initially", numberOfInitialDocs); numDocs += indexDocs(index, numDocs, numberOfInitialDocs); - assertOK(client().performRequest("POST", index + "/_refresh")); // this forces a global checkpoint sync assertSeqNoOnShards(index, nodes, numDocs, newNodeClient); logger.info("allowing shards on all nodes"); updateIndexSetting(index, Settings.builder().putNull("index.routing.allocation.include._name")); @@ -224,7 +223,6 @@ public void testSeqNoCheckpoints() throws Exception { final int numberOfDocsAfterAllowingShardsOnAllNodes = 1 + randomInt(5); logger.info("indexing [{}] docs after allowing shards on all nodes", numberOfDocsAfterAllowingShardsOnAllNodes); numDocs += indexDocs(index, numDocs, numberOfDocsAfterAllowingShardsOnAllNodes); - assertOK(client().performRequest("POST", index + "/_refresh")); // this forces a global checkpoint sync assertSeqNoOnShards(index, nodes, numDocs, newNodeClient); Shard primary = buildShards(index, nodes, newNodeClient).stream().filter(Shard::isPrimary).findFirst().get(); logger.info("moving primary to new node by excluding {}", primary.getNode().getNodeName()); @@ -235,7 +233,6 @@ public void testSeqNoCheckpoints() throws Exception { logger.info("indexing [{}] docs after moving primary", numberOfDocsAfterMovingPrimary); numDocsOnNewPrimary += indexDocs(index, numDocs, numberOfDocsAfterMovingPrimary); numDocs += numberOfDocsAfterMovingPrimary; - assertOK(client().performRequest("POST", index + "/_refresh")); // this forces a global checkpoint sync assertSeqNoOnShards(index, nodes, numDocs, newNodeClient); /* * Dropping the number of replicas to zero, and then increasing it to one triggers a recovery thus exercising any BWC-logic in From 89bbf845d59785ded1a6aa389a6abd33ac1329a8 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 16:58:38 -0400 Subject: [PATCH 07/38] Remove comments --- .../replication/TransportReplicationAction.java | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 297ad82f17eb9..171382b860c9f 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -411,19 +411,6 @@ protected ReplicationOperation { -// try { -// System.out.println(Thread.currentThread().getName()); -// if (syncGlobalCheckpointAfterOperation && -// response.finalResponseIfSuccessful.getShardInfo().getFailed() == 0) { -// threadPool.generic().execute( -// () -> primaryShardReference.indexShard.maybeSyncGlobalCheckpoint("post-operation")); -// } -// } finally { -// listener.onResponse(response); -// } -// }, -// listener::onFailure), newReplicasProxy(primaryTerm), logger, actionName); } } From 1082a31ec10d71e6e24eea2d31cda9c5140a1dd2 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 17:15:34 -0400 Subject: [PATCH 08/38] Close it --- .../main/java/org/elasticsearch/index/IndexService.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 06d4b2082f900..2f9a28e687dfe 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; @@ -739,7 +740,11 @@ private void syncGlobalCheckpoints() { try { shard.acquirePrimaryOperationPermit( ActionListener.wrap( - releasable -> shard.maybeSyncGlobalCheckpoint("background"), + releasable -> { + try (Releasable ignored = releasable) { + shard.maybeSyncGlobalCheckpoint("background"); + } + }, e -> logger.info("failed to execute background global checkpoint sync", e)), ThreadPool.Names.SAME); } catch (final AlreadyClosedException | IndexShardClosedException e) { From aa0c62cc61557f9661fdb9b5ecef6293d175201e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 18:02:24 -0400 Subject: [PATCH 09/38] refresh needed --- .../src/test/java/org/elasticsearch/backwards/IndexingIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java index c158423dfdf66..b5076868f04f0 100644 --- a/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java +++ b/qa/mixed-cluster/src/test/java/org/elasticsearch/backwards/IndexingIT.java @@ -217,6 +217,7 @@ public void testSeqNoCheckpoints() throws Exception { logger.info("allowing shards on all nodes"); updateIndexSetting(index, Settings.builder().putNull("index.routing.allocation.include._name")); ensureGreen(); + assertOK(client().performRequest("POST", index + "/_refresh")); for (final String bwcName : bwcNamesList) { assertCount(index, "_only_nodes:" + bwcName, numDocs); } From 721f7252ae4c6e7a09a93b2dba17d78bbc911cb6 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 18:21:19 -0400 Subject: [PATCH 10/38] remove ensure green --- core/src/test/java/org/elasticsearch/index/IndexServiceIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java index 74ff005bfa4d8..f6433391adf8b 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java @@ -48,7 +48,6 @@ protected Collection> nodePlugins() { public void testGlobalCheckpointSync() throws Exception { internalCluster().startNode(); prepareCreate("test", Settings.builder().put(GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms")).get(); - ensureGreen("test"); final int numberOfDocuments = randomIntBetween(1, 128); for (int i = 0; i < numberOfDocuments; i++) { final String id = Integer.toString(i); From 9bc5155e3e79888717f9dfe72b7c032c87856fe6 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 18:24:05 -0400 Subject: [PATCH 11/38] remove background sync test setting --- .../java/org/elasticsearch/index/IndexService.java | 12 +----------- .../java/org/elasticsearch/index/IndexServiceIT.java | 3 --- .../elasticsearch/test/InternalSettingsPlugin.java | 11 +---------- 3 files changed, 2 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 2f9a28e687dfe..3791819100642 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -933,13 +933,7 @@ public String toString() { final class AsyncGlobalCheckpointTask extends BaseAsyncTask { AsyncGlobalCheckpointTask(final IndexService indexService) { - // index.global_checkpoint_sync_interval is not a real setting, it is only registered in tests - super( - indexService, - indexService - .getIndexSettings() - .getSettings() - .getAsTime("index.global_checkpoint_sync.interval", GLOBAL_CHECKPOINT_SYNC_INTERVAL)); + super(indexService, GLOBAL_CHECKPOINT_SYNC_INTERVAL); } @Override @@ -966,10 +960,6 @@ AsyncTranslogFSync getFsyncTask() { // for tests return fsyncTask; } - AsyncGlobalCheckpointTask getGlobalCheckpointTask() { - return globalCheckpointTask; - } - /** * Clears the caches for the given shard id if the shard is still allocated on this node */ diff --git a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java index f6433391adf8b..4f6907268180f 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.plugins.Plugin; @@ -35,7 +34,6 @@ import java.util.Optional; import java.util.stream.Stream; -import static org.elasticsearch.test.InternalSettingsPlugin.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING; import static org.hamcrest.Matchers.equalTo; public class IndexServiceIT extends ESIntegTestCase { @@ -47,7 +45,6 @@ protected Collection> nodePlugins() { public void testGlobalCheckpointSync() throws Exception { internalCluster().startNode(); - prepareCreate("test", Settings.builder().put(GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms")).get(); final int numberOfDocuments = randomIntBetween(1, 128); for (int i = 0; i < numberOfDocuments; i++) { final String id = Integer.toString(i); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java index 545e86b8a8436..06285849d0f9b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java @@ -42,14 +42,6 @@ public final class InternalSettingsPlugin extends Plugin { Setting.timeSetting("index.translog.retention.check_interval", new TimeValue(10, TimeUnit.MINUTES), new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope); - public static final Setting GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING = - Setting.timeSetting( - "index.global_checkpoint_sync.interval", - new TimeValue(30, TimeUnit.SECONDS), - new TimeValue(0, TimeUnit.MILLISECONDS), - Property.Dynamic, - Property.IndexScope); - @Override public List> getSettings() { return Arrays.asList( @@ -57,7 +49,6 @@ public List> getSettings() { MERGE_ENABLED, INDEX_CREATION_DATE_SETTING, PROVIDED_NAME_SETTING, - TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING, - GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING); + TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING); } } From 63e9d80a5b2194343dcb5541513e7579dcd13c6e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 20:17:23 -0400 Subject: [PATCH 12/38] Revert "remove background sync test setting" This reverts commit 9bc5155e3e79888717f9dfe72b7c032c87856fe6. --- .../java/org/elasticsearch/index/IndexService.java | 12 +++++++++++- .../java/org/elasticsearch/index/IndexServiceIT.java | 3 +++ .../elasticsearch/test/InternalSettingsPlugin.java | 11 ++++++++++- 3 files changed, 24 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 3791819100642..2f9a28e687dfe 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -933,7 +933,13 @@ public String toString() { final class AsyncGlobalCheckpointTask extends BaseAsyncTask { AsyncGlobalCheckpointTask(final IndexService indexService) { - super(indexService, GLOBAL_CHECKPOINT_SYNC_INTERVAL); + // index.global_checkpoint_sync_interval is not a real setting, it is only registered in tests + super( + indexService, + indexService + .getIndexSettings() + .getSettings() + .getAsTime("index.global_checkpoint_sync.interval", GLOBAL_CHECKPOINT_SYNC_INTERVAL)); } @Override @@ -960,6 +966,10 @@ AsyncTranslogFSync getFsyncTask() { // for tests return fsyncTask; } + AsyncGlobalCheckpointTask getGlobalCheckpointTask() { + return globalCheckpointTask; + } + /** * Clears the caches for the given shard id if the shard is still allocated on this node */ diff --git a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java index 4f6907268180f..f6433391adf8b 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.plugins.Plugin; @@ -34,6 +35,7 @@ import java.util.Optional; import java.util.stream.Stream; +import static org.elasticsearch.test.InternalSettingsPlugin.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING; import static org.hamcrest.Matchers.equalTo; public class IndexServiceIT extends ESIntegTestCase { @@ -45,6 +47,7 @@ protected Collection> nodePlugins() { public void testGlobalCheckpointSync() throws Exception { internalCluster().startNode(); + prepareCreate("test", Settings.builder().put(GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms")).get(); final int numberOfDocuments = randomIntBetween(1, 128); for (int i = 0; i < numberOfDocuments; i++) { final String id = Integer.toString(i); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java index 06285849d0f9b..545e86b8a8436 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java @@ -42,6 +42,14 @@ public final class InternalSettingsPlugin extends Plugin { Setting.timeSetting("index.translog.retention.check_interval", new TimeValue(10, TimeUnit.MINUTES), new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope); + public static final Setting GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING = + Setting.timeSetting( + "index.global_checkpoint_sync.interval", + new TimeValue(30, TimeUnit.SECONDS), + new TimeValue(0, TimeUnit.MILLISECONDS), + Property.Dynamic, + Property.IndexScope); + @Override public List> getSettings() { return Arrays.asList( @@ -49,6 +57,7 @@ public List> getSettings() { MERGE_ENABLED, INDEX_CREATION_DATE_SETTING, PROVIDED_NAME_SETTING, - TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING); + TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING, + GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING); } } From 6ade72085c4f12cb11f2c60c959bc01903b93d47 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 20:27:19 -0400 Subject: [PATCH 13/38] test iteratoin --- .../elasticsearch/index/IndexServiceIT.java | 3 -- .../elasticsearch/recovery/RelocationIT.java | 28 +++++++++++-------- 2 files changed, 17 insertions(+), 14 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java index f6433391adf8b..4f6907268180f 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.plugins.Plugin; @@ -35,7 +34,6 @@ import java.util.Optional; import java.util.stream.Stream; -import static org.elasticsearch.test.InternalSettingsPlugin.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING; import static org.hamcrest.Matchers.equalTo; public class IndexServiceIT extends ESIntegTestCase { @@ -47,7 +45,6 @@ protected Collection> nodePlugins() { public void testGlobalCheckpointSync() throws Exception { internalCluster().startNode(); - prepareCreate("test", Settings.builder().put(GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms")).get(); final int numberOfDocuments = randomIntBetween(1, 128); for (int i = 0; i < numberOfDocuments; i++) { final String id = Integer.toString(i); diff --git a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java index adb00d981bd8a..3fd5df554ad20 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -63,6 +63,7 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; +import org.elasticsearch.test.InternalSettingsPlugin; import org.elasticsearch.test.MockIndexEventListener; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; @@ -89,6 +90,7 @@ import java.util.stream.Stream; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; +import static org.elasticsearch.test.InternalSettingsPlugin.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; @@ -104,7 +106,7 @@ public class RelocationIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class, MockIndexEventListener.TestPlugin.class); + return Arrays.asList(InternalSettingsPlugin.class, MockTransportService.TestPlugin.class, MockIndexEventListener.TestPlugin.class); } @Override @@ -301,11 +303,14 @@ public void testRelocationWhileRefreshing() throws Exception { nodes[0] = internalCluster().startNode(); logger.info("--> creating test index ..."); - prepareCreate("test", Settings.builder() - .put("index.number_of_shards", 1) - .put("index.number_of_replicas", numberOfReplicas) - .put("index.refresh_interval", -1) // we want to control refreshes c - ).get(); + prepareCreate( + "test", + Settings.builder() + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", numberOfReplicas) + .put("index.refresh_interval", -1) // we want to control refreshes c + .put(GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms")) + .get(); for (int i = 1; i < numberOfNodes; i++) { logger.info("--> starting [node_{}] ...", i); @@ -478,11 +483,12 @@ public void testIndexAndRelocateConcurrently() throws ExecutionException, Interr logger.info("red nodes: {}", (Object)redNodes); ensureStableCluster(halfNodes * 2); - assertAcked(prepareCreate("test", Settings.builder() - .put("index.routing.allocation.exclude.color", "blue") - .put(indexSettings()) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(halfNodes - 1)) - )); + final Settings.Builder settings = Settings.builder() + .put("index.routing.allocation.exclude.color", "blue") + .put(indexSettings()) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(halfNodes - 1)) + .put(GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms"); + assertAcked(prepareCreate("test", settings)); assertAllShardsOnNodes("test", redNodes); int numDocs = randomIntBetween(100, 150); ArrayList ids = new ArrayList<>(); From 354df1ccbe4fd10fcbdbda668c1b428eed86373f Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 20:27:56 -0400 Subject: [PATCH 14/38] disable bwc tests They can not work right not until this is backported; a primary running 6.x code without this patch will not be sending the global checkpoint sync yet. --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index cfc8401a934e0..7b1e517a8586b 100644 --- a/build.gradle +++ b/build.gradle @@ -186,7 +186,7 @@ task verifyVersions { * after the backport of the backcompat code is complete. */ allprojects { - ext.bwc_tests_enabled = true + ext.bwc_tests_enabled = false } task verifyBwcTestsEnabled { From 79742eac434bf3adb86b277b53bba5bed1627a9e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 20 Sep 2017 20:33:59 -0400 Subject: [PATCH 15/38] imports --- .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 2 -- 1 file changed, 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 18baa458435e3..56c2d847723b9 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -21,7 +21,6 @@ import com.carrotsearch.hppc.ObjectLongMap; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CheckIndex; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexOptions; @@ -104,7 +103,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.GlobalCheckpointSyncAction; import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; From d9fc19d02b3f6485d61a5771e4762aa2fe5c50ef Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 05:57:19 -0400 Subject: [PATCH 16/38] Formatting of method --- .../support/replication/TransportReplicationAction.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 171382b860c9f..fc3d60743ee88 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -405,12 +405,9 @@ public void onFailure(Exception e) { } protected ReplicationOperation> createReplicatedOperation( - Request request, ActionListener> listener, - PrimaryShardReference primaryShardReference) { - return new ReplicationOperation<>( - request, - primaryShardReference, - listener, + Request request, ActionListener> listener, + PrimaryShardReference primaryShardReference) { + return new ReplicationOperation<>(request, primaryShardReference, listener, newReplicasProxy(primaryTerm), logger, actionName); } } From afb082fc88d926908fd49681f6d306d2ffa68c9c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 05:58:21 -0400 Subject: [PATCH 17/38] Remove leftover code --- .../org/elasticsearch/index/seqno/GlobalCheckpointTracker.java | 3 --- .../org/elasticsearch/index/seqno/SequenceNumbersService.java | 1 - 2 files changed, 4 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 6ffadf9beb26e..34c731edaaf8b 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -618,8 +618,6 @@ public synchronized void updateLocalCheckpoint(final String allocationId, final assert invariant(); } - private ObjectLongMap globalCheckpoints = new ObjectLongHashMap<>(); - /** * Computes the global checkpoint based on the given local checkpoints. In case where there are entries preventing the * computation to happen (for example due to blocking), it returns the fallback value. @@ -719,7 +717,6 @@ public synchronized void completeRelocationHandoff() { } } }); - globalCheckpoints.clear(); assert invariant(); } 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 6e2b2ac46924e..6f828fdc05124 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -21,7 +21,6 @@ import com.carrotsearch.hppc.ObjectLongMap; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.ReplicationGroup; From 9bf875e86115c72f5cbd2d03e835ded9a04f9ff9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 06:02:50 -0400 Subject: [PATCH 18/38] State handling --- core/src/main/java/org/elasticsearch/index/IndexService.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 2f9a28e687dfe..c0eca65d5f27f 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -730,13 +730,14 @@ private void syncGlobalCheckpoints() { for (final IndexShard shard : this.shards.values()) { if (shard.routingEntry().active() && shard.routingEntry().primary()) { switch (shard.state()) { + case CLOSED: case CREATED: case RECOVERING: - case CLOSED: + case RELOCATED: continue; case POST_RECOVERY: + assert false : "shard " + shard.shardId() + " is in post-recovery but marked as active"; case STARTED: - case RELOCATED: try { shard.acquirePrimaryOperationPermit( ActionListener.wrap( From 007d5c425dfdafd03410227c39fab1fa995985a4 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 06:05:12 -0400 Subject: [PATCH 19/38] Logging on failed sync --- .../action/support/replication/TransportReplicationAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index fc3d60743ee88..d15c7f1c41b9d 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -379,7 +379,8 @@ public void onResponse(Response response) { try { primaryShardReference.indexShard.maybeSyncGlobalCheckpoint("post-operation"); } catch (final Exception e) { - logger.trace("ignored", e); + logger.info("post-operation global checkpoint sync failed", e); + // intentionally swallow, a missed global checkpoint sync should not fail this operation } } primaryShardReference.close(); // release shard operation lock before responding to caller From cb9373bfaf746a046ae2b5e2393a9a06b9685935 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 06:12:33 -0400 Subject: [PATCH 20/38] no fallthrough --- core/src/main/java/org/elasticsearch/index/IndexService.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index c0eca65d5f27f..7d98c9ae101cf 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -737,6 +737,7 @@ private void syncGlobalCheckpoints() { continue; case POST_RECOVERY: assert false : "shard " + shard.shardId() + " is in post-recovery but marked as active"; + continue; case STARTED: try { shard.acquirePrimaryOperationPermit( From 7e6d1bf46c266c7ea586a4ec2f2c54a503a708e8 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 06:19:42 -0400 Subject: [PATCH 21/38] Setting --- .../org/elasticsearch/index/IndexService.java | 18 +++++++++++------- .../elasticsearch/index/IndexServiceIT.java | 5 ----- .../elasticsearch/recovery/RelocationIT.java | 5 ++--- .../test/InternalSettingsPlugin.java | 11 ++--------- 4 files changed, 15 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 7d98c9ae101cf..68145d3e0cae2 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -32,6 +32,8 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; @@ -927,7 +929,14 @@ public String toString() { } } - private static final TimeValue GLOBAL_CHECKPOINT_SYNC_INTERVAL = TimeValue.timeValueSeconds(30); + // this setting is intentionally not registered, it is only used in tests + public static final Setting GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING = + Setting.timeSetting( + "index.global_checkpoint_sync.interval", + new TimeValue(30, TimeUnit.SECONDS), + new TimeValue(0, TimeUnit.MILLISECONDS), + Property.Dynamic, + Property.IndexScope); /** * Background task that syncs the global checkpoint to replicas. @@ -936,12 +945,7 @@ final class AsyncGlobalCheckpointTask extends BaseAsyncTask { AsyncGlobalCheckpointTask(final IndexService indexService) { // index.global_checkpoint_sync_interval is not a real setting, it is only registered in tests - super( - indexService, - indexService - .getIndexSettings() - .getSettings() - .getAsTime("index.global_checkpoint_sync.interval", GLOBAL_CHECKPOINT_SYNC_INTERVAL)); + super(indexService, GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.get(indexService.getIndexSettings().getSettings())); } @Override diff --git a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java index 4f6907268180f..344c194049a63 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java @@ -38,11 +38,6 @@ public class IndexServiceIT extends ESIntegTestCase { - @Override - protected Collection> nodePlugins() { - return Collections.singletonList(InternalSettingsPlugin.class); - } - public void testGlobalCheckpointSync() throws Exception { internalCluster().startNode(); final int numberOfDocuments = randomIntBetween(1, 128); diff --git a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java index 3fd5df554ad20..51822c2030a49 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -90,7 +90,6 @@ import java.util.stream.Stream; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; -import static org.elasticsearch.test.InternalSettingsPlugin.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; @@ -309,7 +308,7 @@ public void testRelocationWhileRefreshing() throws Exception { .put("index.number_of_shards", 1) .put("index.number_of_replicas", numberOfReplicas) .put("index.refresh_interval", -1) // we want to control refreshes c - .put(GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms")) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms")) .get(); for (int i = 1; i < numberOfNodes; i++) { @@ -487,7 +486,7 @@ public void testIndexAndRelocateConcurrently() throws ExecutionException, Interr .put("index.routing.allocation.exclude.color", "blue") .put(indexSettings()) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(halfNodes - 1)) - .put(GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms"); + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms"); assertAcked(prepareCreate("test", settings)); assertAllShardsOnNodes("test", redNodes); int numDocs = randomIntBetween(100, 150); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java index 545e86b8a8436..e1c555b811064 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.IndexService; import org.elasticsearch.plugins.Plugin; import java.util.Arrays; @@ -42,14 +43,6 @@ public final class InternalSettingsPlugin extends Plugin { Setting.timeSetting("index.translog.retention.check_interval", new TimeValue(10, TimeUnit.MINUTES), new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope); - public static final Setting GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING = - Setting.timeSetting( - "index.global_checkpoint_sync.interval", - new TimeValue(30, TimeUnit.SECONDS), - new TimeValue(0, TimeUnit.MILLISECONDS), - Property.Dynamic, - Property.IndexScope); - @Override public List> getSettings() { return Arrays.asList( @@ -58,6 +51,6 @@ public List> getSettings() { INDEX_CREATION_DATE_SETTING, PROVIDED_NAME_SETTING, TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING, - GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING); + IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING); } } From f7295acf139e312ddb988c54cb60128aa4ba21b4 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 06:22:47 -0400 Subject: [PATCH 22/38] Critical fix --- .../java/org/elasticsearch/index/shard/IndexShardTestCase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 6e9fec975292b..d463fdbd17bdd 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -319,7 +319,8 @@ protected IndexShard reinitShard(IndexShard current, ShardRouting routing, Index routing, current.shardPath(), current.indexSettings().getIndexMetaData(), - null, current.engineFactory, + null, + current.engineFactory, current.getGlobalCheckpointSyncer(), listeners); } From f237d884e54ca3cfd10a88a8ea971b8934c3068f Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 06:23:33 -0400 Subject: [PATCH 23/38] fix comment --- core/src/test/java/org/elasticsearch/recovery/RelocationIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java index 51822c2030a49..23ae9cef100d9 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -307,7 +307,7 @@ public void testRelocationWhileRefreshing() throws Exception { Settings.builder() .put("index.number_of_shards", 1) .put("index.number_of_replicas", numberOfReplicas) - .put("index.refresh_interval", -1) // we want to control refreshes c + .put("index.refresh_interval", -1) // we want to control refreshes .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms")) .get(); From e88b92cf0538dd459b5b3d2f4f258e28214ec3cc Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 06:28:03 -0400 Subject: [PATCH 24/38] Revert move, add javadocs --- .../index/seqno/GlobalCheckpointTracker.java | 5 +++++ .../index/seqno/SequenceNumbersService.java | 13 +++++++++---- .../org/elasticsearch/index/shard/IndexShard.java | 5 +++++ 3 files changed, 19 insertions(+), 4 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 34c731edaaf8b..4e7cd4a55d5fe 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -209,6 +209,11 @@ public int hashCode() { } } + /** + * Get the local knowledge of the global checkpoints for all tracked allocation IDs. + * + * @return a map from allocation ID to the local knowledge of the global checkpoint for that allocation ID + */ synchronized ObjectLongMap getGlobalCheckpoints() { assert primaryMode; assert handoffInProgress == 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 6f828fdc05124..7386936c44c0f 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -138,6 +138,15 @@ public void updateGlobalCheckpointForShard(final String allocationId, final long globalCheckpointTracker.updateGlobalCheckpointForShard(allocationId, globalCheckpoint); } + /** + * Get the local knowledge of the global checkpoints for all tracked allocation IDs. + * + * @return a map from allocation ID to the local knowledge of the global checkpoint for that allocation ID + */ + public ObjectLongMap getGlobalCheckpoints() { + return globalCheckpointTracker.getGlobalCheckpoints(); + } + /** * Called when the recovery process for a shard is ready to open the engine on the target shard. * See {@link GlobalCheckpointTracker#initiateTracking(String)} for details. @@ -203,10 +212,6 @@ public synchronized long getTrackedLocalCheckpointForShard(final String allocati return globalCheckpointTracker.getTrackedLocalCheckpointForShard(allocationId).getLocalCheckpoint(); } - public ObjectLongMap getGlobalCheckpoints() { - return globalCheckpointTracker.getGlobalCheckpoints(); - } - /** * Activates the global checkpoint tracker in primary mode (see {@link GlobalCheckpointTracker#primaryMode}. * Called on primary activation or promotion. 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 56c2d847723b9..00adff68de2a7 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1763,6 +1763,11 @@ public long getGlobalCheckpoint() { return getEngine().seqNoService().getGlobalCheckpoint(); } + /** + * Get the local knowledge of the global checkpoints for all tracked allocation IDs. + * + * @return a map from allocation ID to the local knowledge of the global checkpoint for that allocation ID + */ public ObjectLongMap getGlobalCheckpoints() { verifyPrimary(); verifyNotClosed(); From 030156d1315fdb7a39edfd557b6910f42471b109 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 06:32:11 -0400 Subject: [PATCH 25/38] Handle execute future --- .../elasticsearch/index/seqno/GlobalCheckpointSyncAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java index 0f3caa8a15b7f..49dba5f5159a1 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.seqno; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.ReplicationOperation; @@ -77,7 +78,7 @@ public GlobalCheckpointSyncAction( } public void updateGlobalCheckpointForShard(final ShardId shardId) { - execute(new Request(shardId)); + execute(new Request(shardId), ActionListener.wrap(r -> {}, e -> logger.info(shardId + " global checkpoint sync failed", e))); } @Override From 03433726b4f26234f56e1589d4e4f29756c23ed7 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 07:06:42 -0400 Subject: [PATCH 26/38] Only check in-sync shards --- .../index/seqno/GlobalCheckpointTracker.java | 14 ++++++++------ .../index/seqno/SequenceNumbersService.java | 6 +++--- .../org/elasticsearch/index/shard/IndexShard.java | 8 ++++---- .../org/elasticsearch/recovery/RelocationIT.java | 2 +- 4 files changed, 16 insertions(+), 14 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 4e7cd4a55d5fe..d2b53aac1a045 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -210,17 +210,19 @@ public int hashCode() { } /** - * Get the local knowledge of the global checkpoints for all tracked allocation IDs. + * Get the local knowledge of the global checkpoints for all in-sync allocation IDs. * * @return a map from allocation ID to the local knowledge of the global checkpoint for that allocation ID */ - synchronized ObjectLongMap getGlobalCheckpoints() { + synchronized ObjectLongMap getInSyncGlobalCheckpoints() { assert primaryMode; assert handoffInProgress == false; - final ObjectLongMap globalCheckpoints = new ObjectLongHashMap<>(checkpoints.size()); - for (final Map.Entry cps : checkpoints.entrySet()) { - globalCheckpoints.put(cps.getKey(), cps.getValue().globalCheckpoint); - } + final ObjectLongMap globalCheckpoints = new ObjectLongHashMap<>(checkpoints.size()); // upper bound on the size + checkpoints + .entrySet() + .stream() + .filter(e -> e.getValue().inSync) + .forEach(e -> globalCheckpoints.put(e.getKey(), e.getValue().globalCheckpoint)); return globalCheckpoints; } 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 7386936c44c0f..1c8911a0cd886 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -139,12 +139,12 @@ public void updateGlobalCheckpointForShard(final String allocationId, final long } /** - * Get the local knowledge of the global checkpoints for all tracked allocation IDs. + * Get the local knowledge of the global checkpoints for all in-sync allocation IDs. * * @return a map from allocation ID to the local knowledge of the global checkpoint for that allocation ID */ - public ObjectLongMap getGlobalCheckpoints() { - return globalCheckpointTracker.getGlobalCheckpoints(); + public ObjectLongMap getInSyncGlobalCheckpoints() { + return globalCheckpointTracker.getInSyncGlobalCheckpoints(); } /** 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 00adff68de2a7..edd37aa5c1739 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1764,14 +1764,14 @@ public long getGlobalCheckpoint() { } /** - * Get the local knowledge of the global checkpoints for all tracked allocation IDs. + * Get the local knowledge of the global checkpoints for all in-sync allocation IDs. * * @return a map from allocation ID to the local knowledge of the global checkpoint for that allocation ID */ - public ObjectLongMap getGlobalCheckpoints() { + public ObjectLongMap getInSyncGlobalCheckpoints() { verifyPrimary(); verifyNotClosed(); - return getEngine().seqNoService().getGlobalCheckpoints(); + return getEngine().seqNoService().getInSyncGlobalCheckpoints(); } /** @@ -1787,7 +1787,7 @@ public void maybeSyncGlobalCheckpoint(final String reason) { // only sync if there are not operations in flight final SeqNoStats stats = getEngine().seqNoService().stats(); if (stats.getMaxSeqNo() == stats.getGlobalCheckpoint()) { - final ObjectLongMap globalCheckpoints = getGlobalCheckpoints(); + final ObjectLongMap globalCheckpoints = getInSyncGlobalCheckpoints(); final String allocationId = routingEntry().allocationId().getId(); assert globalCheckpoints.containsKey(allocationId); final long globalCheckpoint = globalCheckpoints.get(allocationId); diff --git a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java index 23ae9cef100d9..15617ab876db8 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -130,7 +130,7 @@ protected void beforeIndexDeletion() throws Exception { final IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node.getName()); final IndexShard indexShard = indicesService.getShardOrNull(primaryShardRouting.shardId()); - final ObjectLongMap globalCheckpoints = indexShard.getGlobalCheckpoints(); + final ObjectLongMap globalCheckpoints = indexShard.getInSyncGlobalCheckpoints(); for (ShardStats shardStats : indexShardStats) { final SeqNoStats seqNoStats = shardStats.getSeqNoStats(); assertThat(shardStats.getShardRouting() + " local checkpoint mismatch", From cf4e67b9ddb9654239add2330c0f595dacf87749 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 07:18:50 -0400 Subject: [PATCH 27/38] Test iteration --- .../GlobalCheckpointSyncIT.java} | 26 ++++++++++++++++--- 1 file changed, 22 insertions(+), 4 deletions(-) rename core/src/test/java/org/elasticsearch/index/{IndexServiceIT.java => seqno/GlobalCheckpointSyncIT.java} (72%) diff --git a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java similarity index 72% rename from core/src/test/java/org/elasticsearch/index/IndexServiceIT.java rename to core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java index 344c194049a63..a64d868b31efd 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexServiceIT.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java @@ -17,13 +17,15 @@ * under the License. */ -package org.elasticsearch.index; +package org.elasticsearch.index.seqno; import org.elasticsearch.action.admin.indices.stats.IndexShardStats; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -32,15 +34,27 @@ import java.util.Collection; import java.util.Collections; import java.util.Optional; +import java.util.stream.Collectors; import java.util.stream.Stream; +import java.util.stream.StreamSupport; import static org.hamcrest.Matchers.equalTo; -public class IndexServiceIT extends ESIntegTestCase { +public class GlobalCheckpointSyncIT extends ESIntegTestCase { - public void testGlobalCheckpointSync() throws Exception { + @Override + protected Collection> nodePlugins() { + return Stream.concat(super.nodePlugins().stream(), Stream.of(InternalSettingsPlugin.class)).collect(Collectors.toList()); + } + + public void testPostOperationGlobalCheckpointSync() throws Exception { internalCluster().startNode(); - final int numberOfDocuments = randomIntBetween(1, 128); + // set the sync interval high so it does not execute during this test + prepareCreate("test", Settings.builder().put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "24h")).get(); + if (randomBoolean()) { + ensureGreen(); + } + final int numberOfDocuments = randomIntBetween(0, 128); for (int i = 0; i < numberOfDocuments; i++) { final String id = Integer.toString(i); client().prepareIndex("test", "test", id).setSource("{\"foo\": " + id + "}", XContentType.JSON).get(); @@ -60,6 +74,10 @@ public void testGlobalCheckpointSync() throws Exception { final SeqNoStats primarySeqNoStats = primary.getSeqNoStats(); for (final ShardStats shardStats : indexShardStats) { final SeqNoStats seqNoStats = shardStats.getSeqNoStats(); + if (seqNoStats == null) { + // the shard is initializing + continue; + } assertThat(seqNoStats.getGlobalCheckpoint(), equalTo(primarySeqNoStats.getGlobalCheckpoint())); } } From f82df3048e525c7be057515b0b2adb3f9a9db488 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 07:29:49 -0400 Subject: [PATCH 28/38] More testing --- .../index/seqno/GlobalCheckpointSyncIT.java | 47 +++++++++++++++++-- 1 file changed, 43 insertions(+), 4 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java index a64d868b31efd..406fecfc47f32 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java @@ -31,9 +31,13 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalSettingsPlugin; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Optional; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -54,11 +58,42 @@ public void testPostOperationGlobalCheckpointSync() throws Exception { if (randomBoolean()) { ensureGreen(); } - final int numberOfDocuments = randomIntBetween(0, 128); - for (int i = 0; i < numberOfDocuments; i++) { - final String id = Integer.toString(i); - client().prepareIndex("test", "test", id).setSource("{\"foo\": " + id + "}", XContentType.JSON).get(); + + final int numberOfDocuments = randomIntBetween(0, 256); + + final int numberOfThreads = randomIntBetween(1, 4); + final CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads); + + // start concurrent indexing threads + final List threads = new ArrayList<>(numberOfThreads); + for (int i = 0; i < numberOfThreads; i++) { + final int index = i; + final Thread thread = new Thread(() -> { + try { + barrier.await(); + } catch (BrokenBarrierException | InterruptedException e) { + throw new RuntimeException(e); + } + for (int j = 0; j < numberOfDocuments; j++) { + final String id = Integer.toString(index * numberOfDocuments + j); + client().prepareIndex("test", "test", id).setSource("{\"foo\": " + id + "}", XContentType.JSON).get(); + } + try { + barrier.await(); + } catch (BrokenBarrierException | InterruptedException e) { + throw new RuntimeException(e); + } + }); + threads.add(thread); + thread.start(); } + + // synchronize the start of the threads + barrier.await(); + + // wait for the threads to finish + barrier.await(); + assertBusy(() -> { final IndicesStatsResponse stats = client().admin().indices().prepareStats().clear().get(); final IndexStats indexStats = stats.getIndex("test"); @@ -82,6 +117,10 @@ public void testPostOperationGlobalCheckpointSync() throws Exception { } } }); + + for (final Thread thread : threads) { + thread.join(); + } } } From 507806beb3ccd245043eba79a08db50a917c9f20 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 10:37:53 -0400 Subject: [PATCH 29/38] refactor shard creation --- .../main/java/org/elasticsearch/index/IndexModule.java | 3 +-- .../main/java/org/elasticsearch/index/IndexService.java | 8 ++------ .../java/org/elasticsearch/indices/IndicesService.java | 5 +++-- .../indices/cluster/IndicesClusterStateService.java | 5 +++-- .../indices/IndicesLifecycleListenerSingleNodeTests.java | 2 +- .../AbstractIndicesClusterStateServiceTestCase.java | 3 ++- 6 files changed, 12 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexModule.java b/core/src/main/java/org/elasticsearch/index/IndexModule.java index 43c16673f02be..8251244bbb998 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexModule.java +++ b/core/src/main/java/org/elasticsearch/index/IndexModule.java @@ -22,7 +22,6 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.client.Client; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; @@ -367,7 +366,7 @@ public IndexService newIndexService( new SimilarityService(indexSettings, scriptService, similarities), shardStoreDeleter, analysisRegistry, engineFactory.get(), circuitBreakerService, bigArrays, threadPool, scriptService, client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry, - indicesFieldDataCache, searchOperationListeners, indexOperationListeners, namedWriteableRegistry, globalCheckpointSyncer); + indicesFieldDataCache, searchOperationListeners, indexOperationListeners, namedWriteableRegistry); } /** diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 68145d3e0cae2..8a4c0a203c457 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -116,7 +116,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final IndexSettings indexSettings; private final List searchOperationListeners; private final List indexingOperationListeners; - private final Consumer globalCheckpointSyncer; private volatile AsyncRefreshTask refreshTask; private volatile AsyncTranslogFSync fsyncTask; private volatile AsyncGlobalCheckpointTask globalCheckpointTask; @@ -152,8 +151,7 @@ public IndexService( IndicesFieldDataCache indicesFieldDataCache, List searchOperationListeners, List indexingOperationListeners, - NamedWriteableRegistry namedWriteableRegistry, - Consumer globalCheckpointSyncer) throws IOException { + NamedWriteableRegistry namedWriteableRegistry) throws IOException { super(indexSettings); this.indexSettings = indexSettings; this.xContentRegistry = xContentRegistry; @@ -192,7 +190,6 @@ public IndexService( this.searcherWrapper = wrapperFactory.newWrapper(this); this.searchOperationListeners = Collections.unmodifiableList(searchOperationListeners); this.indexingOperationListeners = Collections.unmodifiableList(indexingOperationListeners); - this.globalCheckpointSyncer = globalCheckpointSyncer; // kick off async ops for the first shard in this index this.refreshTask = new AsyncRefreshTask(this); this.trimTranslogTask = new AsyncTrimTranslogTask(this); @@ -311,8 +308,7 @@ private long getAvgShardSizeInBytes() throws IOException { } } - public synchronized IndexShard createShard(ShardRouting routing) throws IOException { - final boolean primary = routing.primary(); + public synchronized IndexShard createShard(ShardRouting routing, Consumer globalCheckpointSyncer) throws IOException { /* * TODO: we execute this in parallel but it's a synced method. Yet, we might * be able to serialize the execution via the cluster state in the future. for now we just diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index 3c5fd2c8d21cc..4bd549f49da30 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -508,10 +508,11 @@ public synchronized void verifyIndexMetadata(IndexMetaData metaData, IndexMetaDa @Override public IndexShard createShard(ShardRouting shardRouting, RecoveryState recoveryState, PeerRecoveryTargetService recoveryTargetService, PeerRecoveryTargetService.RecoveryListener recoveryListener, RepositoriesService repositoriesService, - Consumer onShardFailure) throws IOException { + Consumer onShardFailure, + Consumer globalCheckpointSyncer) throws IOException { ensureChangesAllowed(); IndexService indexService = indexService(shardRouting.index()); - IndexShard indexShard = indexService.createShard(shardRouting); + IndexShard indexShard = indexService.createShard(shardRouting, globalCheckpointSyncer); indexShard.addShardFailureCallback(onShardFailure); indexShard.startRecovery(recoveryState, recoveryTargetService, recoveryListener, repositoriesService, (type, mapping) -> { 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 81f8c16f4cf31..3ccef4efaea9b 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -550,7 +550,7 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR logger.debug("{} creating shard", shardRouting.shardId()); RecoveryState recoveryState = new RecoveryState(shardRouting, nodes.getLocalNode(), sourceNode); indicesService.createShard(shardRouting, recoveryState, recoveryTargetService, new RecoveryListener(shardRouting), - repositoriesService, failedShardHandler); + repositoriesService, failedShardHandler, globalCheckpointSyncer); } catch (Exception e) { failAndRemoveShard(shardRouting, true, "failed to create shard", e, state); } @@ -841,7 +841,8 @@ U createIndex(IndexMetaData indexMetaData, */ T createShard(ShardRouting shardRouting, RecoveryState recoveryState, PeerRecoveryTargetService recoveryTargetService, PeerRecoveryTargetService.RecoveryListener recoveryListener, RepositoriesService repositoriesService, - Consumer onShardFailure) throws IOException; + Consumer onShardFailure, + Consumer globalCheckpointSyncer) throws IOException; /** * Returns shard for the specified id if it exists otherwise returns null. diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index db6cd91938d9a..4b4e565711a1a 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -130,7 +130,7 @@ public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRem newRouting = newRouting.moveToUnassigned(unassignedInfo) .updateUnassigned(unassignedInfo, RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE); newRouting = ShardRoutingHelper.initialize(newRouting, nodeId); - IndexShard shard = index.createShard(newRouting); + IndexShard shard = index.createShard(newRouting, s -> {}); IndexShardTestCase.updateRoutingEntry(shard, newRouting); assertEquals(5, counter.get()); final DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), 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 d5f860d05cdd6..b84ead4b135da 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -227,7 +227,8 @@ public MockIndexShard createShard(ShardRouting shardRouting, RecoveryState recov PeerRecoveryTargetService recoveryTargetService, PeerRecoveryTargetService.RecoveryListener recoveryListener, RepositoriesService repositoriesService, - Consumer onShardFailure) throws IOException { + Consumer onShardFailure, + Consumer globalCheckpointSyncer) throws IOException { failRandomly(); MockIndexService indexService = indexService(recoveryState.getShardId().getIndex()); MockIndexShard indexShard = indexService.createShard(shardRouting); From 530adddbef7402a9d729eaaad2d7892fc2c52a2a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 10:50:51 -0400 Subject: [PATCH 30/38] handle closed --- .../replication/TransportReplicationAction.java | 10 ++++++++-- .../java/org/elasticsearch/index/IndexService.java | 6 +++++- .../index/seqno/GlobalCheckpointSyncAction.java | 11 ++++++++++- 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index d15c7f1c41b9d..c2f16f509d0ee 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -20,7 +20,9 @@ package org.elasticsearch.action.support.replication; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; @@ -55,6 +57,7 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ReplicationGroup; import org.elasticsearch.index.shard.ShardId; @@ -379,8 +382,11 @@ public void onResponse(Response response) { try { primaryShardReference.indexShard.maybeSyncGlobalCheckpoint("post-operation"); } catch (final Exception e) { - logger.info("post-operation global checkpoint sync failed", e); - // intentionally swallow, a missed global checkpoint sync should not fail this operation + // only log non-closed exceptions + if (ExceptionsHelper.unwrap(e, AlreadyClosedException.class, IndexShardClosedException.class) == null) { + logger.info("post-operation global checkpoint sync failed", e); + // intentionally swallow, a missed global checkpoint sync should not fail this operation + } } } primaryShardReference.close(); // release shard operation lock before responding to caller diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 8a4c0a203c457..ef5fb954170c9 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -745,7 +745,11 @@ private void syncGlobalCheckpoints() { shard.maybeSyncGlobalCheckpoint("background"); } }, - e -> logger.info("failed to execute background global checkpoint sync", e)), + e -> { + if (!(e instanceof AlreadyClosedException || e instanceof IndexShardClosedException)) { + logger.info("failed to execute background global checkpoint sync", e); + } + }), ThreadPool.Names.SAME); } catch (final AlreadyClosedException | IndexShardClosedException e) { // the shard was closed concurrently, continue diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java index 49dba5f5159a1..60b61ccefa51c 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.seqno; +import org.apache.lucene.store.AlreadyClosedException; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; @@ -35,6 +37,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; @@ -78,7 +81,13 @@ public GlobalCheckpointSyncAction( } public void updateGlobalCheckpointForShard(final ShardId shardId) { - execute(new Request(shardId), ActionListener.wrap(r -> {}, e -> logger.info(shardId + " global checkpoint sync failed", e))); + execute( + new Request(shardId), + ActionListener.wrap(r -> {}, e -> { + if (ExceptionsHelper.unwrap(e, AlreadyClosedException.class, IndexShardClosedException.class) == null) { + logger.info(shardId + " global checkpoint sync failed", e); + } + })); } @Override From f3b04dcec38e0c6a50603a3a5144eb903c67733d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 11:53:03 -0400 Subject: [PATCH 31/38] assertSeqNos --- .../org/elasticsearch/index/IndexService.java | 4 +- .../discovery/ClusterDisruptionIT.java | 20 ++++---- .../elasticsearch/recovery/RelocationIT.java | 47 +---------------- .../elasticsearch/test/ESIntegTestCase.java | 51 +++++++++++++++++++ 4 files changed, 65 insertions(+), 57 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index ef5fb954170c9..ae5ea432855af 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -724,7 +724,7 @@ private void maybeTrimTranslog() { } } - private void syncGlobalCheckpoints() { + private void maybeSyncGlobalCheckpoints() { for (final IndexShard shard : this.shards.values()) { if (shard.routingEntry().active() && shard.routingEntry().primary()) { switch (shard.state()) { @@ -950,7 +950,7 @@ final class AsyncGlobalCheckpointTask extends BaseAsyncTask { @Override protected void runInternal() { - indexService.syncGlobalCheckpoints(); + indexService.maybeSyncGlobalCheckpoints(); } @Override diff --git a/core/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java b/core/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java index 38c9bcb72459f..0445c8ce3439b 100644 --- a/core/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java @@ -92,10 +92,10 @@ public void testAckedIndexing() throws Exception { final List nodes = startCluster(rarely() ? 5 : 3); assertAcked(prepareCreate("test") - .setSettings(Settings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1 + randomInt(2)) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(2)) - )); + .setSettings(Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1 + randomInt(2)) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(2)) + )); ensureGreen(); ServiceDisruptionScheme disruptionScheme = addRandomDisruptionScheme(); @@ -142,8 +142,8 @@ public void testAckedIndexing() throws Exception { exceptedExceptions.add(e); final String docId = id; logger.trace( - (Supplier) - () -> new ParameterizedMessage("[{}] failed id [{}] through node [{}]", name, docId, node), e); + (Supplier) + () -> new ParameterizedMessage("[{}] failed id [{}] through node [{}]", name, docId, node), e); } finally { countDownLatchRef.get().countDown(); logger.trace("[{}] decreased counter : {}", name, countDownLatchRef.get().getCount()); @@ -190,12 +190,12 @@ public void testAckedIndexing() throws Exception { disruptionScheme.stopDisrupting(); for (String node : internalCluster().getNodeNames()) { ensureStableCluster(nodes.size(), TimeValue.timeValueMillis(disruptionScheme.expectedTimeToHeal().millis() + - DISRUPTION_HEALING_OVERHEAD.millis()), true, node); + DISRUPTION_HEALING_OVERHEAD.millis()), true, node); } // in case of a bridge partition, shard allocation can fail "index.allocation.max_retries" times if the master // is the super-connected node and recovery source and target are on opposite sides of the bridge if (disruptionScheme instanceof NetworkDisruption && - ((NetworkDisruption) disruptionScheme).getDisruptedLinks() instanceof Bridge) { + ((NetworkDisruption) disruptionScheme).getDisruptedLinks() instanceof Bridge) { assertAcked(client().admin().cluster().prepareReroute().setRetryFailed(true)); } ensureGreen("test"); @@ -207,7 +207,7 @@ public void testAckedIndexing() throws Exception { logger.debug("validating through node [{}] ([{}] acked docs)", node, ackedDocs.size()); for (String id : ackedDocs.keySet()) { assertTrue("doc [" + id + "] indexed via node [" + ackedDocs.get(id) + "] not found", - client(node).prepareGet("test", "type", id).setPreference("_local").get().isExists()); + client(node).prepareGet("test", "type", id).setPreference("_local").get().isExists()); } } catch (AssertionError | NoShardAvailableActionException e) { throw new AssertionError(e.getMessage() + " (checked via node [" + node + "]", e); @@ -215,6 +215,8 @@ public void testAckedIndexing() throws Exception { } }, 30, TimeUnit.SECONDS); + assertSeqNos(); + logger.info("done validating (iteration [{}])", iter); } } finally { diff --git a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java index 15617ab876db8..1c7032fa02e87 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -20,22 +20,16 @@ package org.elasticsearch.recovery; import com.carrotsearch.hppc.IntHashSet; -import com.carrotsearch.hppc.ObjectLongMap; import com.carrotsearch.hppc.procedures.IntProcedure; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.util.English; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; -import org.elasticsearch.action.admin.indices.stats.IndexShardStats; -import org.elasticsearch.action.admin.indices.stats.IndexStats; -import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; -import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; @@ -47,13 +41,10 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.seqno.SeqNoStats; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest; import org.elasticsearch.plugins.Plugin; @@ -82,7 +73,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Semaphore; @@ -111,42 +101,7 @@ protected Collection> nodePlugins() { @Override protected void beforeIndexDeletion() throws Exception { super.beforeIndexDeletion(); - assertBusy(() -> { - IndicesStatsResponse stats = client().admin().indices().prepareStats().clear().get(); - for (IndexStats indexStats : stats.getIndices().values()) { - for (IndexShardStats indexShardStats : indexStats.getIndexShards().values()) { - Optional maybePrimary = Stream.of(indexShardStats.getShards()) - .filter(s -> s.getShardRouting().active() && s.getShardRouting().primary()) - .findFirst(); - if (maybePrimary.isPresent() == false) { - continue; - } - ShardStats primary = maybePrimary.get(); - final SeqNoStats primarySeqNoStats = primary.getSeqNoStats(); - final ShardRouting primaryShardRouting = primary.getShardRouting(); - assertThat(primaryShardRouting + " should have set the global checkpoint", - primarySeqNoStats.getGlobalCheckpoint(), not(equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO))); - final DiscoveryNode node = clusterService().state().nodes().get(primaryShardRouting.currentNodeId()); - final IndicesService indicesService = - internalCluster().getInstance(IndicesService.class, node.getName()); - final IndexShard indexShard = indicesService.getShardOrNull(primaryShardRouting.shardId()); - final ObjectLongMap globalCheckpoints = indexShard.getInSyncGlobalCheckpoints(); - for (ShardStats shardStats : indexShardStats) { - final SeqNoStats seqNoStats = shardStats.getSeqNoStats(); - assertThat(shardStats.getShardRouting() + " local checkpoint mismatch", - seqNoStats.getLocalCheckpoint(), equalTo(primarySeqNoStats.getLocalCheckpoint())); - assertThat(shardStats.getShardRouting() + " global checkpoint mismatch", - seqNoStats.getGlobalCheckpoint(), equalTo(primarySeqNoStats.getGlobalCheckpoint())); - assertThat(shardStats.getShardRouting() + " max seq no mismatch", - seqNoStats.getMaxSeqNo(), equalTo(primarySeqNoStats.getMaxSeqNo())); - // the local knowledge on the primary of the global checkpoint equals the global checkpoint on the shard - assertThat( - seqNoStats.getGlobalCheckpoint(), - equalTo(globalCheckpoints.get(shardStats.getShardRouting().allocationId().getId()))); - } - } - } - }); + assertSeqNos(); } public void testSimpleRelocationNoIndexing() { diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 2753e4013c181..ffa3cd6bed082 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -19,6 +19,7 @@ package org.elasticsearch.test; +import com.carrotsearch.hppc.ObjectLongMap; import com.carrotsearch.randomizedtesting.RandomizedContext; import com.carrotsearch.randomizedtesting.annotations.TestGroup; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; @@ -49,6 +50,10 @@ import org.elasticsearch.action.admin.indices.segments.IndexShardSegments; import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse; import org.elasticsearch.action.admin.indices.segments.ShardSegments; +import org.elasticsearch.action.admin.indices.stats.IndexShardStats; +import org.elasticsearch.action.admin.indices.stats.IndexStats; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; @@ -69,6 +74,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -114,6 +120,9 @@ import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.Segment; import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.seqno.SeqNoStats; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesQueryCache; import org.elasticsearch.indices.IndicesRequestCache; @@ -161,6 +170,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; @@ -191,6 +201,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.startsWith; @@ -2194,4 +2205,44 @@ public static Index resolveIndex(String index) { String uuid = getIndexResponse.getSettings().get(index).get(IndexMetaData.SETTING_INDEX_UUID); return new Index(index, uuid); } + + protected void assertSeqNos() throws Exception { + assertBusy(() -> { + IndicesStatsResponse stats = client().admin().indices().prepareStats().clear().get(); + for (IndexStats indexStats : stats.getIndices().values()) { + for (IndexShardStats indexShardStats : indexStats.getIndexShards().values()) { + Optional maybePrimary = Stream.of(indexShardStats.getShards()) + .filter(s -> s.getShardRouting().active() && s.getShardRouting().primary()) + .findFirst(); + if (maybePrimary.isPresent() == false) { + continue; + } + ShardStats primary = maybePrimary.get(); + final SeqNoStats primarySeqNoStats = primary.getSeqNoStats(); + final ShardRouting primaryShardRouting = primary.getShardRouting(); + assertThat(primaryShardRouting + " should have set the global checkpoint", + primarySeqNoStats.getGlobalCheckpoint(), not(equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO))); + final DiscoveryNode node = clusterService().state().nodes().get(primaryShardRouting.currentNodeId()); + final IndicesService indicesService = + internalCluster().getInstance(IndicesService.class, node.getName()); + final IndexShard indexShard = indicesService.getShardOrNull(primaryShardRouting.shardId()); + final ObjectLongMap globalCheckpoints = indexShard.getInSyncGlobalCheckpoints(); + for (ShardStats shardStats : indexShardStats) { + final SeqNoStats seqNoStats = shardStats.getSeqNoStats(); + assertThat(shardStats.getShardRouting() + " local checkpoint mismatch", + seqNoStats.getLocalCheckpoint(), equalTo(primarySeqNoStats.getLocalCheckpoint())); + assertThat(shardStats.getShardRouting() + " global checkpoint mismatch", + seqNoStats.getGlobalCheckpoint(), equalTo(primarySeqNoStats.getGlobalCheckpoint())); + assertThat(shardStats.getShardRouting() + " max seq no mismatch", + seqNoStats.getMaxSeqNo(), equalTo(primarySeqNoStats.getMaxSeqNo())); + // the local knowledge on the primary of the global checkpoint equals the global checkpoint on the shard + assertThat( + seqNoStats.getGlobalCheckpoint(), + equalTo(globalCheckpoints.get(shardStats.getShardRouting().allocationId().getId()))); + } + } + } + }); + } + } From 5030a776cfd859e89522cfd9c087c59afdee4b22 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 12:53:39 -0400 Subject: [PATCH 32/38] Add distruption test --- .../index/seqno/GlobalCheckpointSyncIT.java | 98 ++++++++++++++++++- 1 file changed, 95 insertions(+), 3 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java index 406fecfc47f32..b52161a73e16f 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java @@ -23,21 +23,39 @@ import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.ShardIterator; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalSettingsPlugin; +import org.elasticsearch.test.disruption.NetworkDisruption; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Optional; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CyclicBarrier; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -48,17 +66,89 @@ public class GlobalCheckpointSyncIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return Stream.concat(super.nodePlugins().stream(), Stream.of(InternalSettingsPlugin.class)).collect(Collectors.toList()); + return Stream.concat(super.nodePlugins().stream(), Stream.of(InternalSettingsPlugin.class, MockTransportService.TestPlugin.class)).collect(Collectors.toList()); } public void testPostOperationGlobalCheckpointSync() throws Exception { - internalCluster().startNode(); + runGlobalCheckpointSyncTest(TimeValue.timeValueHours(24), client -> {}, client -> {}); + } + + /* + * This test swallows the post-operation global checkpoint syncs, and then restores the ability to send these requests at the end of the + * test so that a background sync can fire and sync the global checkpoint. + */ + public void testBackgroundGlobalCheckpointSync() throws Exception { + runGlobalCheckpointSyncTest( + TimeValue.timeValueSeconds(randomIntBetween(1, 3)), + client -> { + // prevent global checkpoint syncs between all nodes + final DiscoveryNodes nodes = client.admin().cluster().prepareState().get().getState().getNodes(); + for (final DiscoveryNode node : nodes) { + for (final DiscoveryNode other : nodes) { + if (node == other) { + continue; + } + final MockTransportService senderTransportService = + (MockTransportService) internalCluster().getInstance(TransportService.class, node.getName()); + final MockTransportService receiverTransportService = + (MockTransportService) internalCluster().getInstance(TransportService.class, other.getName()); + + senderTransportService.addDelegate(receiverTransportService, + new MockTransportService.DelegateTransport(senderTransportService.original()) { + @Override + protected void sendRequest( + final Connection connection, + final long requestId, + final String action, + final TransportRequest request, + final TransportRequestOptions options) throws IOException { + if ("indices:admin/seq_no/global_checkpoint_sync[r]".equals(action)) { + throw new IllegalStateException("blocking indices:admin/seq_no/global_checkpoint_sync[r]"); + } else { + super.sendRequest(connection, requestId, action, request, options); + } + } + }); + } + } + }, + client -> { + // restore global checkpoint syncs between all nodes + final DiscoveryNodes nodes = client.admin().cluster().prepareState().get().getState().getNodes(); + for (final DiscoveryNode node : nodes) { + for (final DiscoveryNode other : nodes) { + if (node == other) { + continue; + } + final MockTransportService senderTransportService = + (MockTransportService) internalCluster().getInstance(TransportService.class, node.getName()); + final MockTransportService receiverTransportService = + (MockTransportService) internalCluster().getInstance(TransportService.class, other.getName()); + senderTransportService.clearRule(receiverTransportService); + } + } + }); + } + + private void runGlobalCheckpointSyncTest( + final TimeValue globalCheckpointSyncInterval, + final Consumer beforeIndexing, + final Consumer afterIndexing) throws Exception { + final int numberOfReplicas = randomIntBetween(1, 4); + internalCluster().ensureAtLeastNumDataNodes(1 + numberOfReplicas); // set the sync interval high so it does not execute during this test - prepareCreate("test", Settings.builder().put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "24h")).get(); + prepareCreate( + "test", + Settings.builder() + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), globalCheckpointSyncInterval) + .put("index.number_of_replicas", numberOfReplicas)) + .get(); if (randomBoolean()) { ensureGreen(); } + beforeIndexing.accept(client()); + final int numberOfDocuments = randomIntBetween(0, 256); final int numberOfThreads = randomIntBetween(1, 4); @@ -94,6 +184,8 @@ public void testPostOperationGlobalCheckpointSync() throws Exception { // wait for the threads to finish barrier.await(); + afterIndexing.accept(client()); + assertBusy(() -> { final IndicesStatsResponse stats = client().admin().indices().prepareStats().clear().get(); final IndexStats indexStats = stats.getIndex("test"); From b640b109a0c224be68a8b11c5af12ca93a47f5c0 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 13:42:23 -0400 Subject: [PATCH 33/38] revert formatting changes --- .../discovery/ClusterDisruptionIT.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java b/core/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java index 0445c8ce3439b..0701d9d10e3ac 100644 --- a/core/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java @@ -92,10 +92,10 @@ public void testAckedIndexing() throws Exception { final List nodes = startCluster(rarely() ? 5 : 3); assertAcked(prepareCreate("test") - .setSettings(Settings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1 + randomInt(2)) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(2)) - )); + .setSettings(Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1 + randomInt(2)) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(2)) + )); ensureGreen(); ServiceDisruptionScheme disruptionScheme = addRandomDisruptionScheme(); @@ -142,8 +142,8 @@ public void testAckedIndexing() throws Exception { exceptedExceptions.add(e); final String docId = id; logger.trace( - (Supplier) - () -> new ParameterizedMessage("[{}] failed id [{}] through node [{}]", name, docId, node), e); + (Supplier) + () -> new ParameterizedMessage("[{}] failed id [{}] through node [{}]", name, docId, node), e); } finally { countDownLatchRef.get().countDown(); logger.trace("[{}] decreased counter : {}", name, countDownLatchRef.get().getCount()); @@ -190,12 +190,12 @@ public void testAckedIndexing() throws Exception { disruptionScheme.stopDisrupting(); for (String node : internalCluster().getNodeNames()) { ensureStableCluster(nodes.size(), TimeValue.timeValueMillis(disruptionScheme.expectedTimeToHeal().millis() + - DISRUPTION_HEALING_OVERHEAD.millis()), true, node); + DISRUPTION_HEALING_OVERHEAD.millis()), true, node); } // in case of a bridge partition, shard allocation can fail "index.allocation.max_retries" times if the master // is the super-connected node and recovery source and target are on opposite sides of the bridge if (disruptionScheme instanceof NetworkDisruption && - ((NetworkDisruption) disruptionScheme).getDisruptedLinks() instanceof Bridge) { + ((NetworkDisruption) disruptionScheme).getDisruptedLinks() instanceof Bridge) { assertAcked(client().admin().cluster().prepareReroute().setRetryFailed(true)); } ensureGreen("test"); @@ -207,7 +207,7 @@ public void testAckedIndexing() throws Exception { logger.debug("validating through node [{}] ([{}] acked docs)", node, ackedDocs.size()); for (String id : ackedDocs.keySet()) { assertTrue("doc [" + id + "] indexed via node [" + ackedDocs.get(id) + "] not found", - client(node).prepareGet("test", "type", id).setPreference("_local").get().isExists()); + client(node).prepareGet("test", "type", id).setPreference("_local").get().isExists()); } } catch (AssertionError | NoShardAvailableActionException e) { throw new AssertionError(e.getMessage() + " (checked via node [" + node + "]", e); From cccdec603c0ae377149a9302bf53b261da394173 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 13:43:25 -0400 Subject: [PATCH 34/38] checkstyle --- .../elasticsearch/index/seqno/GlobalCheckpointSyncIT.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java index b52161a73e16f..4c296db884c75 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java @@ -66,7 +66,10 @@ public class GlobalCheckpointSyncIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return Stream.concat(super.nodePlugins().stream(), Stream.of(InternalSettingsPlugin.class, MockTransportService.TestPlugin.class)).collect(Collectors.toList()); + return Stream.concat( + super.nodePlugins().stream(), + Stream.of(InternalSettingsPlugin.class, MockTransportService.TestPlugin.class)) + .collect(Collectors.toList()); } public void testPostOperationGlobalCheckpointSync() throws Exception { From d43f79423ed14def9e8d6cb5bf4d3fb566a064f2 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 13:52:26 -0400 Subject: [PATCH 35/38] imports --- .../index/seqno/GlobalCheckpointSyncIT.java | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java index 4c296db884c75..bf910ae97f4d2 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java @@ -24,33 +24,23 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.cluster.routing.ShardIterator; -import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalSettingsPlugin; -import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.Optional; import java.util.concurrent.BrokenBarrierException; @@ -58,7 +48,6 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; -import java.util.stream.StreamSupport; import static org.hamcrest.Matchers.equalTo; From 26e4c7689b9e850df7affcd7b0881c31de1e6029 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 13:58:39 -0400 Subject: [PATCH 36/38] remove dead code from create index --- .../metadata/MetaDataCreateIndexService.java | 2 +- .../metadata/MetaDataIndexAliasesService.java | 2 +- .../metadata/MetaDataIndexTemplateService.java | 2 +- .../cluster/metadata/MetaDataMappingService.java | 2 +- .../java/org/elasticsearch/index/IndexModule.java | 3 +-- .../org/elasticsearch/indices/IndicesService.java | 13 ++++--------- .../indices/cluster/IndicesClusterStateService.java | 6 ++---- .../cluster/metadata/IndexCreationTaskTests.java | 2 +- .../org/elasticsearch/index/IndexModuleTests.java | 2 +- .../IndicesLifecycleListenerSingleNodeTests.java | 2 +- .../AbstractIndicesClusterStateServiceTestCase.java | 3 +-- .../indices/cluster/ClusterStateChanges.java | 3 +-- 12 files changed, 16 insertions(+), 26 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index f2495edfc6b24..8b262a249c844 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -428,7 +428,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { (tmpImd.getNumberOfReplicas() + 1) + "]"); } // create the index here (on the master) to validate it can be created, as well as adding the mapping - final IndexService indexService = indicesService.createIndex(tmpImd, Collections.emptyList(), s -> {}); + final IndexService indexService = indicesService.createIndex(tmpImd, Collections.emptyList()); createdIndex = indexService.index(); // now add the mappings MapperService mapperService = indexService.mapperService(); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java index ea4bdc83b564c..0949032db1665 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java @@ -139,7 +139,7 @@ ClusterState innerExecute(ClusterState currentState, Iterable actio if (indexService == null) { // temporarily create the index and add mappings so we can parse the filter try { - indexService = indicesService.createIndex(index, emptyList(), s -> {}); + indexService = indicesService.createIndex(index, emptyList()); indicesToClose.add(index.getIndex()); } catch (IOException e) { throw new ElasticsearchException("Failed to create temporary index for parsing the alias", e); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java index c404a7972a9ce..c96895b94e793 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java @@ -213,7 +213,7 @@ private static void validateAndAddTemplate(final PutRequest request, IndexTempla .build(); final IndexMetaData tmpIndexMetadata = IndexMetaData.builder(temporaryIndexName).settings(dummySettings).build(); - IndexService dummyIndexService = indicesService.createIndex(tmpIndexMetadata, Collections.emptyList(), s -> {}); + IndexService dummyIndexService = indicesService.createIndex(tmpIndexMetadata, Collections.emptyList()); createdIndex = dummyIndexService.index(); templateBuilder.order(request.order); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java index c56a2c8716f65..865b58c468a52 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java @@ -145,7 +145,7 @@ ClusterState executeRefresh(final ClusterState currentState, final List {}); + indexService = indicesService.createIndex(indexMetaData, Collections.emptyList()); removeIndex = true; indexService.mapperService().merge(indexMetaData, MergeReason.MAPPING_RECOVERY, true); } diff --git a/core/src/main/java/org/elasticsearch/index/IndexModule.java b/core/src/main/java/org/elasticsearch/index/IndexModule.java index 8251244bbb998..f806c210f0014 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexModule.java +++ b/core/src/main/java/org/elasticsearch/index/IndexModule.java @@ -330,8 +330,7 @@ public IndexService newIndexService( IndicesQueryCache indicesQueryCache, MapperRegistry mapperRegistry, IndicesFieldDataCache indicesFieldDataCache, - NamedWriteableRegistry namedWriteableRegistry, - Consumer globalCheckpointSyncer) + NamedWriteableRegistry namedWriteableRegistry) throws IOException { final IndexEventListener eventListener = freeze(); IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index 4bd549f49da30..caffa1b7befda 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -378,14 +378,11 @@ public IndexService indexServiceSafe(Index index) { * @param indexMetaData the index metadata to create the index for * @param builtInListeners a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with the * per-index listeners - * @param globalCheckpointSyncer callback for syncing global checkpoints * @throws ResourceAlreadyExistsException if the index already exists. */ @Override public synchronized IndexService createIndex( - final IndexMetaData indexMetaData, - final List builtInListeners, - final Consumer globalCheckpointSyncer) throws IOException { + final IndexMetaData indexMetaData, final List builtInListeners) throws IOException { ensureChangesAllowed(); if (indexMetaData.getIndexUUID().equals(IndexMetaData.INDEX_UUID_NA_VALUE)) { throw new IllegalArgumentException("index must have a real UUID found value: [" + indexMetaData.getIndexUUID() + "]"); @@ -409,7 +406,6 @@ public void onStoreClosed(ShardId shardId) { indexMetaData, indicesQueryCache, indicesFieldDataCache, - globalCheckpointSyncer, finalListeners, indexingMemoryController); boolean success = false; @@ -432,7 +428,6 @@ private synchronized IndexService createIndexService(final String reason, IndexMetaData indexMetaData, IndicesQueryCache indicesQueryCache, IndicesFieldDataCache indicesFieldDataCache, - Consumer globalCheckpointSyncer, List builtInListeners, IndexingOperationListener... indexingOperationListeners) throws IOException { final IndexSettings idxSettings = new IndexSettings(indexMetaData, this.settings, indexScopeSetting); @@ -462,8 +457,8 @@ private synchronized IndexService createIndexService(final String reason, indicesQueryCache, mapperRegistry, indicesFieldDataCache, - namedWriteableRegistry, - globalCheckpointSyncer); + namedWriteableRegistry + ); } /** @@ -494,7 +489,7 @@ public synchronized void verifyIndexMetadata(IndexMetaData metaData, IndexMetaDa closeables.add(indicesQueryCache); // this will also fail if some plugin fails etc. which is nice since we can verify that early final IndexService service = - createIndexService("metadata verification", metaData, indicesQueryCache, indicesFieldDataCache, s -> {}, emptyList()); + createIndexService("metadata verification", metaData, indicesQueryCache, indicesFieldDataCache, emptyList()); closeables.add(() -> service.close("metadata verification", false)); service.mapperService().merge(metaData, MapperService.MergeReason.MAPPING_RECOVERY, true); if (metaData.equals(metaDataUpdate) == false) { 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 3ccef4efaea9b..5aa8b5f3ee1b3 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -451,7 +451,7 @@ private void createIndices(final ClusterState state) { AllocatedIndex indexService = null; try { - indexService = indicesService.createIndex(indexMetaData, buildInIndexListener, globalCheckpointSyncer); + indexService = indicesService.createIndex(indexMetaData, buildInIndexListener); if (indexService.updateMapping(indexMetaData) && sendRefreshMapping) { nodeMappingRefreshAction.nodeMappingRefresh(state.nodes().getMasterNode(), new NodeMappingRefreshAction.NodeMappingRefreshRequest(indexMetaData.getIndex().getName(), @@ -798,12 +798,10 @@ public interface AllocatedIndices> * @param indexMetaData the index metadata to create the index for * @param builtInIndexListener a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with * the per-index listeners - * @param globalCheckpointSyncer callback for syncing global checkpoints * @throws ResourceAlreadyExistsException if the index already exists. */ U createIndex(IndexMetaData indexMetaData, - List builtInIndexListener, - Consumer globalCheckpointSyncer) throws IOException; + List builtInIndexListener) throws IOException; /** * Verify that the contents on disk for the given index is deleted; if not, delete the contents. diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java index 1db44840d11ba..9dc989961f32b 100644 --- a/core/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java @@ -443,6 +443,6 @@ private void setupIndicesService() throws Exception { when(service.getIndexSortSupplier()).thenReturn(supplier); when(service.getIndexEventListener()).thenReturn(mock(IndexEventListener.class)); - when(indicesService.createIndex(anyObject(), anyObject(), anyObject())).thenReturn(service); + when(indicesService.createIndex(anyObject(), anyObject())).thenReturn(service); } } diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 69f8f4adc8d31..4b9645a3af87d 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -140,7 +140,7 @@ public void tearDown() throws Exception { private IndexService newIndexService(IndexModule module) throws IOException { return module.newIndexService(nodeEnvironment, xContentRegistry(), deleter, circuitBreakerService, bigArrays, threadPool, scriptService, null, indicesQueryCache, mapperRegistry, - new IndicesFieldDataCache(settings, listener), writableRegistry(), s -> {}); + new IndicesFieldDataCache(settings, listener), writableRegistry()); } public void testWrapperIsBound() throws IOException { diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index 4b4e565711a1a..787c6c815dc52 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -121,7 +121,7 @@ public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRem }; indicesService.removeIndex(idx, DELETED, "simon says"); try { - IndexService index = indicesService.createIndex(metaData, Arrays.asList(countingListener), s -> {}); + IndexService index = indicesService.createIndex(metaData, Arrays.asList(countingListener)); assertEquals(3, counter.get()); idx = index.index(); ShardRouting newRouting = shardRouting; 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 b84ead4b135da..35bbc497838f2 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -190,8 +190,7 @@ protected class MockIndicesService implements AllocatedIndices buildInIndexListener, - Consumer globalCheckpointSyncer) throws IOException { + List buildInIndexListener) throws IOException { MockIndexService indexService = new MockIndexService(new IndexSettings(indexMetaData, Settings.EMPTY)); indices = newMapBuilder(indices).put(indexMetaData.getIndexUUID(), indexService).immutableMap(); return indexService; diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index 847e271925e23..b1725ead326d9 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -95,7 +95,6 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyList; -import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -140,7 +139,7 @@ public ClusterStateChanges(NamedXContentRegistry xContentRegistry, ThreadPool th // MetaDataCreateIndexService creates indices using its IndicesService instance to check mappings -> fake it here try { @SuppressWarnings("unchecked") final List listeners = anyList(); - when(indicesService.createIndex(any(IndexMetaData.class), listeners, anyObject())) + when(indicesService.createIndex(any(IndexMetaData.class), listeners)) .then(invocationOnMock -> { IndexService indexService = mock(IndexService.class); IndexMetaData indexMetaData = (IndexMetaData)invocationOnMock.getArguments()[0]; From b80b728d968887b91b7be101d137fc71b44e523f Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 13:59:39 -0400 Subject: [PATCH 37/38] everything in the right place --- .../org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java index bf910ae97f4d2..b2c828cb73f0c 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java @@ -62,6 +62,7 @@ protected Collection> nodePlugins() { } public void testPostOperationGlobalCheckpointSync() throws Exception { + // set the sync interval high so it does not execute during this test runGlobalCheckpointSyncTest(TimeValue.timeValueHours(24), client -> {}, client -> {}); } @@ -128,7 +129,6 @@ private void runGlobalCheckpointSyncTest( final Consumer afterIndexing) throws Exception { final int numberOfReplicas = randomIntBetween(1, 4); internalCluster().ensureAtLeastNumDataNodes(1 + numberOfReplicas); - // set the sync interval high so it does not execute during this test prepareCreate( "test", Settings.builder() From b8adcce90e1af3c4ebf1e2fa8f7cba09a2c643ea Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 21 Sep 2017 14:00:49 -0400 Subject: [PATCH 38/38] revert formatting changes --- .../support/replication/TransportReplicationAction.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index c2f16f509d0ee..77f7ff1d4460a 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -412,8 +412,8 @@ public void onFailure(Exception e) { } protected ReplicationOperation> createReplicatedOperation( - Request request, ActionListener> listener, - PrimaryShardReference primaryShardReference) { + Request request, ActionListener> listener, + PrimaryShardReference primaryShardReference) { return new ReplicationOperation<>(request, primaryShardReference, listener, newReplicasProxy(primaryTerm), logger, actionName); }