From 82d464d4b20d8602c59e64aaeaef23dce5b91262 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 12 Feb 2019 22:22:40 -0500 Subject: [PATCH 01/30] Integrate retention leases to recovery from remote This commit is the first step in integrating shard history retention leases with CCR. In this commit we integrate shard history retention leases with recovery from remote. Before we start transferring files, we take out a retention lease on the primary. Then during the file copy phase, we repeatedly renew the retention lease. Finally, when recovery from remote is complete, we disable the background renewing of the retention lease. --- .../index/engine/SoftDeletesPolicyTests.java | 1 + .../ClearCcrRestoreSessionAction.java | 2 +- .../xpack/ccr/repository/CcrRepository.java | 89 +++- .../elasticsearch/xpack/CcrIntegTestCase.java | 74 +++- .../xpack/ccr/CcrRepositoryIT.java | 55 --- .../xpack/ccr/CcrRetentionLeaseIT.java | 409 ++++++++++++++++++ 6 files changed, 572 insertions(+), 58 deletions(-) create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java diff --git a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java index 8257aa99d0486..892d4c14f15bf 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java @@ -36,6 +36,7 @@ import java.util.function.Supplier; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; +import static org.elasticsearch.index.seqno.SequenceNumbers.min; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/ClearCcrRestoreSessionAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/ClearCcrRestoreSessionAction.java index 317890edb4206..c4651e877fadd 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/ClearCcrRestoreSessionAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/ClearCcrRestoreSessionAction.java @@ -23,7 +23,7 @@ public class ClearCcrRestoreSessionAction extends Action { public static final ClearCcrRestoreSessionAction INSTANCE = new ClearCcrRestoreSessionAction(); - private static final String NAME = "internal:admin/ccr/restore/session/clear"; + public static final String NAME = "internal:admin/ccr/restore/session/clear"; private ClearCcrRestoreSessionAction() { super(NAME); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index d613200531c5c..75e0f4ede0493 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -8,6 +8,9 @@ import com.carrotsearch.hppc.cursors.IntObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.IndexCommit; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; @@ -29,6 +32,7 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.metrics.CounterMetric; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; @@ -36,6 +40,9 @@ import org.elasticsearch.index.Index; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.seqno.LocalCheckpointTracker; +import org.elasticsearch.index.seqno.RetentionLeaseActions; +import org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException; +import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardRecoveryException; import org.elasticsearch.index.shard.ShardId; @@ -55,6 +62,7 @@ import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotShardFailure; import org.elasticsearch.snapshots.SnapshotState; +import org.elasticsearch.threadpool.Scheduler; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.Ccr; import org.elasticsearch.xpack.ccr.CcrLicenseChecker; @@ -77,11 +85,14 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.LongConsumer; import java.util.function.Supplier; +import static org.elasticsearch.index.seqno.RetentionLeaseActions.RETAIN_ALL; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; @@ -91,6 +102,8 @@ */ public class CcrRepository extends AbstractLifecycleComponent implements Repository { + private static final Logger logger = LogManager.getLogger(CcrRepository.class); + public static final String LATEST = "_latest_"; public static final String TYPE = "_ccr_"; public static final String NAME_PREFIX = "_ccr_"; @@ -291,12 +304,47 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v store.decRef(); } + Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); + Map ccrMetaData = indexShard.indexSettings().getIndexMetaData().getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY); String leaderUUID = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY); Index leaderIndex = new Index(shardId.getIndexName(), leaderUUID); ShardId leaderShardId = new ShardId(leaderIndex, shardId.getId()); - Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); + final String retentionLeaseId = indexShard.shardId().getIndex().getUUID() + "-following-" + leaderUUID; + logger.trace( + () -> new ParameterizedMessage("{} requesting leader primary to add retention lease [" + retentionLeaseId + "]", shardId)); + final Optional maybeAddAlready = + addRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + maybeAddAlready.ifPresent(addAlready -> { + logger.trace(() -> new ParameterizedMessage("{} retention lease already exists, requesting a renewal", shardId), addAlready); + final Optional maybeRenewNotFound = + renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + maybeRenewNotFound.ifPresent(renewNotFound -> { + logger.trace(() -> new ParameterizedMessage( + "{} retention lease not found while attempting to renew, attempting a final add", shardId), renewNotFound); + final Optional maybeFallbackAddAlready = + addRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + maybeFallbackAddAlready.ifPresent(fallbackAddAlready -> { + /* + * At this point we tried to add the lease and the retention lease already existed. By the time we tried to renew the + * lease, it expired or was removed. We tried to add the lease again and it already exists? Bail. + */ + assert false : fallbackAddAlready; + throw fallbackAddAlready; + }); + }); + }); + + // schedule renewals to run during the restore + final Scheduler.Cancellable renewable = threadPool.scheduleWithFixedDelay( + () -> { + logger.info("{} background renewing retention lease during restore", shardId); + renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + }, + RETENTION_LEASE_RENEW_INTERVAL_SETTING.get(indexShard.indexSettings().getSettings()), + Ccr.CCR_THREAD_POOL_NAME); + // TODO: There should be some local timeout. And if the remote cluster returns an unknown session // response, we should be able to retry by creating a new session. String name = metadata.name(); @@ -305,9 +353,48 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v updateMappings(remoteClient, leaderIndex, restoreSession.mappingVersion, client, indexShard.routingEntry().index()); } catch (Exception e) { throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); + } finally { + renewable.cancel(); } } + private Optional addRetentionLease( + final ShardId leaderShardId, + final String retentionLeaseId, + final Client remoteClient) { + try { + final RetentionLeaseActions.AddRequest request = + new RetentionLeaseActions.AddRequest(leaderShardId, retentionLeaseId, RETAIN_ALL, "ccr"); + remoteClient.execute(RetentionLeaseActions.Add.INSTANCE, request).actionGet(ccrSettings.getRecoveryActionTimeout()); + return Optional.empty(); + } catch (final RetentionLeaseAlreadyExistsException e) { + return Optional.of(e); + } + } + + private Optional renewRetentionLease( + final ShardId leaderShardId, + final String retentionLeaseId, + final Client remoteClient) { + try { + final RetentionLeaseActions.RenewRequest request = + new RetentionLeaseActions.RenewRequest(leaderShardId, retentionLeaseId, RETAIN_ALL, "ccr"); + remoteClient.execute(RetentionLeaseActions.Renew.INSTANCE, request).actionGet(ccrSettings.getRecoveryActionTimeout()); + return Optional.empty(); + } catch (final RetentionLeaseNotFoundException e) { + return Optional.of(e); + } + } + + // this setting is intentionally not registered, it is only used in tests + public static final Setting RETENTION_LEASE_RENEW_INTERVAL_SETTING = + Setting.timeSetting( + "index.ccr.retention_lease.renew_interval", + new TimeValue(5, TimeUnit.MINUTES), + new TimeValue(0, TimeUnit.MILLISECONDS), + Setting.Property.Dynamic, + Setting.Property.IndexScope); + @Override public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId leaderShardId) { throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java index 81b90a3ff60b0..63ede221b2cbb 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack; import org.apache.lucene.store.AlreadyClosedException; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksAction; @@ -22,8 +23,11 @@ import org.elasticsearch.analysis.common.CommonAnalysisPlugin; import org.elasticsearch.client.Client; import org.elasticsearch.client.Requests; +import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -35,6 +39,7 @@ import org.elasticsearch.common.Randomness; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; @@ -58,6 +63,9 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.snapshots.RestoreInfo; +import org.elasticsearch.snapshots.RestoreService; +import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.tasks.TaskInfo; import org.elasticsearch.test.BackgroundIndexer; import org.elasticsearch.test.ESIntegTestCase; @@ -99,10 +107,12 @@ import java.util.function.BooleanSupplier; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING; import static org.elasticsearch.discovery.SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING; +import static org.elasticsearch.snapshots.RestoreService.restoreInProgress; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.hamcrest.Matchers.empty; @@ -114,6 +124,10 @@ public abstract class CcrIntegTestCase extends ESTestCase { private static ClusterGroup clusterGroup; + protected Collection> nodePlugins() { + return Collections.emptyList(); + } + @Before public final void startClusters() throws Exception { if (clusterGroup != null && reuseClusters()) { @@ -224,7 +238,10 @@ public Path nodeConfigPath(int nodeOrdinal) { @Override public Collection> nodePlugins() { - return Arrays.asList(LocalStateCcr.class, CommonAnalysisPlugin.class); + return Stream.concat( + Stream.of(LocalStateCcr.class, CommonAnalysisPlugin.class), + CcrIntegTestCase.this.nodePlugins().stream()) + .collect(Collectors.toList()); } @Override @@ -639,6 +656,61 @@ public long waitForDocs(final long numDocs, int maxWaitTime, TimeUnit maxWaitTim return lastKnownCount.get(); } + protected ActionListener waitForRestore( + final ClusterService clusterService, + final ActionListener listener) { + return new ActionListener() { + + @Override + public void onResponse(RestoreService.RestoreCompletionResponse restoreCompletionResponse) { + if (restoreCompletionResponse.getRestoreInfo() == null) { + final Snapshot snapshot = restoreCompletionResponse.getSnapshot(); + final String uuid = restoreCompletionResponse.getUuid(); + + final ClusterStateListener clusterStateListener = new ClusterStateListener() { + + @Override + public void clusterChanged(ClusterChangedEvent changedEvent) { + final RestoreInProgress.Entry prevEntry = restoreInProgress(changedEvent.previousState(), uuid); + final RestoreInProgress.Entry newEntry = restoreInProgress(changedEvent.state(), uuid); + if (prevEntry == null) { + /* + * When there is a master failure after a restore has been started, this listener might not be registered + * on the current master and as such it might miss some intermediary cluster states due to batching. + * Clean up the listener in that case and acknowledge completion of restore operation to client. + */ + clusterService.removeListener(this); + listener.onResponse(null); + } else if (newEntry == null) { + clusterService.removeListener(this); + ImmutableOpenMap shards = prevEntry.shards(); + RestoreInfo ri = new RestoreInfo(prevEntry.snapshot().getSnapshotId().getName(), + prevEntry.indices(), + shards.size(), + shards.size() - RestoreService.failedShards(shards)); + logger.debug("restore of [{}] completed", snapshot); + listener.onResponse(ri); + } else { + // restore not completed yet, wait for next cluster state update + } + } + + }; + + clusterService.addListener(clusterStateListener); + } else { + listener.onResponse(restoreCompletionResponse.getRestoreInfo()); + } + } + + @Override + public void onFailure(Exception t) { + listener.onFailure(t); + } + + }; + } + static void removeCCRRelatedMetadataFromClusterState(ClusterService clusterService) throws Exception { CountDownLatch latch = new CountDownLatch(1); clusterService.submitStateUpdateTask("remove-ccr-related-metadata", new ClusterStateUpdateTask() { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java index 9f061b9c33099..ed3d10830f853 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java @@ -8,7 +8,6 @@ import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; @@ -16,26 +15,20 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; -import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryMissingException; import org.elasticsearch.snapshots.RestoreInfo; import org.elasticsearch.snapshots.RestoreService; -import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.CcrIntegTestCase; @@ -52,7 +45,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import static java.util.Collections.singletonMap; -import static org.elasticsearch.snapshots.RestoreService.restoreInProgress; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -447,51 +439,4 @@ private void assertExpectedDocument(String followerIndex, final int value) { assertThat(getResponse.getSource().get("f"), equalTo(value)); } - private ActionListener waitForRestore(ClusterService clusterService, - ActionListener listener) { - return new ActionListener() { - @Override - public void onResponse(RestoreService.RestoreCompletionResponse restoreCompletionResponse) { - if (restoreCompletionResponse.getRestoreInfo() == null) { - final Snapshot snapshot = restoreCompletionResponse.getSnapshot(); - final String uuid = restoreCompletionResponse.getUuid(); - - ClusterStateListener clusterStateListener = new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent changedEvent) { - final RestoreInProgress.Entry prevEntry = restoreInProgress(changedEvent.previousState(), uuid); - final RestoreInProgress.Entry newEntry = restoreInProgress(changedEvent.state(), uuid); - if (prevEntry == null) { - // When there is a master failure after a restore has been started, this listener might not be registered - // on the current master and as such it might miss some intermediary cluster states due to batching. - // Clean up listener in that case and acknowledge completion of restore operation to client. - clusterService.removeListener(this); - listener.onResponse(null); - } else if (newEntry == null) { - clusterService.removeListener(this); - ImmutableOpenMap shards = prevEntry.shards(); - RestoreInfo ri = new RestoreInfo(prevEntry.snapshot().getSnapshotId().getName(), - prevEntry.indices(), - shards.size(), - shards.size() - RestoreService.failedShards(shards)); - logger.debug("restore of [{}] completed", snapshot); - listener.onResponse(ri); - } else { - // restore not completed yet, wait for next cluster state update - } - } - }; - - clusterService.addListener(clusterStateListener); - } else { - listener.onResponse(restoreCompletionResponse.getRestoreInfo()); - } - } - - @Override - public void onFailure(Exception t) { - listener.onFailure(t); - } - }; - } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java new file mode 100644 index 0000000000000..187021e144d82 --- /dev/null +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -0,0 +1,409 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.ccr; + +import com.carrotsearch.hppc.cursors.ObjectCursor; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest; +import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.seqno.RetentionLease; +import org.elasticsearch.index.seqno.RetentionLeases; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.snapshots.RestoreInfo; +import org.elasticsearch.snapshots.RestoreService; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.CcrIntegTestCase; +import org.elasticsearch.xpack.ccr.action.repositories.ClearCcrRestoreSessionAction; +import org.elasticsearch.xpack.ccr.repository.CcrRepository; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.arrayWithSize; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasSize; + +public class CcrRetentionLeaseIT extends CcrIntegTestCase { + + public static final class RetentionLeaseRenewIntervalSettingPlugin extends Plugin { + + @Override + public List> getSettings() { + return Collections.singletonList(CcrRepository.RETENTION_LEASE_RENEW_INTERVAL_SETTING); + } + + } + + public static final class RetentionLeaseSyncIntervalSettingPlugin extends Plugin { + + @Override + public List> getSettings() { + return Collections.singletonList(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING); + } + + } + + @Override + protected Collection> nodePlugins() { + return Stream.concat( + super.nodePlugins().stream(), + Stream.of(RetentionLeaseRenewIntervalSettingPlugin.class, RetentionLeaseSyncIntervalSettingPlugin.class)) + .collect(Collectors.toList()); + } + + private final IndicesOptions indicesOptions = IndicesOptions.strictSingleIndexNoExpandForbidClosed(); + + private RestoreSnapshotRequest setUpRestoreSnapshotRequest( + final String leaderIndex, + final int numberOfShards, + final int numberOfReplicas, + final String followerIndex, + final int numberOfDocuments) throws IOException { + final ClusterUpdateSettingsRequest settingsRequest = new ClusterUpdateSettingsRequest(); + final String chunkSize = new ByteSizeValue(randomFrom(4, 128, 1024), ByteSizeUnit.KB).getStringRep(); + settingsRequest.persistentSettings(Settings.builder().put(CcrSettings.RECOVERY_CHUNK_SIZE.getKey(), chunkSize)); + assertAcked(followerClient().admin().cluster().updateSettings(settingsRequest).actionGet()); + + final String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; + + final Map additionalSettings = new HashMap<>(); + additionalSettings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true"); + additionalSettings.put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(200).getStringRep()); + final String leaderIndexSettings = getIndexSettings(numberOfShards, numberOfReplicas, additionalSettings); + assertAcked(leaderClient().admin().indices().prepareCreate(leaderIndex).setSource(leaderIndexSettings, XContentType.JSON)); + ensureLeaderGreen(leaderIndex); + + + logger.info("indexing [{}] docs", numberOfDocuments); + for (int i = 0; i < numberOfDocuments; i++) { + final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); + leaderClient().prepareIndex(leaderIndex, "doc", Integer.toString(i)).setSource(source, XContentType.JSON).get(); + if (rarely()) { + leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).setWaitIfOngoing(true).get(); + } + } + + leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).setWaitIfOngoing(true).get(); + + final Settings.Builder settingsBuilder = Settings.builder() + .put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, followerIndex) + .put(CcrRepository.RETENTION_LEASE_RENEW_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(200)) + .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true); + return new RestoreSnapshotRequest(leaderClusterRepoName, CcrRepository.LATEST) + .indexSettings(settingsBuilder) + .indices(leaderIndex) + .indicesOptions(indicesOptions) + .renamePattern("^(.*)$") + .renameReplacement(followerIndex) + .masterNodeTimeout(new TimeValue(1L, TimeUnit.HOURS)); + } + + public void testRetentionLeaseIsTakenAtTheStartOfRecovery() throws Exception { + final String leaderIndex = "leader"; + final int numberOfShards = randomIntBetween(1, 3); + final int numberOfReplicas = between(0, 1); + final String followerIndex = "follower"; + final int numberOfDocuments = scaledRandomIntBetween(1, 8192); + final RestoreSnapshotRequest restoreRequest = + setUpRestoreSnapshotRequest(leaderIndex, numberOfShards, numberOfReplicas, followerIndex, numberOfDocuments); + final RestoreService restoreService = getFollowerCluster().getCurrentMasterNodeInstance(RestoreService.class); + final ClusterService clusterService = getFollowerCluster().getCurrentMasterNodeInstance(ClusterService.class); + + final PlainActionFuture future = PlainActionFuture.newFuture(); + restoreService.restoreSnapshot(restoreRequest, waitForRestore(clusterService, future)); + + final ClusterStateResponse leaderIndexClusterState = + leaderClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(leaderIndex).get(); + final String leaderUUID = leaderIndexClusterState.getState().metaData().index(leaderIndex).getIndexUUID(); + + // ensure that a retention lease has been put in place on each shard + assertBusy(() -> { + final IndicesStatsResponse stats = + leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + assertNotNull(stats.getShards()); + assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); + final List shardStats = Arrays.stream(stats.getShards()) + .sorted((s, t) -> { + if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { + return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); + } else { + return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); + } + }) + .collect(Collectors.toList()); + for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { + final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); + assertThat(currentRetentionLeases.leases(), hasSize(1)); + final ClusterStateResponse followerIndexClusterState = + followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); + final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); + final RetentionLease retentionLease = + currentRetentionLeases.leases().iterator().next(); + assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + } + }); + + final RestoreInfo restoreInfo = future.actionGet(); + + assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards()); + assertEquals(0, restoreInfo.failedShards()); + for (int i = 0; i < numberOfDocuments; ++i) { + assertExpectedDocument(followerIndex, i); + } + + } + + public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { + final String leaderIndex = "leader"; + final int numberOfShards = randomIntBetween(1, 3); + final int numberOfReplicas = between(0, 1); + final String followerIndex = "follower"; + final int numberOfDocuments = scaledRandomIntBetween(1, 8192); + final RestoreSnapshotRequest restoreRequest = + setUpRestoreSnapshotRequest(leaderIndex, numberOfShards, numberOfReplicas, followerIndex, numberOfDocuments); + final RestoreService restoreService = getFollowerCluster().getCurrentMasterNodeInstance(RestoreService.class); + final ClusterService clusterService = getFollowerCluster().getCurrentMasterNodeInstance(ClusterService.class); + + final CountDownLatch latch = new CountDownLatch(1); + + // block the recovery from completing; this ensures the background sync is still running + final ClusterStateResponse followerClusterState = followerClient().admin().cluster().prepareState().clear().setNodes(true).get(); + for (final ObjectCursor senderNode : followerClusterState.getState().nodes().getDataNodes().values()) { + final MockTransportService senderTransportService = + (MockTransportService) getFollowerCluster().getInstance(TransportService.class, senderNode.value.getName()); + final ClusterStateResponse leaderClusterState = leaderClient().admin().cluster().prepareState().clear().setNodes(true).get(); + for (final ObjectCursor receiverNode : leaderClusterState.getState().nodes().getDataNodes().values()) { + final MockTransportService receiverTransportService = + (MockTransportService) getLeaderCluster().getInstance(TransportService.class, receiverNode.value.getName()); + senderTransportService.addSendBehavior(receiverTransportService, + (connection, requestId, action, request, options) -> { + if (ClearCcrRestoreSessionAction.NAME.equals(action)) { + try { + latch.await(); + } catch (final InterruptedException e) { + fail(e.toString()); + } + } + connection.sendRequest(requestId, action, request, options); + }); + + } + + } + + final PlainActionFuture future = PlainActionFuture.newFuture(); + restoreService.restoreSnapshot(restoreRequest, waitForRestore(clusterService, future)); + + final ClusterStateResponse leaderIndexClusterState = + leaderClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(leaderIndex).get(); + final String leaderUUID = leaderIndexClusterState.getState().metaData().index(leaderIndex).getIndexUUID(); + + // ensure that a retention lease has been put in place on each shard, and grab a copy of them + final List retentionLeases = new ArrayList<>(); + assertBusy(() -> { + retentionLeases.clear(); + final IndicesStatsResponse stats = + leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + assertNotNull(stats.getShards()); + assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); + final List shardStats = Arrays.stream(stats.getShards()) + .sorted((s, t) -> { + if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { + return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); + } else { + return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); + } + }) + .collect(Collectors.toList()); + for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { + final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); + assertThat(currentRetentionLeases.leases(), hasSize(1)); + final ClusterStateResponse followerIndexClusterState = + followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); + final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); + final RetentionLease retentionLease = + currentRetentionLeases.leases().iterator().next(); + assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + retentionLeases.add(currentRetentionLeases); + } + }); + + // now ensure that the retention leases are being renewed + assertBusy(() -> { + final IndicesStatsResponse stats = + leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + assertNotNull(stats.getShards()); + assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); + final List shardStats = Arrays.stream(stats.getShards()) + .sorted((s, t) -> { + if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { + return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); + } else { + return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); + } + }) + .collect(Collectors.toList()); + for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { + final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); + assertThat(currentRetentionLeases.leases(), hasSize(1)); + final ClusterStateResponse followerIndexClusterState = + followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); + final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); + final RetentionLease retentionLease = + currentRetentionLeases.leases().iterator().next(); + assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + // we assert that retention leases are being renewed by an increase in the timestamp + assertThat(retentionLease.timestamp(), greaterThan(retentionLeases.get(i).leases().iterator().next().timestamp())); + } + }); + + for (final ObjectCursor senderNode : followerClusterState.getState().nodes().getDataNodes().values()) { + final MockTransportService senderTransportService = + (MockTransportService) getFollowerCluster().getInstance(TransportService.class, senderNode.value.getName()); + senderTransportService.clearAllRules(); + } + + latch.countDown(); + + final RestoreInfo restoreInfo = future.actionGet(); + + assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards()); + assertEquals(0, restoreInfo.failedShards()); + for (int i = 0; i < numberOfDocuments; ++i) { + assertExpectedDocument(followerIndex, i); + } + } + + public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws Exception { + final String leaderIndex = "leader"; + final int numberOfShards = randomIntBetween(1, 3); + final int numberOfReplicas = between(0, 1); + final String followerIndex = "follower"; + final int numberOfDocuments = scaledRandomIntBetween(1, 8192); + final RestoreSnapshotRequest restoreRequest = + setUpRestoreSnapshotRequest(leaderIndex, numberOfShards, numberOfReplicas, followerIndex, numberOfDocuments); + final RestoreService restoreService = getFollowerCluster().getCurrentMasterNodeInstance(RestoreService.class); + final ClusterService clusterService = getFollowerCluster().getCurrentMasterNodeInstance(ClusterService.class); + + final PlainActionFuture future = PlainActionFuture.newFuture(); + restoreService.restoreSnapshot(restoreRequest, waitForRestore(clusterService, future)); + + final RestoreInfo restoreInfo = future.actionGet(); + final long start = System.nanoTime(); + + final ClusterStateResponse leaderIndexClusterState = + leaderClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(leaderIndex).get(); + final String leaderUUID = leaderIndexClusterState.getState().metaData().index(leaderIndex).getIndexUUID(); + + // sample the leases after recovery + final List retentionLeases = new ArrayList<>(); + assertBusy(() -> { + retentionLeases.clear(); + final IndicesStatsResponse stats = + leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + assertNotNull(stats.getShards()); + assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); + final List shardStats = Arrays.stream(stats.getShards()) + .sorted((s, t) -> { + if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { + return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); + } else { + return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); + } + }) + .collect(Collectors.toList()); + for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { + final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); + assertThat(currentRetentionLeases.leases(), hasSize(1)); + final ClusterStateResponse followerIndexClusterState = + followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); + final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); + final RetentionLease retentionLease = + currentRetentionLeases.leases().iterator().next(); + assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + retentionLeases.add(currentRetentionLeases); + } + }); + + final long end = System.nanoTime(); + Thread.sleep(Math.max(0, randomIntBetween(2, 4) * 200 - TimeUnit.NANOSECONDS.toMillis(end - start))); + + // now ensure that the retention leases are the same + assertBusy(() -> { + final IndicesStatsResponse stats = + leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + assertNotNull(stats.getShards()); + assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); + final List shardStats = Arrays.stream(stats.getShards()) + .sorted((s, t) -> { + if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { + return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); + } else { + return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); + } + }) + .collect(Collectors.toList()); + for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { + final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); + assertThat(currentRetentionLeases.leases(), hasSize(1)); + final ClusterStateResponse followerIndexClusterState = + followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); + final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); + final RetentionLease retentionLease = + currentRetentionLeases.leases().iterator().next(); + assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + // we assert that retention leases are being renewed by an increase in the timestamp + assertThat(retentionLease.timestamp(), equalTo(retentionLeases.get(i).leases().iterator().next().timestamp())); + } + }); + + assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards()); + assertEquals(0, restoreInfo.failedShards()); + for (int i = 0; i < numberOfDocuments; ++i) { + assertExpectedDocument(followerIndex, i); + } + } + + private void assertExpectedDocument(final String followerIndex, final int value) { + final GetResponse getResponse = followerClient().prepareGet(followerIndex, "doc", Integer.toString(value)).get(); + assertTrue("doc with id [" + value + "] is missing", getResponse.isExists()); + assertTrue((getResponse.getSource().containsKey("f"))); + assertThat(getResponse.getSource().get("f"), equalTo(value)); + } + +} From 6138176152e8247b56d68b7bb4287719668a2526 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 12 Feb 2019 22:28:39 -0500 Subject: [PATCH 02/30] Fix accidental import --- .../org/elasticsearch/index/engine/SoftDeletesPolicyTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java index 892d4c14f15bf..8257aa99d0486 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java @@ -36,7 +36,6 @@ import java.util.function.Supplier; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; -import static org.elasticsearch.index.seqno.SequenceNumbers.min; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; From 9ca7526e2a4e618ba66911d54203ae0b7a6054f2 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 12 Feb 2019 22:34:43 -0500 Subject: [PATCH 03/30] Update x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java --- .../org/elasticsearch/xpack/ccr/repository/CcrRepository.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 75e0f4ede0493..750a983dcd014 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -339,7 +339,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v // schedule renewals to run during the restore final Scheduler.Cancellable renewable = threadPool.scheduleWithFixedDelay( () -> { - logger.info("{} background renewing retention lease during restore", shardId); + logger.trace("{} background renewing retention lease during restore", shardId); renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); }, RETENTION_LEASE_RENEW_INTERVAL_SETTING.get(indexShard.indexSettings().getSettings()), From 530b55b98dcec513f82a3f7f0f5f68ee83c418dd Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 12 Feb 2019 23:46:59 -0500 Subject: [PATCH 04/30] Update x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java --- .../org/elasticsearch/xpack/ccr/repository/CcrRepository.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 750a983dcd014..6b7527be518ee 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -313,7 +313,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v final String retentionLeaseId = indexShard.shardId().getIndex().getUUID() + "-following-" + leaderUUID; logger.trace( - () -> new ParameterizedMessage("{} requesting leader primary to add retention lease [" + retentionLeaseId + "]", shardId)); + () -> new ParameterizedMessage("{} requesting leader primary to add retention lease [{}]", shardId, retentionLeaseId)); final Optional maybeAddAlready = addRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeAddAlready.ifPresent(addAlready -> { From f6e129e4d50b4c6bb195d9b2e3e09f219b49a0ca Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 13 Feb 2019 08:06:11 -0500 Subject: [PATCH 05/30] Force follower to fall behind --- .../xpack/ccr/CcrRetentionLeases.java | 24 ++++++++++++++ .../xpack/ccr/repository/CcrRepository.java | 3 +- .../xpack/ccr/CcrRetentionLeaseIT.java | 12 +++---- .../xpack/ccr/IndexFollowingIT.java | 32 ++++++++++++++++++- 4 files changed, 63 insertions(+), 8 deletions(-) create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java new file mode 100644 index 0000000000000..ee0f350602edf --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java @@ -0,0 +1,24 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.ccr; + +import java.util.Locale; + +public class CcrRetentionLeases { + + /** + * The retention lease ID used by followers. + * + * @param followerUUID the follower index UUID + * @param leaderUUID the leader index UUID + * @return the retention lease ID + */ + public static String retentionLeaseId(final String followerUUID, final String leaderUUID) { + return String.format(Locale.ROOT, "%s-following-%s", followerUUID, leaderUUID); + } + +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 6b7527be518ee..c652b2a1ea339 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -94,6 +94,7 @@ import static org.elasticsearch.index.seqno.RetentionLeaseActions.RETAIN_ALL; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; +import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.retentionLeaseId; /** @@ -311,7 +312,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v Index leaderIndex = new Index(shardId.getIndexName(), leaderUUID); ShardId leaderShardId = new ShardId(leaderIndex, shardId.getId()); - final String retentionLeaseId = indexShard.shardId().getIndex().getUUID() + "-following-" + leaderUUID; + final String retentionLeaseId = retentionLeaseId(indexShard.shardId().getIndex().getUUID(), leaderUUID); logger.trace( () -> new ParameterizedMessage("{} requesting leader primary to add retention lease [{}]", shardId, retentionLeaseId)); final Optional maybeAddAlready = diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index 187021e144d82..b2aa9f02b8c68 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -53,6 +53,7 @@ import java.util.stream.Stream; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.retentionLeaseId; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -108,7 +109,6 @@ private RestoreSnapshotRequest setUpRestoreSnapshotRequest( assertAcked(leaderClient().admin().indices().prepareCreate(leaderIndex).setSource(leaderIndexSettings, XContentType.JSON)); ensureLeaderGreen(leaderIndex); - logger.info("indexing [{}] docs", numberOfDocuments); for (int i = 0; i < numberOfDocuments; i++) { final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); @@ -174,7 +174,7 @@ public void testRetentionLeaseIsTakenAtTheStartOfRecovery() throws Exception { final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); } }); @@ -258,7 +258,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); retentionLeases.add(currentRetentionLeases); } }); @@ -286,7 +286,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); // we assert that retention leases are being renewed by an increase in the timestamp assertThat(retentionLease.timestamp(), greaterThan(retentionLeases.get(i).leases().iterator().next().timestamp())); } @@ -355,7 +355,7 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); retentionLeases.add(currentRetentionLeases); } }); @@ -386,7 +386,7 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(followerUUID + "-following-" + leaderUUID)); + assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); // we assert that retention leases are being renewed by an increase in the timestamp assertThat(retentionLease.timestamp(), equalTo(retentionLeases.get(i).leases().iterator().next().timestamp())); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index ade96b4614171..c5bd21cd2ca31 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -15,6 +15,7 @@ import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest; @@ -43,6 +44,7 @@ import org.elasticsearch.cluster.health.ClusterShardHealth; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; +import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.bytes.BytesReference; @@ -55,6 +57,7 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.seqno.RetentionLeaseActions; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.rest.RestStatus; @@ -92,6 +95,7 @@ import static java.util.Collections.singletonMap; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.retentionLeaseId; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -983,7 +987,7 @@ public void testMustCloseIndexAndPauseToRestartWithPutFollowing() throws Excepti public void testIndexFallBehind() throws Exception { final int numberOfPrimaryShards = randomIntBetween(1, 3); final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, between(0, 1), - singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); assertAcked(leaderClient().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); ensureLeaderYellow("index1"); @@ -1007,6 +1011,32 @@ public void testIndexFallBehind() throws Exception { pauseFollow("index2"); + // we have to remove the retention leases on the leader shards to ensure the follower falls behind + final ClusterStateResponse followerIndexClusterState = + followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices("index2").get(); + final String followerUUID = followerIndexClusterState.getState().metaData().index("index2").getIndexUUID(); + final ClusterStateResponse leaderIndexClusterState = + leaderClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices("index1").get(); + final String leaderUUID = leaderIndexClusterState.getState().metaData().index("index1").getIndexUUID(); + + final RoutingTable leaderRoutingTable = leaderClient() + .admin() + .cluster() + .prepareState() + .clear() + .setIndices("index1") + .setRoutingTable(true) + .get() + .getState() + .routingTable(); + for (int i = 0; i < numberOfPrimaryShards; i++) { + final ShardId shardId = leaderRoutingTable.index("index1").shard(i).shardId(); + leaderClient().execute( + RetentionLeaseActions.Remove.INSTANCE, + new RetentionLeaseActions.RemoveRequest(shardId, retentionLeaseId(followerUUID, leaderUUID))) + .get(); + } + for (int i = 0; i < numDocs; i++) { final String source = String.format(Locale.ROOT, "{\"f\":%d}", i * 2); leaderClient().prepareIndex("index1", "doc", Integer.toString(i)).setSource(source, XContentType.JSON).get(); From d294e9988d93a83cb2393c7a1c8c410062e302ff Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 13 Feb 2019 08:26:02 -0500 Subject: [PATCH 06/30] Add more tests --- .../xpack/ccr/IndexFollowingIT.java | 102 ++++++++++++------ 1 file changed, 68 insertions(+), 34 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index c5bd21cd2ca31..ac311307b19ef 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.ccr; +import com.carrotsearch.hppc.cursors.ObjectCursor; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ExceptionsHelper; @@ -44,8 +45,10 @@ import org.elasticsearch.cluster.health.ClusterShardHealth; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; @@ -89,7 +92,9 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; +import java.util.function.Consumer; import java.util.stream.Collectors; import static java.util.Collections.singletonMap; @@ -985,6 +990,60 @@ public void testMustCloseIndexAndPauseToRestartWithPutFollowing() throws Excepti } public void testIndexFallBehind() throws Exception { + runFallBehindTest( + () -> { + // we have to remove the retention leases on the leader shards to ensure the follower falls behind + final ClusterStateResponse followerIndexClusterState = + followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices("index2").get(); + final String followerUUID = followerIndexClusterState.getState().metaData().index("index2").getIndexUUID(); + final ClusterStateResponse leaderIndexClusterState = + leaderClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices("index1").get(); + final String leaderUUID = leaderIndexClusterState.getState().metaData().index("index1").getIndexUUID(); + + final RoutingTable leaderRoutingTable = leaderClient() + .admin() + .cluster() + .prepareState() + .clear() + .setIndices("index1") + .setRoutingTable(true) + .get() + .getState() + .routingTable(); + + for (ObjectCursor shardRoutingTable : leaderRoutingTable.index("index1").shards().values()) { + final ShardId shardId = shardRoutingTable.value.shardId(); + leaderClient().execute( + RetentionLeaseActions.Remove.INSTANCE, + new RetentionLeaseActions.RemoveRequest(shardId, retentionLeaseId(followerUUID, leaderUUID))) + .get(); + } + }, + exceptions -> assertThat(exceptions.size(), greaterThan(0))); + } + + public void testIndexDoesNotFallBehind() throws Exception { + runFallBehindTest( + () -> {}, + exceptions -> assertThat(exceptions.size(), equalTo(0))); + } + + /** + * Runs a fall behind test. In this test, we construct a situation where a follower is paused. While the follower is paused we index + * more documents that was causes soft deletes into the leader, flush them, and run a force merge. This is to set up a situation where + * the operations will not necessarily be there. With retention leases in place, we would actually expect the operations to be there. + * After pausing the follower, the specified callback is executed. This gives a test an opportunity to set up assumptions. For example, + * a test might remove all the retention leases on the leader to set up a situation where the follower will fall behind when it is + * resumed because the operations will no longer be held on the leader. The specified exceptions callback is invoked after resuming the + * follower to give a test an opportunity to assert on the resource not found exceptions (either present or not present). + * + * @param afterPausingFollower the callback to run after pausing the follower + * @param exceptionConsumer the callback to run on a collection of resource not found exceptions after resuming the follower + * @throws Exception + */ + private void runFallBehindTest( + final CheckedRunnable afterPausingFollower, + final Consumer> exceptionConsumer) throws Exception { final int numberOfPrimaryShards = randomIntBetween(1, 3); final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, between(0, 1), singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); @@ -1011,31 +1070,7 @@ public void testIndexFallBehind() throws Exception { pauseFollow("index2"); - // we have to remove the retention leases on the leader shards to ensure the follower falls behind - final ClusterStateResponse followerIndexClusterState = - followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices("index2").get(); - final String followerUUID = followerIndexClusterState.getState().metaData().index("index2").getIndexUUID(); - final ClusterStateResponse leaderIndexClusterState = - leaderClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices("index1").get(); - final String leaderUUID = leaderIndexClusterState.getState().metaData().index("index1").getIndexUUID(); - - final RoutingTable leaderRoutingTable = leaderClient() - .admin() - .cluster() - .prepareState() - .clear() - .setIndices("index1") - .setRoutingTable(true) - .get() - .getState() - .routingTable(); - for (int i = 0; i < numberOfPrimaryShards; i++) { - final ShardId shardId = leaderRoutingTable.index("index1").shard(i).shardId(); - leaderClient().execute( - RetentionLeaseActions.Remove.INSTANCE, - new RetentionLeaseActions.RemoveRequest(shardId, retentionLeaseId(followerUUID, leaderUUID))) - .get(); - } + afterPausingFollower.run(); for (int i = 0; i < numDocs; i++) { final String source = String.format(Locale.ROOT, "{\"f\":%d}", i * 2); @@ -1053,20 +1088,19 @@ public void testIndexFallBehind() throws Exception { assertBusy(() -> { List statuses = getFollowTaskStatuses("index2"); Set exceptions = statuses.stream() - .map(ShardFollowNodeTaskStatus::getFatalException) - .filter(Objects::nonNull) - .map(ExceptionsHelper::unwrapCause) - .filter(e -> e instanceof ResourceNotFoundException) - .map(e -> (ResourceNotFoundException) e) - .filter(e -> e.getMetadataKeys().contains("es.requested_operations_missing")) - .collect(Collectors.toSet()); - assertThat(exceptions.size(), greaterThan(0)); + .map(ShardFollowNodeTaskStatus::getFatalException) + .filter(Objects::nonNull) + .map(ExceptionsHelper::unwrapCause) + .filter(e -> e instanceof ResourceNotFoundException) + .map(e -> (ResourceNotFoundException) e) + .filter(e -> e.getMetadataKeys().contains("es.requested_operations_missing")) + .collect(Collectors.toSet()); + exceptionConsumer.accept(exceptions); }); followerClient().admin().indices().prepareClose("index2").get(); pauseFollow("index2"); - final PutFollowAction.Request followRequest2 = putFollow("index1", "index2"); PutFollowAction.Response response2 = followerClient().execute(PutFollowAction.INSTANCE, followRequest2).get(); assertTrue(response2.isFollowIndexCreated()); From b13c6c5ad984fc3ca4d9d945fa63fddc9ad58449 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 13 Feb 2019 10:04:35 -0500 Subject: [PATCH 07/30] Make linter happy --- .../test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index ac311307b19ef..d1aa3cc21d5d9 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -1039,7 +1039,7 @@ public void testIndexDoesNotFallBehind() throws Exception { * * @param afterPausingFollower the callback to run after pausing the follower * @param exceptionConsumer the callback to run on a collection of resource not found exceptions after resuming the follower - * @throws Exception + * @throws Exception if a checked exception is thrown during the test */ private void runFallBehindTest( final CheckedRunnable afterPausingFollower, From ab4d61c8fcafd0b9571807e0f694ef3d06fd0790 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 13 Feb 2019 10:24:33 -0500 Subject: [PATCH 08/30] Fix lost import --- .../test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java index db0d06b0ee087..e80a663c03f46 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java @@ -8,6 +8,7 @@ import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; From 30f136684f2a43a70538d4a69e7b8e8146a40cdd Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 13 Feb 2019 11:00:52 -0500 Subject: [PATCH 09/30] Finally --- .../xpack/ccr/repository/CcrRepository.java | 1 + .../xpack/ccr/CcrRetentionLeaseIT.java | 137 +++++++++--------- .../xpack/ccr/IndexFollowingIT.java | 2 - 3 files changed, 72 insertions(+), 68 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index c652b2a1ea339..6144f44c39588 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -355,6 +355,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v } catch (Exception e) { throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); } finally { + logger.trace("{} stopping background renewing retention lease at the end of recovery", shardId); renewable.cancel(); } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index b2aa9f02b8c68..5045d6fb984b3 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -233,80 +233,85 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { leaderClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(leaderIndex).get(); final String leaderUUID = leaderIndexClusterState.getState().metaData().index(leaderIndex).getIndexUUID(); - // ensure that a retention lease has been put in place on each shard, and grab a copy of them - final List retentionLeases = new ArrayList<>(); - assertBusy(() -> { - retentionLeases.clear(); - final IndicesStatsResponse stats = - leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); - assertNotNull(stats.getShards()); - assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); - final List shardStats = Arrays.stream(stats.getShards()) - .sorted((s, t) -> { - if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { - return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); - } else { - return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); - } - }) - .collect(Collectors.toList()); - for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { - final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(currentRetentionLeases.leases(), hasSize(1)); - final ClusterStateResponse followerIndexClusterState = - followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); - final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); - final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); - retentionLeases.add(currentRetentionLeases); - } - }); - - // now ensure that the retention leases are being renewed - assertBusy(() -> { - final IndicesStatsResponse stats = - leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); - assertNotNull(stats.getShards()); - assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); - final List shardStats = Arrays.stream(stats.getShards()) - .sorted((s, t) -> { - if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { - return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); - } else { - return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); - } - }) - .collect(Collectors.toList()); - for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { - final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(currentRetentionLeases.leases(), hasSize(1)); - final ClusterStateResponse followerIndexClusterState = - followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); - final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); - final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); - // we assert that retention leases are being renewed by an increase in the timestamp - assertThat(retentionLease.timestamp(), greaterThan(retentionLeases.get(i).leases().iterator().next().timestamp())); + try { + // ensure that a retention lease has been put in place on each shard, and grab a copy of them + final List retentionLeases = new ArrayList<>(); + assertBusy(() -> { + retentionLeases.clear(); + final IndicesStatsResponse stats = + leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + assertNotNull(stats.getShards()); + assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); + final List shardStats = Arrays.stream(stats.getShards()) + .sorted((s, t) -> { + if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { + return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); + } else { + return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); + } + }) + .collect(Collectors.toList()); + for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { + final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); + assertThat(currentRetentionLeases.leases(), hasSize(1)); + final ClusterStateResponse followerIndexClusterState = + followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); + final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); + final RetentionLease retentionLease = + currentRetentionLeases.leases().iterator().next(); + assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); + retentionLeases.add(currentRetentionLeases); + } + }); + + // now ensure that the retention leases are being renewed + assertBusy(() -> { + final IndicesStatsResponse stats = + leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + assertNotNull(stats.getShards()); + assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); + final List shardStats = Arrays.stream(stats.getShards()) + .sorted((s, t) -> { + if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { + return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); + } else { + return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); + } + }) + .collect(Collectors.toList()); + for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { + final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); + assertThat(currentRetentionLeases.leases(), hasSize(1)); + final ClusterStateResponse followerIndexClusterState = + followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); + final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); + final RetentionLease retentionLease = + currentRetentionLeases.leases().iterator().next(); + assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); + // we assert that retention leases are being renewed by an increase in the timestamp + assertThat(retentionLease.timestamp(), greaterThan(retentionLeases.get(i).leases().iterator().next().timestamp())); + } + }); + latch.countDown(); + } finally { + for (final ObjectCursor senderNode : followerClusterState.getState().nodes().getDataNodes().values()) { + final MockTransportService senderTransportService = + (MockTransportService) getFollowerCluster().getInstance(TransportService.class, senderNode.value.getName()); + senderTransportService.clearAllRules(); } - }); - - for (final ObjectCursor senderNode : followerClusterState.getState().nodes().getDataNodes().values()) { - final MockTransportService senderTransportService = - (MockTransportService) getFollowerCluster().getInstance(TransportService.class, senderNode.value.getName()); - senderTransportService.clearAllRules(); } - latch.countDown(); - final RestoreInfo restoreInfo = future.actionGet(); - assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards()); + assertEquals(restoreInfo.totalShards(), restoreInfo. + + successfulShards()); + assertEquals(0, restoreInfo.failedShards()); - for (int i = 0; i < numberOfDocuments; ++i) { + for (int i = 0; i < numberOfDocuments; i++) { assertExpectedDocument(followerIndex, i); } + } public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws Exception { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index d1aa3cc21d5d9..fdf9fc7529849 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -48,7 +48,6 @@ import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; @@ -92,7 +91,6 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.stream.Collectors; From d5dee9bad7ee70d549f294a6d6bd58a38d700132 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 13 Feb 2019 14:32:27 -0500 Subject: [PATCH 10/30] Tweak some log messages --- .../elasticsearch/xpack/ccr/repository/CcrRepository.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 61ed9ff640d23..55e85c84993e7 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -315,7 +315,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v final String retentionLeaseId = retentionLeaseId(indexShard.shardId().getIndex().getUUID(), leaderUUID); logger.trace( - () -> new ParameterizedMessage("{} requesting leader primary to add retention lease [{}]", shardId, retentionLeaseId)); + () -> new ParameterizedMessage("{} requesting leader to add retention lease [{}]", shardId, retentionLeaseId)); final Optional maybeAddAlready = addRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeAddAlready.ifPresent(addAlready -> { @@ -324,7 +324,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeRenewNotFound.ifPresent(renewNotFound -> { logger.trace(() -> new ParameterizedMessage( - "{} retention lease not found while attempting to renew, attempting a final add", shardId), renewNotFound); + "{} retention lease not found while attempting to renew, requesting a final add", shardId), renewNotFound); final Optional maybeFallbackAddAlready = addRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeFallbackAddAlready.ifPresent(fallbackAddAlready -> { @@ -356,7 +356,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v } catch (Exception e) { throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); } finally { - logger.trace("{} stopping background renewing retention lease at the end of recovery", shardId); + logger.trace("{} stopping background retention lease renewing at the end of recovery", shardId); renewable.cancel(); } } From ae9cff221be39cae485daf1dc4294ff46e322e4e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 13 Feb 2019 23:50:17 -0500 Subject: [PATCH 11/30] wip --- .../CcrRepositoryRetentionLeaseTests.java | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java new file mode 100644 index 0000000000000..138e3b829b7d5 --- /dev/null +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -0,0 +1,62 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.ccr.repository; + +import org.elasticsearch.Version; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.RepositoryMetaData; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.ccr.CcrLicenseChecker; +import org.elasticsearch.xpack.ccr.CcrSettings; + +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CcrRepositoryRetentionLeaseTests extends ESTestCase { + + public void test() { + final RepositoryMetaData repositoryMetaData = mock(RepositoryMetaData.class); + when(repositoryMetaData.name()).thenReturn(CcrRepository.NAME_PREFIX); + final Set> settings = + Stream.concat( + ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.stream(), + CcrSettings.getSettings().stream().filter(Setting::hasNodeScope)) + .collect(Collectors.toSet()); + + final CcrRepository repository = new CcrRepository( + repositoryMetaData, + mock(Client.class), + new CcrLicenseChecker(() -> true, () -> true), + Settings.EMPTY, + new CcrSettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, settings)), + mock(ThreadPool.class)); + repository.restoreShard( + mock(IndexShard.class), + new SnapshotId("snapshot-name", "snapshot-uuid"), + Version.CURRENT, + new IndexId("name", "id"), + new ShardId(new Index("index", "index-uuid"), 0), + mock(RecoveryState.class)); + } + +} \ No newline at end of file From a42d229cbd8fa74baa67d7074b9844b45e5120db Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 14 Feb 2019 08:29:09 -0500 Subject: [PATCH 12/30] Fix under security --- .../index/seqno/RetentionLeaseActions.java | 18 +++++++++--------- .../xpack/ccr/repository/CcrRepository.java | 15 ++++++++++++--- .../CcrRepositoryRetentionLeaseTests.java | 2 -- .../authz/privilege/SystemPrivilege.java | 3 +++ .../authz/privilege/PrivilegeTests.java | 4 ++++ .../authz/AuthorizationServiceTests.java | 2 ++ 6 files changed, 30 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java index f35e4906131e1..1e094dbc88236 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java @@ -136,10 +136,10 @@ protected boolean resolveIndex(final T request) { public static class Add extends Action { public static final Add INSTANCE = new Add(); - public static final String NAME = "indices:admin/seq_no/add_retention_lease"; + public static final String ACTION_NAME = "indices:admin/seq_no/add_retention_lease"; private Add() { - super(NAME); + super(ACTION_NAME); } public static class TransportAction extends TransportRetentionLeaseAction { @@ -153,7 +153,7 @@ public TransportAction( final IndexNameExpressionResolver indexNameExpressionResolver, final IndicesService indicesService) { super( - NAME, + ACTION_NAME, threadPool, clusterService, transportService, @@ -186,10 +186,10 @@ public Response newResponse() { public static class Renew extends Action { public static final Renew INSTANCE = new Renew(); - public static final String NAME = "indices:admin/seq_no/renew_retention_lease"; + public static final String ACTION_NAME = "indices:admin/seq_no/renew_retention_lease"; private Renew() { - super(NAME); + super(ACTION_NAME); } public static class TransportAction extends TransportRetentionLeaseAction { @@ -203,7 +203,7 @@ public TransportAction( final IndexNameExpressionResolver indexNameExpressionResolver, final IndicesService indicesService) { super( - NAME, + ACTION_NAME, threadPool, clusterService, transportService, @@ -232,10 +232,10 @@ public Response newResponse() { public static class Remove extends Action { public static final Remove INSTANCE = new Remove(); - public static final String NAME = "indices:admin/seq_no/remove_retention_lease"; + public static final String ACTION_NAME = "indices:admin/seq_no/remove_retention_lease"; private Remove() { - super(NAME); + super(ACTION_NAME); } public static class TransportAction extends TransportRetentionLeaseAction { @@ -249,7 +249,7 @@ public TransportAction( final IndexNameExpressionResolver indexNameExpressionResolver, final IndicesService indicesService) { super( - NAME, + ACTION_NAME, threadPool, clusterService, transportService, diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 55e85c84993e7..094d54966e76e 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.seqno.LocalCheckpointTracker; @@ -341,8 +342,16 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v // schedule renewals to run during the restore final Scheduler.Cancellable renewable = threadPool.scheduleWithFixedDelay( () -> { - logger.trace("{} background renewing retention lease during restore", shardId); - renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + final ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + // we have to execute under the system context so that if security is enabled the renewal is authorized + threadContext.markAsSystemContext(); + logger.trace("{} background renewal of retention lease during restore", shardId); + renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + } catch (final Exception e) { + logger.warn(new ParameterizedMessage("{} background renewal of retention lease failed during restore", shardId), e); + throw e; + } }, RETENTION_LEASE_RENEW_INTERVAL_SETTING.get(indexShard.indexSettings().getSettings()), Ccr.CCR_THREAD_POOL_NAME); @@ -356,7 +365,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v } catch (Exception e) { throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); } finally { - logger.trace("{} stopping background retention lease renewing at the end of recovery", shardId); + logger.trace("{} canceling background renewal of retention lease at the end of restore", shardId); renewable.cancel(); } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java index 138e3b829b7d5..71f91b9238638 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -8,7 +8,6 @@ import org.elasticsearch.Version; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; @@ -28,7 +27,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.junit.Assert.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/SystemPrivilege.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/SystemPrivilege.java index ec3305a963c55..523a810174912 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/SystemPrivilege.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/SystemPrivilege.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.core.security.authz.privilege; +import org.elasticsearch.index.seqno.RetentionLeaseActions; import org.elasticsearch.index.seqno.RetentionLeaseBackgroundSyncAction; import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; import org.elasticsearch.transport.TransportActionProxy; @@ -29,6 +30,8 @@ public final class SystemPrivilege extends Privilege { "indices:admin/seq_no/global_checkpoint_sync*", // needed for global checkpoint syncs RetentionLeaseSyncAction.ACTION_NAME + "*", // needed for retention lease syncs RetentionLeaseBackgroundSyncAction.ACTION_NAME + "*", // needed for background retention lease syncs + RetentionLeaseActions.Add.ACTION_NAME + "*", // needed for CCR to add retention leases + RetentionLeaseActions.Renew.ACTION_NAME + "*", // needed for CCR to renew retention leases "indices:admin/settings/update" // needed for DiskThresholdMonitor.markIndicesReadOnly ); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/PrivilegeTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/PrivilegeTests.java index 906b00ccab0fc..46db9e83f7740 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/PrivilegeTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/PrivilegeTests.java @@ -133,6 +133,10 @@ public void testSystem() throws Exception { assertThat(predicate.test("indices:admin/seq_no/retention_lease_background_sync"), is(true)); assertThat(predicate.test("indices:admin/seq_no/retention_lease_background_sync[p]"), is(true)); assertThat(predicate.test("indices:admin/seq_no/retention_lease_background_sync[r]"), is(true)); + assertThat(predicate.test("indices:admin/seq_no/add_retention_lease"), is(true)); + assertThat(predicate.test("indices:admin/seq_no/add_retention_lease[s]"), is(true)); + assertThat(predicate.test("indices:admin/seq_no/renew_retention_lease"), is(true)); + assertThat(predicate.test("indices:admin/seq_no/renew_retention_lease[s]"), is(true)); assertThat(predicate.test("indices:admin/settings/update"), is(true)); assertThat(predicate.test("indices:admin/settings/foo"), is(false)); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java index 7c4cd564e9993..bde5949d378b3 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java @@ -261,6 +261,8 @@ public void testActionsForSystemUserIsAuthorized() throws IOException { "indices:admin/seq_no/global_checkpoint_sync", "indices:admin/seq_no/retention_lease_sync", "indices:admin/seq_no/retention_lease_background_sync", + "indices:admin/seq_no/add_retention_lease", + "indices:admin/seq_no/renew_retention_lease", "indices:admin/settings/update" }; for (String action : actions) { authorize(authentication, action, request); From 2e6e7055b05aa98ba26d65e52d4d2c91b633a9fe Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 14 Feb 2019 08:29:55 -0500 Subject: [PATCH 13/30] Shift log message --- .../org/elasticsearch/xpack/ccr/repository/CcrRepository.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 094d54966e76e..6d78ab607234d 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -342,11 +342,11 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v // schedule renewals to run during the restore final Scheduler.Cancellable renewable = threadPool.scheduleWithFixedDelay( () -> { + logger.trace("{} background renewal of retention lease during restore", shardId); final ThreadContext threadContext = threadPool.getThreadContext(); try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { // we have to execute under the system context so that if security is enabled the renewal is authorized threadContext.markAsSystemContext(); - logger.trace("{} background renewal of retention lease during restore", shardId); renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); } catch (final Exception e) { logger.warn(new ParameterizedMessage("{} background renewal of retention lease failed during restore", shardId), e); From cbe044c468ac23feb468c2a263ba581b0199853c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 14 Feb 2019 11:42:12 -0500 Subject: [PATCH 14/30] More tests --- .../RetentionLeaseAlreadyExistsException.java | 2 +- .../RetentionLeaseNotFoundException.java | 2 +- .../xpack/ccr/CcrRetentionLeases.java | 18 ++- .../xpack/ccr/repository/CcrRepository.java | 114 ++++++++++------ .../xpack/ccr/CcrRetentionLeaseIT.java | 27 +++- .../xpack/ccr/IndexFollowingIT.java | 5 +- .../CcrRepositoryRetentionLeaseTests.java | 126 ++++++++++++++++-- 7 files changed, 227 insertions(+), 67 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseAlreadyExistsException.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseAlreadyExistsException.java index aaa41a7b400c1..ffd5e96e6a526 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseAlreadyExistsException.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseAlreadyExistsException.java @@ -27,7 +27,7 @@ public class RetentionLeaseAlreadyExistsException extends ResourceAlreadyExistsException { - RetentionLeaseAlreadyExistsException(final String id) { + public RetentionLeaseAlreadyExistsException(final String id) { super("retention lease with ID [" + Objects.requireNonNull(id) + "] already exists"); } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseNotFoundException.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseNotFoundException.java index d975077327fa3..2b13ae6b448e0 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseNotFoundException.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseNotFoundException.java @@ -27,7 +27,7 @@ public class RetentionLeaseNotFoundException extends ResourceNotFoundException { - RetentionLeaseNotFoundException(final String id) { + public RetentionLeaseNotFoundException(final String id) { super("retention lease with ID [" + Objects.requireNonNull(id) + "] not found"); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java index ee0f350602edf..98f1b8ef8e2fa 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java @@ -17,8 +17,22 @@ public class CcrRetentionLeases { * @param leaderUUID the leader index UUID * @return the retention lease ID */ - public static String retentionLeaseId(final String followerUUID, final String leaderUUID) { - return String.format(Locale.ROOT, "%s-following-%s", followerUUID, leaderUUID); + public static String retentionLeaseId( + final String localClusterName, + final String remoteClusterName, + final String followerIndexName, + final String followerUUID, + final String leaderIndexName, + final String leaderUUID) { + return String.format( + Locale.ROOT, + "%s/%s/%s-following-%s/%s/%s", + localClusterName, + followerIndexName, + followerUUID, + remoteClusterName, + leaderIndexName, + leaderUUID); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 6d78ab607234d..33a75840c976e 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -13,6 +13,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.IndexCommit; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; @@ -22,6 +23,7 @@ import org.elasticsearch.action.support.ListenerTimeouts; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -115,6 +117,7 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit private final RepositoryMetaData metadata; private final CcrSettings ccrSettings; + private final String localClusterName; private final String remoteClusterAlias; private final Client client; private final CcrLicenseChecker ccrLicenseChecker; @@ -126,6 +129,7 @@ public CcrRepository(RepositoryMetaData metadata, Client client, CcrLicenseCheck CcrSettings ccrSettings, ThreadPool threadPool) { this.metadata = metadata; this.ccrSettings = ccrSettings; + this.localClusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings).value(); assert metadata.name().startsWith(NAME_PREFIX) : "CcrRepository metadata.name() must start with: " + NAME_PREFIX; this.remoteClusterAlias = Strings.split(metadata.name(), NAME_PREFIX)[1]; this.ccrLicenseChecker = ccrLicenseChecker; @@ -153,10 +157,15 @@ public RepositoryMetaData getMetadata() { return metadata; } + private Client getRemoteClusterClient() { + return client.getRemoteClusterClient(remoteClusterAlias); + } + + @Override public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { assert SNAPSHOT_ID.equals(snapshotId) : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId"; - Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); + Client remoteClient = getRemoteClusterClient(); ClusterStateResponse response = remoteClient.admin().cluster().prepareState().clear().setMetaData(true).setNodes(true) .get(ccrSettings.getRecoveryActionTimeout()); ImmutableOpenMap indicesMap = response.getState().metaData().indices(); @@ -169,7 +178,7 @@ public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { @Override public MetaData getSnapshotGlobalMetaData(SnapshotId snapshotId) { assert SNAPSHOT_ID.equals(snapshotId) : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId"; - Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); + Client remoteClient = getRemoteClusterClient(); // We set a single dummy index name to avoid fetching all the index data ClusterStateRequest clusterStateRequest = CcrRequests.metaDataRequest("dummy_index_name"); ClusterStateResponse clusterState = remoteClient.admin().cluster().state(clusterStateRequest) @@ -181,7 +190,7 @@ public MetaData getSnapshotGlobalMetaData(SnapshotId snapshotId) { public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId index) throws IOException { assert SNAPSHOT_ID.equals(snapshotId) : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId"; String leaderIndex = index.getName(); - Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); + Client remoteClient = getRemoteClusterClient(); ClusterStateRequest clusterStateRequest = CcrRequests.metaDataRequest(leaderIndex); ClusterStateResponse clusterState = remoteClient.admin().cluster().state(clusterStateRequest) @@ -220,7 +229,7 @@ public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId ind @Override public RepositoryData getRepositoryData() { - Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); + Client remoteClient = getRemoteClusterClient(); ClusterStateResponse response = remoteClient.admin().cluster().prepareState().clear().setMetaData(true) .get(ccrSettings.getRecoveryActionTimeout()); MetaData remoteMetaData = response.getState().getMetaData(); @@ -297,24 +306,71 @@ public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId, RecoveryState recoveryState) { // TODO: Add timeouts to network calls / the restore process. + createEmptyStore(indexShard, shardId); + + final String followerIndexName = indexShard.shardId().getIndex().getName(); + final String followerUUID = indexShard.shardId().getIndex().getUUID(); + final Map ccrMetaData = indexShard.indexSettings().getIndexMetaData().getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY); + final String leaderIndexName = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_NAME_KEY); + final String leaderUUID = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY); + final Index leaderIndex = new Index(leaderIndexName, leaderUUID); + final ShardId leaderShardId = new ShardId(leaderIndex, shardId.getId()); + + final Client remoteClient = getRemoteClusterClient(); + + final String retentionLeaseId = + retentionLeaseId(localClusterName, remoteClusterAlias, followerIndexName, followerUUID, leaderIndexName, leaderUUID); + + acquireRetentionLeaseOnLeader(shardId, retentionLeaseId, leaderShardId, remoteClient); + + // schedule renewals to run during the restore + final Scheduler.Cancellable renewable = threadPool.scheduleWithFixedDelay( + () -> { + logger.trace("{} background renewal of retention lease during restore", shardId); + final ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + // we have to execute under the system context so that if security is enabled the renewal is authorized + threadContext.markAsSystemContext(); + renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + } catch (final Exception e) { + assert e instanceof ElasticsearchSecurityException == false : e; + logger.warn(new ParameterizedMessage("{} background renewal of retention lease failed during restore", shardId), e); + throw e; + } + }, + RETENTION_LEASE_RENEW_INTERVAL_SETTING.get(indexShard.indexSettings().getSettings()), + Ccr.CCR_THREAD_POOL_NAME); + + // TODO: There should be some local timeout. And if the remote cluster returns an unknown session + // response, we should be able to retry by creating a new session. + try (RestoreSession restoreSession = openSession(metadata.name(), remoteClient, leaderShardId, indexShard, recoveryState)) { + restoreSession.restoreFiles(); + updateMappings(remoteClient, leaderIndex, restoreSession.mappingVersion, client, indexShard.routingEntry().index()); + } catch (Exception e) { + throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); + } finally { + logger.trace("{} canceling background renewal of retention lease at the end of restore", shardId); + renewable.cancel(); + } + } + + private void createEmptyStore(final IndexShard indexShard, final ShardId shardId) { final Store store = indexShard.store(); store.incRef(); try { store.createEmpty(indexShard.indexSettings().getIndexMetaData().getCreationVersion().luceneVersion); - } catch (EngineException | IOException e) { + } catch (final EngineException | IOException e) { throw new IndexShardRecoveryException(shardId, "failed to create empty store", e); } finally { store.decRef(); } + } - Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); - - Map ccrMetaData = indexShard.indexSettings().getIndexMetaData().getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY); - String leaderUUID = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY); - Index leaderIndex = new Index(shardId.getIndexName(), leaderUUID); - ShardId leaderShardId = new ShardId(leaderIndex, shardId.getId()); - - final String retentionLeaseId = retentionLeaseId(indexShard.shardId().getIndex().getUUID(), leaderUUID); + void acquireRetentionLeaseOnLeader( + final ShardId shardId, + final String retentionLeaseId, + final ShardId leaderShardId, + final Client remoteClient) { logger.trace( () -> new ParameterizedMessage("{} requesting leader to add retention lease [{}]", shardId, retentionLeaseId)); final Optional maybeAddAlready = @@ -338,36 +394,6 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v }); }); }); - - // schedule renewals to run during the restore - final Scheduler.Cancellable renewable = threadPool.scheduleWithFixedDelay( - () -> { - logger.trace("{} background renewal of retention lease during restore", shardId); - final ThreadContext threadContext = threadPool.getThreadContext(); - try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { - // we have to execute under the system context so that if security is enabled the renewal is authorized - threadContext.markAsSystemContext(); - renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); - } catch (final Exception e) { - logger.warn(new ParameterizedMessage("{} background renewal of retention lease failed during restore", shardId), e); - throw e; - } - }, - RETENTION_LEASE_RENEW_INTERVAL_SETTING.get(indexShard.indexSettings().getSettings()), - Ccr.CCR_THREAD_POOL_NAME); - - // TODO: There should be some local timeout. And if the remote cluster returns an unknown session - // response, we should be able to retry by creating a new session. - String name = metadata.name(); - try (RestoreSession restoreSession = openSession(name, remoteClient, leaderShardId, indexShard, recoveryState)) { - restoreSession.restoreFiles(); - updateMappings(remoteClient, leaderIndex, restoreSession.mappingVersion, client, indexShard.routingEntry().index()); - } catch (Exception e) { - throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); - } finally { - logger.trace("{} canceling background renewal of retention lease at the end of restore", shardId); - renewable.cancel(); - } } private Optional addRetentionLease( @@ -430,7 +456,7 @@ private void updateMappings(Client leaderClient, Index leaderIndex, long leaderM } } - private RestoreSession openSession(String repositoryName, Client remoteClient, ShardId leaderShardId, IndexShard indexShard, + RestoreSession openSession(String repositoryName, Client remoteClient, ShardId leaderShardId, IndexShard indexShard, RecoveryState recoveryState) { String sessionUUID = UUIDs.randomBase64UUID(); PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index 5045d6fb984b3..713bfe1bd8dd0 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -174,7 +174,7 @@ public void testRetentionLeaseIsTakenAtTheStartOfRecovery() throws Exception { final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); + assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID))); } }); @@ -259,7 +259,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); + assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID))); retentionLeases.add(currentRetentionLeases); } }); @@ -287,7 +287,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); + assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID))); // we assert that retention leases are being renewed by an increase in the timestamp assertThat(retentionLease.timestamp(), greaterThan(retentionLeases.get(i).leases().iterator().next().timestamp())); } @@ -360,7 +360,14 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); + final String expectedRetentionLeaseId = retentionLeaseId( + getFollowerCluster().getClusterName(), + getLeaderCluster().getClusterName(), + followerIndex, + followerUUID, + leaderIndex, + leaderUUID); + assertThat(retentionLease.id(), equalTo(expectedRetentionLeaseId)); retentionLeases.add(currentRetentionLeases); } }); @@ -391,7 +398,7 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = currentRetentionLeases.leases().iterator().next(); - assertThat(retentionLease.id(), equalTo(retentionLeaseId(followerUUID, leaderUUID))); + assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID))); // we assert that retention leases are being renewed by an increase in the timestamp assertThat(retentionLease.timestamp(), equalTo(retentionLeases.get(i).leases().iterator().next().timestamp())); } @@ -404,6 +411,16 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws } } + private String getRetentionLeaseId(String followerIndex, String followerUUID, String leaderIndex, String leaderUUID) { + return retentionLeaseId( + getFollowerCluster().getClusterName(), + getLeaderCluster().getClusterName(), + followerIndex, + followerUUID, + leaderIndex, + leaderUUID); + } + private void assertExpectedDocument(final String followerIndex, final int value) { final GetResponse getResponse = followerClient().prepareGet(followerIndex, "doc", Integer.toString(value)).get(); assertTrue("doc with id [" + value + "] is missing", getResponse.isExists()); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index fdf9fc7529849..c1b78c48f3a56 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -1009,11 +1009,14 @@ public void testIndexFallBehind() throws Exception { .getState() .routingTable(); + final String retentionLeaseId = + retentionLeaseId(getFollowerCluster().getClusterName(), getLeaderCluster().getClusterName(), "index2", followerUUID, "index1", leaderUUID); + for (ObjectCursor shardRoutingTable : leaderRoutingTable.index("index1").shards().values()) { final ShardId shardId = shardRoutingTable.value.shardId(); leaderClient().execute( RetentionLeaseActions.Remove.INSTANCE, - new RetentionLeaseActions.RemoveRequest(shardId, retentionLeaseId(followerUUID, leaderUUID))) + new RetentionLeaseActions.RemoveRequest(shardId, retentionLeaseId)) .get(); } }, diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java index 71f91b9238638..cfdf263b62a99 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -6,33 +6,41 @@ package org.elasticsearch.xpack.ccr.repository; -import org.elasticsearch.Version; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; -import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.seqno.RetentionLeaseActions; +import org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException; +import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.indices.recovery.RecoveryState; -import org.elasticsearch.repositories.IndexId; -import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.CcrLicenseChecker; +import org.elasticsearch.xpack.ccr.CcrRetentionLeases; import org.elasticsearch.xpack.ccr.CcrSettings; +import org.mockito.ArgumentCaptor; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.elasticsearch.index.seqno.RetentionLeaseActions.RETAIN_ALL; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.same; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; public class CcrRepositoryRetentionLeaseTests extends ESTestCase { - public void test() { + public void testWhenRetentionLeaseAlreadyExistsWeTryToRenewIt() { final RepositoryMetaData repositoryMetaData = mock(RepositoryMetaData.class); when(repositoryMetaData.name()).thenReturn(CcrRepository.NAME_PREFIX); final Set> settings = @@ -48,13 +56,105 @@ public void test() { Settings.EMPTY, new CcrSettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, settings)), mock(ThreadPool.class)); - repository.restoreShard( - mock(IndexShard.class), - new SnapshotId("snapshot-name", "snapshot-uuid"), - Version.CURRENT, - new IndexId("name", "id"), - new ShardId(new Index("index", "index-uuid"), 0), - mock(RecoveryState.class)); + + final ShardId followerShardId = new ShardId(new Index("follower-index-name", "follower-index-uuid"), 0); + final ShardId leaderShardId = new ShardId(new Index("leader-index-name", "leader-index-uuid"), 0); + + final String retentionLeaseId = CcrRetentionLeases.retentionLeaseId( + "local-cluster", + "remote-cluster", + followerShardId.getIndex().getName(), + followerShardId.getIndex().getUUID(), + leaderShardId.getIndex().getName(), + leaderShardId.getIndex().getUUID()); + + // simulate that the the retention lease already exists on the leader, and verify that we attempt to renew it + final Client remoteClient = mock(Client.class); + final ArgumentCaptor addRequestCaptor = + ArgumentCaptor.forClass(RetentionLeaseActions.AddRequest.class); + when(remoteClient.execute(same(RetentionLeaseActions.Add.INSTANCE), addRequestCaptor.capture())) + .thenThrow(new RetentionLeaseAlreadyExistsException(retentionLeaseId)); + final ArgumentCaptor renewRequestCaptor = + ArgumentCaptor.forClass(RetentionLeaseActions.RenewRequest.class); + final PlainActionFuture response = new PlainActionFuture<>(); + response.onResponse(new RetentionLeaseActions.Response()); + when(remoteClient.execute(same(RetentionLeaseActions.Renew.INSTANCE), renewRequestCaptor.capture())) + .thenReturn(response); + + repository.acquireRetentionLeaseOnLeader(followerShardId, retentionLeaseId, leaderShardId, remoteClient); + + verify(remoteClient).execute(same(RetentionLeaseActions.Add.INSTANCE), any(RetentionLeaseActions.AddRequest.class)); + assertThat(addRequestCaptor.getValue().getShardId(), equalTo(leaderShardId)); + assertThat(addRequestCaptor.getValue().getId(), equalTo(retentionLeaseId)); + assertThat(addRequestCaptor.getValue().getRetainingSequenceNumber(), equalTo(RETAIN_ALL)); + assertThat(addRequestCaptor.getValue().getSource(), equalTo("ccr")); + + verify(remoteClient).execute(same(RetentionLeaseActions.Renew.INSTANCE), any(RetentionLeaseActions.RenewRequest.class)); + assertThat(renewRequestCaptor.getValue().getShardId(), equalTo(leaderShardId)); + assertThat(renewRequestCaptor.getValue().getId(), equalTo(retentionLeaseId)); + assertThat(renewRequestCaptor.getValue().getRetainingSequenceNumber(), equalTo(RETAIN_ALL)); + assertThat(renewRequestCaptor.getValue().getSource(), equalTo("ccr")); + + verifyNoMoreInteractions(remoteClient); + } + + public void testWhenRetentionLeaseExpiresBeforeWeCanRenewIt() { + final RepositoryMetaData repositoryMetaData = mock(RepositoryMetaData.class); + when(repositoryMetaData.name()).thenReturn(CcrRepository.NAME_PREFIX); + final Set> settings = + Stream.concat( + ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.stream(), + CcrSettings.getSettings().stream().filter(Setting::hasNodeScope)) + .collect(Collectors.toSet()); + + final CcrRepository repository = new CcrRepository( + repositoryMetaData, + mock(Client.class), + new CcrLicenseChecker(() -> true, () -> true), + Settings.EMPTY, + new CcrSettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, settings)), + mock(ThreadPool.class)); + + final ShardId followerShardId = new ShardId(new Index("follower-index-name", "follower-index-uuid"), 0); + final ShardId leaderShardId = new ShardId(new Index("leader-index-name", "leader-index-uuid"), 0); + + final String retentionLeaseId = CcrRetentionLeases.retentionLeaseId( + "local-cluster", + "remote-cluster", + followerShardId.getIndex().getName(), + followerShardId.getIndex().getUUID(), + leaderShardId.getIndex().getName(), + leaderShardId.getIndex().getUUID()); + + // simulate that the the retention lease already exists on the leader, expires before we renew, and verify that we attempt to add it + final Client remoteClient = mock(Client.class); + final ArgumentCaptor addRequestCaptor = + ArgumentCaptor.forClass(RetentionLeaseActions.AddRequest.class); + final PlainActionFuture response = new PlainActionFuture<>(); + response.onResponse(new RetentionLeaseActions.Response()); + when(remoteClient.execute(same(RetentionLeaseActions.Add.INSTANCE), addRequestCaptor.capture())) + .thenThrow(new RetentionLeaseAlreadyExistsException(retentionLeaseId)) + .thenReturn(response); + final ArgumentCaptor renewRequestCaptor = + ArgumentCaptor.forClass(RetentionLeaseActions.RenewRequest.class); + when(remoteClient.execute(same(RetentionLeaseActions.Renew.INSTANCE), renewRequestCaptor.capture())) + .thenThrow(new RetentionLeaseNotFoundException(retentionLeaseId)); + + repository.acquireRetentionLeaseOnLeader(followerShardId, retentionLeaseId, leaderShardId, remoteClient); + + verify(remoteClient, times(2)).execute(same(RetentionLeaseActions.Add.INSTANCE), any(RetentionLeaseActions.AddRequest.class)); + assertThat(addRequestCaptor.getValue().getShardId(), equalTo(leaderShardId)); + assertThat(addRequestCaptor.getValue().getId(), equalTo(retentionLeaseId)); + assertThat(addRequestCaptor.getValue().getRetainingSequenceNumber(), equalTo(RETAIN_ALL)); + assertThat(addRequestCaptor.getValue().getSource(), equalTo("ccr")); + + verify(remoteClient).execute(same(RetentionLeaseActions.Renew.INSTANCE), any(RetentionLeaseActions.RenewRequest.class)); + assertThat(renewRequestCaptor.getValue().getShardId(), equalTo(leaderShardId)); + assertThat(renewRequestCaptor.getValue().getId(), equalTo(retentionLeaseId)); + assertThat(renewRequestCaptor.getValue().getRetainingSequenceNumber(), equalTo(RETAIN_ALL)); + assertThat(renewRequestCaptor.getValue().getSource(), equalTo("ccr")); + + verifyNoMoreInteractions(remoteClient); } } \ No newline at end of file From 0ac55338bc5dbdb32300890a341d5e6a98e91f1b Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 14 Feb 2019 11:56:17 -0500 Subject: [PATCH 15/30] Fix comment --- .../elasticsearch/xpack/ccr/IndexFollowingIT.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index c1b78c48f3a56..2191f73db5da6 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -1031,12 +1031,12 @@ public void testIndexDoesNotFallBehind() throws Exception { /** * Runs a fall behind test. In this test, we construct a situation where a follower is paused. While the follower is paused we index - * more documents that was causes soft deletes into the leader, flush them, and run a force merge. This is to set up a situation where - * the operations will not necessarily be there. With retention leases in place, we would actually expect the operations to be there. - * After pausing the follower, the specified callback is executed. This gives a test an opportunity to set up assumptions. For example, - * a test might remove all the retention leases on the leader to set up a situation where the follower will fall behind when it is - * resumed because the operations will no longer be held on the leader. The specified exceptions callback is invoked after resuming the - * follower to give a test an opportunity to assert on the resource not found exceptions (either present or not present). + * more documents that causes soft deletes on the leader, flush them, and run a force merge. This is to set up a situation where the + * operations will not necessarily be there. With retention leases in place, we would actually expect the operations to be there. After + * pausing the follower, the specified callback is executed. This gives a test an opportunity to set up assumptions. For example, a test + * might remove all the retention leases on the leader to set up a situation where the follower will fall behind when it is resumed + * because the operations will no longer be held on the leader. The specified exceptions callback is invoked after resuming the follower + * to give a test an opportunity to assert on the resource not found exceptions (either present or not present). * * @param afterPausingFollower the callback to run after pausing the follower * @param exceptionConsumer the callback to run on a collection of resource not found exceptions after resuming the follower From 67918270804b8586cbc0b827dd6dec67f4b1428d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 14 Feb 2019 12:09:08 -0500 Subject: [PATCH 16/30] Fix imports --- .../elasticsearch/xpack/ccr/IndexFollowingIT.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index 2191f73db5da6..3f8cf008eb8df 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -1009,10 +1009,16 @@ public void testIndexFallBehind() throws Exception { .getState() .routingTable(); - final String retentionLeaseId = - retentionLeaseId(getFollowerCluster().getClusterName(), getLeaderCluster().getClusterName(), "index2", followerUUID, "index1", leaderUUID); - - for (ObjectCursor shardRoutingTable : leaderRoutingTable.index("index1").shards().values()) { + final String retentionLeaseId = retentionLeaseId( + getFollowerCluster().getClusterName(), + getLeaderCluster().getClusterName(), + "index2", + followerUUID, + "index1", + leaderUUID); + + for (final ObjectCursor shardRoutingTable + : leaderRoutingTable.index("index1").shards().values()) { final ShardId shardId = shardRoutingTable.value.shardId(); leaderClient().execute( RetentionLeaseActions.Remove.INSTANCE, From 0dcf6c919d0542ccb879141ec4f6200b882adb6e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 14 Feb 2019 12:11:56 -0500 Subject: [PATCH 17/30] Remove newline --- .../org/elasticsearch/xpack/ccr/repository/CcrRepository.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 33a75840c976e..994f779ae03bf 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -161,7 +161,6 @@ private Client getRemoteClusterClient() { return client.getRemoteClusterClient(remoteClusterAlias); } - @Override public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { assert SNAPSHOT_ID.equals(snapshotId) : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId"; From 086e87e7f06362724a14dd276b0aa044de99f251 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 14 Feb 2019 17:25:28 -0500 Subject: [PATCH 18/30] Refactor to helper --- .../xpack/ccr/CcrRetentionLeaseIT.java | 69 +++++++------------ 1 file changed, 24 insertions(+), 45 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index 713bfe1bd8dd0..ad5302b6d34b0 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -157,15 +157,7 @@ public void testRetentionLeaseIsTakenAtTheStartOfRecovery() throws Exception { leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); - final List shardStats = Arrays.stream(stats.getShards()) - .sorted((s, t) -> { - if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { - return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); - } else { - return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); - } - }) - .collect(Collectors.toList()); + final List shardStats = getShardStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); assertThat(currentRetentionLeases.leases(), hasSize(1)); @@ -242,15 +234,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); - final List shardStats = Arrays.stream(stats.getShards()) - .sorted((s, t) -> { - if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { - return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); - } else { - return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); - } - }) - .collect(Collectors.toList()); + final List shardStats = getShardStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); assertThat(currentRetentionLeases.leases(), hasSize(1)); @@ -270,15 +254,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); - final List shardStats = Arrays.stream(stats.getShards()) - .sorted((s, t) -> { - if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { - return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); - } else { - return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); - } - }) - .collect(Collectors.toList()); + final List shardStats = getShardStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); assertThat(currentRetentionLeases.leases(), hasSize(1)); @@ -343,15 +319,7 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); - final List shardStats = Arrays.stream(stats.getShards()) - .sorted((s, t) -> { - if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { - return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); - } else { - return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); - } - }) - .collect(Collectors.toList()); + final List shardStats = getShardStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); assertThat(currentRetentionLeases.leases(), hasSize(1)); @@ -381,15 +349,7 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); - final List shardStats = Arrays.stream(stats.getShards()) - .sorted((s, t) -> { - if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { - return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); - } else { - return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); - } - }) - .collect(Collectors.toList()); + final List shardStats = getShardStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); assertThat(currentRetentionLeases.leases(), hasSize(1)); @@ -411,6 +371,25 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws } } + /** + * Extract the shard stats from an indices stats response, with the stats ordered by shard ID with primaries first. This is to have a + * consistent ordering when comparing two responses. + * + * @param stats the indices stats + * @return the shard stats in sorted order with (shard ID, primary) as the sort key + */ + private List getShardStats(final IndicesStatsResponse stats) { + return Arrays.stream(stats.getShards()) + .sorted((s, t) -> { + if (s.getShardRouting().shardId().id() == t.getShardRouting().shardId().id()) { + return Boolean.compare(s.getShardRouting().primary(), t.getShardRouting().primary()); + } else { + return Integer.compare(s.getShardRouting().shardId().id(), t.getShardRouting().shardId().id()); + } + }) + .collect(Collectors.toList()); + } + private String getRetentionLeaseId(String followerIndex, String followerUUID, String leaderIndex, String leaderUUID) { return retentionLeaseId( getFollowerCluster().getClusterName(), From 69d757cd7cc8707ec0e91d4cca8ab4582b9c46df Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 10:33:58 -0500 Subject: [PATCH 19/30] Enhance logging messages --- .../index/seqno/RetentionLeaseActions.java | 2 +- .../xpack/ccr/repository/CcrRepository.java | 19 +++++++++++++++---- .../xpack/ccr/CcrRetentionLeaseIT.java | 10 +++++----- 3 files changed, 21 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java index 1e094dbc88236..6fa1fd7fb3f3e 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseActions.java @@ -115,7 +115,7 @@ public void onFailure(final Exception e) { } @Override - protected Response shardOperation(final T request, final ShardId shardId) throws IOException { + protected Response shardOperation(final T request, final ShardId shardId) { throw new UnsupportedOperationException(); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 994f779ae03bf..5fcde2fcdbd41 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -325,7 +325,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v // schedule renewals to run during the restore final Scheduler.Cancellable renewable = threadPool.scheduleWithFixedDelay( () -> { - logger.trace("{} background renewal of retention lease during restore", shardId); + logger.trace("{} background renewal of retention lease [{}] during restore", shardId, retentionLeaseId); final ThreadContext threadContext = threadPool.getThreadContext(); try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { // we have to execute under the system context so that if security is enabled the renewal is authorized @@ -333,7 +333,11 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); } catch (final Exception e) { assert e instanceof ElasticsearchSecurityException == false : e; - logger.warn(new ParameterizedMessage("{} background renewal of retention lease failed during restore", shardId), e); + logger.warn(new ParameterizedMessage( + "{} background renewal of retention lease [{}] failed during restore", + shardId, + retentionLeaseId), + e); throw e; } }, @@ -375,12 +379,19 @@ void acquireRetentionLeaseOnLeader( final Optional maybeAddAlready = addRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeAddAlready.ifPresent(addAlready -> { - logger.trace(() -> new ParameterizedMessage("{} retention lease already exists, requesting a renewal", shardId), addAlready); + logger.trace(() -> new ParameterizedMessage( + "{} retention lease [{}] already exists, requesting a renewal", + shardId, + retentionLeaseId), + addAlready); final Optional maybeRenewNotFound = renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeRenewNotFound.ifPresent(renewNotFound -> { logger.trace(() -> new ParameterizedMessage( - "{} retention lease not found while attempting to renew, requesting a final add", shardId), renewNotFound); + "{} retention lease [{}] not found while attempting to renew, requesting a final add", + shardId, + retentionLeaseId), + renewNotFound); final Optional maybeFallbackAddAlready = addRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeFallbackAddAlready.ifPresent(fallbackAddAlready -> { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index ad5302b6d34b0..04b7d7e7d0440 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -154,7 +154,7 @@ public void testRetentionLeaseIsTakenAtTheStartOfRecovery() throws Exception { // ensure that a retention lease has been put in place on each shard assertBusy(() -> { final IndicesStatsResponse stats = - leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); final List shardStats = getShardStats(stats); @@ -231,7 +231,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { assertBusy(() -> { retentionLeases.clear(); final IndicesStatsResponse stats = - leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); final List shardStats = getShardStats(stats); @@ -251,7 +251,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { // now ensure that the retention leases are being renewed assertBusy(() -> { final IndicesStatsResponse stats = - leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); final List shardStats = getShardStats(stats); @@ -316,7 +316,7 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws assertBusy(() -> { retentionLeases.clear(); final IndicesStatsResponse stats = - leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); final List shardStats = getShardStats(stats); @@ -346,7 +346,7 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws // now ensure that the retention leases are the same assertBusy(() -> { final IndicesStatsResponse stats = - leaderClient().admin().indices().stats(new IndicesStatsRequest().indices(leaderIndex)).actionGet(); + leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet(); assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); final List shardStats = getShardStats(stats); From cdf9bdc8d1c15441ed29980a08c9f0d34aafc304 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 10:58:02 -0500 Subject: [PATCH 20/30] Refactor --- .../xpack/ccr/CcrRetentionLeases.java | 26 ++++++++++--------- .../xpack/ccr/repository/CcrRepository.java | 4 +-- .../xpack/ccr/CcrRetentionLeaseIT.java | 13 ++++------ .../xpack/ccr/IndexFollowingIT.java | 7 +++-- .../CcrRepositoryRetentionLeaseTests.java | 19 ++++---------- 5 files changed, 28 insertions(+), 41 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java index 98f1b8ef8e2fa..122fbdb969aa7 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRetentionLeases.java @@ -6,6 +6,8 @@ package org.elasticsearch.xpack.ccr; +import org.elasticsearch.index.Index; + import java.util.Locale; public class CcrRetentionLeases { @@ -13,26 +15,26 @@ public class CcrRetentionLeases { /** * The retention lease ID used by followers. * - * @param followerUUID the follower index UUID - * @param leaderUUID the leader index UUID + * @param localClusterName the local cluster name + * @param followerIndex the follower index + * @param remoteClusterAlias the remote cluster alias + * @param leaderIndex the leader index * @return the retention lease ID */ public static String retentionLeaseId( final String localClusterName, - final String remoteClusterName, - final String followerIndexName, - final String followerUUID, - final String leaderIndexName, - final String leaderUUID) { + final Index followerIndex, + final String remoteClusterAlias, + final Index leaderIndex) { return String.format( Locale.ROOT, "%s/%s/%s-following-%s/%s/%s", localClusterName, - followerIndexName, - followerUUID, - remoteClusterName, - leaderIndexName, - leaderUUID); + followerIndex.getName(), + followerIndex.getUUID(), + remoteClusterAlias, + leaderIndex.getName(), + leaderIndex.getUUID()); } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 5fcde2fcdbd41..97ed0c8212c8f 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -307,8 +307,6 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v // TODO: Add timeouts to network calls / the restore process. createEmptyStore(indexShard, shardId); - final String followerIndexName = indexShard.shardId().getIndex().getName(); - final String followerUUID = indexShard.shardId().getIndex().getUUID(); final Map ccrMetaData = indexShard.indexSettings().getIndexMetaData().getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY); final String leaderIndexName = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_NAME_KEY); final String leaderUUID = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY); @@ -318,7 +316,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v final Client remoteClient = getRemoteClusterClient(); final String retentionLeaseId = - retentionLeaseId(localClusterName, remoteClusterAlias, followerIndexName, followerUUID, leaderIndexName, leaderUUID); + retentionLeaseId(localClusterName, indexShard.shardId().getIndex(), remoteClusterAlias, leaderIndex); acquireRetentionLeaseOnLeader(shardId, retentionLeaseId, leaderShardId, remoteClient); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index 04b7d7e7d0440..bbf6b69a081a8 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.seqno.RetentionLease; @@ -330,11 +331,9 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws currentRetentionLeases.leases().iterator().next(); final String expectedRetentionLeaseId = retentionLeaseId( getFollowerCluster().getClusterName(), + new Index(followerIndex, followerUUID), getLeaderCluster().getClusterName(), - followerIndex, - followerUUID, - leaderIndex, - leaderUUID); + new Index(leaderIndex, leaderUUID)); assertThat(retentionLease.id(), equalTo(expectedRetentionLeaseId)); retentionLeases.add(currentRetentionLeases); } @@ -393,11 +392,9 @@ private List getShardStats(final IndicesStatsResponse stats) { private String getRetentionLeaseId(String followerIndex, String followerUUID, String leaderIndex, String leaderUUID) { return retentionLeaseId( getFollowerCluster().getClusterName(), + new Index(followerIndex, followerUUID), getLeaderCluster().getClusterName(), - followerIndex, - followerUUID, - leaderIndex, - leaderUUID); + new Index(leaderIndex, leaderUUID)); } private void assertExpectedDocument(final String followerIndex, final int value) { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index 3f8cf008eb8df..5f37ba3a90411 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -57,6 +57,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.seqno.RetentionLeaseActions; @@ -1011,11 +1012,9 @@ public void testIndexFallBehind() throws Exception { final String retentionLeaseId = retentionLeaseId( getFollowerCluster().getClusterName(), + new Index("index2", followerUUID), getLeaderCluster().getClusterName(), - "index2", - followerUUID, - "index1", - leaderUUID); + new Index("index1", leaderUUID)); for (final ObjectCursor shardRoutingTable : leaderRoutingTable.index("index1").shards().values()) { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java index cfdf263b62a99..3970176da548b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -29,6 +29,7 @@ import java.util.stream.Stream; import static org.elasticsearch.index.seqno.RetentionLeaseActions.RETAIN_ALL; +import static org.elasticsearch.xpack.ccr.CcrRetentionLeases.retentionLeaseId; import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; import static org.mockito.Matchers.same; @@ -60,13 +61,8 @@ public void testWhenRetentionLeaseAlreadyExistsWeTryToRenewIt() { final ShardId followerShardId = new ShardId(new Index("follower-index-name", "follower-index-uuid"), 0); final ShardId leaderShardId = new ShardId(new Index("leader-index-name", "leader-index-uuid"), 0); - final String retentionLeaseId = CcrRetentionLeases.retentionLeaseId( - "local-cluster", - "remote-cluster", - followerShardId.getIndex().getName(), - followerShardId.getIndex().getUUID(), - leaderShardId.getIndex().getName(), - leaderShardId.getIndex().getUUID()); + final String retentionLeaseId = + retentionLeaseId("local-cluster", followerShardId.getIndex(), "remote-cluster", leaderShardId.getIndex()); // simulate that the the retention lease already exists on the leader, and verify that we attempt to renew it final Client remoteClient = mock(Client.class); @@ -118,13 +114,8 @@ public void testWhenRetentionLeaseExpiresBeforeWeCanRenewIt() { final ShardId followerShardId = new ShardId(new Index("follower-index-name", "follower-index-uuid"), 0); final ShardId leaderShardId = new ShardId(new Index("leader-index-name", "leader-index-uuid"), 0); - final String retentionLeaseId = CcrRetentionLeases.retentionLeaseId( - "local-cluster", - "remote-cluster", - followerShardId.getIndex().getName(), - followerShardId.getIndex().getUUID(), - leaderShardId.getIndex().getName(), - leaderShardId.getIndex().getUUID()); + final String retentionLeaseId = + retentionLeaseId("local-cluster", followerShardId.getIndex(), "remote-cluster", leaderShardId.getIndex()); // simulate that the the retention lease already exists on the leader, expires before we renew, and verify that we attempt to add it final Client remoteClient = mock(Client.class); From 5bbb07ff602d4de7418b862e5ab2d268dcd734f3 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 12:13:14 -0500 Subject: [PATCH 21/30] Fix imports --- .../xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java index 3970176da548b..4c097cd3feb2e 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -20,7 +20,6 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.ccr.CcrLicenseChecker; -import org.elasticsearch.xpack.ccr.CcrRetentionLeases; import org.elasticsearch.xpack.ccr.CcrSettings; import org.mockito.ArgumentCaptor; From 6fc440b50bae12053cd1051a56b49ab772549d93 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 13:13:44 -0500 Subject: [PATCH 22/30] Background syncs should be async --- .../xpack/ccr/repository/CcrRepository.java | 68 +++++++++++++------ 1 file changed, 48 insertions(+), 20 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 6cb5f9c539f63..2ce3dad9b7c84 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -16,6 +16,7 @@ import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; @@ -47,6 +48,7 @@ import org.elasticsearch.index.seqno.RetentionLeaseActions; import org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException; import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException; +import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardRecoveryException; import org.elasticsearch.index.shard.ShardId; @@ -326,15 +328,21 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { // we have to execute under the system context so that if security is enabled the renewal is authorized threadContext.markAsSystemContext(); - renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); - } catch (final Exception e) { - assert e instanceof ElasticsearchSecurityException == false : e; - logger.warn(new ParameterizedMessage( - "{} background renewal of retention lease [{}] failed during restore", - shardId, - retentionLeaseId), - e); - throw e; + // ignore the response, we fired the request asynchronously + asyncRenewRetentionLease( + leaderShardId, + retentionLeaseId, + remoteClient, + ActionListener.wrap( + r -> {}, + e -> { + assert e instanceof ElasticsearchSecurityException == false : e; + logger.warn(new ParameterizedMessage( + "{} background renewal of retention lease [{}] failed during restore", + shardId, + retentionLeaseId), + e); + })); } }, RETENTION_LEASE_RENEW_INTERVAL_SETTING.get(indexShard.indexSettings().getSettings()), @@ -373,7 +381,7 @@ void acquireRetentionLeaseOnLeader( logger.trace( () -> new ParameterizedMessage("{} requesting leader to add retention lease [{}]", shardId, retentionLeaseId)); final Optional maybeAddAlready = - addRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + syncAddRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeAddAlready.ifPresent(addAlready -> { logger.trace(() -> new ParameterizedMessage( "{} retention lease [{}] already exists, requesting a renewal", @@ -381,7 +389,7 @@ void acquireRetentionLeaseOnLeader( retentionLeaseId), addAlready); final Optional maybeRenewNotFound = - renewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + syncRenewRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeRenewNotFound.ifPresent(renewNotFound -> { logger.trace(() -> new ParameterizedMessage( "{} retention lease [{}] not found while attempting to renew, requesting a final add", @@ -389,7 +397,7 @@ void acquireRetentionLeaseOnLeader( retentionLeaseId), renewNotFound); final Optional maybeFallbackAddAlready = - addRetentionLease(leaderShardId, retentionLeaseId, remoteClient); + syncAddRetentionLease(leaderShardId, retentionLeaseId, remoteClient); maybeFallbackAddAlready.ifPresent(fallbackAddAlready -> { /* * At this point we tried to add the lease and the retention lease already existed. By the time we tried to renew the @@ -402,34 +410,54 @@ void acquireRetentionLeaseOnLeader( }); } - private Optional addRetentionLease( + private Optional syncAddRetentionLease( final ShardId leaderShardId, final String retentionLeaseId, final Client remoteClient) { try { - final RetentionLeaseActions.AddRequest request = - new RetentionLeaseActions.AddRequest(leaderShardId, retentionLeaseId, RETAIN_ALL, "ccr"); - remoteClient.execute(RetentionLeaseActions.Add.INSTANCE, request).actionGet(ccrSettings.getRecoveryActionTimeout()); + final PlainActionFuture response = new PlainActionFuture<>(); + asyncAddRetentionLease(leaderShardId, retentionLeaseId, remoteClient, response); + response.actionGet(ccrSettings.getRecoveryActionTimeout()); return Optional.empty(); } catch (final RetentionLeaseAlreadyExistsException e) { return Optional.of(e); } } - private Optional renewRetentionLease( + private void asyncAddRetentionLease( + final ShardId leaderShardId, + final String retentionLeaseId, + final Client remoteClient, + final ActionListener listener) { + final RetentionLeaseActions.AddRequest request = + new RetentionLeaseActions.AddRequest(leaderShardId, retentionLeaseId, RETAIN_ALL, "ccr"); + remoteClient.execute(RetentionLeaseActions.Add.INSTANCE, request, listener); + } + + private Optional syncRenewRetentionLease( final ShardId leaderShardId, final String retentionLeaseId, final Client remoteClient) { try { - final RetentionLeaseActions.RenewRequest request = - new RetentionLeaseActions.RenewRequest(leaderShardId, retentionLeaseId, RETAIN_ALL, "ccr"); - remoteClient.execute(RetentionLeaseActions.Renew.INSTANCE, request).actionGet(ccrSettings.getRecoveryActionTimeout()); + final PlainActionFuture response = new PlainActionFuture<>(); + asyncRenewRetentionLease(leaderShardId, retentionLeaseId, remoteClient, response); + response.actionGet(ccrSettings.getRecoveryActionTimeout()); return Optional.empty(); } catch (final RetentionLeaseNotFoundException e) { return Optional.of(e); } } + private void asyncRenewRetentionLease( + final ShardId leaderShardId, + final String retentionLeaseId, + final Client remoteClient, + final ActionListener listener) { + final RetentionLeaseActions.RenewRequest request = + new RetentionLeaseActions.RenewRequest(leaderShardId, retentionLeaseId, RETAIN_ALL, "ccr"); + remoteClient.execute(RetentionLeaseActions.Renew.INSTANCE, request, listener); + } + // this setting is intentionally not registered, it is only used in tests public static final Setting RETENTION_LEASE_RENEW_INTERVAL_SETTING = Setting.timeSetting( From 30232bb73674db2cdb4600301f997b591464a454 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 13:15:23 -0500 Subject: [PATCH 23/30] Remove leftover comment from previous iteration --- .../org/elasticsearch/xpack/ccr/repository/CcrRepository.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 2ce3dad9b7c84..570c3ab95dc41 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -328,7 +328,6 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { // we have to execute under the system context so that if security is enabled the renewal is authorized threadContext.markAsSystemContext(); - // ignore the response, we fired the request asynchronously asyncRenewRetentionLease( leaderShardId, retentionLeaseId, From 8fd90fe8ba833b14c36110cec92f3ad15db0dfa8 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 16:10:20 -0500 Subject: [PATCH 24/30] Fix imports --- .../org/elasticsearch/xpack/ccr/repository/CcrRepository.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 570c3ab95dc41..1efa95f3acd6f 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -16,7 +16,6 @@ import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; @@ -48,7 +47,6 @@ import org.elasticsearch.index.seqno.RetentionLeaseActions; import org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException; import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException; -import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardRecoveryException; import org.elasticsearch.index.shard.ShardId; From 21edf1d91aa2b21ccf574c615e1dee7ab3028b6e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 22:19:46 -0500 Subject: [PATCH 25/30] Fix mocking tests --- .../CcrRepositoryRetentionLeaseTests.java | 63 ++++++++++++++----- 1 file changed, 48 insertions(+), 15 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java index 4c097cd3feb2e..d0729a74c848b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.ccr.repository; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.RepositoryMetaData; @@ -24,6 +25,7 @@ import org.mockito.ArgumentCaptor; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -32,6 +34,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; import static org.mockito.Matchers.same; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -67,24 +70,36 @@ public void testWhenRetentionLeaseAlreadyExistsWeTryToRenewIt() { final Client remoteClient = mock(Client.class); final ArgumentCaptor addRequestCaptor = ArgumentCaptor.forClass(RetentionLeaseActions.AddRequest.class); - when(remoteClient.execute(same(RetentionLeaseActions.Add.INSTANCE), addRequestCaptor.capture())) - .thenThrow(new RetentionLeaseAlreadyExistsException(retentionLeaseId)); + doAnswer( + invocation -> { + @SuppressWarnings("unchecked") final ActionListener listener = + (ActionListener) invocation.getArguments()[2]; + listener.onFailure(new RetentionLeaseAlreadyExistsException(retentionLeaseId)); + return null; + }) + .when(remoteClient) + .execute(same(RetentionLeaseActions.Add.INSTANCE), addRequestCaptor.capture(), any()); final ArgumentCaptor renewRequestCaptor = ArgumentCaptor.forClass(RetentionLeaseActions.RenewRequest.class); - final PlainActionFuture response = new PlainActionFuture<>(); - response.onResponse(new RetentionLeaseActions.Response()); - when(remoteClient.execute(same(RetentionLeaseActions.Renew.INSTANCE), renewRequestCaptor.capture())) - .thenReturn(response); + doAnswer( + invocation -> { + @SuppressWarnings("unchecked") final ActionListener listener = + (ActionListener) invocation.getArguments()[2]; + listener.onResponse(new RetentionLeaseActions.Response()); + return null; + }) + .when(remoteClient) + .execute(same(RetentionLeaseActions.Renew.INSTANCE), renewRequestCaptor.capture(), any()); repository.acquireRetentionLeaseOnLeader(followerShardId, retentionLeaseId, leaderShardId, remoteClient); - verify(remoteClient).execute(same(RetentionLeaseActions.Add.INSTANCE), any(RetentionLeaseActions.AddRequest.class)); + verify(remoteClient).execute(same(RetentionLeaseActions.Add.INSTANCE), any(RetentionLeaseActions.AddRequest.class), any()); assertThat(addRequestCaptor.getValue().getShardId(), equalTo(leaderShardId)); assertThat(addRequestCaptor.getValue().getId(), equalTo(retentionLeaseId)); assertThat(addRequestCaptor.getValue().getRetainingSequenceNumber(), equalTo(RETAIN_ALL)); assertThat(addRequestCaptor.getValue().getSource(), equalTo("ccr")); - verify(remoteClient).execute(same(RetentionLeaseActions.Renew.INSTANCE), any(RetentionLeaseActions.RenewRequest.class)); + verify(remoteClient).execute(same(RetentionLeaseActions.Renew.INSTANCE), any(RetentionLeaseActions.RenewRequest.class), any()); assertThat(renewRequestCaptor.getValue().getShardId(), equalTo(leaderShardId)); assertThat(renewRequestCaptor.getValue().getId(), equalTo(retentionLeaseId)); assertThat(renewRequestCaptor.getValue().getRetainingSequenceNumber(), equalTo(RETAIN_ALL)); @@ -122,23 +137,41 @@ public void testWhenRetentionLeaseExpiresBeforeWeCanRenewIt() { ArgumentCaptor.forClass(RetentionLeaseActions.AddRequest.class); final PlainActionFuture response = new PlainActionFuture<>(); response.onResponse(new RetentionLeaseActions.Response()); - when(remoteClient.execute(same(RetentionLeaseActions.Add.INSTANCE), addRequestCaptor.capture())) - .thenThrow(new RetentionLeaseAlreadyExistsException(retentionLeaseId)) - .thenReturn(response); + final AtomicBoolean firstInvocation = new AtomicBoolean(true); + doAnswer( + invocation -> { + @SuppressWarnings("unchecked") final ActionListener listener = + (ActionListener) invocation.getArguments()[2]; + if (firstInvocation.compareAndSet(true, false)) { + listener.onFailure(new RetentionLeaseAlreadyExistsException(retentionLeaseId)); + } else { + listener.onResponse(new RetentionLeaseActions.Response()); + } + return null; + }) + .when(remoteClient).execute(same(RetentionLeaseActions.Add.INSTANCE), addRequestCaptor.capture(), any()); final ArgumentCaptor renewRequestCaptor = ArgumentCaptor.forClass(RetentionLeaseActions.RenewRequest.class); - when(remoteClient.execute(same(RetentionLeaseActions.Renew.INSTANCE), renewRequestCaptor.capture())) - .thenThrow(new RetentionLeaseNotFoundException(retentionLeaseId)); + doAnswer( + invocation -> { + @SuppressWarnings("unchecked") final ActionListener listener = + (ActionListener) invocation.getArguments()[2]; + listener.onFailure(new RetentionLeaseNotFoundException(retentionLeaseId)); + return null; + } + ).when(remoteClient) + .execute(same(RetentionLeaseActions.Renew.INSTANCE), renewRequestCaptor.capture(), any()); repository.acquireRetentionLeaseOnLeader(followerShardId, retentionLeaseId, leaderShardId, remoteClient); - verify(remoteClient, times(2)).execute(same(RetentionLeaseActions.Add.INSTANCE), any(RetentionLeaseActions.AddRequest.class)); + verify(remoteClient, times(2)) + .execute(same(RetentionLeaseActions.Add.INSTANCE), any(RetentionLeaseActions.AddRequest.class), any()); assertThat(addRequestCaptor.getValue().getShardId(), equalTo(leaderShardId)); assertThat(addRequestCaptor.getValue().getId(), equalTo(retentionLeaseId)); assertThat(addRequestCaptor.getValue().getRetainingSequenceNumber(), equalTo(RETAIN_ALL)); assertThat(addRequestCaptor.getValue().getSource(), equalTo("ccr")); - verify(remoteClient).execute(same(RetentionLeaseActions.Renew.INSTANCE), any(RetentionLeaseActions.RenewRequest.class)); + verify(remoteClient).execute(same(RetentionLeaseActions.Renew.INSTANCE), any(RetentionLeaseActions.RenewRequest.class), any()); assertThat(renewRequestCaptor.getValue().getShardId(), equalTo(leaderShardId)); assertThat(renewRequestCaptor.getValue().getId(), equalTo(retentionLeaseId)); assertThat(renewRequestCaptor.getValue().getRetainingSequenceNumber(), equalTo(RETAIN_ALL)); From 7e547eec32710251f5f201f88362dccf940c876e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 22:22:07 -0500 Subject: [PATCH 26/30] Refactor --- .../CcrRepositoryRetentionLeaseTests.java | 38 +++++++++++-------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java index d0729a74c848b..c3bab0510f70c 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -23,6 +23,8 @@ import org.elasticsearch.xpack.ccr.CcrLicenseChecker; import org.elasticsearch.xpack.ccr.CcrSettings; import org.mockito.ArgumentCaptor; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -71,9 +73,9 @@ public void testWhenRetentionLeaseAlreadyExistsWeTryToRenewIt() { final ArgumentCaptor addRequestCaptor = ArgumentCaptor.forClass(RetentionLeaseActions.AddRequest.class); doAnswer( - invocation -> { + invocationOnMock -> { @SuppressWarnings("unchecked") final ActionListener listener = - (ActionListener) invocation.getArguments()[2]; + (ActionListener) invocationOnMock.getArguments()[2]; listener.onFailure(new RetentionLeaseAlreadyExistsException(retentionLeaseId)); return null; }) @@ -82,9 +84,9 @@ public void testWhenRetentionLeaseAlreadyExistsWeTryToRenewIt() { final ArgumentCaptor renewRequestCaptor = ArgumentCaptor.forClass(RetentionLeaseActions.RenewRequest.class); doAnswer( - invocation -> { + invocationOnMock -> { @SuppressWarnings("unchecked") final ActionListener listener = - (ActionListener) invocation.getArguments()[2]; + (ActionListener) invocationOnMock.getArguments()[2]; listener.onResponse(new RetentionLeaseActions.Response()); return null; }) @@ -137,25 +139,31 @@ public void testWhenRetentionLeaseExpiresBeforeWeCanRenewIt() { ArgumentCaptor.forClass(RetentionLeaseActions.AddRequest.class); final PlainActionFuture response = new PlainActionFuture<>(); response.onResponse(new RetentionLeaseActions.Response()); - final AtomicBoolean firstInvocation = new AtomicBoolean(true); doAnswer( - invocation -> { - @SuppressWarnings("unchecked") final ActionListener listener = - (ActionListener) invocation.getArguments()[2]; - if (firstInvocation.compareAndSet(true, false)) { - listener.onFailure(new RetentionLeaseAlreadyExistsException(retentionLeaseId)); - } else { - listener.onResponse(new RetentionLeaseActions.Response()); + new Answer() { + + final AtomicBoolean firstInvocation = new AtomicBoolean(true); + + @Override + public Object answer(final InvocationOnMock invocationOnMock) { + @SuppressWarnings("unchecked") final ActionListener listener = + (ActionListener) invocationOnMock.getArguments()[2]; + if (firstInvocation.compareAndSet(true, false)) { + listener.onFailure(new RetentionLeaseAlreadyExistsException(retentionLeaseId)); + } else { + listener.onResponse(new RetentionLeaseActions.Response()); + } + return null; } - return null; + }) .when(remoteClient).execute(same(RetentionLeaseActions.Add.INSTANCE), addRequestCaptor.capture(), any()); final ArgumentCaptor renewRequestCaptor = ArgumentCaptor.forClass(RetentionLeaseActions.RenewRequest.class); doAnswer( - invocation -> { + invocationOnMock -> { @SuppressWarnings("unchecked") final ActionListener listener = - (ActionListener) invocation.getArguments()[2]; + (ActionListener) invocationOnMock.getArguments()[2]; listener.onFailure(new RetentionLeaseNotFoundException(retentionLeaseId)); return null; } From d471872d82e78621b37a48b9183d7a203a201a0d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 22:37:58 -0500 Subject: [PATCH 27/30] Update x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java --- .../ccr/repository/CcrRepositoryRetentionLeaseTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java index c3bab0510f70c..e78521b527e2b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -140,7 +140,7 @@ public void testWhenRetentionLeaseExpiresBeforeWeCanRenewIt() { final PlainActionFuture response = new PlainActionFuture<>(); response.onResponse(new RetentionLeaseActions.Response()); doAnswer( - new Answer() { + new Answer() { final AtomicBoolean firstInvocation = new AtomicBoolean(true); @@ -188,4 +188,4 @@ public Object answer(final InvocationOnMock invocationOnMock) { verifyNoMoreInteractions(remoteClient); } -} \ No newline at end of file +} From fc9a300a6f83a72b83cd7d127294ef0737fe872d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 15 Feb 2019 22:38:10 -0500 Subject: [PATCH 28/30] Update x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java --- .../xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java index e78521b527e2b..2e382f739300b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -145,7 +145,7 @@ public void testWhenRetentionLeaseExpiresBeforeWeCanRenewIt() { final AtomicBoolean firstInvocation = new AtomicBoolean(true); @Override - public Object answer(final InvocationOnMock invocationOnMock) { + public Void answer(final InvocationOnMock invocationOnMock) { @SuppressWarnings("unchecked") final ActionListener listener = (ActionListener) invocationOnMock.getArguments()[2]; if (firstInvocation.compareAndSet(true, false)) { From 6da443a75d975d29140ead840aa8e824da01d912 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sat, 16 Feb 2019 08:01:31 -0500 Subject: [PATCH 29/30] Tweak logging message --- .../org/elasticsearch/xpack/ccr/repository/CcrRepository.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 1efa95f3acd6f..41cce3f5b0b06 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -353,7 +353,7 @@ public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version v } catch (Exception e) { throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); } finally { - logger.trace("{} canceling background renewal of retention lease at the end of restore", shardId); + logger.trace("{} canceling background renewal of retention lease [{}] at the end of restore", shardId, retentionLeaseId); renewable.cancel(); } } From dec3d285177a03c5c6f126a2fa019316aa083d90 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sat, 16 Feb 2019 14:02:33 -0500 Subject: [PATCH 30/30] Test awaits fix --- .../elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java b/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java index b3c93acb97b99..8b4c21ee086aa 100644 --- a/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java +++ b/x-pack/plugin/ilm/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/indexlifecycle/CCRIndexLifecycleIT.java @@ -278,6 +278,7 @@ public void testCcrAndIlmWithRollover() throws Exception { } } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37165") public void testUnfollowInjectedBeforeShrink() throws Exception { final String indexName = "shrink-test"; final String shrunkenIndexName = "shrink-" + indexName;