|
18 | 18 | */
|
19 | 19 | package org.elasticsearch.discovery;
|
20 | 20 |
|
| 21 | +import org.elasticsearch.ExceptionsHelper; |
21 | 22 | import org.elasticsearch.action.ActionFuture;
|
22 | 23 | import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
|
23 | 24 | import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse;
|
|
27 | 28 | import org.elasticsearch.cluster.ClusterStateListener;
|
28 | 29 | import org.elasticsearch.cluster.SnapshotDeletionsInProgress;
|
29 | 30 | import org.elasticsearch.cluster.SnapshotsInProgress;
|
| 31 | +import org.elasticsearch.cluster.metadata.RepositoriesMetaData; |
| 32 | +import org.elasticsearch.cluster.metadata.RepositoryMetaData; |
30 | 33 | import org.elasticsearch.cluster.service.ClusterService;
|
31 | 34 | import org.elasticsearch.common.settings.Settings;
|
32 | 35 | import org.elasticsearch.common.unit.ByteSizeUnit;
|
33 | 36 | import org.elasticsearch.plugins.Plugin;
|
34 | 37 | import org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException;
|
| 38 | +import org.elasticsearch.snapshots.SnapshotException; |
35 | 39 | import org.elasticsearch.snapshots.SnapshotInfo;
|
36 | 40 | import org.elasticsearch.snapshots.SnapshotMissingException;
|
37 | 41 | import org.elasticsearch.snapshots.SnapshotState;
|
|
49 | 53 | import java.util.concurrent.TimeUnit;
|
50 | 54 |
|
51 | 55 | import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
| 56 | +import static org.hamcrest.Matchers.either; |
| 57 | +import static org.hamcrest.Matchers.endsWith; |
| 58 | +import static org.hamcrest.Matchers.is; |
52 | 59 |
|
53 | 60 | /**
|
54 | 61 | * Tests snapshot operations during disruptions.
|
@@ -156,6 +163,95 @@ public void clusterChanged(ClusterChangedEvent event) {
|
156 | 163 | assertAllSnapshotsCompleted();
|
157 | 164 | }
|
158 | 165 |
|
| 166 | + public void testDisruptionAfterFinalization() throws Exception { |
| 167 | + final String idxName = "test"; |
| 168 | + final List<String> allMasterEligibleNodes = internalCluster().startMasterOnlyNodes(3); |
| 169 | + final String dataNode = internalCluster().startDataOnlyNode(); |
| 170 | + ensureStableCluster(4); |
| 171 | + |
| 172 | + createRandomIndex(idxName); |
| 173 | + |
| 174 | + logger.info("--> creating repository"); |
| 175 | + assertAcked(client().admin().cluster().preparePutRepository("test-repo") |
| 176 | + .setType("fs").setSettings(Settings.builder() |
| 177 | + .put("location", randomRepoPath()) |
| 178 | + .put("compress", randomBoolean()) |
| 179 | + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES))); |
| 180 | + |
| 181 | + final String masterNode1 = internalCluster().getMasterName(); |
| 182 | + Set<String> otherNodes = new HashSet<>(allMasterEligibleNodes); |
| 183 | + otherNodes.remove(masterNode1); |
| 184 | + otherNodes.add(dataNode); |
| 185 | + |
| 186 | + NetworkDisruption networkDisruption = |
| 187 | + new NetworkDisruption(new NetworkDisruption.TwoPartitions(Collections.singleton(masterNode1), otherNodes), |
| 188 | + new NetworkDisruption.NetworkUnresponsive()); |
| 189 | + internalCluster().setDisruptionScheme(networkDisruption); |
| 190 | + |
| 191 | + ClusterService clusterService = internalCluster().clusterService(masterNode1); |
| 192 | + CountDownLatch disruptionStarted = new CountDownLatch(1); |
| 193 | + clusterService.addListener(new ClusterStateListener() { |
| 194 | + @Override |
| 195 | + public void clusterChanged(ClusterChangedEvent event) { |
| 196 | + SnapshotsInProgress snapshots = event.state().custom(SnapshotsInProgress.TYPE); |
| 197 | + if (snapshots != null && snapshots.entries().size() > 0) { |
| 198 | + final SnapshotsInProgress.Entry snapshotEntry = snapshots.entries().get(0); |
| 199 | + if (snapshotEntry.state() == SnapshotsInProgress.State.SUCCESS) { |
| 200 | + final RepositoriesMetaData repoMeta = |
| 201 | + event.state().metaData().custom(RepositoriesMetaData.TYPE); |
| 202 | + final RepositoryMetaData metaData = repoMeta.repository("test-repo"); |
| 203 | + if (metaData.generation() == metaData.pendingGeneration() |
| 204 | + && metaData.generation() > snapshotEntry.repositoryStateId()) { |
| 205 | + logger.info("--> starting disruption"); |
| 206 | + networkDisruption.startDisrupting(); |
| 207 | + clusterService.removeListener(this); |
| 208 | + disruptionStarted.countDown(); |
| 209 | + } |
| 210 | + } |
| 211 | + } |
| 212 | + } |
| 213 | + }); |
| 214 | + |
| 215 | + final String snapshot = "test-snap"; |
| 216 | + |
| 217 | + logger.info("--> starting snapshot"); |
| 218 | + ActionFuture<CreateSnapshotResponse> future = client(masterNode1).admin().cluster() |
| 219 | + .prepareCreateSnapshot("test-repo", snapshot).setWaitForCompletion(true) |
| 220 | + .setIndices(idxName).execute(); |
| 221 | + |
| 222 | + logger.info("--> waiting for disruption to start"); |
| 223 | + assertTrue(disruptionStarted.await(1, TimeUnit.MINUTES)); |
| 224 | + |
| 225 | + assertAllSnapshotsCompleted(); |
| 226 | + |
| 227 | + logger.info("--> verify that snapshot was successful or no longer exist"); |
| 228 | + assertBusy(() -> { |
| 229 | + try { |
| 230 | + assertSnapshotExists("test-repo", snapshot); |
| 231 | + } catch (SnapshotMissingException exception) { |
| 232 | + logger.info("--> done verifying, snapshot doesn't exist"); |
| 233 | + } |
| 234 | + }, 1, TimeUnit.MINUTES); |
| 235 | + |
| 236 | + logger.info("--> stopping disrupting"); |
| 237 | + networkDisruption.stopDisrupting(); |
| 238 | + ensureStableCluster(4, masterNode1); |
| 239 | + logger.info("--> done"); |
| 240 | + |
| 241 | + try { |
| 242 | + future.get(); |
| 243 | + fail("Should have failed because the node disconnected from the cluster during snapshot finalization"); |
| 244 | + } catch (Exception ex) { |
| 245 | + final SnapshotException sne = (SnapshotException) ExceptionsHelper.unwrap(ex, SnapshotException.class); |
| 246 | + assertNotNull(sne); |
| 247 | + assertThat( |
| 248 | + sne.getMessage(), either(endsWith(" Failed to remove snapshot from cluster state")).or(endsWith(" no longer master"))); |
| 249 | + assertThat(sne.getSnapshotName(), is(snapshot)); |
| 250 | + } |
| 251 | + |
| 252 | + assertAllSnapshotsCompleted(); |
| 253 | + } |
| 254 | + |
159 | 255 | private void assertAllSnapshotsCompleted() throws Exception {
|
160 | 256 | logger.info("--> wait until the snapshot is done");
|
161 | 257 | assertBusy(() -> {
|
|
0 commit comments