|
@@ -28,8 +28,8 @@ import org.elasticsearch.action.search.ShardSearchFailure;
|
|
|
import org.elasticsearch.action.support.IndicesOptions;
|
|
|
import org.elasticsearch.cluster.*;
|
|
|
import org.elasticsearch.cluster.metadata.*;
|
|
|
-import org.elasticsearch.cluster.metadata.SnapshotMetaData.ShardSnapshotStatus;
|
|
|
-import org.elasticsearch.cluster.metadata.SnapshotMetaData.State;
|
|
|
+import org.elasticsearch.cluster.SnapshotsInProgress.ShardSnapshotStatus;
|
|
|
+import org.elasticsearch.cluster.SnapshotsInProgress.State;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
|
|
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
|
@@ -77,13 +77,13 @@ import static com.google.common.collect.Sets.newHashSet;
|
|
|
* <ul>
|
|
|
* <li>On the master node the {@link #createSnapshot(SnapshotRequest, CreateSnapshotListener)} is called and makes sure that no snapshots is currently running
|
|
|
* and registers the new snapshot in cluster state</li>
|
|
|
- * <li>When cluster state is updated the {@link #beginSnapshot(ClusterState, SnapshotMetaData.Entry, boolean, CreateSnapshotListener)} method
|
|
|
+ * <li>When cluster state is updated the {@link #beginSnapshot(ClusterState, SnapshotsInProgress.Entry, boolean, CreateSnapshotListener)} method
|
|
|
* kicks in and initializes the snapshot in the repository and then populates list of shards that needs to be snapshotted in cluster state</li>
|
|
|
* <li>Each data node is watching for these shards and when new shards scheduled for snapshotting appear in the cluster state, data nodes
|
|
|
* start processing them through {@link SnapshotsService#processIndexShardSnapshots(ClusterChangedEvent)} method</li>
|
|
|
* <li>Once shard snapshot is created data node updates state of the shard in the cluster state using the {@link #updateIndexShardSnapshotStatus(UpdateIndexShardSnapshotStatusRequest)} method</li>
|
|
|
* <li>When last shard is completed master node in {@link #innerUpdateSnapshotState} method marks the snapshot as completed</li>
|
|
|
- * <li>After cluster state is updated, the {@link #endSnapshot(SnapshotMetaData.Entry)} finalizes snapshot in the repository,
|
|
|
+ * <li>After cluster state is updated, the {@link #endSnapshot(SnapshotsInProgress.Entry)} finalizes snapshot in the repository,
|
|
|
* notifies all {@link #snapshotCompletionListeners} that snapshot is completed, and finally calls {@link #removeSnapshotFromClusterState(SnapshotId, SnapshotInfo, Throwable)} to remove snapshot from cluster state</li>
|
|
|
* </ul>
|
|
|
*/
|
|
@@ -135,7 +135,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @throws SnapshotMissingException if snapshot is not found
|
|
|
*/
|
|
|
public Snapshot snapshot(SnapshotId snapshotId) {
|
|
|
- List<SnapshotMetaData.Entry> entries = currentSnapshots(snapshotId.getRepository(), new String[]{snapshotId.getSnapshot()});
|
|
|
+ List<SnapshotsInProgress.Entry> entries = currentSnapshots(snapshotId.getRepository(), new String[]{snapshotId.getSnapshot()});
|
|
|
if (!entries.isEmpty()) {
|
|
|
return inProgressSnapshot(entries.iterator().next());
|
|
|
}
|
|
@@ -150,8 +150,8 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
*/
|
|
|
public List<Snapshot> snapshots(String repositoryName) {
|
|
|
Set<Snapshot> snapshotSet = newHashSet();
|
|
|
- List<SnapshotMetaData.Entry> entries = currentSnapshots(repositoryName, null);
|
|
|
- for (SnapshotMetaData.Entry entry : entries) {
|
|
|
+ List<SnapshotsInProgress.Entry> entries = currentSnapshots(repositoryName, null);
|
|
|
+ for (SnapshotsInProgress.Entry entry : entries) {
|
|
|
snapshotSet.add(inProgressSnapshot(entry));
|
|
|
}
|
|
|
Repository repository = repositoriesService.repository(repositoryName);
|
|
@@ -172,8 +172,8 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
*/
|
|
|
public List<Snapshot> currentSnapshots(String repositoryName) {
|
|
|
List<Snapshot> snapshotList = newArrayList();
|
|
|
- List<SnapshotMetaData.Entry> entries = currentSnapshots(repositoryName, null);
|
|
|
- for (SnapshotMetaData.Entry entry : entries) {
|
|
|
+ List<SnapshotsInProgress.Entry> entries = currentSnapshots(repositoryName, null);
|
|
|
+ for (SnapshotsInProgress.Entry entry : entries) {
|
|
|
snapshotList.add(inProgressSnapshot(entry));
|
|
|
}
|
|
|
CollectionUtil.timSort(snapshotList);
|
|
@@ -193,27 +193,25 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
final SnapshotId snapshotId = new SnapshotId(request.repository(), request.name());
|
|
|
clusterService.submitStateUpdateTask(request.cause(), new TimeoutClusterStateUpdateTask() {
|
|
|
|
|
|
- private SnapshotMetaData.Entry newSnapshot = null;
|
|
|
+ private SnapshotsInProgress.Entry newSnapshot = null;
|
|
|
|
|
|
@Override
|
|
|
public ClusterState execute(ClusterState currentState) {
|
|
|
validate(request, currentState);
|
|
|
|
|
|
MetaData metaData = currentState.metaData();
|
|
|
- MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
|
|
|
- SnapshotMetaData snapshots = metaData.custom(SnapshotMetaData.TYPE);
|
|
|
+ SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE);
|
|
|
if (snapshots == null || snapshots.entries().isEmpty()) {
|
|
|
// Store newSnapshot here to be processed in clusterStateProcessed
|
|
|
ImmutableList<String> indices = ImmutableList.copyOf(metaData.concreteIndices(request.indicesOptions(), request.indices()));
|
|
|
logger.trace("[{}][{}] creating snapshot for indices [{}]", request.repository(), request.name(), indices);
|
|
|
- newSnapshot = new SnapshotMetaData.Entry(snapshotId, request.includeGlobalState(), State.INIT, indices, System.currentTimeMillis(), null);
|
|
|
- snapshots = new SnapshotMetaData(newSnapshot);
|
|
|
+ newSnapshot = new SnapshotsInProgress.Entry(snapshotId, request.includeGlobalState(), State.INIT, indices, System.currentTimeMillis(), null);
|
|
|
+ snapshots = new SnapshotsInProgress(newSnapshot);
|
|
|
} else {
|
|
|
// TODO: What should we do if a snapshot is already running?
|
|
|
throw new ConcurrentSnapshotExecutionException(snapshotId, "a snapshot is already running");
|
|
|
}
|
|
|
- mdBuilder.putCustom(SnapshotMetaData.TYPE, snapshots);
|
|
|
- return ClusterState.builder(currentState).metaData(mdBuilder).build();
|
|
|
+ return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -288,7 +286,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @param partial allow partial snapshots
|
|
|
* @param userCreateSnapshotListener listener
|
|
|
*/
|
|
|
- private void beginSnapshot(ClusterState clusterState, final SnapshotMetaData.Entry snapshot, final boolean partial, final CreateSnapshotListener userCreateSnapshotListener) {
|
|
|
+ private void beginSnapshot(ClusterState clusterState, final SnapshotsInProgress.Entry snapshot, final boolean partial, final CreateSnapshotListener userCreateSnapshotListener) {
|
|
|
boolean snapshotCreated = false;
|
|
|
try {
|
|
|
Repository repository = repositoriesService.repository(snapshot.snapshotId().getRepository());
|
|
@@ -313,26 +311,25 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
}
|
|
|
clusterService.submitStateUpdateTask("update_snapshot [" + snapshot.snapshotId().getSnapshot() + "]", new ProcessedClusterStateUpdateTask() {
|
|
|
boolean accepted = false;
|
|
|
- SnapshotMetaData.Entry updatedSnapshot;
|
|
|
+ SnapshotsInProgress.Entry updatedSnapshot;
|
|
|
String failure = null;
|
|
|
|
|
|
@Override
|
|
|
public ClusterState execute(ClusterState currentState) {
|
|
|
MetaData metaData = currentState.metaData();
|
|
|
- MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
|
|
|
- SnapshotMetaData snapshots = metaData.custom(SnapshotMetaData.TYPE);
|
|
|
- ImmutableList.Builder<SnapshotMetaData.Entry> entries = ImmutableList.builder();
|
|
|
- for (SnapshotMetaData.Entry entry : snapshots.entries()) {
|
|
|
+ SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE);
|
|
|
+ ImmutableList.Builder<SnapshotsInProgress.Entry> entries = ImmutableList.builder();
|
|
|
+ for (SnapshotsInProgress.Entry entry : snapshots.entries()) {
|
|
|
if (entry.snapshotId().equals(snapshot.snapshotId())) {
|
|
|
// Replace the snapshot that was just created
|
|
|
- ImmutableMap<ShardId, SnapshotMetaData.ShardSnapshotStatus> shards = shards(currentState, entry.indices());
|
|
|
+ ImmutableMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards = shards(currentState, entry.indices());
|
|
|
if (!partial) {
|
|
|
Tuple<Set<String>, Set<String>> indicesWithMissingShards = indicesWithMissingShards(shards, currentState.metaData());
|
|
|
Set<String> missing = indicesWithMissingShards.v1();
|
|
|
Set<String> closed = indicesWithMissingShards.v2();
|
|
|
if (missing.isEmpty() == false || closed.isEmpty() == false) {
|
|
|
StringBuilder failureMessage = new StringBuilder();
|
|
|
- updatedSnapshot = new SnapshotMetaData.Entry(entry, State.FAILED, shards);
|
|
|
+ updatedSnapshot = new SnapshotsInProgress.Entry(entry, State.FAILED, shards);
|
|
|
entries.add(updatedSnapshot);
|
|
|
if (missing.isEmpty() == false ) {
|
|
|
failureMessage.append("Indices don't have primary shards ");
|
|
@@ -349,7 +346,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
continue;
|
|
|
}
|
|
|
}
|
|
|
- updatedSnapshot = new SnapshotMetaData.Entry(entry, State.STARTED, shards);
|
|
|
+ updatedSnapshot = new SnapshotsInProgress.Entry(entry, State.STARTED, shards);
|
|
|
entries.add(updatedSnapshot);
|
|
|
if (!completed(shards.values())) {
|
|
|
accepted = true;
|
|
@@ -358,8 +355,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
entries.add(entry);
|
|
|
}
|
|
|
}
|
|
|
- mdBuilder.putCustom(SnapshotMetaData.TYPE, new SnapshotMetaData(entries.build()));
|
|
|
- return ClusterState.builder(currentState).metaData(mdBuilder).build();
|
|
|
+ return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(entries.build())).build();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -407,7 +403,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private Snapshot inProgressSnapshot(SnapshotMetaData.Entry entry) {
|
|
|
+ private Snapshot inProgressSnapshot(SnapshotsInProgress.Entry entry) {
|
|
|
return new Snapshot(entry.snapshotId().getSnapshot(), entry.indices(), entry.startTime());
|
|
|
}
|
|
|
|
|
@@ -421,35 +417,34 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @param snapshots optional list of snapshots that will be used as a filter
|
|
|
* @return list of metadata for currently running snapshots
|
|
|
*/
|
|
|
- public List<SnapshotMetaData.Entry> currentSnapshots(String repository, String[] snapshots) {
|
|
|
- MetaData metaData = clusterService.state().metaData();
|
|
|
- SnapshotMetaData snapshotMetaData = metaData.custom(SnapshotMetaData.TYPE);
|
|
|
- if (snapshotMetaData == null || snapshotMetaData.entries().isEmpty()) {
|
|
|
+ public List<SnapshotsInProgress.Entry> currentSnapshots(String repository, String[] snapshots) {
|
|
|
+ SnapshotsInProgress snapshotsInProgress = clusterService.state().custom(SnapshotsInProgress.TYPE);
|
|
|
+ if (snapshotsInProgress == null || snapshotsInProgress.entries().isEmpty()) {
|
|
|
return ImmutableList.of();
|
|
|
}
|
|
|
if ("_all".equals(repository)) {
|
|
|
- return snapshotMetaData.entries();
|
|
|
+ return snapshotsInProgress.entries();
|
|
|
}
|
|
|
- if (snapshotMetaData.entries().size() == 1) {
|
|
|
+ if (snapshotsInProgress.entries().size() == 1) {
|
|
|
// Most likely scenario - one snapshot is currently running
|
|
|
// Check this snapshot against the query
|
|
|
- SnapshotMetaData.Entry entry = snapshotMetaData.entries().get(0);
|
|
|
+ SnapshotsInProgress.Entry entry = snapshotsInProgress.entries().get(0);
|
|
|
if (!entry.snapshotId().getRepository().equals(repository)) {
|
|
|
return ImmutableList.of();
|
|
|
}
|
|
|
if (snapshots != null && snapshots.length > 0) {
|
|
|
for (String snapshot : snapshots) {
|
|
|
if (entry.snapshotId().getSnapshot().equals(snapshot)) {
|
|
|
- return snapshotMetaData.entries();
|
|
|
+ return snapshotsInProgress.entries();
|
|
|
}
|
|
|
}
|
|
|
return ImmutableList.of();
|
|
|
} else {
|
|
|
- return snapshotMetaData.entries();
|
|
|
+ return snapshotsInProgress.entries();
|
|
|
}
|
|
|
}
|
|
|
- ImmutableList.Builder<SnapshotMetaData.Entry> builder = ImmutableList.builder();
|
|
|
- for (SnapshotMetaData.Entry entry : snapshotMetaData.entries()) {
|
|
|
+ ImmutableList.Builder<SnapshotsInProgress.Entry> builder = ImmutableList.builder();
|
|
|
+ for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) {
|
|
|
if (!entry.snapshotId().getRepository().equals(repository)) {
|
|
|
continue;
|
|
|
}
|
|
@@ -544,8 +539,8 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
processStartedShards(event);
|
|
|
}
|
|
|
}
|
|
|
- SnapshotMetaData prev = event.previousState().metaData().custom(SnapshotMetaData.TYPE);
|
|
|
- SnapshotMetaData curr = event.state().metaData().custom(SnapshotMetaData.TYPE);
|
|
|
+ SnapshotsInProgress prev = event.previousState().custom(SnapshotsInProgress.TYPE);
|
|
|
+ SnapshotsInProgress curr = event.state().custom(SnapshotsInProgress.TYPE);
|
|
|
|
|
|
if (prev == null) {
|
|
|
if (curr != null) {
|
|
@@ -579,16 +574,14 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
@Override
|
|
|
public ClusterState execute(ClusterState currentState) throws Exception {
|
|
|
DiscoveryNodes nodes = currentState.nodes();
|
|
|
- MetaData metaData = currentState.metaData();
|
|
|
- MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
|
|
|
- SnapshotMetaData snapshots = metaData.custom(SnapshotMetaData.TYPE);
|
|
|
+ SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE);
|
|
|
if (snapshots == null) {
|
|
|
return currentState;
|
|
|
}
|
|
|
boolean changed = false;
|
|
|
- ArrayList<SnapshotMetaData.Entry> entries = newArrayList();
|
|
|
- for (final SnapshotMetaData.Entry snapshot : snapshots.entries()) {
|
|
|
- SnapshotMetaData.Entry updatedSnapshot = snapshot;
|
|
|
+ ArrayList<SnapshotsInProgress.Entry> entries = newArrayList();
|
|
|
+ for (final SnapshotsInProgress.Entry snapshot : snapshots.entries()) {
|
|
|
+ SnapshotsInProgress.Entry updatedSnapshot = snapshot;
|
|
|
boolean snapshotChanged = false;
|
|
|
if (snapshot.state() == State.STARTED || snapshot.state() == State.ABORTED) {
|
|
|
ImmutableMap.Builder<ShardId, ShardSnapshotStatus> shards = ImmutableMap.builder();
|
|
@@ -609,10 +602,10 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
changed = true;
|
|
|
ImmutableMap<ShardId, ShardSnapshotStatus> shardsMap = shards.build();
|
|
|
if (!snapshot.state().completed() && completed(shardsMap.values())) {
|
|
|
- updatedSnapshot = new SnapshotMetaData.Entry(snapshot, State.SUCCESS, shardsMap);
|
|
|
+ updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, State.SUCCESS, shardsMap);
|
|
|
endSnapshot(updatedSnapshot);
|
|
|
} else {
|
|
|
- updatedSnapshot = new SnapshotMetaData.Entry(snapshot, snapshot.state(), shardsMap);
|
|
|
+ updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, snapshot.state(), shardsMap);
|
|
|
}
|
|
|
}
|
|
|
entries.add(updatedSnapshot);
|
|
@@ -635,9 +628,8 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
}
|
|
|
}
|
|
|
if (changed) {
|
|
|
- snapshots = new SnapshotMetaData(entries.toArray(new SnapshotMetaData.Entry[entries.size()]));
|
|
|
- mdBuilder.putCustom(SnapshotMetaData.TYPE, snapshots);
|
|
|
- return ClusterState.builder(currentState).metaData(mdBuilder).build();
|
|
|
+ snapshots = new SnapshotsInProgress(entries.toArray(new SnapshotsInProgress.Entry[entries.size()]));
|
|
|
+ return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build();
|
|
|
}
|
|
|
return currentState;
|
|
|
}
|
|
@@ -655,33 +647,30 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
clusterService.submitStateUpdateTask("update snapshot state after shards started", new ClusterStateUpdateTask() {
|
|
|
@Override
|
|
|
public ClusterState execute(ClusterState currentState) throws Exception {
|
|
|
- MetaData metaData = currentState.metaData();
|
|
|
RoutingTable routingTable = currentState.routingTable();
|
|
|
- MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
|
|
|
- SnapshotMetaData snapshots = metaData.custom(SnapshotMetaData.TYPE);
|
|
|
+ SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE);
|
|
|
if (snapshots != null) {
|
|
|
boolean changed = false;
|
|
|
- ArrayList<SnapshotMetaData.Entry> entries = newArrayList();
|
|
|
- for (final SnapshotMetaData.Entry snapshot : snapshots.entries()) {
|
|
|
- SnapshotMetaData.Entry updatedSnapshot = snapshot;
|
|
|
+ ArrayList<SnapshotsInProgress.Entry> entries = newArrayList();
|
|
|
+ for (final SnapshotsInProgress.Entry snapshot : snapshots.entries()) {
|
|
|
+ SnapshotsInProgress.Entry updatedSnapshot = snapshot;
|
|
|
if (snapshot.state() == State.STARTED) {
|
|
|
ImmutableMap<ShardId, ShardSnapshotStatus> shards = processWaitingShards(snapshot.shards(), routingTable);
|
|
|
if (shards != null) {
|
|
|
changed = true;
|
|
|
if (!snapshot.state().completed() && completed(shards.values())) {
|
|
|
- updatedSnapshot = new SnapshotMetaData.Entry(snapshot, State.SUCCESS, shards);
|
|
|
+ updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, State.SUCCESS, shards);
|
|
|
endSnapshot(updatedSnapshot);
|
|
|
} else {
|
|
|
- updatedSnapshot = new SnapshotMetaData.Entry(snapshot, shards);
|
|
|
+ updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, shards);
|
|
|
}
|
|
|
}
|
|
|
entries.add(updatedSnapshot);
|
|
|
}
|
|
|
}
|
|
|
if (changed) {
|
|
|
- snapshots = new SnapshotMetaData(entries.toArray(new SnapshotMetaData.Entry[entries.size()]));
|
|
|
- mdBuilder.putCustom(SnapshotMetaData.TYPE, snapshots);
|
|
|
- return ClusterState.builder(currentState).metaData(mdBuilder).build();
|
|
|
+ snapshots = new SnapshotsInProgress(entries.toArray(new SnapshotsInProgress.Entry[entries.size()]));
|
|
|
+ return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build();
|
|
|
}
|
|
|
}
|
|
|
return currentState;
|
|
@@ -735,9 +724,9 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
}
|
|
|
|
|
|
private boolean waitingShardsStartedOrUnassigned(ClusterChangedEvent event) {
|
|
|
- SnapshotMetaData curr = event.state().metaData().custom(SnapshotMetaData.TYPE);
|
|
|
+ SnapshotsInProgress curr = event.state().custom(SnapshotsInProgress.TYPE);
|
|
|
if (curr != null) {
|
|
|
- for (SnapshotMetaData.Entry entry : curr.entries()) {
|
|
|
+ for (SnapshotsInProgress.Entry entry : curr.entries()) {
|
|
|
if (entry.state() == State.STARTED && !entry.waitingIndices().isEmpty()) {
|
|
|
for (String index : entry.waitingIndices().keySet()) {
|
|
|
if (event.indexRoutingTableChanged(index)) {
|
|
@@ -759,11 +748,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
private boolean removedNodesCleanupNeeded(ClusterChangedEvent event) {
|
|
|
// Check if we just became the master
|
|
|
boolean newMaster = !event.previousState().nodes().localNodeMaster();
|
|
|
- SnapshotMetaData snapshotMetaData = event.state().getMetaData().custom(SnapshotMetaData.TYPE);
|
|
|
- if (snapshotMetaData == null) {
|
|
|
+ SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE);
|
|
|
+ if (snapshotsInProgress == null) {
|
|
|
return false;
|
|
|
}
|
|
|
- for (SnapshotMetaData.Entry snapshot : snapshotMetaData.entries()) {
|
|
|
+ for (SnapshotsInProgress.Entry snapshot : snapshotsInProgress.entries()) {
|
|
|
if (newMaster && (snapshot.state() == State.SUCCESS || snapshot.state() == State.INIT)) {
|
|
|
// We just replaced old master and snapshots in intermediate states needs to be cleaned
|
|
|
return true;
|
|
@@ -786,11 +775,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @param event cluster state changed event
|
|
|
*/
|
|
|
private void processIndexShardSnapshots(ClusterChangedEvent event) {
|
|
|
- SnapshotMetaData snapshotMetaData = event.state().metaData().custom(SnapshotMetaData.TYPE);
|
|
|
+ SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE);
|
|
|
Map<SnapshotId, SnapshotShards> survivors = newHashMap();
|
|
|
// First, remove snapshots that are no longer there
|
|
|
for (Map.Entry<SnapshotId, SnapshotShards> entry : shardSnapshots.entrySet()) {
|
|
|
- if (snapshotMetaData != null && snapshotMetaData.snapshot(entry.getKey()) != null) {
|
|
|
+ if (snapshotsInProgress != null && snapshotsInProgress.snapshot(entry.getKey()) != null) {
|
|
|
survivors.put(entry.getKey(), entry.getValue());
|
|
|
}
|
|
|
}
|
|
@@ -800,12 +789,12 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
Map<SnapshotId, Map<ShardId, IndexShardSnapshotStatus>> newSnapshots = newHashMap();
|
|
|
// Now go through all snapshots and update existing or create missing
|
|
|
final String localNodeId = clusterService.localNode().id();
|
|
|
- if (snapshotMetaData != null) {
|
|
|
- for (SnapshotMetaData.Entry entry : snapshotMetaData.entries()) {
|
|
|
+ if (snapshotsInProgress != null) {
|
|
|
+ for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) {
|
|
|
if (entry.state() == State.STARTED) {
|
|
|
Map<ShardId, IndexShardSnapshotStatus> startedShards = newHashMap();
|
|
|
SnapshotShards snapshotShards = shardSnapshots.get(entry.snapshotId());
|
|
|
- for (Map.Entry<ShardId, SnapshotMetaData.ShardSnapshotStatus> shard : entry.shards().entrySet()) {
|
|
|
+ for (Map.Entry<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shard : entry.shards().entrySet()) {
|
|
|
// Add all new shards to start processing on
|
|
|
if (localNodeId.equals(shard.getValue().nodeId())) {
|
|
|
if (shard.getValue().state() == State.INIT && (snapshotShards == null || !snapshotShards.shards.containsKey(shard.getKey()))) {
|
|
@@ -833,7 +822,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
// Abort all running shards for this snapshot
|
|
|
SnapshotShards snapshotShards = shardSnapshots.get(entry.snapshotId());
|
|
|
if (snapshotShards != null) {
|
|
|
- for (Map.Entry<ShardId, SnapshotMetaData.ShardSnapshotStatus> shard : entry.shards().entrySet()) {
|
|
|
+ for (Map.Entry<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shard : entry.shards().entrySet()) {
|
|
|
IndexShardSnapshotStatus snapshotStatus = snapshotShards.shards.get(shard.getKey());
|
|
|
if (snapshotStatus != null) {
|
|
|
switch (snapshotStatus.stage()) {
|
|
@@ -843,7 +832,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
case DONE:
|
|
|
logger.debug("[{}] trying to cancel snapshot on the shard [{}] that is already done, updating status on the master", entry.snapshotId(), shard.getKey());
|
|
|
updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.snapshotId(), shard.getKey(),
|
|
|
- new ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotMetaData.State.SUCCESS)));
|
|
|
+ new ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotsInProgress.State.SUCCESS)));
|
|
|
break;
|
|
|
case FAILURE:
|
|
|
logger.debug("[{}] trying to cancel snapshot on the shard [{}] that has already failed, updating status on the master", entry.snapshotId(), shard.getKey());
|
|
@@ -883,15 +872,15 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
public void run() {
|
|
|
try {
|
|
|
shardSnapshotService.snapshot(entry.getKey(), shardEntry.getValue());
|
|
|
- updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.getKey(), shardEntry.getKey(), new ShardSnapshotStatus(localNodeId, SnapshotMetaData.State.SUCCESS)));
|
|
|
+ updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.getKey(), shardEntry.getKey(), new ShardSnapshotStatus(localNodeId, SnapshotsInProgress.State.SUCCESS)));
|
|
|
} catch (Throwable t) {
|
|
|
logger.warn("[{}] [{}] failed to create snapshot", t, shardEntry.getKey(), entry.getKey());
|
|
|
- updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.getKey(), shardEntry.getKey(), new ShardSnapshotStatus(localNodeId, SnapshotMetaData.State.FAILED, ExceptionsHelper.detailedMessage(t))));
|
|
|
+ updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.getKey(), shardEntry.getKey(), new ShardSnapshotStatus(localNodeId, SnapshotsInProgress.State.FAILED, ExceptionsHelper.detailedMessage(t))));
|
|
|
}
|
|
|
}
|
|
|
});
|
|
|
} catch (Throwable t) {
|
|
|
- updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.getKey(), shardEntry.getKey(), new ShardSnapshotStatus(localNodeId, SnapshotMetaData.State.FAILED, ExceptionsHelper.detailedMessage(t))));
|
|
|
+ updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(entry.getKey(), shardEntry.getKey(), new ShardSnapshotStatus(localNodeId, SnapshotsInProgress.State.FAILED, ExceptionsHelper.detailedMessage(t))));
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -903,11 +892,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @param event
|
|
|
*/
|
|
|
private void syncShardStatsOnNewMaster(ClusterChangedEvent event) {
|
|
|
- SnapshotMetaData snapshotMetaData = event.state().getMetaData().custom(SnapshotMetaData.TYPE);
|
|
|
- if (snapshotMetaData == null) {
|
|
|
+ SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE);
|
|
|
+ if (snapshotsInProgress == null) {
|
|
|
return;
|
|
|
}
|
|
|
- for (SnapshotMetaData.Entry snapshot : snapshotMetaData.entries()) {
|
|
|
+ for (SnapshotsInProgress.Entry snapshot : snapshotsInProgress.entries()) {
|
|
|
if (snapshot.state() == State.STARTED || snapshot.state() == State.ABORTED) {
|
|
|
ImmutableMap<ShardId, IndexShardSnapshotStatus> localShards = currentSnapshotShards(snapshot.snapshotId());
|
|
|
if (localShards != null) {
|
|
@@ -922,7 +911,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
// but we think the shard is done - we need to make new master know that the shard is done
|
|
|
logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard is done locally, updating status on the master", snapshot.snapshotId(), shardId);
|
|
|
updateIndexShardSnapshotStatus(new UpdateIndexShardSnapshotStatusRequest(snapshot.snapshotId(), shardId,
|
|
|
- new ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotMetaData.State.SUCCESS)));
|
|
|
+ new ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotsInProgress.State.SUCCESS)));
|
|
|
} else if (localShard.getValue().stage() == IndexShardSnapshotStatus.Stage.FAILURE) {
|
|
|
// but we think the shard failed - we need to make new master know that the shard failed
|
|
|
logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard failed locally, updating status on master", snapshot.snapshotId(), shardId);
|
|
@@ -961,7 +950,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @param shards list of shard statuses
|
|
|
* @return true if all shards have completed (either successfully or failed), false otherwise
|
|
|
*/
|
|
|
- private boolean completed(Collection<SnapshotMetaData.ShardSnapshotStatus> shards) {
|
|
|
+ private boolean completed(Collection<SnapshotsInProgress.ShardSnapshotStatus> shards) {
|
|
|
for (ShardSnapshotStatus status : shards) {
|
|
|
if (!status.state().completed()) {
|
|
|
return false;
|
|
@@ -976,10 +965,10 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @param shards list of shard statuses
|
|
|
* @return list of failed and closed indices
|
|
|
*/
|
|
|
- private Tuple<Set<String>, Set<String>> indicesWithMissingShards(ImmutableMap<ShardId, SnapshotMetaData.ShardSnapshotStatus> shards, MetaData metaData) {
|
|
|
+ private Tuple<Set<String>, Set<String>> indicesWithMissingShards(ImmutableMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards, MetaData metaData) {
|
|
|
Set<String> missing = newHashSet();
|
|
|
Set<String> closed = newHashSet();
|
|
|
- for (ImmutableMap.Entry<ShardId, SnapshotMetaData.ShardSnapshotStatus> entry : shards.entrySet()) {
|
|
|
+ for (ImmutableMap.Entry<ShardId, SnapshotsInProgress.ShardSnapshotStatus> entry : shards.entrySet()) {
|
|
|
if (entry.getValue().state() == State.MISSING) {
|
|
|
if (metaData.hasIndex(entry.getKey().getIndex()) && metaData.index(entry.getKey().getIndex()).getState() == IndexMetaData.State.CLOSE) {
|
|
|
closed.add(entry.getKey().getIndex());
|
|
@@ -1019,12 +1008,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
return currentState;
|
|
|
}
|
|
|
|
|
|
- final MetaData metaData = currentState.metaData();
|
|
|
- final SnapshotMetaData snapshots = metaData.custom(SnapshotMetaData.TYPE);
|
|
|
+ final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE);
|
|
|
if (snapshots != null) {
|
|
|
int changedCount = 0;
|
|
|
- final List<SnapshotMetaData.Entry> entries = newArrayList();
|
|
|
- for (SnapshotMetaData.Entry entry : snapshots.entries()) {
|
|
|
+ final List<SnapshotsInProgress.Entry> entries = newArrayList();
|
|
|
+ for (SnapshotsInProgress.Entry entry : snapshots.entries()) {
|
|
|
HashMap<ShardId, ShardSnapshotStatus> shards = null;
|
|
|
|
|
|
for (int i = 0; i < batchSize; i++) {
|
|
@@ -1043,11 +1031,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
|
|
|
if (shards != null) {
|
|
|
if (!completed(shards.values())) {
|
|
|
- entries.add(new SnapshotMetaData.Entry(entry, ImmutableMap.copyOf(shards)));
|
|
|
+ entries.add(new SnapshotsInProgress.Entry(entry, ImmutableMap.copyOf(shards)));
|
|
|
} else {
|
|
|
// Snapshot is finished - mark it as done
|
|
|
// TODO: Add PARTIAL_SUCCESS status?
|
|
|
- SnapshotMetaData.Entry updatedEntry = new SnapshotMetaData.Entry(entry, State.SUCCESS, ImmutableMap.copyOf(shards));
|
|
|
+ SnapshotsInProgress.Entry updatedEntry = new SnapshotsInProgress.Entry(entry, State.SUCCESS, ImmutableMap.copyOf(shards));
|
|
|
entries.add(updatedEntry);
|
|
|
// Finalize snapshot in the repository
|
|
|
endSnapshot(updatedEntry);
|
|
@@ -1060,9 +1048,8 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
if (changedCount > 0) {
|
|
|
logger.trace("changed cluster state triggered by {} snapshot state updates", changedCount);
|
|
|
|
|
|
- final SnapshotMetaData updatedSnapshots = new SnapshotMetaData(entries.toArray(new SnapshotMetaData.Entry[entries.size()]));
|
|
|
- final MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData()).putCustom(SnapshotMetaData.TYPE, updatedSnapshots);
|
|
|
- return ClusterState.builder(currentState).metaData(mdBuilder).build();
|
|
|
+ final SnapshotsInProgress updatedSnapshots = new SnapshotsInProgress(entries.toArray(new SnapshotsInProgress.Entry[entries.size()]));
|
|
|
+ return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, updatedSnapshots).build();
|
|
|
}
|
|
|
}
|
|
|
return currentState;
|
|
@@ -1084,7 +1071,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
*
|
|
|
* @param entry snapshot
|
|
|
*/
|
|
|
- private void endSnapshot(SnapshotMetaData.Entry entry) {
|
|
|
+ private void endSnapshot(SnapshotsInProgress.Entry entry) {
|
|
|
endSnapshot(entry, null);
|
|
|
}
|
|
|
|
|
@@ -1097,7 +1084,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @param entry snapshot
|
|
|
* @param failure failure reason or null if snapshot was successful
|
|
|
*/
|
|
|
- private void endSnapshot(final SnapshotMetaData.Entry entry, final String failure) {
|
|
|
+ private void endSnapshot(final SnapshotsInProgress.Entry entry, final String failure) {
|
|
|
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new Runnable() {
|
|
|
@Override
|
|
|
public void run() {
|
|
@@ -1136,13 +1123,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
clusterService.submitStateUpdateTask("remove snapshot metadata", new ProcessedClusterStateUpdateTask() {
|
|
|
@Override
|
|
|
public ClusterState execute(ClusterState currentState) {
|
|
|
- MetaData metaData = currentState.metaData();
|
|
|
- MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
|
|
|
- SnapshotMetaData snapshots = metaData.custom(SnapshotMetaData.TYPE);
|
|
|
+ SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE);
|
|
|
if (snapshots != null) {
|
|
|
boolean changed = false;
|
|
|
- ArrayList<SnapshotMetaData.Entry> entries = newArrayList();
|
|
|
- for (SnapshotMetaData.Entry entry : snapshots.entries()) {
|
|
|
+ ArrayList<SnapshotsInProgress.Entry> entries = newArrayList();
|
|
|
+ for (SnapshotsInProgress.Entry entry : snapshots.entries()) {
|
|
|
if (entry.snapshotId().equals(snapshotId)) {
|
|
|
changed = true;
|
|
|
} else {
|
|
@@ -1150,9 +1135,8 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
}
|
|
|
}
|
|
|
if (changed) {
|
|
|
- snapshots = new SnapshotMetaData(entries.toArray(new SnapshotMetaData.Entry[entries.size()]));
|
|
|
- mdBuilder.putCustom(SnapshotMetaData.TYPE, snapshots);
|
|
|
- return ClusterState.builder(currentState).metaData(mdBuilder).build();
|
|
|
+ snapshots = new SnapshotsInProgress(entries.toArray(new SnapshotsInProgress.Entry[entries.size()]));
|
|
|
+ return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build();
|
|
|
}
|
|
|
}
|
|
|
return currentState;
|
|
@@ -1196,14 +1180,12 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
|
|
|
@Override
|
|
|
public ClusterState execute(ClusterState currentState) throws Exception {
|
|
|
- MetaData metaData = currentState.metaData();
|
|
|
- MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
|
|
|
- SnapshotMetaData snapshots = metaData.custom(SnapshotMetaData.TYPE);
|
|
|
+ SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE);
|
|
|
if (snapshots == null) {
|
|
|
// No snapshots running - we can continue
|
|
|
return currentState;
|
|
|
}
|
|
|
- SnapshotMetaData.Entry snapshot = snapshots.snapshot(snapshotId);
|
|
|
+ SnapshotsInProgress.Entry snapshot = snapshots.snapshot(snapshotId);
|
|
|
if (snapshot == null) {
|
|
|
// This snapshot is not running - continue
|
|
|
if (!snapshots.entries().isEmpty()) {
|
|
@@ -1252,10 +1234,9 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
endSnapshot(snapshot);
|
|
|
}
|
|
|
}
|
|
|
- SnapshotMetaData.Entry newSnapshot = new SnapshotMetaData.Entry(snapshot, State.ABORTED, shards);
|
|
|
- snapshots = new SnapshotMetaData(newSnapshot);
|
|
|
- mdBuilder.putCustom(SnapshotMetaData.TYPE, snapshots);
|
|
|
- return ClusterState.builder(currentState).metaData(mdBuilder).build();
|
|
|
+ SnapshotsInProgress.Entry newSnapshot = new SnapshotsInProgress.Entry(snapshot, State.ABORTED, shards);
|
|
|
+ snapshots = new SnapshotsInProgress(newSnapshot);
|
|
|
+ return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1303,10 +1284,9 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @return true if repository is currently in use by one of the running snapshots
|
|
|
*/
|
|
|
public static boolean isRepositoryInUse(ClusterState clusterState, String repository) {
|
|
|
- MetaData metaData = clusterState.metaData();
|
|
|
- SnapshotMetaData snapshots = metaData.custom(SnapshotMetaData.TYPE);
|
|
|
+ SnapshotsInProgress snapshots = clusterState.custom(SnapshotsInProgress.TYPE);
|
|
|
if (snapshots != null) {
|
|
|
- for (SnapshotMetaData.Entry snapshot : snapshots.entries()) {
|
|
|
+ for (SnapshotsInProgress.Entry snapshot : snapshots.entries()) {
|
|
|
if (repository.equals(snapshot.snapshotId().getRepository())) {
|
|
|
return true;
|
|
|
}
|
|
@@ -1343,18 +1323,18 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
* @param indices list of indices to be snapshotted
|
|
|
* @return list of shard to be included into current snapshot
|
|
|
*/
|
|
|
- private ImmutableMap<ShardId, SnapshotMetaData.ShardSnapshotStatus> shards(ClusterState clusterState, ImmutableList<String> indices) {
|
|
|
- ImmutableMap.Builder<ShardId, SnapshotMetaData.ShardSnapshotStatus> builder = ImmutableMap.builder();
|
|
|
+ private ImmutableMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards(ClusterState clusterState, ImmutableList<String> indices) {
|
|
|
+ ImmutableMap.Builder<ShardId, SnapshotsInProgress.ShardSnapshotStatus> builder = ImmutableMap.builder();
|
|
|
MetaData metaData = clusterState.metaData();
|
|
|
for (String index : indices) {
|
|
|
IndexMetaData indexMetaData = metaData.index(index);
|
|
|
if (indexMetaData == null) {
|
|
|
// The index was deleted before we managed to start the snapshot - mark it as missing.
|
|
|
- builder.put(new ShardId(index, 0), new SnapshotMetaData.ShardSnapshotStatus(null, State.MISSING, "missing index"));
|
|
|
+ builder.put(new ShardId(index, 0), new SnapshotsInProgress.ShardSnapshotStatus(null, State.MISSING, "missing index"));
|
|
|
} else if (indexMetaData.getState() == IndexMetaData.State.CLOSE) {
|
|
|
for (int i = 0; i < indexMetaData.numberOfShards(); i++) {
|
|
|
ShardId shardId = new ShardId(index, i);
|
|
|
- builder.put(shardId, new SnapshotMetaData.ShardSnapshotStatus(null, State.MISSING, "index is closed"));
|
|
|
+ builder.put(shardId, new SnapshotsInProgress.ShardSnapshotStatus(null, State.MISSING, "index is closed"));
|
|
|
}
|
|
|
} else {
|
|
|
IndexRoutingTable indexRoutingTable = clusterState.getRoutingTable().index(index);
|
|
@@ -1363,17 +1343,17 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
if (indexRoutingTable != null) {
|
|
|
ShardRouting primary = indexRoutingTable.shard(i).primaryShard();
|
|
|
if (primary == null || !primary.assignedToNode()) {
|
|
|
- builder.put(shardId, new SnapshotMetaData.ShardSnapshotStatus(null, State.MISSING, "primary shard is not allocated"));
|
|
|
+ builder.put(shardId, new SnapshotsInProgress.ShardSnapshotStatus(null, State.MISSING, "primary shard is not allocated"));
|
|
|
} else if (primary.relocating() || primary.initializing()) {
|
|
|
// The WAITING state was introduced in V1.2.0 - don't use it if there are nodes with older version in the cluster
|
|
|
- builder.put(shardId, new SnapshotMetaData.ShardSnapshotStatus(primary.currentNodeId(), State.WAITING));
|
|
|
+ builder.put(shardId, new SnapshotsInProgress.ShardSnapshotStatus(primary.currentNodeId(), State.WAITING));
|
|
|
} else if (!primary.started()) {
|
|
|
- builder.put(shardId, new SnapshotMetaData.ShardSnapshotStatus(primary.currentNodeId(), State.MISSING, "primary shard hasn't been started yet"));
|
|
|
+ builder.put(shardId, new SnapshotsInProgress.ShardSnapshotStatus(primary.currentNodeId(), State.MISSING, "primary shard hasn't been started yet"));
|
|
|
} else {
|
|
|
- builder.put(shardId, new SnapshotMetaData.ShardSnapshotStatus(primary.currentNodeId()));
|
|
|
+ builder.put(shardId, new SnapshotsInProgress.ShardSnapshotStatus(primary.currentNodeId()));
|
|
|
}
|
|
|
} else {
|
|
|
- builder.put(shardId, new SnapshotMetaData.ShardSnapshotStatus(null, State.MISSING, "missing routing table"));
|
|
|
+ builder.put(shardId, new SnapshotsInProgress.ShardSnapshotStatus(null, State.MISSING, "missing routing table"));
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -1656,7 +1636,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
private static class UpdateIndexShardSnapshotStatusRequest extends TransportRequest {
|
|
|
private SnapshotId snapshotId;
|
|
|
private ShardId shardId;
|
|
|
- private SnapshotMetaData.ShardSnapshotStatus status;
|
|
|
+ private SnapshotsInProgress.ShardSnapshotStatus status;
|
|
|
|
|
|
volatile boolean processed; // state field, no need to serialize
|
|
|
|
|
@@ -1664,7 +1644,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
|
|
|
}
|
|
|
|
|
|
- private UpdateIndexShardSnapshotStatusRequest(SnapshotId snapshotId, ShardId shardId, SnapshotMetaData.ShardSnapshotStatus status) {
|
|
|
+ private UpdateIndexShardSnapshotStatusRequest(SnapshotId snapshotId, ShardId shardId, SnapshotsInProgress.ShardSnapshotStatus status) {
|
|
|
this.snapshotId = snapshotId;
|
|
|
this.shardId = shardId;
|
|
|
this.status = status;
|
|
@@ -1675,7 +1655,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
super.readFrom(in);
|
|
|
snapshotId = SnapshotId.readSnapshotId(in);
|
|
|
shardId = ShardId.readShardId(in);
|
|
|
- status = SnapshotMetaData.ShardSnapshotStatus.readShardSnapshotStatus(in);
|
|
|
+ status = SnapshotsInProgress.ShardSnapshotStatus.readShardSnapshotStatus(in);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -1694,7 +1674,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
|
|
return shardId;
|
|
|
}
|
|
|
|
|
|
- public SnapshotMetaData.ShardSnapshotStatus status() {
|
|
|
+ public SnapshotsInProgress.ShardSnapshotStatus status() {
|
|
|
return status;
|
|
|
}
|
|
|
|