|
79 | 79 | import org.opensearch.cluster.service.ClusterService;
|
80 | 80 | import org.opensearch.common.Nullable;
|
81 | 81 | import org.opensearch.common.Priority;
|
| 82 | +import org.opensearch.common.SetOnce; |
82 | 83 | import org.opensearch.common.UUIDs;
|
83 | 84 | import org.opensearch.common.collect.Tuple;
|
84 | 85 | import org.opensearch.common.lifecycle.AbstractLifecycleComponent;
|
@@ -466,34 +467,35 @@ public TimeValue timeout() {
|
466 | 467 | * @param listener snapshot creation listener
|
467 | 468 | */
|
468 | 469 | public void createSnapshotV2(final CreateSnapshotRequest request, final ActionListener<SnapshotInfo> listener) {
|
469 |
| - long pinnedTimestamp = System.currentTimeMillis(); |
470 | 470 | final String repositoryName = request.repository();
|
471 | 471 | final String snapshotName = indexNameExpressionResolver.resolveDateMathExpression(request.snapshot());
|
| 472 | + validate(repositoryName, snapshotName); |
| 473 | + |
| 474 | + final SnapshotId snapshotId = new SnapshotId(snapshotName, UUIDs.randomBase64UUID()); // new UUID for the snapshot |
| 475 | + Snapshot snapshot = new Snapshot(repositoryName, snapshotId); |
| 476 | + long pinnedTimestamp = System.currentTimeMillis(); |
| 477 | + try { |
| 478 | + updateSnapshotPinnedTimestamp(snapshot, pinnedTimestamp); |
| 479 | + } catch (Exception e) { |
| 480 | + listener.onFailure(e); |
| 481 | + return; |
| 482 | + } |
472 | 483 |
|
473 | 484 | Repository repository = repositoriesService.repository(repositoryName);
|
474 |
| - validate(repositoryName, snapshotName); |
475 | 485 | repository.executeConsistentStateUpdate(repositoryData -> new ClusterStateUpdateTask(Priority.URGENT) {
|
476 | 486 | private SnapshotsInProgress.Entry newEntry;
|
477 |
| - |
478 |
| - private SnapshotId snapshotId; |
479 |
| - |
480 |
| - private Snapshot snapshot; |
481 |
| - |
482 | 487 | boolean enteredLoop;
|
483 | 488 |
|
484 | 489 | @Override
|
485 | 490 | public ClusterState execute(ClusterState currentState) {
|
486 | 491 | // move to in progress
|
487 |
| - snapshotId = new SnapshotId(snapshotName, UUIDs.randomBase64UUID()); // new UUID for the snapshot |
488 | 492 | Repository repository = repositoriesService.repository(repositoryName);
|
489 |
| - |
490 | 493 | if (repository.isReadOnly()) {
|
491 | 494 | listener.onFailure(
|
492 | 495 | new RepositoryException(repository.getMetadata().name(), "cannot create snapshot-v2 in a readonly repository")
|
493 | 496 | );
|
494 | 497 | }
|
495 | 498 |
|
496 |
| - snapshot = new Snapshot(repositoryName, snapshotId); |
497 | 499 | final Map<String, Object> userMeta = repository.adaptUserMetadata(request.userMetadata());
|
498 | 500 |
|
499 | 501 | createSnapshotPreValidations(currentState, repositoryData, repositoryName, snapshotName);
|
@@ -593,59 +595,46 @@ public void clusterStateProcessed(String source, ClusterState oldState, final Cl
|
593 | 595 | pinnedTimestamp
|
594 | 596 | );
|
595 | 597 | final Version version = minCompatibleVersion(newState.nodes().getMinNodeVersion(), repositoryData, null);
|
596 |
| - final StepListener<RepositoryData> pinnedTimestampListener = new StepListener<>(); |
597 |
| - pinnedTimestampListener.whenComplete(repoData -> { |
598 |
| - repository.finalizeSnapshot( |
599 |
| - shardGenerations, |
600 |
| - repositoryData.getGenId(), |
601 |
| - metadataForSnapshot(newState.metadata(), request.includeGlobalState(), false, dataStreams, newEntry.indices()), |
602 |
| - snapshotInfo, |
603 |
| - version, |
604 |
| - state -> stateWithoutSnapshot(state, snapshot), |
605 |
| - Priority.IMMEDIATE, |
606 |
| - new ActionListener<RepositoryData>() { |
607 |
| - @Override |
608 |
| - public void onResponse(RepositoryData repositoryData) { |
609 |
| - if (clusterService.state().nodes().isLocalNodeElectedClusterManager() == false) { |
610 |
| - leaveRepoLoop(repositoryName); |
611 |
| - failSnapshotCompletionListeners( |
612 |
| - snapshot, |
613 |
| - new SnapshotException(snapshot, "Aborting snapshot-v2, no longer cluster manager") |
614 |
| - ); |
615 |
| - listener.onFailure( |
616 |
| - new SnapshotException( |
617 |
| - repositoryName, |
618 |
| - snapshotName, |
619 |
| - "Aborting snapshot-v2, no longer cluster manager" |
620 |
| - ) |
621 |
| - ); |
622 |
| - return; |
623 |
| - } |
624 |
| - listener.onResponse(snapshotInfo); |
625 |
| - // For snapshot-v2, we don't allow concurrent snapshots . But meanwhile non-v2 snapshot operations |
626 |
| - // can get queued . This is triggering them. |
627 |
| - runNextQueuedOperation(repositoryData, repositoryName, true); |
628 |
| - cleanOrphanTimestamp(repositoryName, repositoryData); |
629 |
| - } |
630 |
| - |
631 |
| - @Override |
632 |
| - public void onFailure(Exception e) { |
633 |
| - logger.error("Failed to finalize snapshot repo {} for snapshot-v2 {} ", repositoryName, snapshotName); |
| 598 | + repository.finalizeSnapshot( |
| 599 | + shardGenerations, |
| 600 | + repositoryData.getGenId(), |
| 601 | + metadataForSnapshot(newState.metadata(), request.includeGlobalState(), false, dataStreams, newEntry.indices()), |
| 602 | + snapshotInfo, |
| 603 | + version, |
| 604 | + state -> stateWithoutSnapshot(state, snapshot), |
| 605 | + Priority.IMMEDIATE, |
| 606 | + new ActionListener<RepositoryData>() { |
| 607 | + @Override |
| 608 | + public void onResponse(RepositoryData repositoryData) { |
| 609 | + if (clusterService.state().nodes().isLocalNodeElectedClusterManager() == false) { |
634 | 610 | leaveRepoLoop(repositoryName);
|
635 |
| - // cleaning up in progress snapshot here |
636 |
| - stateWithoutSnapshotV2(newState); |
637 |
| - listener.onFailure(e); |
| 611 | + failSnapshotCompletionListeners( |
| 612 | + snapshot, |
| 613 | + new SnapshotException(snapshot, "Aborting snapshot-v2, no longer cluster manager") |
| 614 | + ); |
| 615 | + listener.onFailure( |
| 616 | + new SnapshotException(repositoryName, snapshotName, "Aborting snapshot-v2, no longer cluster manager") |
| 617 | + ); |
| 618 | + return; |
638 | 619 | }
|
| 620 | + listener.onResponse(snapshotInfo); |
| 621 | + logger.info("created snapshot-v2 [{}] in repository [{}]", repositoryName, snapshotName); |
| 622 | + // For snapshot-v2, we don't allow concurrent snapshots . But meanwhile non-v2 snapshot operations |
| 623 | + // can get queued . This is triggering them. |
| 624 | + runNextQueuedOperation(repositoryData, repositoryName, true); |
| 625 | + cleanOrphanTimestamp(repositoryName, repositoryData); |
639 | 626 | }
|
640 |
| - ); |
641 |
| - }, e -> { |
642 |
| - logger.error("Failed to update pinned timestamp for snapshot-v2 {} {} {} ", repositoryName, snapshotName, e); |
643 |
| - leaveRepoLoop(repositoryName); |
644 |
| - // cleaning up in progress snapshot here |
645 |
| - stateWithoutSnapshotV2(newState); |
646 |
| - listener.onFailure(e); |
647 |
| - }); |
648 |
| - updateSnapshotPinnedTimestamp(repositoryData, snapshot, pinnedTimestamp, pinnedTimestampListener); |
| 627 | + |
| 628 | + @Override |
| 629 | + public void onFailure(Exception e) { |
| 630 | + logger.error("Failed to finalize snapshot repo {} for snapshot-v2 {} ", repositoryName, snapshotName); |
| 631 | + leaveRepoLoop(repositoryName); |
| 632 | + // cleaning up in progress snapshot here |
| 633 | + stateWithoutSnapshotV2(newState); |
| 634 | + listener.onFailure(e); |
| 635 | + } |
| 636 | + } |
| 637 | + ); |
649 | 638 | }
|
650 | 639 |
|
651 | 640 | @Override
|
@@ -733,30 +722,30 @@ private void createSnapshotPreValidations(
|
733 | 722 | ensureNoCleanupInProgress(currentState, repositoryName, snapshotName);
|
734 | 723 | }
|
735 | 724 |
|
736 |
| - private void updateSnapshotPinnedTimestamp( |
737 |
| - RepositoryData repositoryData, |
738 |
| - Snapshot snapshot, |
739 |
| - long timestampToPin, |
740 |
| - ActionListener<RepositoryData> listener |
741 |
| - ) { |
| 725 | + private void updateSnapshotPinnedTimestamp(Snapshot snapshot, long timestampToPin) throws Exception { |
| 726 | + CountDownLatch latch = new CountDownLatch(1); |
| 727 | + SetOnce<Exception> ex = new SetOnce<>(); |
| 728 | + ActionListener<Void> listener = new ActionListener<>() { |
| 729 | + @Override |
| 730 | + public void onResponse(Void unused) { |
| 731 | + logger.debug("Timestamp pinned successfully for snapshot {}", snapshot.getSnapshotId().getName()); |
| 732 | + } |
| 733 | + |
| 734 | + @Override |
| 735 | + public void onFailure(Exception e) { |
| 736 | + logger.error("Failed to pin timestamp for snapshot {} with exception {}", snapshot.getSnapshotId().getName(), e); |
| 737 | + ex.set(e); |
| 738 | + } |
| 739 | + }; |
742 | 740 | remoteStorePinnedTimestampService.pinTimestamp(
|
743 | 741 | timestampToPin,
|
744 | 742 | getPinningEntity(snapshot.getRepository(), snapshot.getSnapshotId().getUUID()),
|
745 |
| - new ActionListener<Void>() { |
746 |
| - @Override |
747 |
| - public void onResponse(Void unused) { |
748 |
| - logger.debug("Timestamp pinned successfully for snapshot {}", snapshot.getSnapshotId().getName()); |
749 |
| - listener.onResponse(repositoryData); |
750 |
| - } |
751 |
| - |
752 |
| - @Override |
753 |
| - public void onFailure(Exception e) { |
754 |
| - logger.error("Failed to pin timestamp for snapshot {} with exception {}", snapshot.getSnapshotId().getName(), e); |
755 |
| - listener.onFailure(e); |
756 |
| - |
757 |
| - } |
758 |
| - } |
| 743 | + new LatchedActionListener<>(listener, latch) |
759 | 744 | );
|
| 745 | + latch.await(); |
| 746 | + if (ex.get() != null) { |
| 747 | + throw ex.get(); |
| 748 | + } |
760 | 749 | }
|
761 | 750 |
|
762 | 751 | public static String getPinningEntity(String repositoryName, String snapshotUUID) {
|
|
0 commit comments