Skip to content

Commit

Permalink
Do not start snapshots that are deleted during initialization (#27931)
Browse files Browse the repository at this point in the history
When a new snapshot is created it is added to the cluster state as a
snapshot-in-progress in INIT state, and the initialization is kicked
off in a new runnable task by SnapshotService.beginSnapshot(). The
initialization writes multiple files before updating the cluster state
to change the snapshot-in-progress to STARTED state. This leaves a
short window in which the snapshot could be deleted (let's say, because
the snapshot is stuck in INIT or because it takes too much time to
upload all the initialization files for all snapshotted indices). If
the INIT snapshot is deleted, the snapshot-in-progress becomes ABORTED
but once the initialization in SnapshotService.beginSnapshot() finished
it is change back to STARTED state again.

This commit avoids an ABORTED snapshot to be started if it has been
deleted during initialization. It also adds a test that would have failed
with the previous behavior, and changes few method names here and there.
  • Loading branch information
tlrx committed Jan 15, 2018
1 parent 9875fd2 commit 9b6d37a
Show file tree
Hide file tree
Showing 5 changed files with 181 additions and 71 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ public SnapshotException(final String repositoryName, final String snapshotName,
}

public SnapshotException(final String repositoryName, final String snapshotName, final String msg, final Throwable cause) {
super("[" + repositoryName + ":" + snapshotName + "]" + msg, cause);
super("[" + repositoryName + ":" + snapshotName + "] " + msg, cause);
this.repositoryName = repositoryName;
this.snapshotName = snapshotName;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,15 +30,15 @@
public class SnapshotMissingException extends SnapshotException {

public SnapshotMissingException(final String repositoryName, final SnapshotId snapshotId, final Throwable cause) {
super(repositoryName, snapshotId, " is missing", cause);
super(repositoryName, snapshotId, "is missing", cause);
}

public SnapshotMissingException(final String repositoryName, final SnapshotId snapshotId) {
super(repositoryName, snapshotId, " is missing");
super(repositoryName, snapshotId, "is missing");
}

public SnapshotMissingException(final String repositoryName, final String snapshotName) {
super(repositoryName, snapshotName, " is missing");
super(repositoryName, snapshotName, "is missing");
}

public SnapshotMissingException(StreamInput in) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
Expand Down Expand Up @@ -146,7 +147,6 @@ protected void doStop() {
} finally {
shutdownLock.unlock();
}

}

@Override
Expand All @@ -157,14 +157,16 @@ protected void doClose() {
@Override
public void applyClusterState(ClusterChangedEvent event) {
try {
SnapshotsInProgress prev = event.previousState().custom(SnapshotsInProgress.TYPE);
SnapshotsInProgress curr = event.state().custom(SnapshotsInProgress.TYPE);

if ((prev == null && curr != null) || (prev != null && prev.equals(curr) == false)) {
SnapshotsInProgress previousSnapshots = event.previousState().custom(SnapshotsInProgress.TYPE);
SnapshotsInProgress currentSnapshots = event.state().custom(SnapshotsInProgress.TYPE);
if ((previousSnapshots == null && currentSnapshots != null)
|| (previousSnapshots != null && previousSnapshots.equals(currentSnapshots) == false)) {
processIndexShardSnapshots(event);
}
String masterNodeId = event.state().nodes().getMasterNodeId();
if (masterNodeId != null && masterNodeId.equals(event.previousState().nodes().getMasterNodeId()) == false) {

String previousMasterNodeId = event.previousState().nodes().getMasterNodeId();
String currentMasterNodeId = event.state().nodes().getMasterNodeId();
if (currentMasterNodeId != null && currentMasterNodeId.equals(previousMasterNodeId) == false) {
syncShardStatsOnNewMaster(event);
}

Expand Down Expand Up @@ -281,17 +283,18 @@ private void processIndexShardSnapshots(ClusterChangedEvent event) {
snapshotStatus.abort();
break;
case FINALIZE:
logger.debug("[{}] trying to cancel snapshot on shard [{}] that is finalizing, letting it finish", entry.snapshot(), shard.key);
logger.debug("[{}] trying to cancel snapshot on shard [{}] that is finalizing, " +
"letting it finish", entry.snapshot(), shard.key);
break;
case DONE:
logger.debug("[{}] trying to cancel snapshot on the shard [{}] that is already done, updating status on the master", entry.snapshot(), shard.key);
updateIndexShardSnapshotStatus(entry.snapshot(), shard.key,
new ShardSnapshotStatus(localNodeId, State.SUCCESS), masterNode);
logger.debug("[{}] trying to cancel snapshot on the shard [{}] that is already done, " +
"updating status on the master", entry.snapshot(), shard.key);
notifySuccessfulSnapshotShard(entry.snapshot(), shard.key, localNodeId, masterNode);
break;
case FAILURE:
logger.debug("[{}] trying to cancel snapshot on the shard [{}] that has already failed, updating status on the master", entry.snapshot(), shard.key);
updateIndexShardSnapshotStatus(entry.snapshot(), shard.key,
new ShardSnapshotStatus(localNodeId, State.FAILED, snapshotStatus.failure()), masterNode);
logger.debug("[{}] trying to cancel snapshot on the shard [{}] that has already failed, " +
"updating status on the master", entry.snapshot(), shard.key);
notifyFailedSnapshotShard(entry.snapshot(), shard.key, localNodeId, snapshotStatus.failure(), masterNode);
break;
default:
throw new IllegalStateException("Unknown snapshot shard stage " + snapshotStatus.stage());
Expand Down Expand Up @@ -320,34 +323,47 @@ private void processIndexShardSnapshots(ClusterChangedEvent event) {
if (newSnapshots.isEmpty() == false) {
Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
for (final Map.Entry<Snapshot, Map<ShardId, IndexShardSnapshotStatus>> entry : newSnapshots.entrySet()) {
Map<String, IndexId> indicesMap = snapshotIndices.get(entry.getKey());
final Snapshot snapshot = entry.getKey();
final Map<String, IndexId> indicesMap = snapshotIndices.get(snapshot);
assert indicesMap != null;

for (final Map.Entry<ShardId, IndexShardSnapshotStatus> shardEntry : entry.getValue().entrySet()) {
final ShardId shardId = shardEntry.getKey();
try {
final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id());
final IndexId indexId = indicesMap.get(shardId.getIndexName());
assert indexId != null;
executor.execute(new AbstractRunnable() {
@Override
public void doRun() {
snapshot(indexShard, entry.getKey(), indexId, shardEntry.getValue());
updateIndexShardSnapshotStatus(entry.getKey(), shardId,
new ShardSnapshotStatus(localNodeId, State.SUCCESS), masterNode);
}
final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id());
final IndexId indexId = indicesMap.get(shardId.getIndexName());
assert indexId != null;
executor.execute(new AbstractRunnable() {

@Override
public void onFailure(Exception e) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("[{}] [{}] failed to create snapshot", shardId, entry.getKey()), e);
updateIndexShardSnapshotStatus(entry.getKey(), shardId,
new ShardSnapshotStatus(localNodeId, State.FAILED, ExceptionsHelper.detailedMessage(e)), masterNode);
}
final SetOnce<Exception> failure = new SetOnce<>();

});
} catch (Exception e) {
updateIndexShardSnapshotStatus(entry.getKey(), shardId,
new ShardSnapshotStatus(localNodeId, State.FAILED, ExceptionsHelper.detailedMessage(e)), masterNode);
}
@Override
public void doRun() {
snapshot(indexShard, snapshot, indexId, shardEntry.getValue());
}

@Override
public void onFailure(Exception e) {
logger.warn((Supplier<?>) () ->
new ParameterizedMessage("[{}][{}] failed to snapshot shard", shardId, snapshot), e);
failure.set(e);
}

@Override
public void onRejection(Exception e) {
failure.set(e);
}

@Override
public void onAfter() {
final Exception exception = failure.get();
if (exception != null) {
final String failure = ExceptionsHelper.detailedMessage(exception);
notifyFailedSnapshotShard(snapshot, shardId, localNodeId, failure, masterNode);
} else {
notifySuccessfulSnapshotShard(snapshot, shardId, localNodeId, masterNode);
}
}
});
}
}
}
Expand All @@ -360,34 +376,36 @@ public void onFailure(Exception e) {
* @param snapshotStatus snapshot status
*/
private void snapshot(final IndexShard indexShard, final Snapshot snapshot, final IndexId indexId, final IndexShardSnapshotStatus snapshotStatus) {
Repository repository = snapshotsService.getRepositoriesService().repository(snapshot.getRepository());
ShardId shardId = indexShard.shardId();
if (!indexShard.routingEntry().primary()) {
final ShardId shardId = indexShard.shardId();
if (indexShard.routingEntry().primary() == false) {
throw new IndexShardSnapshotFailedException(shardId, "snapshot should be performed only on primary");
}
if (indexShard.routingEntry().relocating()) {
// do not snapshot when in the process of relocation of primaries so we won't get conflicts
throw new IndexShardSnapshotFailedException(shardId, "cannot snapshot while relocating");
}
if (indexShard.state() == IndexShardState.CREATED || indexShard.state() == IndexShardState.RECOVERING) {

final IndexShardState indexShardState = indexShard.state();
if (indexShardState == IndexShardState.CREATED || indexShardState == IndexShardState.RECOVERING) {
// shard has just been created, or still recovering
throw new IndexShardSnapshotFailedException(shardId, "shard didn't fully recover yet");
}

final Repository repository = snapshotsService.getRepositoriesService().repository(snapshot.getRepository());
try {
// we flush first to make sure we get the latest writes snapshotted
try (Engine.IndexCommitRef snapshotRef = indexShard.acquireIndexCommit(true)) {
repository.snapshotShard(indexShard, snapshot.getSnapshotId(), indexId, snapshotRef.getIndexCommit(), snapshotStatus);
if (logger.isDebugEnabled()) {
StringBuilder sb = new StringBuilder();
sb.append(" index : version [").append(snapshotStatus.indexVersion()).append("], number_of_files [").append(snapshotStatus.numberOfFiles()).append("] with total_size [").append(new ByteSizeValue(snapshotStatus.totalSize())).append("]\n");
StringBuilder details = new StringBuilder();
details.append(" index : version [").append(snapshotStatus.indexVersion());
details.append("], number_of_files [").append(snapshotStatus.numberOfFiles());
details.append("] with total_size [").append(new ByteSizeValue(snapshotStatus.totalSize())).append("]\n");
logger.debug("snapshot ({}) completed to {}, took [{}]\n{}", snapshot, repository,
TimeValue.timeValueMillis(snapshotStatus.time()), sb);
TimeValue.timeValueMillis(snapshotStatus.time()), details);
}
}
} catch (SnapshotFailedEngineException e) {
throw e;
} catch (IndexShardSnapshotFailedException e) {
} catch (SnapshotFailedEngineException | IndexShardSnapshotFailedException e) {
throw e;
} catch (Exception e) {
throw new IndexShardSnapshotFailedException(shardId, "Failed to snapshot", e);
Expand All @@ -402,6 +420,7 @@ private void syncShardStatsOnNewMaster(ClusterChangedEvent event) {
if (snapshotsInProgress == null) {
return;
}

final String localNodeId = event.state().nodes().getLocalNodeId();
final DiscoveryNode masterNode = event.state().nodes().getMasterNode();
for (SnapshotsInProgress.Entry snapshot : snapshotsInProgress.entries()) {
Expand All @@ -417,15 +436,16 @@ private void syncShardStatsOnNewMaster(ClusterChangedEvent event) {
// Master knows about the shard and thinks it has not completed
if (localShardStatus.stage() == Stage.DONE) {
// 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.snapshot(), shardId);
updateIndexShardSnapshotStatus(snapshot.snapshot(), shardId,
new ShardSnapshotStatus(localNodeId, State.SUCCESS), masterNode);
logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard is done locally, " +
"updating status on the master", snapshot.snapshot(), shardId);
notifySuccessfulSnapshotShard(snapshot.snapshot(), shardId, localNodeId, masterNode);

} else if (localShard.getValue().stage() == 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.snapshot(), shardId);
updateIndexShardSnapshotStatus(snapshot.snapshot(), shardId,
new ShardSnapshotStatus(localNodeId, State.FAILED, localShardStatus.failure()), masterNode);

logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard failed locally, " +
"updating status on master", snapshot.snapshot(), shardId);
final String failure = localShardStatus.failure();
notifyFailedSnapshotShard(snapshot.snapshot(), shardId, localNodeId, failure, masterNode);
}
}
}
Expand All @@ -445,7 +465,6 @@ private SnapshotShards(Map<ShardId, IndexShardSnapshotStatus> shards) {
}
}


/**
* Internal request that is used to send changes in snapshot status to master
*/
Expand Down Expand Up @@ -498,15 +517,33 @@ public String toString() {
}
}

/**
* Updates the shard status
*/
public void updateIndexShardSnapshotStatus(Snapshot snapshot, ShardId shardId, ShardSnapshotStatus status, DiscoveryNode master) {
UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status);
/** Notify the master node that the given shard has been successfully snapshotted **/
void notifySuccessfulSnapshotShard(final Snapshot snapshot,
final ShardId shardId,
final String localNodeId,
final DiscoveryNode masterNode) {
sendSnapshotShardUpdate(snapshot, shardId, new ShardSnapshotStatus(localNodeId, State.SUCCESS), masterNode);
}

/** Notify the master node that the given shard failed to be snapshotted **/
void notifyFailedSnapshotShard(final Snapshot snapshot,
final ShardId shardId,
final String localNodeId,
final String failure,
final DiscoveryNode masterNode) {
sendSnapshotShardUpdate(snapshot, shardId, new ShardSnapshotStatus(localNodeId, State.FAILED, failure), masterNode);
}

/** Updates the shard snapshot status by sending a {@link UpdateIndexShardSnapshotStatusRequest} to the master node */
void sendSnapshotShardUpdate(final Snapshot snapshot,
final ShardId shardId,
final ShardSnapshotStatus status,
final DiscoveryNode masterNode) {
try {
transportService.sendRequest(master, UPDATE_SNAPSHOT_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
final UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status);
transportService.sendRequest(masterNode, UPDATE_SNAPSHOT_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
} catch (Exception e) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("[{}] [{}] failed to update snapshot state", request.snapshot(), request.status()), e);
logger.warn((Supplier<?>) () -> new ParameterizedMessage("[{}] [{}] failed to update snapshot state", snapshot, status), e);
}
}

Expand Down
Loading

0 comments on commit 9b6d37a

Please sign in to comment.