Browse Source

Implement basic `CcrRepository` restore (#36287)

This is related to #35975. It implements a basic restore functionality
for the CcrRepository. When the restore process is kicked off, it
configures the new index as expected for a follower index. This means
that the index has a different uuid, the version is not incremented, and
the Ccr metadata is installed.

When the restore shard method is called, an empty shard is initialized.
Tim Brooks 6 years ago
parent
commit
8a53f2b464

+ 4 - 0
server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java

@@ -235,6 +235,10 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
         this(snapshotId, indices, state, null, null, 0L, 0L, 0, 0, Collections.emptyList(), null);
     }
 
+    public SnapshotInfo(SnapshotId snapshotId, List<String> indices, SnapshotState state, Version version) {
+        this(snapshotId, indices, state, null, version, 0L, 0L, 0, 0, Collections.emptyList(), null);
+    }
+
     public SnapshotInfo(SnapshotId snapshotId, List<String> indices, long startTime, Boolean includeGlobalState) {
         this(snapshotId, indices, SnapshotState.IN_PROGRESS, null, Version.CURRENT, startTime, 0L, 0, 0,
             Collections.emptyList(), includeGlobalState);

+ 4 - 0
test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java

@@ -1455,6 +1455,10 @@ public final class InternalTestCluster extends TestCluster {
         return getInstances(clazz, new DataNodePredicate());
     }
 
+    public synchronized <T> T getCurrentMasterNodeInstance(Class<T> clazz) {
+        return getInstance(clazz, new NodeNamePredicate(getMasterName()));
+    }
+
     /**
      * Returns an Iterable to all instances for the given class &gt;T&lt; across all data and master nodes
      * in the cluster.

+ 3 - 1
x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java

@@ -112,6 +112,7 @@ public class Ccr extends Plugin implements ActionPlugin, PersistentTaskPlugin, E
     private final Settings settings;
     private final CcrLicenseChecker ccrLicenseChecker;
     private final SetOnce<CcrRepositoryManager> repositoryManager = new SetOnce<>();
+    private Client client;
 
     /**
      * Construct an instance of the CCR container with the specified settings.
@@ -146,6 +147,7 @@ public class Ccr extends Plugin implements ActionPlugin, PersistentTaskPlugin, E
             final Environment environment,
             final NodeEnvironment nodeEnvironment,
             final NamedWriteableRegistry namedWriteableRegistry) {
+        this.client = client;
         if (enabled == false) {
             return emptyList();
         }
@@ -275,7 +277,7 @@ public class Ccr extends Plugin implements ActionPlugin, PersistentTaskPlugin, E
 
     @Override
     public Map<String, Repository.Factory> getInternalRepositories(Environment env, NamedXContentRegistry namedXContentRegistry) {
-        Repository.Factory repositoryFactory = (metadata) -> new CcrRepository(metadata, settings);
+        Repository.Factory repositoryFactory = (metadata) -> new CcrRepository(metadata, client, ccrLicenseChecker, settings);
         return Collections.singletonMap(CcrRepository.TYPE, repositoryFactory);
     }
 

+ 103 - 7
x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java

@@ -8,13 +8,21 @@ package org.elasticsearch.xpack.ccr.repository;
 
 import org.apache.lucene.index.IndexCommit;
 import org.elasticsearch.Version;
+import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
+import org.elasticsearch.action.support.PlainActionFuture;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.metadata.RepositoryMetaData;
 import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.Strings;
+import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.component.AbstractLifecycleComponent;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.index.Index;
+import org.elasticsearch.index.engine.EngineException;
 import org.elasticsearch.index.shard.IndexShard;
+import org.elasticsearch.index.shard.IndexShardRecoveryException;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
 import org.elasticsearch.index.store.Store;
@@ -25,9 +33,17 @@ import org.elasticsearch.repositories.RepositoryData;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SnapshotInfo;
 import org.elasticsearch.snapshots.SnapshotShardFailure;
+import org.elasticsearch.snapshots.SnapshotState;
+import org.elasticsearch.xpack.ccr.Ccr;
+import org.elasticsearch.xpack.ccr.CcrLicenseChecker;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * This repository relies on a remote cluster for Ccr restores. It is read-only so it can only be used to
@@ -35,14 +51,23 @@ import java.util.List;
  */
 public class CcrRepository extends AbstractLifecycleComponent implements Repository {
 
+    public static final String LATEST = "_latest_";
     public static final String TYPE = "_ccr_";
     public static final String NAME_PREFIX = "_ccr_";
+    private static final SnapshotId SNAPSHOT_ID = new SnapshotId(LATEST, LATEST);
 
     private final RepositoryMetaData metadata;
+    private final String remoteClusterAlias;
+    private final Client client;
+    private final CcrLicenseChecker ccrLicenseChecker;
 
-    public CcrRepository(RepositoryMetaData metadata, Settings settings) {
+    public CcrRepository(RepositoryMetaData metadata, Client client, CcrLicenseChecker ccrLicenseChecker, Settings settings) {
         super(settings);
         this.metadata = metadata;
+        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;
+        this.client = client;
     }
 
     @Override
@@ -67,22 +92,85 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
 
     @Override
     public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) {
-        throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE);
+        assert SNAPSHOT_ID.equals(snapshotId) : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId";
+        Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias);
+        ClusterStateResponse response = remoteClient.admin().cluster().prepareState().clear().setMetaData(true).setNodes(true).get();
+        ImmutableOpenMap<String, IndexMetaData> indicesMap = response.getState().metaData().indices();
+        ArrayList<String> indices = new ArrayList<>(indicesMap.size());
+        indicesMap.keysIt().forEachRemaining(indices::add);
+
+        return new SnapshotInfo(snapshotId, indices, SnapshotState.SUCCESS, response.getState().getNodes().getMaxNodeVersion());
     }
 
     @Override
     public MetaData getSnapshotGlobalMetaData(SnapshotId snapshotId) {
-        throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE);
+        assert SNAPSHOT_ID.equals(snapshotId) : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId";
+        Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias);
+        ClusterStateResponse response = remoteClient
+            .admin()
+            .cluster()
+            .prepareState()
+            .clear()
+            .setMetaData(true)
+            .setIndices("dummy_index_name") // We set a single dummy index name to avoid fetching all the index data
+            .get();
+        return response.getState().metaData();
     }
 
     @Override
     public IndexMetaData getSnapshotIndexMetaData(SnapshotId snapshotId, IndexId index) throws IOException {
-        throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE);
+        assert SNAPSHOT_ID.equals(snapshotId) : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId";
+        String leaderIndex = index.getName();
+        Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias);
+
+        ClusterStateResponse response = remoteClient
+            .admin()
+            .cluster()
+            .prepareState()
+            .clear()
+            .setMetaData(true)
+            .setIndices(leaderIndex)
+            .get();
+
+        // Validates whether the leader cluster has been configured properly:
+        PlainActionFuture<String[]> future = PlainActionFuture.newFuture();
+        IndexMetaData leaderIndexMetaData = response.getState().metaData().index(leaderIndex);
+        ccrLicenseChecker.fetchLeaderHistoryUUIDs(remoteClient, leaderIndexMetaData, future::onFailure, future::onResponse);
+        String[] leaderHistoryUUIDs = future.actionGet();
+
+        IndexMetaData.Builder imdBuilder = IndexMetaData.builder(leaderIndexMetaData);
+        // Adding the leader index uuid for each shard as custom metadata:
+        Map<String, String> metadata = new HashMap<>();
+        metadata.put(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_SHARD_HISTORY_UUIDS, String.join(",", leaderHistoryUUIDs));
+        metadata.put(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY, leaderIndexMetaData.getIndexUUID());
+        metadata.put(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_NAME_KEY, leaderIndexMetaData.getIndex().getName());
+        metadata.put(Ccr.CCR_CUSTOM_METADATA_REMOTE_CLUSTER_NAME_KEY, remoteClusterAlias);
+        imdBuilder.putCustom(Ccr.CCR_CUSTOM_METADATA_KEY, metadata);
+
+        return imdBuilder.build();
     }
 
     @Override
     public RepositoryData getRepositoryData() {
-        throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE);
+        Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias);
+        ClusterStateResponse response = remoteClient.admin().cluster().prepareState().clear().setMetaData(true).get();
+        MetaData remoteMetaData = response.getState().getMetaData();
+
+        Map<String, SnapshotId> copiedSnapshotIds = new HashMap<>();
+        Map<String, SnapshotState> snapshotStates = new HashMap<>(copiedSnapshotIds.size());
+        Map<IndexId, Set<SnapshotId>> indexSnapshots = new HashMap<>(copiedSnapshotIds.size());
+
+        ImmutableOpenMap<String, IndexMetaData> remoteIndices = remoteMetaData.getIndices();
+        for (String indexName : remoteMetaData.getConcreteAllIndices()) {
+            // Both the Snapshot name and UUID are set to _latest_
+            SnapshotId snapshotId = new SnapshotId(LATEST, LATEST);
+            copiedSnapshotIds.put(indexName, snapshotId);
+            snapshotStates.put(indexName, SnapshotState.SUCCESS);
+            Index index = remoteIndices.get(indexName).getIndex();
+            indexSnapshots.put(new IndexId(indexName, index.getUUID()), Collections.singleton(snapshotId));
+        }
+
+        return new RepositoryData(1, copiedSnapshotIds, snapshotStates, indexSnapshots, Collections.emptyList());
     }
 
     @Override
@@ -137,9 +225,17 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
     }
 
     @Override
-    public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId,
+    public void restoreShard(IndexShard indexShard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId,
                              RecoveryState recoveryState) {
-        throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE);
+        final Store store = indexShard.store();
+        store.incRef();
+        try {
+            store.createEmpty(indexShard.indexSettings().getIndexMetaData().getCreationVersion().luceneVersion);
+        } catch (EngineException | IOException e) {
+            throw new IndexShardRecoveryException(shardId, "failed to recover from gateway", e);
+        } finally {
+            store.decRef();
+        }
     }
 
     @Override

+ 200 - 0
x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java

@@ -0,0 +1,200 @@
+/*
+ * 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 org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
+import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
+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.service.ClusterService;
+import org.elasticsearch.common.collect.ImmutableOpenMap;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.XContentType;
+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.transport.TransportService;
+import org.elasticsearch.xpack.CcrIntegTestCase;
+import org.elasticsearch.xpack.ccr.repository.CcrRepository;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static java.util.Collections.singletonMap;
+import static org.elasticsearch.snapshots.RestoreService.restoreInProgress;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+
+// TODO: Fold this integration test into a more expansive integration test as more bootstrap from remote work
+// TODO: is completed.
+public class CcrRepositoryIT extends CcrIntegTestCase {
+
+    private final IndicesOptions indicesOptions = IndicesOptions.strictSingleIndexNoExpandForbidClosed();
+
+    public void testThatRepositoryIsPutAndRemovedWhenRemoteClusterIsUpdated() throws Exception {
+        String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster";
+        final RepositoriesService repositoriesService =
+            getFollowerCluster().getDataOrMasterNodeInstances(RepositoriesService.class).iterator().next();
+        try {
+            Repository repository = repositoriesService.repository(leaderClusterRepoName);
+            assertEquals(CcrRepository.TYPE, repository.getMetadata().type());
+            assertEquals(leaderClusterRepoName, repository.getMetadata().name());
+        } catch (RepositoryMissingException e) {
+            fail("need repository");
+        }
+
+        ClusterUpdateSettingsRequest putSecondCluster = new ClusterUpdateSettingsRequest();
+        String address = getFollowerCluster().getDataNodeInstance(TransportService.class).boundAddress().publishAddress().toString();
+        putSecondCluster.persistentSettings(Settings.builder().put("cluster.remote.follower_cluster_copy.seeds", address));
+        assertAcked(followerClient().admin().cluster().updateSettings(putSecondCluster).actionGet());
+
+        String followerCopyRepoName = CcrRepository.NAME_PREFIX + "follower_cluster_copy";
+        try {
+            Repository repository = repositoriesService.repository(followerCopyRepoName);
+            assertEquals(CcrRepository.TYPE, repository.getMetadata().type());
+            assertEquals(followerCopyRepoName, repository.getMetadata().name());
+        } catch (RepositoryMissingException e) {
+            fail("need repository");
+        }
+
+        ClusterUpdateSettingsRequest deleteLeaderCluster = new ClusterUpdateSettingsRequest();
+        deleteLeaderCluster.persistentSettings(Settings.builder().put("cluster.remote.leader_cluster.seeds", ""));
+        assertAcked(followerClient().admin().cluster().updateSettings(deleteLeaderCluster).actionGet());
+
+        expectThrows(RepositoryMissingException.class, () -> repositoriesService.repository(leaderClusterRepoName));
+
+        ClusterUpdateSettingsRequest deleteSecondCluster = new ClusterUpdateSettingsRequest();
+        deleteSecondCluster.persistentSettings(Settings.builder().put("cluster.remote.follower_cluster_copy.seeds", ""));
+        assertAcked(followerClient().admin().cluster().updateSettings(deleteSecondCluster).actionGet());
+
+        expectThrows(RepositoryMissingException.class, () -> repositoriesService.repository(followerCopyRepoName));
+
+        ClusterUpdateSettingsRequest putLeaderRequest = new ClusterUpdateSettingsRequest();
+        address = getLeaderCluster().getDataNodeInstance(TransportService.class).boundAddress().publishAddress().toString();
+        putLeaderRequest.persistentSettings(Settings.builder().put("cluster.remote.leader_cluster.seeds", address));
+        assertAcked(followerClient().admin().cluster().updateSettings(putLeaderRequest).actionGet());
+    }
+
+    public void testThatRepositoryRecoversEmptyIndexBasedOnLeaderSettings() throws IOException {
+        String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster";
+        String leaderIndex = "index1";
+        String followerIndex = "index2";
+
+        final int numberOfPrimaryShards = randomIntBetween(1, 3);
+        final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, between(0, 1),
+            singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true"));
+        assertAcked(leaderClient().admin().indices().prepareCreate(leaderIndex).setSource(leaderIndexSettings, XContentType.JSON));
+        ensureLeaderGreen(leaderIndex);
+
+        final RestoreService restoreService = getFollowerCluster().getCurrentMasterNodeInstance(RestoreService.class);
+        final ClusterService clusterService = getFollowerCluster().getCurrentMasterNodeInstance(ClusterService.class);
+
+        Settings.Builder settingsBuilder = Settings.builder()
+            .put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, followerIndex)
+            .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true);
+        RestoreService.RestoreRequest restoreRequest = new RestoreService.RestoreRequest(leaderClusterRepoName,
+            CcrRepository.LATEST, new String[]{leaderIndex}, indicesOptions,
+            "^(.*)$", followerIndex, Settings.EMPTY, new TimeValue(1, TimeUnit.HOURS), false,
+            false, true, settingsBuilder.build(), new String[0],
+            "restore_snapshot[" + leaderClusterRepoName + ":" + leaderIndex + "]");
+
+        PlainActionFuture<RestoreInfo> future = PlainActionFuture.newFuture();
+        restoreService.restoreSnapshot(restoreRequest, waitForRestore(clusterService, future));
+        RestoreInfo restoreInfo = future.actionGet();
+
+        assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards());
+        assertEquals(0, restoreInfo.failedShards());
+
+        ClusterStateResponse leaderState = leaderClient()
+            .admin()
+            .cluster()
+            .prepareState()
+            .clear()
+            .setMetaData(true)
+            .setIndices(leaderIndex)
+            .get();
+        ClusterStateResponse followerState = followerClient()
+            .admin()
+            .cluster()
+            .prepareState()
+            .clear()
+            .setMetaData(true)
+            .setIndices(followerIndex)
+            .get();
+
+        IndexMetaData leaderMetadata = leaderState.getState().metaData().index(leaderIndex);
+        IndexMetaData followerMetadata = followerState.getState().metaData().index(followerIndex);
+        assertEquals(leaderMetadata.getNumberOfShards(), followerMetadata.getNumberOfShards());
+        Map<String, String> ccrMetadata = followerMetadata.getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY);
+        assertEquals(leaderIndex, ccrMetadata.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_NAME_KEY));
+        assertEquals(leaderMetadata.getIndexUUID(), ccrMetadata.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY));
+        assertEquals("leader_cluster", ccrMetadata.get(Ccr.CCR_CUSTOM_METADATA_REMOTE_CLUSTER_NAME_KEY));
+        assertEquals(followerIndex, followerMetadata.getSettings().get(IndexMetaData.SETTING_INDEX_PROVIDED_NAME));
+        assertEquals(true, IndexSettings.INDEX_SOFT_DELETES_SETTING.get(followerMetadata.getSettings()));
+
+        // UUID is changed so that we can follow indexes on same cluster
+        assertNotEquals(leaderMetadata.getIndexUUID(), followerMetadata.getIndexUUID());
+    }
+
+    private ActionListener<RestoreService.RestoreCompletionResponse> waitForRestore(ClusterService clusterService,
+                                                                                    ActionListener<RestoreInfo> listener) {
+        return new ActionListener<RestoreService.RestoreCompletionResponse>() {
+            @Override
+            public void onResponse(RestoreService.RestoreCompletionResponse restoreCompletionResponse) {
+                if (restoreCompletionResponse.getRestoreInfo() == null) {
+                    final Snapshot snapshot = restoreCompletionResponse.getSnapshot();
+
+                    ClusterStateListener clusterStateListener = new ClusterStateListener() {
+                        @Override
+                        public void clusterChanged(ClusterChangedEvent changedEvent) {
+                            final RestoreInProgress.Entry prevEntry = restoreInProgress(changedEvent.previousState(), snapshot);
+                            final RestoreInProgress.Entry newEntry = restoreInProgress(changedEvent.state(), snapshot);
+                            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<ShardId, RestoreInProgress.ShardRestoreStatus> 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);
+            }
+        };
+    }
+}

+ 0 - 62
x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryManagerIT.java

@@ -1,62 +0,0 @@
-/*
- * 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 org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.repositories.RepositoriesService;
-import org.elasticsearch.repositories.Repository;
-import org.elasticsearch.repositories.RepositoryMissingException;
-import org.elasticsearch.transport.TransportService;
-import org.elasticsearch.xpack.CcrIntegTestCase;
-import org.elasticsearch.xpack.ccr.repository.CcrRepository;
-
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
-
-// TODO: Fold this integration test into a more expansive integration test as more bootstrap from remote work
-// TODO: is completed.
-public class CcrRepositoryManagerIT extends CcrIntegTestCase {
-
-    public void testThatRepositoryIsPutAndRemovedWhenRemoteClusterIsUpdated() throws Exception {
-        String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster";
-        final RepositoriesService repositoriesService =
-            getFollowerCluster().getDataOrMasterNodeInstances(RepositoriesService.class).iterator().next();
-        try {
-            Repository repository = repositoriesService.repository(leaderClusterRepoName);
-            assertEquals(CcrRepository.TYPE, repository.getMetadata().type());
-            assertEquals(leaderClusterRepoName, repository.getMetadata().name());
-        } catch (RepositoryMissingException e) {
-            fail("need repository");
-        }
-
-        ClusterUpdateSettingsRequest putFollowerRequest = new ClusterUpdateSettingsRequest();
-        String address = getFollowerCluster().getDataNodeInstance(TransportService.class).boundAddress().publishAddress().toString();
-        putFollowerRequest.persistentSettings(Settings.builder().put("cluster.remote.follower_cluster_copy.seeds", address));
-        assertAcked(followerClient().admin().cluster().updateSettings(putFollowerRequest).actionGet());
-
-        String followerCopyRepoName = CcrRepository.NAME_PREFIX + "follower_cluster_copy";
-        try {
-            Repository repository = repositoriesService.repository(followerCopyRepoName);
-            assertEquals(CcrRepository.TYPE, repository.getMetadata().type());
-            assertEquals(followerCopyRepoName, repository.getMetadata().name());
-        } catch (RepositoryMissingException e) {
-            fail("need repository");
-        }
-
-        ClusterUpdateSettingsRequest deleteLeaderRequest = new ClusterUpdateSettingsRequest();
-        deleteLeaderRequest.persistentSettings(Settings.builder().put("cluster.remote.leader_cluster.seeds", ""));
-        assertAcked(followerClient().admin().cluster().updateSettings(deleteLeaderRequest).actionGet());
-
-        expectThrows(RepositoryMissingException.class, () -> repositoriesService.repository(leaderClusterRepoName));
-
-        ClusterUpdateSettingsRequest deleteFollowerRequest = new ClusterUpdateSettingsRequest();
-        deleteFollowerRequest.persistentSettings(Settings.builder().put("cluster.remote.follower_cluster_copy.seeds", ""));
-        assertAcked(followerClient().admin().cluster().updateSettings(deleteFollowerRequest).actionGet());
-
-        expectThrows(RepositoryMissingException.class, () -> repositoriesService.repository(followerCopyRepoName));
-    }
-}