Adds UUIDs to snapshots

This commit adds a UUID for each snapshot, in addition to the already
existing repository and snapshot name. The addition of UUIDs will enable
more robust handling of the deletion of previous snapshots and lingering
files from partially failed delete operations, on top of being able to
uniquely track each snapshot.

Closes #18228
Relates #18156
This commit is contained in:
Ali Beyad 2016-06-02 17:01:14 -04:00
parent f82ab787a5
commit b720216395
57 changed files with 2019 additions and 951 deletions

View File

@ -26,10 +26,10 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotInfo;
import org.elasticsearch.snapshots.SnapshotsService;
import org.elasticsearch.threadpool.ThreadPool;
@ -72,7 +72,7 @@ public class TransportCreateSnapshotAction extends TransportMasterNodeAction<Cre
@Override
protected void masterOperation(final CreateSnapshotRequest request, ClusterState state, final ActionListener<CreateSnapshotResponse> listener) {
SnapshotsService.SnapshotRequest snapshotRequest =
new SnapshotsService.SnapshotRequest("create_snapshot [" + request.snapshot() + "]", request.snapshot(), request.repository())
new SnapshotsService.SnapshotRequest(request.repository(), request.snapshot(), "create_snapshot [" + request.snapshot() + "]")
.indices(request.indices())
.indicesOptions(request.indicesOptions())
.partial(request.partial())
@ -84,19 +84,19 @@ public class TransportCreateSnapshotAction extends TransportMasterNodeAction<Cre
public void onResponse() {
if (request.waitForCompletion()) {
snapshotsService.addListener(new SnapshotsService.SnapshotCompletionListener() {
SnapshotId snapshotId = new SnapshotId(request.repository(), request.snapshot());
@Override
public void onSnapshotCompletion(SnapshotId snapshotId, SnapshotInfo snapshot) {
if (this.snapshotId.equals(snapshotId)) {
listener.onResponse(new CreateSnapshotResponse(snapshot));
public void onSnapshotCompletion(Snapshot snapshot, SnapshotInfo snapshotInfo) {
if (snapshot.getRepository().equals(request.repository()) &&
snapshot.getSnapshotId().getName().equals(request.snapshot())) {
listener.onResponse(new CreateSnapshotResponse(snapshotInfo));
snapshotsService.removeListener(this);
}
}
@Override
public void onSnapshotFailure(SnapshotId snapshotId, Throwable t) {
if (this.snapshotId.equals(snapshotId)) {
public void onSnapshotFailure(Snapshot snapshot, Throwable t) {
if (snapshot.getRepository().equals(request.repository()) &&
snapshot.getSnapshotId().getName().equals(request.snapshot())) {
listener.onFailure(t);
snapshotsService.removeListener(this);
}

View File

@ -26,7 +26,6 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -66,8 +65,7 @@ public class TransportDeleteSnapshotAction extends TransportMasterNodeAction<Del
@Override
protected void masterOperation(final DeleteSnapshotRequest request, ClusterState state, final ActionListener<DeleteSnapshotResponse> listener) {
SnapshotId snapshotIds = new SnapshotId(request.repository(), request.snapshot());
snapshotsService.deleteSnapshot(snapshotIds, new SnapshotsService.DeleteSnapshotListener() {
snapshotsService.deleteSnapshot(request.repository(), request.snapshot(), new SnapshotsService.DeleteSnapshotListener() {
@Override
public void onResponse() {
listener.onResponse(new DeleteSnapshotResponse(true));

View File

@ -42,7 +42,7 @@ public class GetSnapshotsResponse extends ActionResponse implements ToXContent {
}
GetSnapshotsResponse(List<SnapshotInfo> snapshots) {
this.snapshots = snapshots;
this.snapshots = Collections.unmodifiableList(snapshots);
}
/**

View File

@ -26,20 +26,22 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.snapshots.SnapshotInfo;
import org.elasticsearch.snapshots.SnapshotMissingException;
import org.elasticsearch.snapshots.SnapshotsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
@ -52,7 +54,8 @@ public class TransportGetSnapshotsAction extends TransportMasterNodeAction<GetSn
public TransportGetSnapshotsAction(Settings settings, TransportService transportService, ClusterService clusterService,
ThreadPool threadPool, SnapshotsService snapshotsService, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, GetSnapshotsAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, GetSnapshotsRequest::new);
super(settings, GetSnapshotsAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver,
GetSnapshotsRequest::new);
this.snapshotsService = snapshotsService;
}
@ -72,36 +75,51 @@ public class TransportGetSnapshotsAction extends TransportMasterNodeAction<GetSn
}
@Override
protected void masterOperation(final GetSnapshotsRequest request, ClusterState state, final ActionListener<GetSnapshotsResponse> listener) {
protected void masterOperation(final GetSnapshotsRequest request, ClusterState state,
final ActionListener<GetSnapshotsResponse> listener) {
try {
final String repository = request.repository();
List<SnapshotInfo> snapshotInfoBuilder = new ArrayList<>();
if (isAllSnapshots(request.snapshots())) {
snapshotInfoBuilder.addAll(snapshotsService.snapshots(request.repository(), request.ignoreUnavailable()));
snapshotInfoBuilder.addAll(snapshotsService.currentSnapshots(repository));
snapshotInfoBuilder.addAll(snapshotsService.snapshots(repository,
snapshotsService.snapshotIds(repository),
request.ignoreUnavailable()));
} else if (isCurrentSnapshots(request.snapshots())) {
snapshotInfoBuilder.addAll(snapshotsService.currentSnapshots(request.repository()));
snapshotInfoBuilder.addAll(snapshotsService.currentSnapshots(repository));
} else {
Set<String> snapshotsToGet = new LinkedHashSet<>(); // to keep insertion order
List<SnapshotInfo> snapshots = null;
final Map<String, SnapshotId> allSnapshotIds = new HashMap<>();
for (SnapshotInfo snapshotInfo : snapshotsService.currentSnapshots(repository)) {
SnapshotId snapshotId = snapshotInfo.snapshotId();
allSnapshotIds.put(snapshotId.getName(), snapshotId);
}
for (SnapshotId snapshotId : snapshotsService.snapshotIds(repository)) {
allSnapshotIds.put(snapshotId.getName(), snapshotId);
}
final Set<SnapshotId> toResolve = new LinkedHashSet<>(); // maintain order
for (String snapshotOrPattern : request.snapshots()) {
if (Regex.isSimpleMatchPattern(snapshotOrPattern) == false) {
snapshotsToGet.add(snapshotOrPattern);
} else {
if (snapshots == null) { // lazily load snapshots
snapshots = snapshotsService.snapshots(request.repository(), request.ignoreUnavailable());
if (allSnapshotIds.containsKey(snapshotOrPattern)) {
toResolve.add(allSnapshotIds.get(snapshotOrPattern));
} else if (request.ignoreUnavailable() == false) {
throw new SnapshotMissingException(repository, snapshotOrPattern);
}
for (SnapshotInfo snapshot : snapshots) {
if (Regex.simpleMatch(snapshotOrPattern, snapshot.name())) {
snapshotsToGet.add(snapshot.name());
} else {
for (Map.Entry<String, SnapshotId> entry : allSnapshotIds.entrySet()) {
if (Regex.simpleMatch(snapshotOrPattern, entry.getKey())) {
toResolve.add(entry.getValue());
}
}
}
}
for (String snapshot : snapshotsToGet) {
SnapshotId snapshotId = new SnapshotId(request.repository(), snapshot);
snapshotInfoBuilder.add(snapshotsService.snapshot(snapshotId));
if (toResolve.isEmpty() && request.ignoreUnavailable() == false) {
throw new SnapshotMissingException(repository, request.snapshots()[0]);
}
snapshotInfoBuilder.addAll(snapshotsService.snapshots(repository, new ArrayList<>(toResolve), request.ignoreUnavailable()));
}
listener.onResponse(new GetSnapshotsResponse(Collections.unmodifiableList(snapshotInfoBuilder)));
listener.onResponse(new GetSnapshotsResponse(snapshotInfoBuilder));
} catch (Throwable t) {
listener.onFailure(t);
}

View File

@ -26,12 +26,12 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.snapshots.RestoreInfo;
import org.elasticsearch.snapshots.RestoreService;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -72,23 +72,22 @@ public class TransportRestoreSnapshotAction extends TransportMasterNodeAction<Re
}
@Override
protected void masterOperation(final RestoreSnapshotRequest request, ClusterState state, final ActionListener<RestoreSnapshotResponse> listener) {
RestoreService.RestoreRequest restoreRequest = new RestoreService.RestoreRequest(
"restore_snapshot[" + request.snapshot() + "]", request.repository(), request.snapshot(),
protected void masterOperation(final RestoreSnapshotRequest request, final ClusterState state, final ActionListener<RestoreSnapshotResponse> listener) {
RestoreService.RestoreRequest restoreRequest = new RestoreService.RestoreRequest(request.repository(), request.snapshot(),
request.indices(), request.indicesOptions(), request.renamePattern(), request.renameReplacement(),
request.settings(), request.masterNodeTimeout(), request.includeGlobalState(), request.partial(), request.includeAliases(),
request.indexSettings(), request.ignoreIndexSettings());
request.indexSettings(), request.ignoreIndexSettings(), "restore_snapshot[" + request.snapshot() + "]");
restoreService.restoreSnapshot(restoreRequest, new ActionListener<RestoreInfo>() {
@Override
public void onResponse(RestoreInfo restoreInfo) {
if (restoreInfo == null && request.waitForCompletion()) {
restoreService.addListener(new ActionListener<RestoreService.RestoreCompletionResponse>() {
SnapshotId snapshotId = new SnapshotId(request.repository(), request.snapshot());
@Override
public void onResponse(RestoreService.RestoreCompletionResponse restoreCompletionResponse) {
if (this.snapshotId.equals(restoreCompletionResponse.getSnapshotId())) {
final Snapshot snapshot = restoreCompletionResponse.getSnapshot();
if (snapshot.getRepository().equals(request.repository()) &&
snapshot.getSnapshotId().getName().equals(request.snapshot())) {
listener.onResponse(new RestoreSnapshotResponse(restoreCompletionResponse.getRestoreInfo()));
restoreService.removeListener(this);
}

View File

@ -20,7 +20,7 @@
package org.elasticsearch.action.admin.cluster.snapshots.status;
import org.elasticsearch.cluster.SnapshotsInProgress.State;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
@ -35,6 +35,7 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import static java.util.Collections.unmodifiableMap;
@ -44,7 +45,7 @@ import static java.util.Collections.unmodifiableMap;
*/
public class SnapshotStatus implements ToXContent, Streamable {
private SnapshotId snapshotId;
private Snapshot snapshot;
private State state;
@ -56,11 +57,10 @@ public class SnapshotStatus implements ToXContent, Streamable {
private SnapshotStats stats;
SnapshotStatus(SnapshotId snapshotId, State state, List<SnapshotIndexShardStatus> shards) {
this.snapshotId = snapshotId;
this.state = state;
this.shards = shards;
SnapshotStatus(final Snapshot snapshot, final State state, final List<SnapshotIndexShardStatus> shards) {
this.snapshot = Objects.requireNonNull(snapshot);
this.state = Objects.requireNonNull(state);
this.shards = Objects.requireNonNull(shards);
shardsStats = new SnapshotShardsStats(shards);
updateShardStats();
}
@ -69,10 +69,10 @@ public class SnapshotStatus implements ToXContent, Streamable {
}
/**
* Returns snapshot id
* Returns snapshot
*/
public SnapshotId getSnapshotId() {
return snapshotId;
public Snapshot getSnapshot() {
return snapshot;
}
/**
@ -124,7 +124,7 @@ public class SnapshotStatus implements ToXContent, Streamable {
@Override
public void readFrom(StreamInput in) throws IOException {
snapshotId = SnapshotId.readSnapshotId(in);
snapshot = new Snapshot(in);
state = State.fromValue(in.readByte());
int size = in.readVInt();
List<SnapshotIndexShardStatus> builder = new ArrayList<>();
@ -137,7 +137,7 @@ public class SnapshotStatus implements ToXContent, Streamable {
@Override
public void writeTo(StreamOutput out) throws IOException {
snapshotId.writeTo(out);
snapshot.writeTo(out);
out.writeByte(state.value());
out.writeVInt(shards.size());
for (SnapshotIndexShardStatus shard : shards) {
@ -170,7 +170,6 @@ public class SnapshotStatus implements ToXContent, Streamable {
}
}
/**
* Returns number of files in the snapshot
*/
@ -178,22 +177,22 @@ public class SnapshotStatus implements ToXContent, Streamable {
return stats;
}
static final class Fields {
static final String SNAPSHOT = "snapshot";
static final String REPOSITORY = "repository";
static final String STATE = "state";
static final String INDICES = "indices";
}
private static final String SNAPSHOT = "snapshot";
private static final String REPOSITORY = "repository";
private static final String UUID = "uuid";
private static final String STATE = "state";
private static final String INDICES = "indices";
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field(Fields.SNAPSHOT, snapshotId.getSnapshot());
builder.field(Fields.REPOSITORY, snapshotId.getRepository());
builder.field(Fields.STATE, state.name());
builder.field(SNAPSHOT, snapshot.getSnapshotId().getName());
builder.field(REPOSITORY, snapshot.getRepository());
builder.field(UUID, snapshot.getSnapshotId().getUUID());
builder.field(STATE, state.name());
shardsStats.toXContent(builder, params);
stats.toXContent(builder, params);
builder.startObject(Fields.INDICES);
builder.startObject(INDICES);
for (SnapshotIndexStatus indexStatus : getIndices().values()) {
indexStatus.toXContent(builder, params);
}

View File

@ -29,7 +29,7 @@ import org.elasticsearch.action.support.nodes.BaseNodesResponse;
import org.elasticsearch.action.support.nodes.TransportNodesAction;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
@ -94,11 +94,11 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction<Transpor
@Override
protected NodeSnapshotStatus nodeOperation(NodeRequest request) {
Map<SnapshotId, Map<ShardId, SnapshotIndexShardStatus>> snapshotMapBuilder = new HashMap<>();
Map<Snapshot, Map<ShardId, SnapshotIndexShardStatus>> snapshotMapBuilder = new HashMap<>();
try {
String nodeId = clusterService.localNode().getId();
for (SnapshotId snapshotId : request.snapshotIds) {
Map<ShardId, IndexShardSnapshotStatus> shardsStatus = snapshotShardsService.currentSnapshotShards(snapshotId);
for (Snapshot snapshot : request.snapshots) {
Map<ShardId, IndexShardSnapshotStatus> shardsStatus = snapshotShardsService.currentSnapshotShards(snapshot);
if (shardsStatus == null) {
continue;
}
@ -114,7 +114,7 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction<Transpor
}
shardMapBuilder.put(shardEntry.getKey(), shardStatus);
}
snapshotMapBuilder.put(snapshotId, unmodifiableMap(shardMapBuilder));
snapshotMapBuilder.put(snapshot, unmodifiableMap(shardMapBuilder));
}
return new NodeSnapshotStatus(clusterService.localNode(), unmodifiableMap(snapshotMapBuilder));
} catch (Exception e) {
@ -129,7 +129,7 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction<Transpor
public static class Request extends BaseNodesRequest<Request> {
private SnapshotId[] snapshotIds;
private Snapshot[] snapshots;
public Request() {
}
@ -138,8 +138,8 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction<Transpor
super(nodesIds);
}
public Request snapshotIds(SnapshotId[] snapshotIds) {
this.snapshotIds = snapshotIds;
public Request snapshots(Snapshot[] snapshots) {
this.snapshots = snapshots;
return this;
}
@ -179,42 +179,42 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction<Transpor
public static class NodeRequest extends BaseNodeRequest {
private List<SnapshotId> snapshotIds;
private List<Snapshot> snapshots;
public NodeRequest() {
}
NodeRequest(String nodeId, TransportNodesSnapshotsStatus.Request request) {
super(nodeId);
snapshotIds = Arrays.asList(request.snapshotIds);
snapshots = Arrays.asList(request.snapshots);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
snapshotIds = in.readList(SnapshotId::readSnapshotId);
snapshots = in.readList(Snapshot::new);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeStreamableList(snapshotIds);
out.writeList(snapshots);
}
}
public static class NodeSnapshotStatus extends BaseNodeResponse {
private Map<SnapshotId, Map<ShardId, SnapshotIndexShardStatus>> status;
private Map<Snapshot, Map<ShardId, SnapshotIndexShardStatus>> status;
NodeSnapshotStatus() {
}
public NodeSnapshotStatus(DiscoveryNode node, Map<SnapshotId, Map<ShardId, SnapshotIndexShardStatus>> status) {
public NodeSnapshotStatus(DiscoveryNode node, Map<Snapshot, Map<ShardId, SnapshotIndexShardStatus>> status) {
super(node);
this.status = status;
}
public Map<SnapshotId, Map<ShardId, SnapshotIndexShardStatus>> status() {
public Map<Snapshot, Map<ShardId, SnapshotIndexShardStatus>> status() {
return status;
}
@ -222,9 +222,9 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction<Transpor
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
int numberOfSnapshots = in.readVInt();
Map<SnapshotId, Map<ShardId, SnapshotIndexShardStatus>> snapshotMapBuilder = new HashMap<>(numberOfSnapshots);
Map<Snapshot, Map<ShardId, SnapshotIndexShardStatus>> snapshotMapBuilder = new HashMap<>(numberOfSnapshots);
for (int i = 0; i < numberOfSnapshots; i++) {
SnapshotId snapshotId = SnapshotId.readSnapshotId(in);
Snapshot snapshot = new Snapshot(in);
int numberOfShards = in.readVInt();
Map<ShardId, SnapshotIndexShardStatus> shardMapBuilder = new HashMap<>(numberOfShards);
for (int j = 0; j < numberOfShards; j++) {
@ -232,7 +232,7 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction<Transpor
SnapshotIndexShardStatus status = SnapshotIndexShardStatus.readShardSnapshotStatus(in);
shardMapBuilder.put(shardId, status);
}
snapshotMapBuilder.put(snapshotId, unmodifiableMap(shardMapBuilder));
snapshotMapBuilder.put(snapshot, unmodifiableMap(shardMapBuilder));
}
status = unmodifiableMap(snapshotMapBuilder);
}
@ -242,7 +242,7 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction<Transpor
super.writeTo(out);
if (status != null) {
out.writeVInt(status.size());
for (Map.Entry<SnapshotId, Map<ShardId, SnapshotIndexShardStatus>> entry : status.entrySet()) {
for (Map.Entry<Snapshot, Map<ShardId, SnapshotIndexShardStatus>> entry : status.entrySet()) {
entry.getKey().writeTo(out);
out.writeVInt(entry.getValue().size());
for (Map.Entry<ShardId, SnapshotIndexShardStatus> shardEntry : entry.getValue().entrySet()) {

View File

@ -29,26 +29,32 @@ import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.snapshots.SnapshotInfo;
import org.elasticsearch.snapshots.SnapshotMissingException;
import org.elasticsearch.snapshots.SnapshotsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
/**
*/
@ -87,8 +93,8 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
protected void masterOperation(final SnapshotsStatusRequest request,
final ClusterState state,
final ActionListener<SnapshotsStatusResponse> listener) throws Exception {
List<SnapshotsInProgress.Entry> currentSnapshots = snapshotsService.currentSnapshots(request.repository(), request.snapshots());
List<SnapshotsInProgress.Entry> currentSnapshots =
snapshotsService.currentSnapshots(request.repository(), Arrays.asList(request.snapshots()));
if (currentSnapshots.isEmpty()) {
listener.onResponse(buildResponse(request, currentSnapshots, null));
return;
@ -105,19 +111,19 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
if (!nodesIds.isEmpty()) {
// There are still some snapshots running - check their progress
SnapshotId[] snapshotIds = new SnapshotId[currentSnapshots.size()];
Snapshot[] snapshots = new Snapshot[currentSnapshots.size()];
for (int i = 0; i < currentSnapshots.size(); i++) {
snapshotIds[i] = currentSnapshots.get(i).snapshotId();
snapshots[i] = currentSnapshots.get(i).snapshot();
}
TransportNodesSnapshotsStatus.Request nodesRequest = new TransportNodesSnapshotsStatus.Request(nodesIds.toArray(new String[nodesIds.size()]))
.snapshotIds(snapshotIds).timeout(request.masterNodeTimeout());
.snapshots(snapshots).timeout(request.masterNodeTimeout());
transportNodesSnapshotsStatus.execute(nodesRequest, new ActionListener<TransportNodesSnapshotsStatus.NodesSnapshotStatus>() {
@Override
public void onResponse(TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses) {
try {
List<SnapshotsInProgress.Entry> currentSnapshots =
snapshotsService.currentSnapshots(request.repository(), request.snapshots());
snapshotsService.currentSnapshots(request.repository(), Arrays.asList(request.snapshots()));
listener.onResponse(buildResponse(request, currentSnapshots, nodeSnapshotStatuses));
} catch (Throwable e) {
listener.onFailure(e);
@ -136,12 +142,12 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
}
private SnapshotsStatusResponse buildResponse(SnapshotsStatusRequest request, List<SnapshotsInProgress.Entry> currentSnapshots,
private SnapshotsStatusResponse buildResponse(SnapshotsStatusRequest request, List<SnapshotsInProgress.Entry> currentSnapshotEntries,
TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses) throws IOException {
// First process snapshot that are currently processed
List<SnapshotStatus> builder = new ArrayList<>();
Set<SnapshotId> currentSnapshotIds = new HashSet<>();
if (!currentSnapshots.isEmpty()) {
Set<String> currentSnapshotNames = new HashSet<>();
if (!currentSnapshotEntries.isEmpty()) {
Map<String, TransportNodesSnapshotsStatus.NodeSnapshotStatus> nodeSnapshotStatusMap;
if (nodeSnapshotStatuses != null) {
nodeSnapshotStatusMap = nodeSnapshotStatuses.getNodesMap();
@ -149,8 +155,8 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
nodeSnapshotStatusMap = new HashMap<>();
}
for (SnapshotsInProgress.Entry entry : currentSnapshots) {
currentSnapshotIds.add(entry.snapshotId());
for (SnapshotsInProgress.Entry entry : currentSnapshotEntries) {
currentSnapshotNames.add(entry.snapshot().getSnapshotId().getName());
List<SnapshotIndexShardStatus> shardStatusBuilder = new ArrayList<>();
for (ObjectObjectCursor<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shardEntry : entry.shards()) {
SnapshotsInProgress.ShardSnapshotStatus status = shardEntry.value;
@ -158,7 +164,7 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
// We should have information about this shard from the shard:
TransportNodesSnapshotsStatus.NodeSnapshotStatus nodeStatus = nodeSnapshotStatusMap.get(status.nodeId());
if (nodeStatus != null) {
Map<ShardId, SnapshotIndexShardStatus> shardStatues = nodeStatus.status().get(entry.snapshotId());
Map<ShardId, SnapshotIndexShardStatus> shardStatues = nodeStatus.status().get(entry.snapshot());
if (shardStatues != null) {
SnapshotIndexShardStatus shardStatus = shardStatues.get(shardEntry.key);
if (shardStatus != null) {
@ -190,41 +196,50 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
SnapshotIndexShardStatus shardStatus = new SnapshotIndexShardStatus(shardEntry.key, stage);
shardStatusBuilder.add(shardStatus);
}
builder.add(new SnapshotStatus(entry.snapshotId(), entry.state(), Collections.unmodifiableList(shardStatusBuilder)));
builder.add(new SnapshotStatus(entry.snapshot(), entry.state(), Collections.unmodifiableList(shardStatusBuilder)));
}
}
// Now add snapshots on disk that are not currently running
if (Strings.hasText(request.repository())) {
if (request.snapshots() != null && request.snapshots().length > 0) {
for (String snapshotName : request.snapshots()) {
SnapshotId snapshotId = new SnapshotId(request.repository(), snapshotName);
if (currentSnapshotIds.contains(snapshotId)) {
// This is a snapshot the is currently running - skipping
final String repositoryName = request.repository();
if (Strings.hasText(repositoryName) && request.snapshots() != null && request.snapshots().length > 0) {
final Set<String> requestedSnapshotNames = Sets.newHashSet(request.snapshots());
final Map<String, SnapshotId> matchedSnapshotIds = snapshotsService.snapshotIds(repositoryName).stream()
.filter(s -> requestedSnapshotNames.contains(s.getName()))
.collect(Collectors.toMap(SnapshotId::getName, Function.identity()));
for (final String snapshotName : request.snapshots()) {
SnapshotId snapshotId = matchedSnapshotIds.get(snapshotName);
if (snapshotId == null) {
if (currentSnapshotNames.contains(snapshotName)) {
// we've already found this snapshot in the current snapshot entries, so skip over
continue;
} else {
// neither in the current snapshot entries nor found in the repository
throw new SnapshotMissingException(repositoryName, snapshotName);
}
SnapshotInfo snapshot = snapshotsService.snapshot(snapshotId);
List<SnapshotIndexShardStatus> shardStatusBuilder = new ArrayList<>();
if (snapshot.state().completed()) {
Map<ShardId, IndexShardSnapshotStatus> shardStatues = snapshotsService.snapshotShards(snapshotId);
for (Map.Entry<ShardId, IndexShardSnapshotStatus> shardStatus : shardStatues.entrySet()) {
shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), shardStatus.getValue()));
}
final SnapshotsInProgress.State state;
switch (snapshot.state()) {
case FAILED:
state = SnapshotsInProgress.State.FAILED;
break;
case SUCCESS:
case PARTIAL:
// Translating both PARTIAL and SUCCESS to SUCCESS for now
// TODO: add the differentiation on the metadata level in the next major release
state = SnapshotsInProgress.State.SUCCESS;
break;
default:
throw new IllegalArgumentException("Unknown snapshot state " + snapshot.state());
}
builder.add(new SnapshotStatus(snapshotId, state, Collections.unmodifiableList(shardStatusBuilder)));
}
SnapshotInfo snapshotInfo = snapshotsService.snapshot(repositoryName, snapshotId);
List<SnapshotIndexShardStatus> shardStatusBuilder = new ArrayList<>();
if (snapshotInfo.state().completed()) {
Map<ShardId, IndexShardSnapshotStatus> shardStatues =
snapshotsService.snapshotShards(request.repository(), snapshotInfo);
for (Map.Entry<ShardId, IndexShardSnapshotStatus> shardStatus : shardStatues.entrySet()) {
shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), shardStatus.getValue()));
}
final SnapshotsInProgress.State state;
switch (snapshotInfo.state()) {
case FAILED:
state = SnapshotsInProgress.State.FAILED;
break;
case SUCCESS:
case PARTIAL:
// Translating both PARTIAL and SUCCESS to SUCCESS for now
// TODO: add the differentiation on the metadata level in the next major release
state = SnapshotsInProgress.State.SUCCESS;
break;
default:
throw new IllegalArgumentException("Unknown snapshot state " + snapshotInfo.state());
}
builder.add(new SnapshotStatus(new Snapshot(repositoryName, snapshotInfo.snapshotId()), state, Collections.unmodifiableList(shardStatusBuilder)));
}
}
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.cluster;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.cluster.ClusterState.Custom;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -34,6 +34,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
/**
* Meta data about restore processes that are currently executing
@ -73,22 +74,6 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
return this.entries;
}
/**
* Returns currently running restore process with corresponding snapshot id or null if this snapshot is not being
* restored
*
* @param snapshotId snapshot id
* @return restore metadata or null
*/
public Entry snapshot(SnapshotId snapshotId) {
for (Entry entry : entries) {
if (snapshotId.equals(entry.snapshotId())) {
return entry;
}
}
return null;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
@ -111,22 +96,22 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
*/
public static class Entry {
private final State state;
private final SnapshotId snapshotId;
private final Snapshot snapshot;
private final ImmutableOpenMap<ShardId, ShardRestoreStatus> shards;
private final List<String> indices;
/**
* Creates new restore metadata
*
* @param snapshotId snapshot id
* @param snapshot snapshot
* @param state current state of the restore process
* @param indices list of indices being restored
* @param shards map of shards being restored to their current restore status
*/
public Entry(SnapshotId snapshotId, State state, List<String> indices, ImmutableOpenMap<ShardId, ShardRestoreStatus> shards) {
this.snapshotId = snapshotId;
this.state = state;
this.indices = indices;
public Entry(Snapshot snapshot, State state, List<String> indices, ImmutableOpenMap<ShardId, ShardRestoreStatus> shards) {
this.snapshot = Objects.requireNonNull(snapshot);
this.state = Objects.requireNonNull(state);
this.indices = Objects.requireNonNull(indices);
if (shards == null) {
this.shards = ImmutableOpenMap.of();
} else {
@ -135,12 +120,12 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
}
/**
* Returns snapshot id
* Returns snapshot
*
* @return snapshot id
* @return snapshot
*/
public SnapshotId snapshotId() {
return this.snapshotId;
public Snapshot snapshot() {
return this.snapshot;
}
/**
@ -172,26 +157,22 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Entry entry = (Entry) o;
if (!indices.equals(entry.indices)) return false;
if (!snapshotId.equals(entry.snapshotId)) return false;
if (!shards.equals(entry.shards)) return false;
if (state != entry.state) return false;
return true;
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
@SuppressWarnings("unchecked") Entry entry = (Entry) o;
return snapshot.equals(entry.snapshot) &&
state == entry.state &&
indices.equals(entry.indices) &&
shards.equals(entry.shards);
}
@Override
public int hashCode() {
int result = state.hashCode();
result = 31 * result + snapshotId.hashCode();
result = 31 * result + shards.hashCode();
result = 31 * result + indices.hashCode();
return result;
return Objects.hash(snapshot, state, indices, shards);
}
}
@ -301,31 +282,29 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
ShardRestoreStatus status = (ShardRestoreStatus) o;
if (nodeId != null ? !nodeId.equals(status.nodeId) : status.nodeId != null) return false;
if (reason != null ? !reason.equals(status.reason) : status.reason != null) return false;
if (state != status.state) return false;
return true;
@SuppressWarnings("unchecked") ShardRestoreStatus status = (ShardRestoreStatus) o;
return state == status.state &&
Objects.equals(nodeId, status.nodeId) &&
Objects.equals(reason, status.reason);
}
@Override
public int hashCode() {
int result = state != null ? state.hashCode() : 0;
result = 31 * result + (nodeId != null ? nodeId.hashCode() : 0);
result = 31 * result + (reason != null ? reason.hashCode() : 0);
return result;
return Objects.hash(state, nodeId, reason);
}
}
/**
* Shard restore process state
*/
public static enum State {
public enum State {
/**
* Initializing state
*/
@ -409,7 +388,7 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
public RestoreInProgress readFrom(StreamInput in) throws IOException {
Entry[] entries = new Entry[in.readVInt()];
for (int i = 0; i < entries.length; i++) {
SnapshotId snapshotId = SnapshotId.readSnapshotId(in);
Snapshot snapshot = new Snapshot(in);
State state = State.fromValue(in.readByte());
int indices = in.readVInt();
List<String> indexBuilder = new ArrayList<>();
@ -423,7 +402,7 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
ShardRestoreStatus shardState = ShardRestoreStatus.readShardRestoreStatus(in);
builder.put(shardId, shardState);
}
entries[i] = new Entry(snapshotId, state, Collections.unmodifiableList(indexBuilder), builder.build());
entries[i] = new Entry(snapshot, state, Collections.unmodifiableList(indexBuilder), builder.build());
}
return new RestoreInProgress(entries);
}
@ -435,7 +414,7 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(entries.size());
for (Entry entry : entries) {
entry.snapshotId().writeTo(out);
entry.snapshot().writeTo(out);
out.writeByte(entry.state().value());
out.writeVInt(entry.indices().size());
for (String index : entry.indices()) {
@ -471,8 +450,8 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
*/
public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException {
builder.startObject();
builder.field("snapshot", entry.snapshotId().getSnapshot());
builder.field("repository", entry.snapshotId().getRepository());
builder.field("snapshot", entry.snapshot().getSnapshotId().getName());
builder.field("repository", entry.snapshot().getRepository());
builder.field("state", entry.state());
builder.startArray("indices");
{

View File

@ -23,13 +23,13 @@ import com.carrotsearch.hppc.ObjectContainer;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.cluster.ClusterState.Custom;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.snapshots.Snapshot;
import java.io.IOException;
import java.util.ArrayList;
@ -66,7 +66,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
public static class Entry {
private final State state;
private final SnapshotId snapshotId;
private final Snapshot snapshot;
private final boolean includeGlobalState;
private final boolean partial;
private final ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards;
@ -74,9 +74,10 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
private final ImmutableOpenMap<String, List<ShardId>> waitingIndices;
private final long startTime;
public Entry(SnapshotId snapshotId, boolean includeGlobalState, boolean partial, State state, List<String> indices, long startTime, ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
public Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, State state, List<String> indices, long startTime,
ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
this.state = state;
this.snapshotId = snapshotId;
this.snapshot = snapshot;
this.includeGlobalState = includeGlobalState;
this.partial = partial;
this.indices = indices;
@ -91,15 +92,15 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
}
public Entry(Entry entry, State state, ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
this(entry.snapshotId, entry.includeGlobalState, entry.partial, state, entry.indices, entry.startTime, shards);
this(entry.snapshot, entry.includeGlobalState, entry.partial, state, entry.indices, entry.startTime, shards);
}
public Entry(Entry entry, ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
this(entry, entry.state, shards);
}
public SnapshotId snapshotId() {
return this.snapshotId;
public Snapshot snapshot() {
return this.snapshot;
}
public ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards() {
@ -142,7 +143,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
if (startTime != entry.startTime) return false;
if (!indices.equals(entry.indices)) return false;
if (!shards.equals(entry.shards)) return false;
if (!snapshotId.equals(entry.snapshotId)) return false;
if (!snapshot.equals(entry.snapshot)) return false;
if (state != entry.state) return false;
if (!waitingIndices.equals(entry.waitingIndices)) return false;
@ -152,7 +153,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
@Override
public int hashCode() {
int result = state.hashCode();
result = 31 * result + snapshotId.hashCode();
result = 31 * result + snapshot.hashCode();
result = 31 * result + (includeGlobalState ? 1 : 0);
result = 31 * result + (partial ? 1 : 0);
result = 31 * result + shards.hashCode();
@ -162,6 +163,11 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
return result;
}
@Override
public String toString() {
return snapshot.toString();
}
private ImmutableOpenMap<String, List<ShardId>> findWaitingIndices(ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards) {
Map<String, List<ShardId>> waitingIndicesMap = new HashMap<>();
for (ObjectObjectCursor<ShardId, ShardSnapshotStatus> entry : shards) {
@ -277,7 +283,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
}
}
public static enum State {
public enum State {
INIT((byte) 0, false, false),
STARTED((byte) 1, false, false),
SUCCESS((byte) 2, true, false),
@ -347,9 +353,10 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
return this.entries;
}
public Entry snapshot(SnapshotId snapshotId) {
public Entry snapshot(final Snapshot snapshot) {
for (Entry entry : entries) {
if (snapshotId.equals(entry.snapshotId())) {
final Snapshot curr = entry.snapshot();
if (curr.equals(snapshot)) {
return entry;
}
}
@ -365,7 +372,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
public SnapshotsInProgress readFrom(StreamInput in) throws IOException {
Entry[] entries = new Entry[in.readVInt()];
for (int i = 0; i < entries.length; i++) {
SnapshotId snapshotId = SnapshotId.readSnapshotId(in);
Snapshot snapshot = new Snapshot(in);
boolean includeGlobalState = in.readBoolean();
boolean partial = in.readBoolean();
State state = State.fromValue(in.readByte());
@ -383,7 +390,13 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
State shardState = State.fromValue(in.readByte());
builder.put(shardId, new ShardSnapshotStatus(nodeId, shardState));
}
entries[i] = new Entry(snapshotId, includeGlobalState, partial, state, Collections.unmodifiableList(indexBuilder), startTime, builder.build());
entries[i] = new Entry(snapshot,
includeGlobalState,
partial,
state,
Collections.unmodifiableList(indexBuilder),
startTime,
builder.build());
}
return new SnapshotsInProgress(entries);
}
@ -392,7 +405,7 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(entries.size());
for (Entry entry : entries) {
entry.snapshotId().writeTo(out);
entry.snapshot().writeTo(out);
out.writeBoolean(entry.includeGlobalState());
out.writeBoolean(entry.partial());
out.writeByte(entry.state().value());
@ -410,25 +423,24 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
}
}
static final class Fields {
static final String REPOSITORY = "repository";
static final String SNAPSHOTS = "snapshots";
static final String SNAPSHOT = "snapshot";
static final String INCLUDE_GLOBAL_STATE = "include_global_state";
static final String PARTIAL = "partial";
static final String STATE = "state";
static final String INDICES = "indices";
static final String START_TIME_MILLIS = "start_time_millis";
static final String START_TIME = "start_time";
static final String SHARDS = "shards";
static final String INDEX = "index";
static final String SHARD = "shard";
static final String NODE = "node";
}
private static final String REPOSITORY = "repository";
private static final String SNAPSHOTS = "snapshots";
private static final String SNAPSHOT = "snapshot";
private static final String UUID = "uuid";
private static final String INCLUDE_GLOBAL_STATE = "include_global_state";
private static final String PARTIAL = "partial";
private static final String STATE = "state";
private static final String INDICES = "indices";
private static final String START_TIME_MILLIS = "start_time_millis";
private static final String START_TIME = "start_time";
private static final String SHARDS = "shards";
private static final String INDEX = "index";
private static final String SHARD = "shard";
private static final String NODE = "node";
@Override
public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException {
builder.startArray(Fields.SNAPSHOTS);
builder.startArray(SNAPSHOTS);
for (Entry entry : entries) {
toXContent(entry, builder, params);
}
@ -438,30 +450,31 @@ public class SnapshotsInProgress extends AbstractDiffable<Custom> implements Cus
public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException {
builder.startObject();
builder.field(Fields.REPOSITORY, entry.snapshotId().getRepository());
builder.field(Fields.SNAPSHOT, entry.snapshotId().getSnapshot());
builder.field(Fields.INCLUDE_GLOBAL_STATE, entry.includeGlobalState());
builder.field(Fields.PARTIAL, entry.partial());
builder.field(Fields.STATE, entry.state());
builder.startArray(Fields.INDICES);
builder.field(REPOSITORY, entry.snapshot().getRepository());
builder.field(SNAPSHOT, entry.snapshot().getSnapshotId().getName());
builder.field(UUID, entry.snapshot().getSnapshotId().getUUID());
builder.field(INCLUDE_GLOBAL_STATE, entry.includeGlobalState());
builder.field(PARTIAL, entry.partial());
builder.field(STATE, entry.state());
builder.startArray(INDICES);
{
for (String index : entry.indices()) {
builder.value(index);
}
}
builder.endArray();
builder.timeValueField(Fields.START_TIME_MILLIS, Fields.START_TIME, entry.startTime());
builder.startArray(Fields.SHARDS);
builder.timeValueField(START_TIME_MILLIS, START_TIME, entry.startTime());
builder.startArray(SHARDS);
{
for (ObjectObjectCursor<ShardId, ShardSnapshotStatus> shardEntry : entry.shards) {
ShardId shardId = shardEntry.key;
ShardSnapshotStatus status = shardEntry.value;
builder.startObject();
{
builder.field(Fields.INDEX, shardId.getIndex());
builder.field(Fields.SHARD, shardId.getId());
builder.field(Fields.STATE, status.state());
builder.field(Fields.NODE, status.nodeId());
builder.field(INDEX, shardId.getIndex());
builder.field(SHARD, shardId.getId());
builder.field(STATE, status.state());
builder.field(NODE, status.nodeId());
}
builder.endObject();
}

View File

@ -1,127 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import java.io.IOException;
/**
* Snapshot ID - repository name + snapshot name
*/
public class SnapshotId implements Streamable {
private String repository;
private String snapshot;
// Caching hash code
private int hashCode;
private SnapshotId() {
}
/**
* Constructs new snapshot id
*
* @param repository repository name
* @param snapshot snapshot name
*/
public SnapshotId(String repository, String snapshot) {
this.repository = repository;
this.snapshot = snapshot;
this.hashCode = computeHashCode();
}
/**
* Returns repository name
*
* @return repository name
*/
public String getRepository() {
return repository;
}
/**
* Returns snapshot name
*
* @return snapshot name
*/
public String getSnapshot() {
return snapshot;
}
@Override
public String toString() {
return repository + ":" + snapshot;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null) return false;
SnapshotId snapshotId = (SnapshotId) o;
return snapshot.equals(snapshotId.snapshot) && repository.equals(snapshotId.repository);
}
@Override
public int hashCode() {
return hashCode;
}
private int computeHashCode() {
int result = repository != null ? repository.hashCode() : 0;
result = 31 * result + snapshot.hashCode();
return result;
}
/**
* Reads snapshot id from stream input
*
* @param in stream input
* @return snapshot id
*/
public static SnapshotId readSnapshotId(StreamInput in) throws IOException {
SnapshotId snapshot = new SnapshotId();
snapshot.readFrom(in);
return snapshot;
}
/**
* {@inheritDoc}
*/
@Override
public void readFrom(StreamInput in) throws IOException {
repository = in.readString();
snapshot = in.readString();
hashCode = computeHashCode();
}
/**
* {@inheritDoc}
*/
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(repository);
out.writeString(snapshot);
}
}

View File

@ -376,14 +376,20 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
* Initializes a new empty index, to be restored from a snapshot
*/
public Builder initializeAsNewRestore(IndexMetaData indexMetaData, RestoreSource restoreSource, IntSet ignoreShards) {
return initializeAsRestore(indexMetaData, restoreSource, ignoreShards, true, new UnassignedInfo(UnassignedInfo.Reason.NEW_INDEX_RESTORED, "restore_source[" + restoreSource.snapshotId().getRepository() + "/" + restoreSource.snapshotId().getSnapshot() + "]"));
final UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NEW_INDEX_RESTORED,
"restore_source[" + restoreSource.snapshot().getRepository() + "/" +
restoreSource.snapshot().getSnapshotId().getName() + "]");
return initializeAsRestore(indexMetaData, restoreSource, ignoreShards, true, unassignedInfo);
}
/**
* Initializes an existing index, to be restored from a snapshot
*/
public Builder initializeAsRestore(IndexMetaData indexMetaData, RestoreSource restoreSource) {
return initializeAsRestore(indexMetaData, restoreSource, null, false, new UnassignedInfo(UnassignedInfo.Reason.EXISTING_INDEX_RESTORED, "restore_source[" + restoreSource.snapshotId().getRepository() + "/" + restoreSource.snapshotId().getSnapshot() + "]"));
final UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.EXISTING_INDEX_RESTORED,
"restore_source[" + restoreSource.snapshot().getRepository() + "/" +
restoreSource.snapshot().getSnapshotId().getName() + "]");
return initializeAsRestore(indexMetaData, restoreSource, null, false, unassignedInfo);
}
/**

View File

@ -20,7 +20,7 @@
package org.elasticsearch.cluster.routing;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
@ -28,13 +28,14 @@ import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Objects;
/**
* Represents snapshot and index from which a recovering index should be restored
*/
public class RestoreSource implements Streamable, ToXContent {
private SnapshotId snapshotId;
private Snapshot snapshot;
private String index;
@ -43,14 +44,14 @@ public class RestoreSource implements Streamable, ToXContent {
RestoreSource() {
}
public RestoreSource(SnapshotId snapshotId, Version version, String index) {
this.snapshotId = snapshotId;
this.version = version;
this.index = index;
public RestoreSource(Snapshot snapshot, Version version, String index) {
this.snapshot = Objects.requireNonNull(snapshot);
this.version = Objects.requireNonNull(version);
this.index = Objects.requireNonNull(index);
}
public SnapshotId snapshotId() {
return snapshotId;
public Snapshot snapshot() {
return snapshot;
}
public String index() {
@ -61,26 +62,20 @@ public class RestoreSource implements Streamable, ToXContent {
return version;
}
public static RestoreSource readRestoreSource(StreamInput in) throws IOException {
RestoreSource restoreSource = new RestoreSource();
restoreSource.readFrom(in);
return restoreSource;
}
public static RestoreSource readOptionalRestoreSource(StreamInput in) throws IOException {
return in.readOptionalStreamable(RestoreSource::new);
}
@Override
public void readFrom(StreamInput in) throws IOException {
snapshotId = SnapshotId.readSnapshotId(in);
snapshot = new Snapshot(in);
version = Version.readVersion(in);
index = in.readString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
snapshotId.writeTo(out);
snapshot.writeTo(out);
Version.writeVersion(version, out);
out.writeString(index);
}
@ -88,8 +83,8 @@ public class RestoreSource implements Streamable, ToXContent {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
return builder.startObject()
.field("repository", snapshotId.getRepository())
.field("snapshot", snapshotId.getSnapshot())
.field("repository", snapshot.getRepository())
.field("snapshot", snapshot.getSnapshotId().getName())
.field("version", version.toString())
.field("index", index)
.endObject();
@ -97,26 +92,24 @@ public class RestoreSource implements Streamable, ToXContent {
@Override
public String toString() {
return snapshotId.toString();
return snapshot.toString();
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
RestoreSource that = (RestoreSource) o;
if (!index.equals(that.index)) return false;
if (!snapshotId.equals(that.snapshotId)) return false;
return true;
@SuppressWarnings("unchecked") RestoreSource that = (RestoreSource) o;
return snapshot.equals(that.snapshot) && index.equals(that.index);
}
@Override
public int hashCode() {
int result = snapshotId.hashCode();
result = 31 * result + index.hashCode();
return result;
return Objects.hash(snapshot, index);
}
}

View File

@ -39,7 +39,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RestoreSource;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.common.Booleans;
@ -1442,7 +1441,7 @@ public class IndexShard extends AbstractIndexShardComponent {
markAsRecovering("from snapshot", recoveryState); // mark the shard as recovering on the cluster state thread
threadPool.generic().execute(() -> {
try {
final IndexShardRepository indexShardRepository = repositoriesService.indexShardRepository(restoreSource.snapshotId().getRepository());
final IndexShardRepository indexShardRepository = repositoriesService.indexShardRepository(restoreSource.snapshot().getRepository());
if (restoreFromRepository(indexShardRepository)) {
recoveryListener.onRecoveryDone(recoveryState);
}

View File

@ -395,7 +395,7 @@ final class StoreRecovery {
throw new IndexShardRestoreFailedException(shardId, "empty restore source");
}
if (logger.isTraceEnabled()) {
logger.trace("[{}] restoring shard [{}]", restoreSource.snapshotId(), shardId);
logger.trace("[{}] restoring shard [{}]", restoreSource.snapshot(), shardId);
}
try {
translogState.totalOperations(0);
@ -405,7 +405,7 @@ final class StoreRecovery {
if (!shardId.getIndexName().equals(restoreSource.index())) {
snapshotShardId = new ShardId(restoreSource.index(), IndexMetaData.INDEX_UUID_NA_VALUE, shardId.id());
}
indexShardRepository.restore(restoreSource.snapshotId(), restoreSource.version(), shardId, snapshotShardId, indexShard.recoveryState());
indexShardRepository.restore(restoreSource.snapshot().getSnapshotId(), restoreSource.version(), shardId, snapshotShardId, indexShard.recoveryState());
indexShard.skipTranslogRecovery();
indexShard.finalizeRecovery();
indexShard.postRecovery("restore done");

View File

@ -21,7 +21,7 @@ package org.elasticsearch.index.snapshots;
import org.apache.lucene.index.IndexCommit;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.recovery.RecoveryState;

View File

@ -32,7 +32,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.ParseFieldMatcher;
@ -204,7 +204,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
try {
snapshotContext.restore();
} catch (Throwable e) {
throw new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId.getSnapshot() + "]", e);
throw new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId + "]", e);
}
}
@ -318,7 +318,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
int fileListGeneration = tuple.v2();
try {
indexShardSnapshotFormat(version).delete(blobContainer, snapshotId.getSnapshot());
indexShardSnapshotFormat(version).delete(blobContainer, snapshotId.getName());
} catch (IOException e) {
logger.debug("[{}] [{}] failed to delete shard snapshot file", shardId, snapshotId);
}
@ -326,7 +326,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
// Build a list of snapshots that should be preserved
List<SnapshotFiles> newSnapshotsList = new ArrayList<>();
for (SnapshotFiles point : snapshots) {
if (!point.snapshot().equals(snapshotId.getSnapshot())) {
if (!point.snapshot().equals(snapshotId.getName())) {
newSnapshotsList.add(point);
}
}
@ -339,7 +339,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
*/
public BlobStoreIndexShardSnapshot loadSnapshot() {
try {
return indexShardSnapshotFormat(version).read(blobContainer, snapshotId.getSnapshot());
return indexShardSnapshotFormat(version).read(blobContainer, snapshotId.getName());
} catch (IOException ex) {
throw new IndexShardRestoreFailedException(shardId, "failed to read shard snapshot file", ex);
}
@ -605,14 +605,14 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
// now create and write the commit point
snapshotStatus.updateStage(IndexShardSnapshotStatus.Stage.FINALIZE);
BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getSnapshot(),
BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(),
snapshotIndexCommit.getGeneration(), indexCommitPointFiles, snapshotStatus.startTime(),
// snapshotStatus.startTime() is assigned on the same machine, so it's safe to use with VLong
System.currentTimeMillis() - snapshotStatus.startTime(), indexNumberOfFiles, indexTotalFilesSize);
//TODO: The time stored in snapshot doesn't include cleanup time.
logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId);
try {
indexShardSnapshotFormat.write(snapshot, blobContainer, snapshotId.getSnapshot());
indexShardSnapshotFormat.write(snapshot, blobContainer, snapshotId.getName());
} catch (IOException e) {
throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e);
}

View File

@ -645,7 +645,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
@Override
public void onRecoveryDone(RecoveryState state) {
if (state.getType() == RecoveryState.Type.SNAPSHOT) {
restoreService.indexShardRestoreCompleted(state.getRestoreSource().snapshotId(), shardRouting.shardId());
restoreService.indexShardRestoreCompleted(state.getRestoreSource().snapshot(), shardRouting.shardId());
}
shardStateAction.shardStarted(shardRouting, message(state), SHARD_STATE_ACTION_LISTENER);
}
@ -666,7 +666,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
if (state.getType() == RecoveryState.Type.SNAPSHOT) {
try {
if (Lucene.isCorruptionException(e.getCause())) {
restoreService.failRestore(state.getRestoreSource().snapshotId(), shardRouting.shardId());
restoreService.failRestore(state.getRestoreSource().snapshot(), shardRouting.shardId());
}
} catch (Throwable inner) {
e.addSuppressed(inner);

View File

@ -20,7 +20,7 @@ package org.elasticsearch.repositories;
import org.apache.lucene.index.IndexCommit;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
@ -29,6 +29,7 @@ import org.elasticsearch.snapshots.SnapshotShardFailure;
import java.io.IOException;
import java.util.List;
import java.util.function.Predicate;
/**
* Snapshot repository interface.
@ -39,7 +40,7 @@ import java.util.List;
* <p>
* Typical snapshot usage pattern:
* <ul>
* <li>Master calls {@link #initializeSnapshot(org.elasticsearch.cluster.metadata.SnapshotId, List, org.elasticsearch.cluster.metadata.MetaData)}
* <li>Master calls {@link #initializeSnapshot(SnapshotId, List, org.elasticsearch.cluster.metadata.MetaData)}
* with list of indices that will be included into the snapshot</li>
* <li>Data nodes call {@link org.elasticsearch.index.snapshots.IndexShardRepository#snapshot(SnapshotId, ShardId, IndexCommit, IndexShardSnapshotStatus)} for each shard</li>
* <li>When all shard calls return master calls {@link #finalizeSnapshot}
@ -51,7 +52,7 @@ public interface Repository extends LifecycleComponent<Repository> {
/**
* Reads snapshot description from repository.
*
* @param snapshotId snapshot ID
* @param snapshotId snapshot id
* @return information about snapshot
*/
SnapshotInfo readSnapshot(SnapshotId snapshotId);
@ -65,10 +66,11 @@ public interface Repository extends LifecycleComponent<Repository> {
* @param indices list of indices
* @return information about snapshot
*/
MetaData readSnapshotMetaData(SnapshotId snapshotId, SnapshotInfo snapshot, List<String> indices) throws IOException;
MetaData readSnapshotMetaData(SnapshotInfo snapshot, List<String> indices) throws IOException;
/**
* Returns the list of snapshots currently stored in the repository
* Returns the list of snapshots currently stored in the repository that match the given predicate on the snapshot name.
* To get all snapshots, the predicate filter should return true regardless of the input.
*
* @return snapshot list
*/

View File

@ -24,7 +24,7 @@ import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.UUIDs;
@ -34,6 +34,7 @@ import org.elasticsearch.common.blobstore.BlobPath;
import org.elasticsearch.common.blobstore.BlobStore;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.compress.NotXContentException;
import org.elasticsearch.common.io.Streams;
@ -56,7 +57,6 @@ import org.elasticsearch.repositories.Repository;
import org.elasticsearch.repositories.RepositoryException;
import org.elasticsearch.repositories.RepositorySettings;
import org.elasticsearch.repositories.RepositoryVerificationException;
import org.elasticsearch.snapshots.InvalidSnapshotNameException;
import org.elasticsearch.snapshots.SnapshotCreationException;
import org.elasticsearch.snapshots.SnapshotException;
import org.elasticsearch.snapshots.SnapshotInfo;
@ -71,6 +71,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* BlobStore - based implementation of Snapshot Repository
@ -130,7 +131,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
private static final String SNAPSHOT_CODEC = "snapshot";
private static final String SNAPSHOTS_FILE = "index";
static final String SNAPSHOTS_FILE = "index"; // package private for unit testing
private static final String TESTS_FILE = "tests-";
@ -268,20 +269,25 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
throw new RepositoryException(this.repositoryName, "cannot create snapshot in a readonly repository");
}
try {
if (snapshotFormat.exists(snapshotsBlobContainer, snapshotId.getSnapshot()) ||
snapshotLegacyFormat.exists(snapshotsBlobContainer, snapshotId.getSnapshot())) {
throw new InvalidSnapshotNameException(snapshotId, "snapshot with such name already exists");
final String snapshotName = snapshotId.getName();
// check if the snapshot name already exists in the repository
if (snapshots().stream().anyMatch(s -> s.getName().equals(snapshotName))) {
throw new SnapshotCreationException(repositoryName, snapshotId, "snapshot with the same name already exists");
}
if (snapshotFormat.exists(snapshotsBlobContainer, blobId(snapshotId)) ||
snapshotLegacyFormat.exists(snapshotsBlobContainer, snapshotName)) {
throw new SnapshotCreationException(repositoryName, snapshotId, "snapshot with such name already exists");
}
// Write Global MetaData
globalMetaDataFormat.write(metaData, snapshotsBlobContainer, snapshotId.getSnapshot());
globalMetaDataFormat.write(metaData, snapshotsBlobContainer, snapshotName);
for (String index : indices) {
final IndexMetaData indexMetaData = metaData.index(index);
final BlobPath indexPath = basePath().add("indices").add(index);
final BlobContainer indexMetaDataBlobContainer = blobStore().blobContainer(indexPath);
indexMetaDataFormat.write(indexMetaData, indexMetaDataBlobContainer, snapshotId.getSnapshot());
indexMetaDataFormat.write(indexMetaData, indexMetaDataBlobContainer, snapshotName);
}
} catch (IOException ex) {
throw new SnapshotCreationException(snapshotId, ex);
throw new SnapshotCreationException(repositoryName, snapshotId, ex);
}
}
@ -314,35 +320,27 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
logger.warn("cannot read metadata for snapshot [{}]", ex, snapshotId);
}
try {
final String snapshotName = snapshotId.getName();
// Delete snapshot file first so we wouldn't end up with partially deleted snapshot that looks OK
if (snapshot != null) {
snapshotFormat(snapshot.version()).delete(snapshotsBlobContainer, snapshotId.getSnapshot());
globalMetaDataFormat(snapshot.version()).delete(snapshotsBlobContainer, snapshotId.getSnapshot());
snapshotFormat(snapshot.version()).delete(snapshotsBlobContainer, blobId(snapshotId));
globalMetaDataFormat(snapshot.version()).delete(snapshotsBlobContainer, snapshotName);
} else {
// We don't know which version was the snapshot created with - try deleting both current and legacy formats
snapshotFormat.delete(snapshotsBlobContainer, snapshotId.getSnapshot());
snapshotLegacyFormat.delete(snapshotsBlobContainer, snapshotId.getSnapshot());
globalMetaDataLegacyFormat.delete(snapshotsBlobContainer, snapshotId.getSnapshot());
globalMetaDataFormat.delete(snapshotsBlobContainer, snapshotId.getSnapshot());
snapshotFormat.delete(snapshotsBlobContainer, blobId(snapshotId));
snapshotLegacyFormat.delete(snapshotsBlobContainer, snapshotName);
globalMetaDataLegacyFormat.delete(snapshotsBlobContainer, snapshotName);
globalMetaDataFormat.delete(snapshotsBlobContainer, snapshotName);
}
// Delete snapshot from the snapshot list
List<SnapshotId> snapshotIds = snapshots();
if (snapshotIds.contains(snapshotId)) {
List<SnapshotId> builder = new ArrayList<>();
for (SnapshotId id : snapshotIds) {
if (!snapshotId.equals(id)) {
builder.add(id);
}
}
snapshotIds = Collections.unmodifiableList(builder);
}
List<SnapshotId> snapshotIds = snapshots().stream().filter(id -> snapshotId.equals(id) == false).collect(Collectors.toList());
writeSnapshotList(snapshotIds);
// Now delete all indices
for (String index : indices) {
BlobPath indexPath = basePath().add("indices").add(index);
BlobContainer indexMetaDataBlobContainer = blobStore().blobContainer(indexPath);
try {
indexMetaDataFormat(snapshot.version()).delete(indexMetaDataBlobContainer, snapshotId.getSnapshot());
indexMetaDataFormat(snapshot.version()).delete(indexMetaDataBlobContainer, snapshotId.getName());
} catch (IOException ex) {
logger.warn("[{}] failed to delete metadata for index [{}]", ex, snapshotId, index);
}
@ -368,10 +366,21 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
* {@inheritDoc}
*/
@Override
public SnapshotInfo finalizeSnapshot(SnapshotId snapshotId, List<String> indices, long startTime, String failure, int totalShards, List<SnapshotShardFailure> shardFailures) {
public SnapshotInfo finalizeSnapshot(final SnapshotId snapshotId,
final List<String> indices,
final long startTime,
final String failure,
final int totalShards,
final List<SnapshotShardFailure> shardFailures) {
try {
SnapshotInfo blobStoreSnapshot = new SnapshotInfo(snapshotId.getSnapshot(), indices, startTime, failure, System.currentTimeMillis(), totalShards, shardFailures);
snapshotFormat.write(blobStoreSnapshot, snapshotsBlobContainer, snapshotId.getSnapshot());
SnapshotInfo blobStoreSnapshot = new SnapshotInfo(snapshotId,
indices,
startTime,
failure,
System.currentTimeMillis(),
totalShards,
shardFailures);
snapshotFormat.write(blobStoreSnapshot, snapshotsBlobContainer, blobId(snapshotId));
List<SnapshotId> snapshotIds = snapshots();
if (!snapshotIds.contains(snapshotId)) {
snapshotIds = new ArrayList<>(snapshotIds);
@ -405,15 +414,22 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
for (BlobMetaData md : blobs.values()) {
String blobName = md.name();
final String name;
String uuid;
if (blobName.startsWith(SNAPSHOT_PREFIX) && blobName.length() > legacyPrefixLength) {
name = blobName.substring(prefixLength, blobName.length() - suffixLength);
final String str = blobName.substring(prefixLength, blobName.length() - suffixLength);
// TODO: this will go away once we make the snapshot file writes atomic and
// use it as the source of truth for the snapshots list instead of listing blobs
Tuple<String, String> pair = parseNameUUIDFromBlobName(str);
name = pair.v1();
uuid = pair.v2();
} else if (blobName.startsWith(LEGACY_SNAPSHOT_PREFIX) && blobName.length() > suffixLength + prefixLength) {
name = blobName.substring(legacyPrefixLength);
uuid = SnapshotId.UNASSIGNED_UUID;
} else {
// not sure what it was - ignore
continue;
}
snapshots.add(new SnapshotId(repositoryName, name));
snapshots.add(new SnapshotId(name, uuid));
}
return Collections.unmodifiableList(snapshots);
} catch (IOException ex) {
@ -425,28 +441,25 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
* {@inheritDoc}
*/
@Override
public MetaData readSnapshotMetaData(SnapshotId snapshotId, SnapshotInfo snapshot, List<String> indices) throws IOException {
return readSnapshotMetaData(snapshotId, snapshot.version(), indices, false);
public MetaData readSnapshotMetaData(SnapshotInfo snapshot, List<String> indices) throws IOException {
return readSnapshotMetaData(snapshot.snapshotId(), snapshot.version(), indices, false);
}
/**
* {@inheritDoc}
*/
@Override
public SnapshotInfo readSnapshot(SnapshotId snapshotId) {
public SnapshotInfo readSnapshot(final SnapshotId snapshotId) {
try {
return snapshotFormat.read(snapshotsBlobContainer, snapshotId.getSnapshot());
return snapshotFormat.read(snapshotsBlobContainer, blobId(snapshotId));
} catch (FileNotFoundException | NoSuchFileException ex) {
// File is missing - let's try legacy format instead
try {
return snapshotLegacyFormat.read(snapshotsBlobContainer, snapshotId.getSnapshot());
return snapshotLegacyFormat.read(snapshotsBlobContainer, snapshotId.getName());
} catch (FileNotFoundException | NoSuchFileException ex1) {
throw new SnapshotMissingException(snapshotId, ex);
throw new SnapshotMissingException(repositoryName, snapshotId, ex);
} catch (IOException | NotXContentException ex1) {
throw new SnapshotException(snapshotId, "failed to get snapshots", ex1);
throw new SnapshotException(repositoryName, snapshotId, "failed to get snapshots", ex1);
}
} catch (IOException | NotXContentException ex) {
throw new SnapshotException(snapshotId, "failed to get snapshots", ex);
throw new SnapshotException(repositoryName, snapshotId, "failed to get snapshots", ex);
}
}
@ -456,27 +469,27 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
// When we delete corrupted snapshots we might not know which version we are dealing with
// We can try detecting the version based on the metadata file format
assert ignoreIndexErrors;
if (globalMetaDataFormat.exists(snapshotsBlobContainer, snapshotId.getSnapshot())) {
if (globalMetaDataFormat.exists(snapshotsBlobContainer, snapshotId.getName())) {
snapshotVersion = Version.CURRENT;
} else if (globalMetaDataLegacyFormat.exists(snapshotsBlobContainer, snapshotId.getSnapshot())) {
throw new SnapshotException(snapshotId, "snapshot is too old");
} else if (globalMetaDataLegacyFormat.exists(snapshotsBlobContainer, snapshotId.getName())) {
throw new SnapshotException(repositoryName, snapshotId, "snapshot is too old");
} else {
throw new SnapshotMissingException(snapshotId);
throw new SnapshotMissingException(repositoryName, snapshotId);
}
}
try {
metaData = globalMetaDataFormat(snapshotVersion).read(snapshotsBlobContainer, snapshotId.getSnapshot());
metaData = globalMetaDataFormat(snapshotVersion).read(snapshotsBlobContainer, snapshotId.getName());
} catch (FileNotFoundException | NoSuchFileException ex) {
throw new SnapshotMissingException(snapshotId, ex);
throw new SnapshotMissingException(repositoryName, snapshotId, ex);
} catch (IOException ex) {
throw new SnapshotException(snapshotId, "failed to get snapshots", ex);
throw new SnapshotException(repositoryName, snapshotId, "failed to get snapshots", ex);
}
MetaData.Builder metaDataBuilder = MetaData.builder(metaData);
for (String index : indices) {
BlobPath indexPath = basePath().add("indices").add(index);
BlobContainer indexMetaDataBlobContainer = blobStore().blobContainer(indexPath);
try {
metaDataBuilder.put(indexMetaDataFormat(snapshotVersion).read(indexMetaDataBlobContainer, snapshotId.getSnapshot()), false);
metaDataBuilder.put(indexMetaDataFormat(snapshotVersion).read(indexMetaDataBlobContainer, snapshotId.getName()), false);
} catch (ElasticsearchParseException | IOException ex) {
if (ignoreIndexErrors) {
logger.warn("[{}] [{}] failed to read metadata for index", ex, snapshotId, index);
@ -547,6 +560,10 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
}
}
private static final String SNAPSHOTS = "snapshots";
private static final String NAME = "name";
private static final String UUID = "uuid";
/**
* Writes snapshot index file
* <p>
@ -561,9 +578,12 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
try(StreamOutput stream = new OutputStreamStreamOutput(bStream)) {
XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream);
builder.startObject();
builder.startArray("snapshots");
builder.startArray(SNAPSHOTS);
for (SnapshotId snapshot : snapshots) {
builder.value(snapshot.getSnapshot());
builder.startObject();
builder.field(NAME, snapshot.getName());
builder.field(UUID, snapshot.getUUID());
builder.endObject();
}
builder.endArray();
builder.endObject();
@ -594,10 +614,29 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
if (parser.nextToken() == XContentParser.Token.START_OBJECT) {
if (parser.nextToken() == XContentParser.Token.FIELD_NAME) {
String currentFieldName = parser.currentName();
if ("snapshots".equals(currentFieldName)) {
if (SNAPSHOTS.equals(currentFieldName)) {
if (parser.nextToken() == XContentParser.Token.START_ARRAY) {
while (parser.nextToken() != XContentParser.Token.END_ARRAY) {
snapshots.add(new SnapshotId(repositoryName, parser.text()));
// the new format from 5.0 which contains the snapshot name and uuid
String name = null;
String uuid = null;
if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
while (parser.nextToken() != XContentParser.Token.END_OBJECT) {
currentFieldName = parser.currentName();
parser.nextToken();
if (NAME.equals(currentFieldName)) {
name = parser.text();
} else if (UUID.equals(currentFieldName)) {
uuid = parser.text();
}
}
snapshots.add(new SnapshotId(name, uuid));
}
// the old format pre 5.0 that only contains the snapshot name, use the name as the uuid too
else {
name = parser.text();
snapshots.add(new SnapshotId(name, SnapshotId.UNASSIGNED_UUID));
}
}
}
}
@ -669,4 +708,42 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
public boolean readOnly() {
return readOnly;
}
// package private, only use for testing
BlobContainer blobContainer() {
return snapshotsBlobContainer;
}
// TODO: this will go away once readSnapshotsList uses the index file instead of listing blobs
// to know which snapshots are part of a repository. See #18156
// Package private for testing.
static Tuple<String, String> parseNameUUIDFromBlobName(final String str) {
final String name;
final String uuid;
final int sizeOfUUID = 22; // uuid is 22 chars in length
// unreliable, but highly unlikely to have a snapshot name with a dash followed by 22 characters,
// and this will go away before a release (see #18156).
//norelease
if (str.length() > sizeOfUUID + 1 && str.charAt(str.length() - sizeOfUUID - 1) == '-') {
// new naming convention, snapshot blob id has name and uuid
final int idx = str.length() - sizeOfUUID - 1;
name = str.substring(0, idx);
uuid = str.substring(idx + 1);
} else {
// old naming convention, before snapshots had UUIDs
name = str;
uuid = SnapshotId.UNASSIGNED_UUID;
}
return Tuple.tuple(name, uuid);
}
// Package private for testing
static String blobId(final SnapshotId snapshotId) {
final String uuid = snapshotId.getUUID();
if (uuid.equals(SnapshotId.UNASSIGNED_UUID)) {
// the old snapshot blob naming
return snapshotId.getName();
}
return snapshotId.getName() + "-" + uuid;
}
}

View File

@ -19,7 +19,7 @@
package org.elasticsearch.repositories.uri;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.common.blobstore.BlobPath;
import org.elasticsearch.common.blobstore.BlobStore;
import org.elasticsearch.common.blobstore.url.URLBlobStore;
@ -42,6 +42,7 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.function.Function;
import java.util.function.Predicate;
/**
* Read-only URL-based implementation of the BlobStoreRepository

View File

@ -154,8 +154,8 @@ public class RestRecoveryAction extends AbstractCatAction {
t.addCell(state.getSourceNode() == null ? "n/a" : state.getSourceNode().getName());
t.addCell(state.getTargetNode().getHostName());
t.addCell(state.getTargetNode().getName());
t.addCell(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshotId().getRepository());
t.addCell(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshotId().getSnapshot());
t.addCell(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshot().getRepository());
t.addCell(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshot().getSnapshotId().getName());
t.addCell(state.getIndex().totalRecoverFiles());
t.addCell(state.getIndex().recoveredFileCount());
t.addCell(String.format(Locale.ROOT, "%1.1f%%", state.getIndex().recoveredFilesPercent()));

View File

@ -79,7 +79,7 @@ public class RestSnapshotAction extends AbstractCatAction {
protected Table getTableWithHeader(RestRequest request) {
return new Table()
.startHeaders()
.addCell("id", "alias:id,snapshotId;desc:unique snapshot id")
.addCell("id", "alias:id,snapshot;desc:unique snapshot")
.addCell("status", "alias:s,status;text-align:right;desc:snapshot name")
.addCell("start_epoch", "alias:ste,startEpoch;desc:start time in seconds since 1970-01-01 00:00:00")
.addCell("start_time", "alias:sti,startTime;desc:start time in HH:MM:SS")
@ -101,7 +101,7 @@ public class RestSnapshotAction extends AbstractCatAction {
for (SnapshotInfo snapshotStatus : getSnapshotsResponse.getSnapshots()) {
table.startRow();
table.addCell(snapshotStatus.name());
table.addCell(snapshotStatus.snapshotId().getName());
table.addCell(snapshotStatus.state());
table.addCell(TimeUnit.SECONDS.convert(snapshotStatus.startTime(), TimeUnit.MILLISECONDS));
table.addCell(dateFormat.print(snapshotStatus.startTime()));

View File

@ -19,7 +19,6 @@
package org.elasticsearch.snapshots;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.rest.RestStatus;
@ -29,7 +28,12 @@ import java.io.IOException;
* Thrown when a user tries to start multiple snapshots at the same time
*/
public class ConcurrentSnapshotExecutionException extends SnapshotException {
public ConcurrentSnapshotExecutionException(SnapshotId snapshot, String msg) {
public ConcurrentSnapshotExecutionException(final String repositoryName, final String snapshotName, final String msg) {
super(repositoryName, snapshotName, msg);
}
public ConcurrentSnapshotExecutionException(final Snapshot snapshot, final String msg) {
super(snapshot, msg);
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.snapshots;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.rest.RestStatus;
@ -30,8 +29,8 @@ import java.io.IOException;
*/
public class InvalidSnapshotNameException extends SnapshotException {
public InvalidSnapshotNameException(SnapshotId snapshot, String desc) {
super(snapshot, "Invalid snapshot name [" + snapshot.getSnapshot() + "], " + desc);
public InvalidSnapshotNameException(final String repositoryName, final String snapshotName, String desc) {
super(repositoryName, snapshotName, "Invalid snapshot name [" + snapshotName + "], " + desc);
}
public InvalidSnapshotNameException(StreamInput in) throws IOException {

View File

@ -39,7 +39,6 @@ import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService;
import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
import org.elasticsearch.cluster.metadata.RepositoriesMetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
@ -85,6 +84,8 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CopyOnWriteArrayList;
@ -116,7 +117,7 @@ import static org.elasticsearch.common.util.set.Sets.newHashSet;
* method, which detects that shard should be restored from snapshot rather than recovered from gateway by looking
* at the {@link org.elasticsearch.cluster.routing.ShardRouting#restoreSource()} property.
* <p>
* At the end of the successful restore process {@code IndexShardSnapshotAndRestoreService} calls {@link #indexShardRestoreCompleted(SnapshotId, ShardId)},
* At the end of the successful restore process {@code IndexShardSnapshotAndRestoreService} calls {@link #indexShardRestoreCompleted(Snapshot, ShardId)},
* which updates {@link RestoreInProgress} in cluster state or removes it when all shards are completed. In case of
* restore failure a normal recovery fail-over process kicks in.
*/
@ -153,8 +154,6 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
private final MetaDataCreateIndexService createIndexService;
private final ClusterSettings dynamicSettings;
private final MetaDataIndexUpgradeService metaDataIndexUpgradeService;
private final CopyOnWriteArrayList<ActionListener<RestoreCompletionResponse>> listeners = new CopyOnWriteArrayList<>();
@ -164,7 +163,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
@Inject
public RestoreService(Settings settings, ClusterService clusterService, RepositoriesService repositoriesService, TransportService transportService,
AllocationService allocationService, MetaDataCreateIndexService createIndexService, ClusterSettings dynamicSettings,
AllocationService allocationService, MetaDataCreateIndexService createIndexService,
MetaDataIndexUpgradeService metaDataIndexUpgradeService, ClusterSettings clusterSettings) {
super(settings);
this.clusterService = clusterService;
@ -172,7 +171,6 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
this.transportService = transportService;
this.allocationService = allocationService;
this.createIndexService = createIndexService;
this.dynamicSettings = dynamicSettings;
this.metaDataIndexUpgradeService = metaDataIndexUpgradeService;
transportService.registerRequestHandler(UPDATE_RESTORE_ACTION_NAME, UpdateIndexShardRestoreStatusRequest::new, ThreadPool.Names.SAME, new UpdateRestoreStateRequestHandler());
clusterService.add(this);
@ -188,14 +186,20 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
public void restoreSnapshot(final RestoreRequest request, final ActionListener<RestoreInfo> listener) {
try {
// Read snapshot info and metadata from the repository
Repository repository = repositoriesService.repository(request.repository());
final SnapshotId snapshotId = new SnapshotId(request.repository(), request.name());
final SnapshotInfo snapshot = repository.readSnapshot(snapshotId);
List<String> filteredIndices = SnapshotUtils.filterIndices(snapshot.indices(), request.indices(), request.indicesOptions());
MetaData metaDataIn = repository.readSnapshotMetaData(snapshotId, snapshot, filteredIndices);
Repository repository = repositoriesService.repository(request.repositoryName);
final Optional<SnapshotId> matchingSnapshotId = repository.snapshots().stream()
.filter(s -> request.snapshotName.equals(s.getName())).findFirst();
if (matchingSnapshotId.isPresent() == false) {
throw new SnapshotRestoreException(request.repositoryName, request.snapshotName, "snapshot does not exist");
}
final SnapshotId snapshotId = matchingSnapshotId.get();
final SnapshotInfo snapshotInfo = repository.readSnapshot(snapshotId);
final Snapshot snapshot = new Snapshot(request.repositoryName, snapshotId);
List<String> filteredIndices = SnapshotUtils.filterIndices(snapshotInfo.indices(), request.indices(), request.indicesOptions());
MetaData metaDataIn = repository.readSnapshotMetaData(snapshotInfo, filteredIndices);
final MetaData metaData;
if (snapshot.version().before(Version.V_2_0_0_beta1)) {
if (snapshotInfo.version().before(Version.V_2_0_0_beta1)) {
// ES 2.0 now requires units for all time and byte-sized settings, so we add the default unit if it's missing in this snapshot:
metaData = MetaData.addDefaultUnitsIfNeeded(logger, metaDataIn);
} else {
@ -204,7 +208,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
}
// Make sure that we can restore from this snapshot
validateSnapshotRestorable(snapshotId, snapshot);
validateSnapshotRestorable(request.repositoryName, snapshotInfo);
// Find list of indices that we need to restore
final Map<String, String> renamedIndices = renamedIndices(request, filteredIndices);
@ -220,7 +224,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
// same time
RestoreInProgress restoreInProgress = currentState.custom(RestoreInProgress.TYPE);
if (restoreInProgress != null && !restoreInProgress.entries().isEmpty()) {
throw new ConcurrentSnapshotExecutionException(snapshotId, "Restore process is already running in this cluster");
throw new ConcurrentSnapshotExecutionException(snapshot, "Restore process is already running in this cluster");
}
// Updating cluster state
@ -236,14 +240,14 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
for (Map.Entry<String, String> indexEntry : renamedIndices.entrySet()) {
String index = indexEntry.getValue();
boolean partial = checkPartial(index);
RestoreSource restoreSource = new RestoreSource(snapshotId, snapshot.version(), index);
RestoreSource restoreSource = new RestoreSource(snapshot, snapshotInfo.version(), index);
String renamedIndexName = indexEntry.getKey();
IndexMetaData snapshotIndexMetaData = metaData.index(index);
snapshotIndexMetaData = updateIndexSettings(snapshotIndexMetaData, request.indexSettings, request.ignoreIndexSettings);
try {
snapshotIndexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(snapshotIndexMetaData);
} catch (Exception ex) {
throw new SnapshotRestoreException(snapshotId, "cannot restore index [" + index + "] because it cannot be upgraded", ex);
throw new SnapshotRestoreException(snapshot, "cannot restore index [" + index + "] because it cannot be upgraded", ex);
}
// Check that the index is closed or doesn't exist
IndexMetaData currentIndexMetaData = currentState.metaData().index(renamedIndexName);
@ -309,7 +313,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
}
shards = shardsBuilder.build();
RestoreInProgress.Entry restoreEntry = new RestoreInProgress.Entry(snapshotId, RestoreInProgress.State.INIT, Collections.unmodifiableList(new ArrayList<>(renamedIndices.keySet())), shards);
RestoreInProgress.Entry restoreEntry = new RestoreInProgress.Entry(snapshot, RestoreInProgress.State.INIT, Collections.unmodifiableList(new ArrayList<>(renamedIndices.keySet())), shards);
builder.putCustom(RestoreInProgress.TYPE, new RestoreInProgress(restoreEntry));
} else {
shards = ImmutableOpenMap.of();
@ -322,28 +326,30 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
if (completed(shards)) {
// We don't have any indices to restore - we are done
restoreInfo = new RestoreInfo(request.name(), Collections.unmodifiableList(new ArrayList<>(renamedIndices.keySet())),
shards.size(), shards.size() - failedShards(shards));
restoreInfo = new RestoreInfo(snapshotId.getName(),
Collections.unmodifiableList(new ArrayList<>(renamedIndices.keySet())),
shards.size(),
shards.size() - failedShards(shards));
}
RoutingTable rt = rtBuilder.build();
ClusterState updatedState = builder.metaData(mdBuilder).blocks(blocks).routingTable(rt).build();
RoutingAllocation.Result routingResult = allocationService.reroute(
ClusterState.builder(updatedState).routingTable(rt).build(),
"restored snapshot [" + snapshotId + "]");
"restored snapshot [" + snapshot + "]");
return ClusterState.builder(updatedState).routingResult(routingResult).build();
}
private void checkAliasNameConflicts(Map<String, String> renamedIndices, Set<String> aliases) {
for (Map.Entry<String, String> renamedIndex : renamedIndices.entrySet()) {
if (aliases.contains(renamedIndex.getKey())) {
throw new SnapshotRestoreException(snapshotId, "cannot rename index [" + renamedIndex.getValue() + "] into [" + renamedIndex.getKey() + "] because of conflict with an alias with the same name");
throw new SnapshotRestoreException(snapshot, "cannot rename index [" + renamedIndex.getValue() + "] into [" + renamedIndex.getKey() + "] because of conflict with an alias with the same name");
}
}
}
private void populateIgnoredShards(String index, IntSet ignoreShards) {
for (SnapshotShardFailure failure : snapshot.shardFailures()) {
for (SnapshotShardFailure failure : snapshotInfo.shardFailures()) {
if (index.equals(failure.index())) {
ignoreShards.add(failure.shardId());
}
@ -352,11 +358,11 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
private boolean checkPartial(String index) {
// Make sure that index was fully snapshotted
if (failed(snapshot, index)) {
if (failed(snapshotInfo, index)) {
if (request.partial()) {
return true;
} else {
throw new SnapshotRestoreException(snapshotId, "index [" + index + "] wasn't fully snapshotted - cannot restore");
throw new SnapshotRestoreException(snapshot, "index [" + index + "] wasn't fully snapshotted - cannot restore");
}
} else {
return false;
@ -367,15 +373,15 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
// Index exist - checking that it's closed
if (currentIndexMetaData.getState() != IndexMetaData.State.CLOSE) {
// TODO: Enable restore for open indices
throw new SnapshotRestoreException(snapshotId, "cannot restore index [" + renamedIndex + "] because it's open");
throw new SnapshotRestoreException(snapshot, "cannot restore index [" + renamedIndex + "] because it's open");
}
// Index exist - checking if it's partial restore
if (partial) {
throw new SnapshotRestoreException(snapshotId, "cannot restore partial index [" + renamedIndex + "] because such index already exists");
throw new SnapshotRestoreException(snapshot, "cannot restore partial index [" + renamedIndex + "] because such index already exists");
}
// Make sure that the number of shards is the same. That's the only thing that we cannot change
if (currentIndexMetaData.getNumberOfShards() != snapshotIndexMetaData.getNumberOfShards()) {
throw new SnapshotRestoreException(snapshotId, "cannot restore index [" + renamedIndex + "] with [" + currentIndexMetaData.getNumberOfShards() +
throw new SnapshotRestoreException(snapshot, "cannot restore index [" + renamedIndex + "] with [" + currentIndexMetaData.getNumberOfShards() +
"] shard from snapshot with [" + snapshotIndexMetaData.getNumberOfShards() + "] shards");
}
}
@ -395,7 +401,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
for (String ignoredSetting : ignoreSettings) {
if (!Regex.isSimpleMatchPattern(ignoredSetting)) {
if (UNREMOVABLE_SETTINGS.contains(ignoredSetting)) {
throw new SnapshotRestoreException(snapshotId, "cannot remove setting [" + ignoredSetting + "] on restore");
throw new SnapshotRestoreException(snapshot, "cannot remove setting [" + ignoredSetting + "] on restore");
} else {
settingsMap.remove(ignoredSetting);
}
@ -417,7 +423,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
}
for(Map.Entry<String, String> entry : normalizedChangeSettings.getAsMap().entrySet()) {
if (UNMODIFIABLE_SETTINGS.contains(entry.getKey())) {
throw new SnapshotRestoreException(snapshotId, "cannot modify setting [" + entry.getKey() + "] on restore");
throw new SnapshotRestoreException(snapshot, "cannot modify setting [" + entry.getKey() + "] on restore");
} else {
settingsMap.put(entry.getKey(), entry.getValue());
}
@ -471,7 +477,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
} catch (Throwable e) {
logger.warn("[{}][{}] failed to restore snapshot", e, request.repository(), request.name());
logger.warn("[{}] failed to restore snapshot", e, request.repositoryName + ":" + request.snapshotName);
listener.onFailure(e);
}
}
@ -480,28 +486,28 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
* This method is used by {@link IndexShard} to notify
* {@code RestoreService} about shard restore completion.
*
* @param snapshotId snapshot id
* @param snapshot snapshot
* @param shardId shard id
*/
public void indexShardRestoreCompleted(SnapshotId snapshotId, ShardId shardId) {
logger.trace("[{}] successfully restored shard [{}]", snapshotId, shardId);
UpdateIndexShardRestoreStatusRequest request = new UpdateIndexShardRestoreStatusRequest(snapshotId, shardId,
public void indexShardRestoreCompleted(Snapshot snapshot, ShardId shardId) {
logger.trace("[{}] successfully restored shard [{}]", snapshot, shardId);
UpdateIndexShardRestoreStatusRequest request = new UpdateIndexShardRestoreStatusRequest(snapshot, shardId,
new ShardRestoreStatus(clusterService.state().nodes().getLocalNodeId(), RestoreInProgress.State.SUCCESS));
transportService.sendRequest(clusterService.state().nodes().getMasterNode(),
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
transportService.sendRequest(clusterService.state().nodes().getMasterNode(),
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
}
public final static class RestoreCompletionResponse {
private final SnapshotId snapshotId;
private final Snapshot snapshot;
private final RestoreInfo restoreInfo;
private RestoreCompletionResponse(SnapshotId snapshotId, RestoreInfo restoreInfo) {
this.snapshotId = snapshotId;
private RestoreCompletionResponse(final Snapshot snapshot, final RestoreInfo restoreInfo) {
this.snapshot = snapshot;
this.restoreInfo = restoreInfo;
}
public SnapshotId getSnapshotId() {
return snapshotId;
public Snapshot getSnapshot() {
return snapshot;
}
public RestoreInfo getRestoreInfo() {
@ -520,7 +526,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
clusterService.submitStateUpdateTask("update snapshot state", new ClusterStateUpdateTask() {
private final List<UpdateIndexShardRestoreStatusRequest> drainedRequests = new ArrayList<>();
private Map<SnapshotId, Tuple<RestoreInfo, ImmutableOpenMap<ShardId, ShardRestoreStatus>>> batchedRestoreInfo = null;
private Map<Snapshot, Tuple<RestoreInfo, ImmutableOpenMap<ShardId, ShardRestoreStatus>>> batchedRestoreInfo = null;
@Override
public ClusterState execute(ClusterState currentState) {
@ -549,8 +555,8 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
final UpdateIndexShardRestoreStatusRequest updateSnapshotState = drainedRequests.get(i);
updateSnapshotState.processed = true;
if (entry.snapshotId().equals(updateSnapshotState.snapshotId())) {
logger.trace("[{}] Updating shard [{}] with status [{}]", updateSnapshotState.snapshotId(), updateSnapshotState.shardId(), updateSnapshotState.status().state());
if (entry.snapshot().equals(updateSnapshotState.snapshot())) {
logger.trace("[{}] Updating shard [{}] with status [{}]", updateSnapshotState.snapshot(), updateSnapshotState.shardId(), updateSnapshotState.status().state());
if (shardsBuilder == null) {
shardsBuilder = ImmutableOpenMap.builder(entry.shards());
}
@ -562,16 +568,19 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
if (shardsBuilder != null) {
ImmutableOpenMap<ShardId, ShardRestoreStatus> shards = shardsBuilder.build();
if (!completed(shards)) {
entries.add(new RestoreInProgress.Entry(entry.snapshotId(), RestoreInProgress.State.STARTED, entry.indices(), shards));
entries.add(new RestoreInProgress.Entry(entry.snapshot(), RestoreInProgress.State.STARTED, entry.indices(), shards));
} else {
logger.info("restore [{}] is done", entry.snapshotId());
logger.info("restore [{}] is done", entry.snapshot());
if (batchedRestoreInfo == null) {
batchedRestoreInfo = new HashMap<>();
}
assert !batchedRestoreInfo.containsKey(entry.snapshotId());
batchedRestoreInfo.put(entry.snapshotId(),
assert !batchedRestoreInfo.containsKey(entry.snapshot());
batchedRestoreInfo.put(entry.snapshot(),
new Tuple<>(
new RestoreInfo(entry.snapshotId().getSnapshot(), entry.indices(), shards.size(), shards.size() - failedShards(shards)),
new RestoreInfo(entry.snapshot().getSnapshotId().getName(),
entry.indices(),
shards.size(),
shards.size() - failedShards(shards)),
shards));
}
} else {
@ -592,15 +601,15 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
@Override
public void onFailure(String source, @Nullable Throwable t) {
for (UpdateIndexShardRestoreStatusRequest request : drainedRequests) {
logger.warn("[{}][{}] failed to update snapshot status to [{}]", t, request.snapshotId(), request.shardId(), request.status());
logger.warn("[{}][{}] failed to update snapshot status to [{}]", t, request.snapshot(), request.shardId(), request.status());
}
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
if (batchedRestoreInfo != null) {
for (final Entry<SnapshotId, Tuple<RestoreInfo, ImmutableOpenMap<ShardId, ShardRestoreStatus>>> entry : batchedRestoreInfo.entrySet()) {
final SnapshotId snapshotId = entry.getKey();
for (final Entry<Snapshot, Tuple<RestoreInfo, ImmutableOpenMap<ShardId, ShardRestoreStatus>>> entry : batchedRestoreInfo.entrySet()) {
final Snapshot snapshot = entry.getKey();
final RestoreInfo restoreInfo = entry.getValue().v1();
final ImmutableOpenMap<ShardId, ShardRestoreStatus> shards = entry.getValue().v2();
RoutingTable routingTable = newState.getRoutingTable();
@ -610,13 +619,13 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
ShardId shardId = shard.key;
ShardRouting shardRouting = findPrimaryShard(routingTable, shardId);
if (shardRouting != null && !shardRouting.active()) {
logger.trace("[{}][{}] waiting for the shard to start", snapshotId, shardId);
logger.trace("[{}][{}] waiting for the shard to start", snapshot, shardId);
waitForStarted.add(shardId);
}
}
}
if (waitForStarted.isEmpty()) {
notifyListeners(snapshotId, restoreInfo);
notifyListeners(snapshot, restoreInfo);
} else {
clusterService.addLast(new ClusterStateListener() {
@Override
@ -629,12 +638,12 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
// Shard disappeared (index deleted) or became active
if (shardRouting == null || shardRouting.active()) {
iterator.remove();
logger.trace("[{}][{}] shard disappeared or started - removing", snapshotId, shardId);
logger.trace("[{}][{}] shard disappeared or started - removing", snapshot, shardId);
}
}
}
if (waitForStarted.isEmpty()) {
notifyListeners(snapshotId, restoreInfo);
notifyListeners(snapshot, restoreInfo);
clusterService.remove(this);
}
}
@ -655,10 +664,10 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
return null;
}
private void notifyListeners(SnapshotId snapshotId, RestoreInfo restoreInfo) {
private void notifyListeners(Snapshot snapshot, RestoreInfo restoreInfo) {
for (ActionListener<RestoreCompletionResponse> listener : listeners) {
try {
listener.onResponse(new RestoreCompletionResponse(snapshotId, restoreInfo));
listener.onResponse(new RestoreCompletionResponse(snapshot, restoreInfo));
} catch (Throwable e) {
logger.warn("failed to update snapshot status for [{}]", e, listener);
}
@ -695,7 +704,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
}
String previousIndex = renamedIndices.put(renamedIndex, index);
if (previousIndex != null) {
throw new SnapshotRestoreException(new SnapshotId(request.repository(), request.name()),
throw new SnapshotRestoreException(request.repositoryName, request.snapshotName,
"indices [" + index + "] and [" + previousIndex + "] are renamed into the same index [" + renamedIndex + "]");
}
}
@ -705,16 +714,18 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
/**
* Checks that snapshots can be restored and have compatible version
*
* @param snapshotId snapshot id
* @param snapshot snapshot metadata
* @param repository repository name
* @param snapshotInfo snapshot metadata
*/
private void validateSnapshotRestorable(SnapshotId snapshotId, SnapshotInfo snapshot) {
if (!snapshot.state().restorable()) {
throw new SnapshotRestoreException(snapshotId, "unsupported snapshot state [" + snapshot.state() + "]");
private void validateSnapshotRestorable(final String repository, final SnapshotInfo snapshotInfo) {
if (!snapshotInfo.state().restorable()) {
throw new SnapshotRestoreException(new Snapshot(repository, snapshotInfo.snapshotId()),
"unsupported snapshot state [" + snapshotInfo.state() + "]");
}
if (Version.CURRENT.before(snapshot.version())) {
throw new SnapshotRestoreException(snapshotId, "the snapshot was created with Elasticsearch version [" +
snapshot.version() + "] which is higher than the version of this node [" + Version.CURRENT + "]");
if (Version.CURRENT.before(snapshotInfo.version())) {
throw new SnapshotRestoreException(new Snapshot(repository, snapshotInfo.snapshotId()),
"the snapshot was created with Elasticsearch version [" + snapshotInfo.version() +
"] which is higher than the version of this node [" + Version.CURRENT + "]");
}
}
@ -746,8 +757,8 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
}
if (shardsToFail != null) {
for (ShardId shardId : shardsToFail) {
logger.trace("[{}] failing running shard restore [{}]", entry.snapshotId(), shardId);
updateRestoreStateOnMaster(new UpdateIndexShardRestoreStatusRequest(entry.snapshotId(), shardId, new ShardRestoreStatus(null, RestoreInProgress.State.FAILURE, "index was deleted")));
logger.trace("[{}] failing running shard restore [{}]", entry.snapshot(), shardId);
updateRestoreStateOnMaster(new UpdateIndexShardRestoreStatusRequest(entry.snapshot(), shardId, new ShardRestoreStatus(null, RestoreInProgress.State.FAILURE, "index was deleted")));
}
}
}
@ -757,12 +768,12 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
/**
* Fails the given snapshot restore operation for the given shard
*/
public void failRestore(SnapshotId snapshotId, ShardId shardId) {
logger.debug("[{}] failed to restore shard [{}]", snapshotId, shardId);
UpdateIndexShardRestoreStatusRequest request = new UpdateIndexShardRestoreStatusRequest(snapshotId, shardId,
public void failRestore(Snapshot snapshot, ShardId shardId) {
logger.debug("[{}] failed to restore shard [{}]", snapshot, shardId);
UpdateIndexShardRestoreStatusRequest request = new UpdateIndexShardRestoreStatusRequest(snapshot, shardId,
new ShardRestoreStatus(clusterService.state().nodes().getLocalNodeId(), RestoreInProgress.State.FAILURE));
transportService.sendRequest(clusterService.state().nodes().getMasterNode(),
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
transportService.sendRequest(clusterService.state().nodes().getMasterNode(),
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
}
private boolean failed(SnapshotInfo snapshot, String index) {
@ -846,7 +857,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
RestoreInProgress snapshots = clusterState.custom(RestoreInProgress.TYPE);
if (snapshots != null) {
for (RestoreInProgress.Entry snapshot : snapshots.entries()) {
if (repository.equals(snapshot.snapshotId().getRepository())) {
if (repository.equals(snapshot.snapshot().getRepository())) {
return true;
}
}
@ -861,9 +872,9 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
final private String cause;
final private String name;
final private String repositoryName;
final private String repository;
final private String snapshotName;
final private String[] indices;
@ -890,9 +901,8 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
/**
* Constructs new restore request
*
* @param cause cause for restoring the snapshot
* @param repository repository name
* @param name snapshot name
* @param repositoryName repositoryName
* @param snapshotName snapshotName
* @param indices list of indices to restore
* @param indicesOptions indices options
* @param renamePattern pattern to rename indices
@ -903,14 +913,14 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
* @param partial allow partial restore
* @param indexSettings index settings that should be changed on restore
* @param ignoreIndexSettings index settings that shouldn't be restored
* @param cause cause for restoring the snapshot
*/
public RestoreRequest(String cause, String repository, String name, String[] indices, IndicesOptions indicesOptions,
public RestoreRequest(String repositoryName, String snapshotName, String[] indices, IndicesOptions indicesOptions,
String renamePattern, String renameReplacement, Settings settings,
TimeValue masterNodeTimeout, boolean includeGlobalState, boolean partial, boolean includeAliases,
Settings indexSettings, String[] ignoreIndexSettings ) {
this.cause = cause;
this.name = name;
this.repository = repository;
Settings indexSettings, String[] ignoreIndexSettings, String cause) {
this.repositoryName = Objects.requireNonNull(repositoryName);
this.snapshotName = Objects.requireNonNull(snapshotName);
this.indices = indices;
this.renamePattern = renamePattern;
this.renameReplacement = renameReplacement;
@ -922,7 +932,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
this.includeAliases = includeAliases;
this.indexSettings = indexSettings;
this.ignoreIndexSettings = ignoreIndexSettings;
this.cause = cause;
}
/**
@ -934,22 +944,22 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
return cause;
}
/**
* Returns snapshot name
*
* @return snapshot name
*/
public String name() {
return name;
}
/**
* Returns repository name
*
* @return repository name
*/
public String repository() {
return repository;
public String repositoryName() {
return repositoryName;
}
/**
* Returns snapshot name
*
* @return snapshot name
*/
public String snapshotName() {
return snapshotName;
}
/**
@ -1058,7 +1068,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
* Internal class that is used to send notifications about finished shard restore operations to master node
*/
public static class UpdateIndexShardRestoreStatusRequest extends TransportRequest {
private SnapshotId snapshotId;
private Snapshot snapshot;
private ShardId shardId;
private ShardRestoreStatus status;
@ -1068,8 +1078,8 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
}
private UpdateIndexShardRestoreStatusRequest(SnapshotId snapshotId, ShardId shardId, ShardRestoreStatus status) {
this.snapshotId = snapshotId;
private UpdateIndexShardRestoreStatusRequest(Snapshot snapshot, ShardId shardId, ShardRestoreStatus status) {
this.snapshot = snapshot;
this.shardId = shardId;
this.status = status;
}
@ -1077,7 +1087,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
snapshotId = SnapshotId.readSnapshotId(in);
snapshot = new Snapshot(in);
shardId = ShardId.readShardId(in);
status = ShardRestoreStatus.readShardRestoreStatus(in);
}
@ -1085,13 +1095,13 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
snapshotId.writeTo(out);
snapshot.writeTo(out);
shardId.writeTo(out);
status.writeTo(out);
}
public SnapshotId snapshotId() {
return snapshotId;
public Snapshot snapshot() {
return snapshot;
}
public ShardId shardId() {
@ -1104,7 +1114,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
@Override
public String toString() {
return "" + snapshotId + ", shardId [" + shardId + "], status [" + status.state() + "]";
return "" + snapshot + ", shardId [" + shardId + "], status [" + status.state() + "]";
}
}

View File

@ -0,0 +1,102 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.snapshots;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import java.io.IOException;
import java.util.Objects;
/**
* Basic information about a snapshot - a SnapshotId and the repository that the snapshot belongs to.
*/
public final class Snapshot implements Writeable {
private final String repository;
private final SnapshotId snapshotId;
private final int hashCode;
/**
* Constructs a snapshot.
*/
public Snapshot(final String repository, final SnapshotId snapshotId) {
this.repository = Objects.requireNonNull(repository);
this.snapshotId = Objects.requireNonNull(snapshotId);
this.hashCode = computeHashCode();
}
/**
* Constructs a snapshot from the stream input.
*/
public Snapshot(final StreamInput in) throws IOException {
repository = in.readString();
snapshotId = new SnapshotId(in);
hashCode = computeHashCode();
}
/**
* Gets the repository name for the snapshot.
*/
public String getRepository() {
return repository;
}
/**
* Gets the snapshot id for the snapshot.
*/
public SnapshotId getSnapshotId() {
return snapshotId;
}
@Override
public String toString() {
return repository + ":" + snapshotId.toString();
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
@SuppressWarnings("unchecked") Snapshot that = (Snapshot) o;
return repository.equals(that.repository) && snapshotId.equals(that.snapshotId);
}
@Override
public int hashCode() {
return hashCode;
}
private int computeHashCode() {
return Objects.hash(repository, snapshotId);
}
@Override
public void writeTo(final StreamOutput out) throws IOException {
out.writeString(repository);
snapshotId.writeTo(out);
}
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.snapshots;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.io.stream.StreamInput;
import java.io.IOException;
@ -29,11 +28,19 @@ import java.io.IOException;
*/
public class SnapshotCreationException extends SnapshotException {
public SnapshotCreationException(final String repositoryName, final String snapshotName, final Throwable cause) {
super(repositoryName, snapshotName, "failed to create snapshot", cause);
}
public SnapshotCreationException(final String repositoryName, final SnapshotId snapshotId, final Throwable cause) {
super(repositoryName, snapshotId, "failed to create snapshot", cause);
}
public SnapshotCreationException(final String repositoryName, final SnapshotId snapshotId, final String msg) {
super(repositoryName, snapshotId, msg);
}
public SnapshotCreationException(StreamInput in) throws IOException {
super(in);
}
public SnapshotCreationException(SnapshotId snapshot, Throwable cause) {
super(snapshot, "failed to create snapshot", cause);
}
}

View File

@ -20,7 +20,7 @@
package org.elasticsearch.snapshots;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -30,33 +30,68 @@ import java.io.IOException;
* Generic snapshot exception
*/
public class SnapshotException extends ElasticsearchException {
private final SnapshotId snapshot;
public SnapshotException(SnapshotId snapshot, String msg) {
@Nullable
private final String repositoryName;
@Nullable
private final String snapshotName;
public SnapshotException(final Snapshot snapshot, final String msg) {
this(snapshot, msg, null);
}
public SnapshotException(SnapshotId snapshot, String msg, Throwable cause) {
public SnapshotException(final Snapshot snapshot, final String msg, final Throwable cause) {
super("[" + (snapshot == null ? "_na" : snapshot) + "] " + msg, cause);
this.snapshot = snapshot;
if (snapshot != null) {
this.repositoryName = snapshot.getRepository();
this.snapshotName = snapshot.getSnapshotId().getName();
} else {
this.repositoryName = null;
this.snapshotName = null;
}
}
public SnapshotException(StreamInput in) throws IOException {
public SnapshotException(final String repositoryName, final SnapshotId snapshotId, final String msg) {
this(repositoryName, snapshotId, msg, null);
}
public SnapshotException(final String repositoryName, final SnapshotId snapshotId, final String msg, final Throwable cause) {
super("[" + repositoryName + ":" + snapshotId + "] " + msg, cause);
this.repositoryName = repositoryName;
this.snapshotName = snapshotId.getName();
}
public SnapshotException(final String repositoryName, final String snapshotName, final String msg) {
this(repositoryName, snapshotName, msg, null);
}
public SnapshotException(final String repositoryName, final String snapshotName, final String msg, final Throwable cause) {
super("[" + repositoryName + ":" + snapshotName + "]" + msg, cause);
this.repositoryName = repositoryName;
this.snapshotName = snapshotName;
}
public SnapshotException(final StreamInput in) throws IOException {
super(in);
if (in.readBoolean()) {
snapshot = SnapshotId.readSnapshotId(in);
} else {
snapshot = null;
}
repositoryName = in.readOptionalString();
snapshotName = in.readOptionalString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalStreamable(snapshot);
out.writeOptionalString(repositoryName);
out.writeOptionalString(snapshotName);
}
public SnapshotId snapshot() {
return snapshot;
@Nullable
public String getRepositoryName() {
return repositoryName;
}
@Nullable
public String getSnapshotName() {
return snapshotName;
}
}

View File

@ -0,0 +1,118 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.snapshots;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import java.io.IOException;
import java.util.Objects;
/**
* SnapshotId - snapshot name + snapshot UUID
*/
public final class SnapshotId implements Writeable {
/**
* This value is for older snapshots that don't have a UUID.
*/
public static final String UNASSIGNED_UUID = "_na_";
private final String name;
private final String uuid;
// Caching hash code
private final int hashCode;
/**
* Constructs a new snapshot
*
* @param name snapshot name
* @param uuid snapshot uuid
*/
public SnapshotId(final String name, final String uuid) {
this.name = Objects.requireNonNull(name);
this.uuid = Objects.requireNonNull(uuid);
this.hashCode = computeHashCode();
}
/**
* Constructs a new snapshot from a input stream
*
* @param in input stream
*/
public SnapshotId(final StreamInput in) throws IOException {
name = in.readString();
uuid = in.readString();
hashCode = computeHashCode();
}
/**
* Returns snapshot name
*
* @return snapshot name
*/
public String getName() {
return name;
}
/**
* Returns the snapshot UUID
*
* @return snapshot uuid
*/
public String getUUID() {
return uuid;
}
@Override
public String toString() {
return name + "/" + uuid;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
@SuppressWarnings("unchecked") final SnapshotId that = (SnapshotId) o;
return name.equals(that.name) && uuid.equals(that.uuid);
}
@Override
public int hashCode() {
return hashCode;
}
private int computeHashCode() {
return Objects.hash(name, uuid);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
out.writeString(uuid);
}
}

View File

@ -37,17 +37,19 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
/**
* Information about a snapshot
*/
public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent, FromXContentBuilder<SnapshotInfo>, Writeable {
public static final SnapshotInfo PROTO = new SnapshotInfo("", Collections.emptyList(), 0);
public static final SnapshotInfo PROTO = new SnapshotInfo(new SnapshotId("", ""), Collections.emptyList(), 0);
public static final String CONTEXT_MODE_PARAM = "context_mode";
public static final String CONTEXT_MODE_SNAPSHOT = "SNAPSHOT";
private static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern("strictDateOptionalTime");
private static final String SNAPSHOT = "snapshot";
private static final String UUID = "uuid";
private static final String INDICES = "indices";
private static final String STATE = "state";
private static final String REASON = "reason";
@ -68,7 +70,7 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
private static final String TOTAL_SHARDS = "total_shards";
private static final String SUCCESSFUL_SHARDS = "successful_shards";
private final String name;
private final SnapshotId snapshotId;
private final SnapshotState state;
@ -88,39 +90,35 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
private final List<SnapshotShardFailure> shardFailures;
public SnapshotInfo(String name, List<String> indices, long startTime) {
this(name, indices, SnapshotState.IN_PROGRESS, null, Version.CURRENT, startTime, 0L, 0, 0, Collections.emptyList());
public SnapshotInfo(SnapshotId snapshotId, List<String> indices, long startTime) {
this(snapshotId, indices, SnapshotState.IN_PROGRESS, null, Version.CURRENT, startTime, 0L, 0, 0, Collections.emptyList());
}
public SnapshotInfo(String name, List<String> indices, long startTime, String reason, long endTime,
public SnapshotInfo(SnapshotId snapshotId, List<String> indices, long startTime, String reason, long endTime,
int totalShards, List<SnapshotShardFailure> shardFailures) {
this(name, indices, snapshotState(reason, shardFailures), reason, Version.CURRENT,
this(snapshotId, indices, snapshotState(reason, shardFailures), reason, Version.CURRENT,
startTime, endTime, totalShards, totalShards - shardFailures.size(), shardFailures);
}
private SnapshotInfo(String name, List<String> indices, SnapshotState state, String reason, Version version, long startTime,
long endTime, int totalShards, int successfulShards, List<SnapshotShardFailure> shardFailures) {
assert name != null;
assert indices != null;
assert state != null;
assert shardFailures != null;
this.name = name;
this.indices = indices;
this.state = state;
private SnapshotInfo(SnapshotId snapshotId, List<String> indices, SnapshotState state, String reason, Version version,
long startTime, long endTime, int totalShards, int successfulShards, List<SnapshotShardFailure> shardFailures) {
this.snapshotId = Objects.requireNonNull(snapshotId);
this.indices = Objects.requireNonNull(indices);
this.state = Objects.requireNonNull(state);
this.reason = reason;
this.version = version;
this.startTime = startTime;
this.endTime = endTime;
this.totalShards = totalShards;
this.successfulShards = successfulShards;
this.shardFailures = shardFailures;
this.shardFailures = Objects.requireNonNull(shardFailures);
}
/**
* Constructs snapshot information from stream input
*/
public SnapshotInfo(final StreamInput in) throws IOException {
name = in.readString();
snapshotId = new SnapshotId(in);
int size = in.readVInt();
List<String> indicesListBuilder = new ArrayList<>();
for (int i = 0; i < size; i++) {
@ -147,12 +145,12 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
}
/**
* Returns snapshot name
* Returns snapshot id
*
* @return snapshot name
* @return snapshot id
*/
public String name() {
return name;
public SnapshotId snapshotId() {
return snapshotId;
}
/**
@ -270,16 +268,21 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
}
final SnapshotInfo that = (SnapshotInfo) o;
return startTime == that.startTime && name.equals(that.name);
return startTime == that.startTime && snapshotId.equals(that.snapshotId);
}
@Override
public int hashCode() {
int result = name.hashCode();
int result = snapshotId.hashCode();
result = 31 * result + Long.hashCode(startTime);
return result;
}
@Override
public String toString() {
return "SnapshotInfo[snapshotId=" + snapshotId + ", state=" + state + ", indices=" + indices + "]";
}
/**
* Returns snapshot REST status
*/
@ -303,7 +306,8 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
// write snapshot info for the API and any other situations
builder.startObject();
builder.field(SNAPSHOT, name);
builder.field(SNAPSHOT, snapshotId.getName());
builder.field(UUID, snapshotId.getUUID());
builder.field(VERSION_ID, version.id);
builder.field(VERSION, version.toString());
builder.startArray(INDICES);
@ -342,7 +346,8 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
private XContentBuilder toXContentSnapshot(final XContentBuilder builder, final ToXContent.Params params) throws IOException {
builder.startObject(SNAPSHOT);
builder.field(NAME, name);
builder.field(NAME, snapshotId.getName());
builder.field(UUID, snapshotId.getUUID());
builder.field(VERSION_ID, version.id);
builder.startArray(INDICES);
for (String index : indices) {
@ -380,13 +385,14 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
*/
public static SnapshotInfo fromXContent(final XContentParser parser) throws IOException {
String name = null;
String uuid = null;
Version version = Version.CURRENT;
SnapshotState state = SnapshotState.IN_PROGRESS;
String reason = null;
List<String> indices = Collections.emptyList();
long startTime = 0;
long endTime = 0;
int totalShard = 0;
int totalShards = 0;
int successfulShards = 0;
List<SnapshotShardFailure> shardFailures = Collections.emptyList();
if (parser.currentToken() == null) { // fresh parser? move to the first token
@ -406,6 +412,8 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
if (token.isValue()) {
if (NAME.equals(currentFieldName)) {
name = parser.text();
} else if (UUID.equals(currentFieldName)) {
uuid = parser.text();
} else if (STATE.equals(currentFieldName)) {
state = SnapshotState.valueOf(parser.text());
} else if (REASON.equals(currentFieldName)) {
@ -415,7 +423,7 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
} else if (END_TIME.equals(currentFieldName)) {
endTime = parser.longValue();
} else if (TOTAL_SHARDS.equals(currentFieldName)) {
totalShard = parser.intValue();
totalShards = parser.intValue();
} else if (SUCCESSFUL_SHARDS.equals(currentFieldName)) {
successfulShards = parser.intValue();
} else if (VERSION_ID.equals(currentFieldName)) {
@ -448,12 +456,25 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
} else {
throw new ElasticsearchParseException("unexpected token [" + token + "]");
}
return new SnapshotInfo(name, indices, state, reason, version, startTime, endTime, totalShard, successfulShards, shardFailures);
if (uuid == null) {
// the old format where there wasn't a UUID
uuid = SnapshotId.UNASSIGNED_UUID;
}
return new SnapshotInfo(new SnapshotId(name, uuid),
indices,
state,
reason,
version,
startTime,
endTime,
totalShards,
successfulShards,
shardFailures);
}
@Override
public void writeTo(final StreamOutput out) throws IOException {
out.writeString(name);
snapshotId.writeTo(out);
out.writeVInt(indices.size());
for (String index : indices) {
out.writeString(index);

View File

@ -19,7 +19,6 @@
package org.elasticsearch.snapshots;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.rest.RestStatus;
@ -30,12 +29,16 @@ import java.io.IOException;
*/
public class SnapshotMissingException extends SnapshotException {
public SnapshotMissingException(SnapshotId snapshot, Throwable cause) {
super(snapshot, "is missing", cause);
public SnapshotMissingException(final String repositoryName, final SnapshotId snapshotId, final Throwable cause) {
super(repositoryName, snapshotId, "is missing", cause);
}
public SnapshotMissingException(SnapshotId snapshot) {
super(snapshot, "is missing");
public SnapshotMissingException(final String repositoryName, final SnapshotId snapshotId) {
super(repositoryName, snapshotId, "is missing");
}
public SnapshotMissingException(final String repositoryName, final String snapshotName) {
super(repositoryName, snapshotName, "is missing");
}
public SnapshotMissingException(StreamInput in) throws IOException {

View File

@ -19,7 +19,6 @@
package org.elasticsearch.snapshots;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.io.stream.StreamInput;
import java.io.IOException;
@ -28,11 +27,19 @@ import java.io.IOException;
* Snapshot restore exception
*/
public class SnapshotRestoreException extends SnapshotException {
public SnapshotRestoreException(SnapshotId snapshot, String message) {
public SnapshotRestoreException(final String repositoryName, final String snapshotName, final String message) {
super(repositoryName, snapshotName, message);
}
public SnapshotRestoreException(final String repositoryName, final String snapshotName, final String message, final Throwable cause) {
super(repositoryName, snapshotName, message, cause);
}
public SnapshotRestoreException(final Snapshot snapshot, final String message) {
super(snapshot, message);
}
public SnapshotRestoreException(SnapshotId snapshot, String message, Throwable cause) {
public SnapshotRestoreException(final Snapshot snapshot, final String message, final Throwable cause) {
super(snapshot, message, cause);
}

View File

@ -27,7 +27,6 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.ImmutableOpenMap;
@ -94,7 +93,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
private final Condition shutdownCondition = shutdownLock.newCondition();
private volatile Map<SnapshotId, SnapshotShards> shardSnapshots = emptyMap();
private volatile Map<Snapshot, SnapshotShards> shardSnapshots = emptyMap();
private final BlockingQueue<UpdateIndexShardSnapshotStatusRequest> updatedSnapshotStateQueue = ConcurrentCollections.newBlockingQueue();
@ -176,11 +175,11 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
* This method is executed on data node
* </p>
*
* @param snapshotId snapshot id
* @param snapshot snapshot
* @return map of shard id to snapshot status
*/
public Map<ShardId, IndexShardSnapshotStatus> currentSnapshotShards(SnapshotId snapshotId) {
SnapshotShards snapshotShards = shardSnapshots.get(snapshotId);
public Map<ShardId, IndexShardSnapshotStatus> currentSnapshotShards(Snapshot snapshot) {
SnapshotShards snapshotShards = shardSnapshots.get(snapshot);
if (snapshotShards == null) {
return null;
} else {
@ -195,24 +194,25 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
*/
private void processIndexShardSnapshots(ClusterChangedEvent event) {
SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE);
Map<SnapshotId, SnapshotShards> survivors = new HashMap<>();
Map<Snapshot, SnapshotShards> survivors = new HashMap<>();
// First, remove snapshots that are no longer there
for (Map.Entry<SnapshotId, SnapshotShards> entry : shardSnapshots.entrySet()) {
if (snapshotsInProgress != null && snapshotsInProgress.snapshot(entry.getKey()) != null) {
for (Map.Entry<Snapshot, SnapshotShards> entry : shardSnapshots.entrySet()) {
final Snapshot snapshot = entry.getKey();
if (snapshotsInProgress != null && snapshotsInProgress.snapshot(snapshot) != null) {
survivors.put(entry.getKey(), entry.getValue());
}
}
// For now we will be mostly dealing with a single snapshot at a time but might have multiple simultaneously running
// snapshots in the future
Map<SnapshotId, Map<ShardId, IndexShardSnapshotStatus>> newSnapshots = new HashMap<>();
Map<Snapshot, Map<ShardId, IndexShardSnapshotStatus>> newSnapshots = new HashMap<>();
// Now go through all snapshots and update existing or create missing
final String localNodeId = clusterService.localNode().getId();
if (snapshotsInProgress != null) {
for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) {
if (entry.state() == SnapshotsInProgress.State.STARTED) {
Map<ShardId, IndexShardSnapshotStatus> startedShards = new HashMap<>();
SnapshotShards snapshotShards = shardSnapshots.get(entry.snapshotId());
SnapshotShards snapshotShards = shardSnapshots.get(entry.snapshot());
for (ObjectObjectCursor<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shard : entry.shards()) {
// Add all new shards to start processing on
if (localNodeId.equals(shard.value.nodeId())) {
@ -223,7 +223,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
}
}
if (!startedShards.isEmpty()) {
newSnapshots.put(entry.snapshotId(), startedShards);
newSnapshots.put(entry.snapshot(), startedShards);
if (snapshotShards != null) {
// We already saw this snapshot but we need to add more started shards
Map<ShardId, IndexShardSnapshotStatus> shards = new HashMap<>();
@ -231,15 +231,15 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
shards.putAll(snapshotShards.shards);
// Put all newly started shards
shards.putAll(startedShards);
survivors.put(entry.snapshotId(), new SnapshotShards(unmodifiableMap(shards)));
survivors.put(entry.snapshot(), new SnapshotShards(unmodifiableMap(shards)));
} else {
// Brand new snapshot that we haven't seen before
survivors.put(entry.snapshotId(), new SnapshotShards(unmodifiableMap(startedShards)));
survivors.put(entry.snapshot(), new SnapshotShards(unmodifiableMap(startedShards)));
}
}
} else if (entry.state() == SnapshotsInProgress.State.ABORTED) {
// Abort all running shards for this snapshot
SnapshotShards snapshotShards = shardSnapshots.get(entry.snapshotId());
SnapshotShards snapshotShards = shardSnapshots.get(entry.snapshot());
if (snapshotShards != null) {
for (ObjectObjectCursor<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shard : entry.shards()) {
IndexShardSnapshotStatus snapshotStatus = snapshotShards.shards.get(shard.key);
@ -250,16 +250,16 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
snapshotStatus.abort();
break;
case FINALIZE:
logger.debug("[{}] trying to cancel snapshot on shard [{}] that is finalizing, letting it finish", entry.snapshotId(), 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.snapshotId(), shard.key);
updateIndexShardSnapshotStatus(entry.snapshotId(), shard.key,
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 SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), 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.key);
updateIndexShardSnapshotStatus(entry.snapshotId(), shard.key,
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 SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.FAILED, snapshotStatus.failure()));
break;
default:
@ -288,7 +288,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
// We have new shards to starts
if (newSnapshots.isEmpty() == false) {
Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
for (final Map.Entry<SnapshotId, Map<ShardId, IndexShardSnapshotStatus>> entry : newSnapshots.entrySet()) {
for (final Map.Entry<Snapshot, Map<ShardId, IndexShardSnapshotStatus>> entry : newSnapshots.entrySet()) {
for (final Map.Entry<ShardId, IndexShardSnapshotStatus> shardEntry : entry.getValue().entrySet()) {
final ShardId shardId = shardEntry.getKey();
try {
@ -318,11 +318,11 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
/**
* Creates shard snapshot
*
* @param snapshotId snapshot id
* @param snapshot snapshot
* @param snapshotStatus snapshot status
*/
private void snapshot(final IndexShard indexShard, final SnapshotId snapshotId, final IndexShardSnapshotStatus snapshotStatus) {
IndexShardRepository indexShardRepository = snapshotsService.getRepositoriesService().indexShardRepository(snapshotId.getRepository());
private void snapshot(final IndexShard indexShard, final Snapshot snapshot, final IndexShardSnapshotStatus snapshotStatus) {
IndexShardRepository indexShardRepository = snapshotsService.getRepositoriesService().indexShardRepository(snapshot.getRepository());
ShardId shardId = indexShard.shardId();
if (!indexShard.routingEntry().primary()) {
throw new IndexShardSnapshotFailedException(shardId, "snapshot should be performed only on primary");
@ -340,11 +340,11 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
// we flush first to make sure we get the latest writes snapshotted
IndexCommit snapshotIndexCommit = indexShard.snapshotIndex(true);
try {
indexShardRepository.snapshot(snapshotId, shardId, snapshotIndexCommit, snapshotStatus);
indexShardRepository.snapshot(snapshot.getSnapshotId(), shardId, snapshotIndexCommit, 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");
logger.debug("snapshot ({}) completed to {}, took [{}]\n{}", snapshotId.getSnapshot(), indexShardRepository,
logger.debug("snapshot ({}) completed to {}, took [{}]\n{}", snapshot, indexShardRepository,
TimeValue.timeValueMillis(snapshotStatus.time()), sb);
}
} finally {
@ -369,7 +369,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
}
for (SnapshotsInProgress.Entry snapshot : snapshotsInProgress.entries()) {
if (snapshot.state() == SnapshotsInProgress.State.STARTED || snapshot.state() == SnapshotsInProgress.State.ABORTED) {
Map<ShardId, IndexShardSnapshotStatus> localShards = currentSnapshotShards(snapshot.snapshotId());
Map<ShardId, IndexShardSnapshotStatus> localShards = currentSnapshotShards(snapshot.snapshot());
if (localShards != null) {
ImmutableOpenMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> masterShards = snapshot.shards();
for(Map.Entry<ShardId, IndexShardSnapshotStatus> localShard : localShards.entrySet()) {
@ -380,13 +380,13 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
// Master knows about the shard and thinks it has not completed
if (localShardStatus.stage() == IndexShardSnapshotStatus.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.snapshotId(), shardId);
updateIndexShardSnapshotStatus(snapshot.snapshotId(), shardId,
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 SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), 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);
updateIndexShardSnapshotStatus(snapshot.snapshotId(), shardId,
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 SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.FAILED, localShardStatus.failure()));
}
@ -413,7 +413,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
* Internal request that is used to send changes in snapshot status to master
*/
public static class UpdateIndexShardSnapshotStatusRequest extends TransportRequest {
private SnapshotId snapshotId;
private Snapshot snapshot;
private ShardId shardId;
private SnapshotsInProgress.ShardSnapshotStatus status;
@ -423,8 +423,8 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
}
public UpdateIndexShardSnapshotStatusRequest(SnapshotId snapshotId, ShardId shardId, SnapshotsInProgress.ShardSnapshotStatus status) {
this.snapshotId = snapshotId;
public UpdateIndexShardSnapshotStatusRequest(Snapshot snapshot, ShardId shardId, SnapshotsInProgress.ShardSnapshotStatus status) {
this.snapshot = snapshot;
this.shardId = shardId;
this.status = status;
}
@ -432,7 +432,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
snapshotId = SnapshotId.readSnapshotId(in);
snapshot = new Snapshot(in);
shardId = ShardId.readShardId(in);
status = SnapshotsInProgress.ShardSnapshotStatus.readShardSnapshotStatus(in);
}
@ -440,13 +440,13 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
snapshotId.writeTo(out);
snapshot.writeTo(out);
shardId.writeTo(out);
status.writeTo(out);
}
public SnapshotId snapshotId() {
return snapshotId;
public Snapshot snapshot() {
return snapshot;
}
public ShardId shardId() {
@ -459,7 +459,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
@Override
public String toString() {
return "" + snapshotId + ", shardId [" + shardId + "], status [" + status.state() + "]";
return "" + snapshot + ", shardId [" + shardId + "], status [" + status.state() + "]";
}
public void markAsProcessed() {
@ -474,8 +474,8 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
/**
* Updates the shard status
*/
public void updateIndexShardSnapshotStatus(SnapshotId snapshotId, ShardId shardId, SnapshotsInProgress.ShardSnapshotStatus status) {
UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshotId, shardId, status);
public void updateIndexShardSnapshotStatus(Snapshot snapshot, ShardId shardId, SnapshotsInProgress.ShardSnapshotStatus status) {
UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status);
try {
if (clusterService.state().nodes().isLocalNodeElectedMaster()) {
innerUpdateSnapshotState(request);
@ -484,7 +484,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
UPDATE_SNAPSHOT_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
}
} catch (Throwable t) {
logger.warn("[{}] [{}] failed to update snapshot state", t, request.snapshotId(), request.status());
logger.warn("[{}] [{}] failed to update snapshot state", t, request.snapshot(), request.status());
}
}
@ -528,8 +528,8 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
final UpdateIndexShardSnapshotStatusRequest updateSnapshotState = drainedRequests.get(i);
updateSnapshotState.markAsProcessed();
if (entry.snapshotId().equals(updateSnapshotState.snapshotId())) {
logger.trace("[{}] Updating shard [{}] with status [{}]", updateSnapshotState.snapshotId(), updateSnapshotState.shardId(), updateSnapshotState.status().state());
if (entry.snapshot().equals(updateSnapshotState.snapshot())) {
logger.trace("[{}] Updating shard [{}] with status [{}]", updateSnapshotState.snapshot(), updateSnapshotState.shardId(), updateSnapshotState.status().state());
if (updated == false) {
shards.putAll(entry.shards());
updated = true;
@ -549,7 +549,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
entries.add(updatedEntry);
// Finalize snapshot in the repository
snapshotsService.endSnapshot(updatedEntry);
logger.info("snapshot [{}] is done", updatedEntry.snapshotId());
logger.info("snapshot [{}] is done", updatedEntry.snapshot());
}
} else {
entries.add(entry);
@ -568,7 +568,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
@Override
public void onFailure(String source, Throwable t) {
for (UpdateIndexShardSnapshotStatusRequest request : drainedRequests) {
logger.warn("[{}][{}] failed to update snapshot status to [{}]", t, request.snapshotId(), request.shardId(), request.status());
logger.warn("[{}][{}] failed to update snapshot status to [{}]", t, request.snapshot(), request.shardId(), request.status());
}
}
});

View File

@ -37,7 +37,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.RepositoriesMetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
@ -47,6 +46,7 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
@ -72,8 +72,11 @@ import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.stream.Collectors;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.cluster.SnapshotsInProgress.completed;
@ -92,7 +95,7 @@ import static org.elasticsearch.cluster.SnapshotsInProgress.completed;
* <li>Once shard snapshot is created data node updates state of the shard in the cluster state using the {@link SnapshotShardsService#updateIndexShardSnapshotStatus} method</li>
* <li>When last shard is completed master node in {@link SnapshotShardsService#innerUpdateSnapshotState} method marks the snapshot as completed</li>
* <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>
* notifies all {@link #snapshotCompletionListeners} that snapshot is completed, and finally calls {@link #removeSnapshotFromClusterState(Snapshot, SnapshotInfo, Throwable)} to remove snapshot from cluster state</li>
* </ul>
*/
public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsService> implements ClusterStateListener {
@ -121,49 +124,67 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
}
}
/**
* Retrieves list of snapshot ids that are present in a repository
*
* @param repositoryName repository name
* @return list of snapshot ids
*/
public List<SnapshotId> snapshotIds(final String repositoryName) {
Repository repository = repositoriesService.repository(repositoryName);
assert repository != null; // should only be called once we've validated the repository exists
return repository.snapshots();
}
/**
* Retrieves snapshot from repository
*
* @param snapshotId snapshot id
* @param repositoryName repository name
* @param snapshotId snapshot id
* @return snapshot
* @throws SnapshotMissingException if snapshot is not found
*/
public SnapshotInfo snapshot(SnapshotId snapshotId) {
validate(snapshotId);
List<SnapshotsInProgress.Entry> entries = currentSnapshots(snapshotId.getRepository(), new String[]{snapshotId.getSnapshot()});
public SnapshotInfo snapshot(final String repositoryName, final SnapshotId snapshotId) {
List<SnapshotsInProgress.Entry> entries = currentSnapshots(repositoryName, Arrays.asList(snapshotId.getName()));
if (!entries.isEmpty()) {
return inProgressSnapshot(entries.iterator().next());
}
return repositoriesService.repository(snapshotId.getRepository()).readSnapshot(snapshotId);
return repositoriesService.repository(repositoryName).readSnapshot(snapshotId);
}
/**
* Returns a list of snapshots from repository sorted by snapshot creation date
*
* @param repositoryName repository name
* @param snapshotIds snapshots for which to fetch snapshot information
* @param ignoreUnavailable if true, snapshots that could not be read will only be logged with a warning,
* if false, they will throw an error
* @return list of snapshots
*/
public List<SnapshotInfo> snapshots(String repositoryName, boolean ignoreUnavailable) {
Set<SnapshotInfo> snapshotSet = new HashSet<>();
List<SnapshotsInProgress.Entry> entries = currentSnapshots(repositoryName, null);
public List<SnapshotInfo> snapshots(final String repositoryName, List<SnapshotId> snapshotIds, final boolean ignoreUnavailable) {
final Set<SnapshotInfo> snapshotSet = new HashSet<>();
final Set<SnapshotId> snapshotIdsToIterate = new HashSet<>(snapshotIds);
// first, look at the snapshots in progress
final List<SnapshotsInProgress.Entry> entries =
currentSnapshots(repositoryName, snapshotIdsToIterate.stream().map(SnapshotId::getName).collect(Collectors.toList()));
for (SnapshotsInProgress.Entry entry : entries) {
snapshotSet.add(inProgressSnapshot(entry));
snapshotIdsToIterate.remove(entry.snapshot().getSnapshotId());
}
Repository repository = repositoriesService.repository(repositoryName);
List<SnapshotId> snapshotIds = repository.snapshots();
for (SnapshotId snapshotId : snapshotIds) {
// then, look in the repository
final Repository repository = repositoriesService.repository(repositoryName);
for (SnapshotId snapshotId : snapshotIdsToIterate) {
try {
snapshotSet.add(repository.readSnapshot(snapshotId));
} catch (Exception ex) {
if (ignoreUnavailable) {
logger.warn("failed to get snapshot [{}]", ex, snapshotId);
} else {
throw new SnapshotException(snapshotId, "Snapshot could not be read", ex);
throw new SnapshotException(repositoryName, snapshotId, "Snapshot could not be read", ex);
}
}
}
ArrayList<SnapshotInfo> snapshotList = new ArrayList<>(snapshotSet);
final ArrayList<SnapshotInfo> snapshotList = new ArrayList<>(snapshotSet);
CollectionUtil.timSort(snapshotList);
return Collections.unmodifiableList(snapshotList);
}
@ -174,9 +195,9 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* @param repositoryName repository name
* @return list of snapshots
*/
public List<SnapshotInfo> currentSnapshots(String repositoryName) {
public List<SnapshotInfo> currentSnapshots(final String repositoryName) {
List<SnapshotInfo> snapshotList = new ArrayList<>();
List<SnapshotsInProgress.Entry> entries = currentSnapshots(repositoryName, null);
List<SnapshotsInProgress.Entry> entries = currentSnapshots(repositoryName, Collections.emptyList());
for (SnapshotsInProgress.Entry entry : entries) {
snapshotList.add(inProgressSnapshot(entry));
}
@ -194,8 +215,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* @param listener snapshot creation listener
*/
public void createSnapshot(final SnapshotRequest request, final CreateSnapshotListener listener) {
final SnapshotId snapshotId = new SnapshotId(request.repository(), request.name());
validate(snapshotId);
final String repositoryName = request.repositoryName;
final String snapshotName = request.snapshotName;
validate(repositoryName, snapshotName);
final SnapshotId snapshotId = new SnapshotId(snapshotName, UUIDs.randomBase64UUID()); // new UUID for the snapshot
clusterService.submitStateUpdateTask(request.cause(), new ClusterStateUpdateTask() {
private SnapshotsInProgress.Entry newSnapshot = null;
@ -208,19 +232,25 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
if (snapshots == null || snapshots.entries().isEmpty()) {
// Store newSnapshot here to be processed in clusterStateProcessed
List<String> indices = Arrays.asList(indexNameExpressionResolver.concreteIndexNames(currentState, request.indicesOptions(), request.indices()));
logger.trace("[{}][{}] creating snapshot for indices [{}]", request.repository(), request.name(), indices);
newSnapshot = new SnapshotsInProgress.Entry(snapshotId, request.includeGlobalState(), request.partial(), State.INIT, indices, System.currentTimeMillis(), null);
logger.trace("[{}][{}] creating snapshot for indices [{}]", repositoryName, snapshotName, indices);
newSnapshot = new SnapshotsInProgress.Entry(new Snapshot(repositoryName, snapshotId),
request.includeGlobalState(),
request.partial(),
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");
throw new ConcurrentSnapshotExecutionException(repositoryName, snapshotName, "a snapshot is already running");
}
return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build();
}
@Override
public void onFailure(String source, Throwable t) {
logger.warn("[{}][{}] failed to create snapshot", t, request.repository(), request.name());
logger.warn("[{}][{}] failed to create snapshot", t, repositoryName, snapshotName);
newSnapshot = null;
listener.onFailure(t);
}
@ -228,12 +258,9 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
@Override
public void clusterStateProcessed(String source, ClusterState oldState, final ClusterState newState) {
if (newSnapshot != null) {
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new Runnable() {
@Override
public void run() {
beginSnapshot(newState, newSnapshot, request.partial(), listener);
}
});
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() ->
beginSnapshot(newState, newSnapshot, request.partial(), listener)
);
}
}
@ -253,34 +280,36 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
*/
private void validate(SnapshotRequest request, ClusterState state) {
RepositoriesMetaData repositoriesMetaData = state.getMetaData().custom(RepositoriesMetaData.TYPE);
if (repositoriesMetaData == null || repositoriesMetaData.repository(request.repository()) == null) {
throw new RepositoryMissingException(request.repository());
final String repository = request.repositoryName;
if (repositoriesMetaData == null || repositoriesMetaData.repository(repository) == null) {
throw new RepositoryMissingException(repository);
}
validate(new SnapshotId(request.repository(), request.name()));
validate(repository, request.snapshotName);
}
private static void validate(SnapshotId snapshotId) {
String name = snapshotId.getSnapshot();
if (!Strings.hasLength(name)) {
throw new InvalidSnapshotNameException(snapshotId, "cannot be empty");
private static void validate(final String repositoryName, final String snapshotName) {
if (Strings.hasLength(snapshotName) == false) {
throw new InvalidSnapshotNameException(repositoryName, snapshotName, "cannot be empty");
}
if (name.contains(" ")) {
throw new InvalidSnapshotNameException(snapshotId, "must not contain whitespace");
if (snapshotName.contains(" ")) {
throw new InvalidSnapshotNameException(repositoryName, snapshotName, "must not contain whitespace");
}
if (name.contains(",")) {
throw new InvalidSnapshotNameException(snapshotId, "must not contain ','");
if (snapshotName.contains(",")) {
throw new InvalidSnapshotNameException(repositoryName, snapshotName, "must not contain ','");
}
if (name.contains("#")) {
throw new InvalidSnapshotNameException(snapshotId, "must not contain '#'");
if (snapshotName.contains("#")) {
throw new InvalidSnapshotNameException(repositoryName, snapshotName, "must not contain '#'");
}
if (name.charAt(0) == '_') {
throw new InvalidSnapshotNameException(snapshotId, "must not start with '_'");
if (snapshotName.charAt(0) == '_') {
throw new InvalidSnapshotNameException(repositoryName, snapshotName, "must not start with '_'");
}
if (!name.toLowerCase(Locale.ROOT).equals(name)) {
throw new InvalidSnapshotNameException(snapshotId, "must be lowercase");
if (snapshotName.toLowerCase(Locale.ROOT).equals(snapshotName) == false) {
throw new InvalidSnapshotNameException(repositoryName, snapshotName, "must be lowercase");
}
if (!Strings.validFileName(name)) {
throw new InvalidSnapshotNameException(snapshotId, "must not contain the following characters " + Strings.INVALID_FILENAME_CHARS);
if (Strings.validFileName(snapshotName) == false) {
throw new InvalidSnapshotNameException(repositoryName,
snapshotName,
"must not contain the following characters " + Strings.INVALID_FILENAME_CHARS);
}
}
@ -294,10 +323,13 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* @param partial allow partial snapshots
* @param userCreateSnapshotListener listener
*/
private void beginSnapshot(ClusterState clusterState, final SnapshotsInProgress.Entry snapshot, final boolean partial, final CreateSnapshotListener userCreateSnapshotListener) {
private void beginSnapshot(final ClusterState clusterState,
final SnapshotsInProgress.Entry snapshot,
final boolean partial,
final CreateSnapshotListener userCreateSnapshotListener) {
boolean snapshotCreated = false;
try {
Repository repository = repositoriesService.repository(snapshot.snapshotId().getRepository());
Repository repository = repositoriesService.repository(snapshot.snapshot().getRepository());
MetaData metaData = clusterState.metaData();
if (!snapshot.includeGlobalState()) {
@ -309,7 +341,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
metaData = builder.build();
}
repository.initializeSnapshot(snapshot.snapshotId(), snapshot.indices(), metaData);
repository.initializeSnapshot(snapshot.snapshot().getSnapshotId(), snapshot.indices(), metaData);
snapshotCreated = true;
if (snapshot.indices().isEmpty()) {
// No indices in this snapshot - we are done
@ -317,7 +349,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
endSnapshot(snapshot);
return;
}
clusterService.submitStateUpdateTask("update_snapshot [" + snapshot.snapshotId().getSnapshot() + "]", new ClusterStateUpdateTask() {
clusterService.submitStateUpdateTask("update_snapshot [" + snapshot.snapshot() + "]", new ClusterStateUpdateTask() {
boolean accepted = false;
SnapshotsInProgress.Entry updatedSnapshot;
String failure = null;
@ -327,7 +359,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE);
List<SnapshotsInProgress.Entry> entries = new ArrayList<>();
for (SnapshotsInProgress.Entry entry : snapshots.entries()) {
if (entry.snapshotId().equals(snapshot.snapshotId())) {
if (entry.snapshot().equals(snapshot.snapshot())) {
// Replace the snapshot that was just created
ImmutableOpenMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards = shards(currentState, entry.indices());
if (!partial) {
@ -362,13 +394,15 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
entries.add(entry);
}
}
return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(Collections.unmodifiableList(entries))).build();
return ClusterState.builder(currentState)
.putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(Collections.unmodifiableList(entries)))
.build();
}
@Override
public void onFailure(String source, Throwable t) {
logger.warn("[{}] failed to create snapshot", t, snapshot.snapshotId());
removeSnapshotFromClusterState(snapshot.snapshotId(), null, t, new CleanupAfterErrorListener(snapshot, true, userCreateSnapshotListener, t));
logger.warn("[{}] failed to create snapshot", t, snapshot.snapshot().getSnapshotId());
removeSnapshotFromClusterState(snapshot.snapshot(), null, t, new CleanupAfterErrorListener(snapshot, true, userCreateSnapshotListener, t));
}
@Override
@ -389,8 +423,8 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
}
});
} catch (Throwable t) {
logger.warn("failed to create snapshot [{}]", t, snapshot.snapshotId());
removeSnapshotFromClusterState(snapshot.snapshotId(), null, t, new CleanupAfterErrorListener(snapshot, snapshotCreated, userCreateSnapshotListener, t));
logger.warn("failed to create snapshot [{}]", t, snapshot.snapshot().getSnapshotId());
removeSnapshotFromClusterState(snapshot.snapshot(), null, t, new CleanupAfterErrorListener(snapshot, snapshotCreated, userCreateSnapshotListener, t));
}
}
@ -421,10 +455,15 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
private void cleanupAfterError() {
if(snapshotCreated) {
try {
repositoriesService.repository(snapshot.snapshotId().getRepository()).finalizeSnapshot(
snapshot.snapshotId(), snapshot.indices(), snapshot.startTime(), ExceptionsHelper.detailedMessage(t), 0, Collections.<SnapshotShardFailure>emptyList());
repositoriesService.repository(snapshot.snapshot().getRepository())
.finalizeSnapshot(snapshot.snapshot().getSnapshotId(),
snapshot.indices(),
snapshot.startTime(),
ExceptionsHelper.detailedMessage(t),
0,
Collections.emptyList());
} catch (Throwable t2) {
logger.warn("[{}] failed to close snapshot in repository", snapshot.snapshotId());
logger.warn("[{}] failed to close snapshot in repository", snapshot.snapshot());
}
}
userCreateSnapshotListener.onFailure(t);
@ -433,7 +472,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
}
private SnapshotInfo inProgressSnapshot(SnapshotsInProgress.Entry entry) {
return new SnapshotInfo(entry.snapshotId().getSnapshot(), entry.indices(), entry.startTime());
return new SnapshotInfo(entry.snapshot().getSnapshotId(), entry.indices(), entry.startTime());
}
/**
@ -443,10 +482,10 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* </p>
*
* @param repository repository id
* @param snapshots optional list of snapshots that will be used as a filter
* @param snapshots list of snapshots that will be used as a filter, empty list means no snapshots are filtered
* @return list of metadata for currently running snapshots
*/
public List<SnapshotsInProgress.Entry> currentSnapshots(String repository, String[] snapshots) {
public List<SnapshotsInProgress.Entry> currentSnapshots(final String repository, final List<String> snapshots) {
SnapshotsInProgress snapshotsInProgress = clusterService.state().custom(SnapshotsInProgress.TYPE);
if (snapshotsInProgress == null || snapshotsInProgress.entries().isEmpty()) {
return Collections.emptyList();
@ -458,12 +497,12 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
// Most likely scenario - one snapshot is currently running
// Check this snapshot against the query
SnapshotsInProgress.Entry entry = snapshotsInProgress.entries().get(0);
if (!entry.snapshotId().getRepository().equals(repository)) {
if (entry.snapshot().getRepository().equals(repository) == false) {
return Collections.emptyList();
}
if (snapshots != null && snapshots.length > 0) {
if (snapshots.isEmpty() == false) {
for (String snapshot : snapshots) {
if (entry.snapshotId().getSnapshot().equals(snapshot)) {
if (entry.snapshot().getSnapshotId().getName().equals(snapshot)) {
return snapshotsInProgress.entries();
}
}
@ -474,12 +513,12 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
}
List<SnapshotsInProgress.Entry> builder = new ArrayList<>();
for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) {
if (!entry.snapshotId().getRepository().equals(repository)) {
if (entry.snapshot().getRepository().equals(repository) == false) {
continue;
}
if (snapshots != null && snapshots.length > 0) {
if (snapshots.isEmpty() == false) {
for (String snapshot : snapshots) {
if (entry.snapshotId().getSnapshot().equals(snapshot)) {
if (entry.snapshot().getSnapshotId().getName().equals(snapshot)) {
builder.add(entry);
break;
}
@ -494,34 +533,35 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
/**
* Returns status of shards currently finished snapshots
* <p>
* This method is executed on master node and it's complimentary to the {@link SnapshotShardsService#currentSnapshotShards(SnapshotId)} because it
* This method is executed on master node and it's complimentary to the {@link SnapshotShardsService#currentSnapshotShards(Snapshot)} because it
* returns similar information but for already finished snapshots.
* </p>
*
* @param snapshotId snapshot id
* @param repositoryName repository name
* @param snapshotInfo snapshot info
* @return map of shard id to snapshot status
*/
public Map<ShardId, IndexShardSnapshotStatus> snapshotShards(SnapshotId snapshotId) throws IOException {
validate(snapshotId);
public Map<ShardId, IndexShardSnapshotStatus> snapshotShards(final String repositoryName,
final SnapshotInfo snapshotInfo) throws IOException {
Map<ShardId, IndexShardSnapshotStatus> shardStatus = new HashMap<>();
Repository repository = repositoriesService.repository(snapshotId.getRepository());
IndexShardRepository indexShardRepository = repositoriesService.indexShardRepository(snapshotId.getRepository());
SnapshotInfo snapshot = repository.readSnapshot(snapshotId);
MetaData metaData = repository.readSnapshotMetaData(snapshotId, snapshot, snapshot.indices());
for (String index : snapshot.indices()) {
Repository repository = repositoriesService.repository(repositoryName);
IndexShardRepository indexShardRepository = repositoriesService.indexShardRepository(repositoryName);
MetaData metaData = repository.readSnapshotMetaData(snapshotInfo, snapshotInfo.indices());
for (String index : snapshotInfo.indices()) {
IndexMetaData indexMetaData = metaData.indices().get(index);
if (indexMetaData != null) {
int numberOfShards = indexMetaData.getNumberOfShards();
for (int i = 0; i < numberOfShards; i++) {
ShardId shardId = new ShardId(indexMetaData.getIndex(), i);
SnapshotShardFailure shardFailure = findShardFailure(snapshot.shardFailures(), shardId);
SnapshotShardFailure shardFailure = findShardFailure(snapshotInfo.shardFailures(), shardId);
if (shardFailure != null) {
IndexShardSnapshotStatus shardSnapshotStatus = new IndexShardSnapshotStatus();
shardSnapshotStatus.updateStage(IndexShardSnapshotStatus.Stage.FAILURE);
shardSnapshotStatus.failure(shardFailure.reason());
shardStatus.put(shardId, shardSnapshotStatus);
} else {
IndexShardSnapshotStatus shardSnapshotStatus = indexShardRepository.snapshotStatus(snapshotId, snapshot.version(), shardId);
IndexShardSnapshotStatus shardSnapshotStatus =
indexShardRepository.snapshotStatus(snapshotInfo.snapshotId(), snapshotInfo.version(), shardId);
shardStatus.put(shardId, shardSnapshotStatus);
}
}
@ -606,15 +646,15 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
entries.add(updatedSnapshot);
} else if (snapshot.state() == State.INIT && newMaster) {
// Clean up the snapshot that failed to start from the old master
deleteSnapshot(snapshot.snapshotId(), new DeleteSnapshotListener() {
deleteSnapshot(snapshot.snapshot(), new DeleteSnapshotListener() {
@Override
public void onResponse() {
logger.debug("cleaned up abandoned snapshot {} in INIT state", snapshot.snapshotId());
logger.debug("cleaned up abandoned snapshot {} in INIT state", snapshot.snapshot());
}
@Override
public void onFailure(Throwable t) {
logger.warn("failed to clean up abandoned snapshot {} in INIT state", snapshot.snapshotId());
logger.warn("failed to clean up abandoned snapshot {} in INIT state", snapshot.snapshot());
}
});
} else if (snapshot.state() == State.SUCCESS && newMaster) {
@ -810,10 +850,10 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new Runnable() {
@Override
public void run() {
SnapshotId snapshotId = entry.snapshotId();
final Snapshot snapshot = entry.snapshot();
try {
final Repository repository = repositoriesService.repository(snapshotId.getRepository());
logger.trace("[{}] finalizing snapshot in repository, state: [{}], failure[{}]", snapshotId, entry.state(), failure);
final Repository repository = repositoriesService.repository(snapshot.getRepository());
logger.trace("[{}] finalizing snapshot in repository, state: [{}], failure[{}]", snapshot, entry.state(), failure);
ArrayList<ShardSearchFailure> failures = new ArrayList<>();
ArrayList<SnapshotShardFailure> shardFailures = new ArrayList<>();
for (ObjectObjectCursor<ShardId, ShardSnapshotStatus> shardStatus : entry.shards()) {
@ -824,11 +864,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
shardFailures.add(new SnapshotShardFailure(status.nodeId(), shardId, status.reason()));
}
}
SnapshotInfo snapshot = repository.finalizeSnapshot(snapshotId, entry.indices(), entry.startTime(), failure, entry.shards().size(), Collections.unmodifiableList(shardFailures));
removeSnapshotFromClusterState(snapshotId, snapshot, null);
SnapshotInfo snapshotInfo = repository.finalizeSnapshot(snapshot.getSnapshotId(), entry.indices(), entry.startTime(), failure, entry.shards().size(), Collections.unmodifiableList(shardFailures));
removeSnapshotFromClusterState(snapshot, snapshotInfo, null);
} catch (Throwable t) {
logger.warn("[{}] failed to finalize snapshot", t, snapshotId);
removeSnapshotFromClusterState(snapshotId, null, t);
logger.warn("[{}] failed to finalize snapshot", t, snapshot);
removeSnapshotFromClusterState(snapshot, null, t);
}
}
});
@ -837,23 +877,23 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
/**
* Removes record of running snapshot from cluster state
*
* @param snapshotId snapshot id
* @param snapshot snapshot info if snapshot was successful
* @param t exception if snapshot failed
* @param snapshot snapshot
* @param snapshotInfo snapshot info if snapshot was successful
* @param t exception if snapshot failed
*/
private void removeSnapshotFromClusterState(final SnapshotId snapshotId, final SnapshotInfo snapshot, final Throwable t) {
removeSnapshotFromClusterState(snapshotId, snapshot, t, null);
private void removeSnapshotFromClusterState(final Snapshot snapshot, final SnapshotInfo snapshotInfo, final Throwable t) {
removeSnapshotFromClusterState(snapshot, snapshotInfo, t, null);
}
/**
* Removes record of running snapshot from cluster state and notifies the listener when this action is complete
*
* @param snapshotId snapshot id
* @param snapshot snapshot
* @param snapshot snapshot info if snapshot was successful
* @param t exception if snapshot failed
* @param listener listener to notify when snapshot information is removed from the cluster state
*/
private void removeSnapshotFromClusterState(final SnapshotId snapshotId, final SnapshotInfo snapshot, final Throwable t,
private void removeSnapshotFromClusterState(final Snapshot snapshot, final SnapshotInfo snapshotInfo, final Throwable t,
@Nullable ActionListener<SnapshotInfo> listener) {
clusterService.submitStateUpdateTask("remove snapshot metadata", new ClusterStateUpdateTask() {
@Override
@ -863,7 +903,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
boolean changed = false;
ArrayList<SnapshotsInProgress.Entry> entries = new ArrayList<>();
for (SnapshotsInProgress.Entry entry : snapshots.entries()) {
if (entry.snapshotId().equals(snapshotId)) {
if (entry.snapshot().equals(snapshot)) {
changed = true;
} else {
entries.add(entry);
@ -879,7 +919,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
@Override
public void onFailure(String source, Throwable t) {
logger.warn("[{}] failed to remove snapshot metadata", t, snapshotId);
logger.warn("[{}] failed to remove snapshot metadata", t, snapshot);
if (listener != null) {
listener.onFailure(t);
}
@ -889,32 +929,54 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
for (SnapshotCompletionListener listener : snapshotCompletionListeners) {
try {
if (snapshot != null) {
listener.onSnapshotCompletion(snapshotId, snapshot);
if (snapshotInfo != null) {
listener.onSnapshotCompletion(snapshot, snapshotInfo);
} else {
listener.onSnapshotFailure(snapshotId, t);
listener.onSnapshotFailure(snapshot, t);
}
} catch (Throwable t) {
logger.warn("failed to notify listener [{}]", t, listener);
}
}
if (listener != null) {
listener.onResponse(snapshot);
listener.onResponse(snapshotInfo);
}
}
});
}
/**
* Deletes a snapshot from the repository, looking up the {@link Snapshot} reference before deleting.
* If the snapshot is still running cancels the snapshot first and then deletes it from the repository.
*
* @param repositoryName repositoryName
* @param snapshotName snapshotName
* @param listener listener
*/
public void deleteSnapshot(final String repositoryName, final String snapshotName, final DeleteSnapshotListener listener) {
// First, look for the snapshot in the repository
final Repository repository = repositoriesService.repository(repositoryName);
Optional<SnapshotId> matchedEntry = repository.snapshots().stream().filter(s -> s.getName().equals(snapshotName)).findFirst();
// if nothing found by the same name, then look in the cluster state for current in progress snapshots
if (matchedEntry.isPresent() == false) {
matchedEntry = currentSnapshots(repositoryName, Collections.emptyList()).stream()
.map(e -> e.snapshot().getSnapshotId()).filter(s -> s.getName().equals(snapshotName)).findFirst();
}
if (matchedEntry.isPresent() == false) {
throw new SnapshotMissingException(repositoryName, snapshotName);
}
deleteSnapshot(new Snapshot(repositoryName, matchedEntry.get()), listener);
}
/**
* Deletes snapshot from repository.
* <p>
* If the snapshot is still running cancels the snapshot first and then deletes it from the repository.
*
* @param snapshotId snapshot id
* @param listener listener
* @param snapshot snapshot
* @param listener listener
*/
public void deleteSnapshot(final SnapshotId snapshotId, final DeleteSnapshotListener listener) {
validate(snapshotId);
public void deleteSnapshot(final Snapshot snapshot, final DeleteSnapshotListener listener) {
clusterService.submitStateUpdateTask("delete snapshot", new ClusterStateUpdateTask() {
boolean waitForSnapshot = false;
@ -926,22 +988,22 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
// No snapshots running - we can continue
return currentState;
}
SnapshotsInProgress.Entry snapshot = snapshots.snapshot(snapshotId);
if (snapshot == null) {
SnapshotsInProgress.Entry snapshotEntry = snapshots.snapshot(snapshot);
if (snapshotEntry == null) {
// This snapshot is not running - continue
if (!snapshots.entries().isEmpty()) {
// However other snapshots are running - cannot continue
throw new ConcurrentSnapshotExecutionException(snapshotId, "another snapshot is currently running cannot delete");
throw new ConcurrentSnapshotExecutionException(snapshot, "another snapshot is currently running cannot delete");
}
return currentState;
} else {
// This snapshot is currently running - stopping shards first
waitForSnapshot = true;
ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards;
if (snapshot.state() == State.STARTED && snapshot.shards() != null) {
if (snapshotEntry.state() == State.STARTED && snapshotEntry.shards() != null) {
// snapshot is currently running - stop started shards
ImmutableOpenMap.Builder<ShardId, ShardSnapshotStatus> shardsBuilder = ImmutableOpenMap.builder();
for (ObjectObjectCursor<ShardId, ShardSnapshotStatus> shardEntry : snapshot.shards()) {
for (ObjectObjectCursor<ShardId, ShardSnapshotStatus> shardEntry : snapshotEntry.shards()) {
ShardSnapshotStatus status = shardEntry.value;
if (!status.state().completed()) {
shardsBuilder.put(shardEntry.key, new ShardSnapshotStatus(status.nodeId(), State.ABORTED));
@ -950,14 +1012,14 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
}
}
shards = shardsBuilder.build();
} else if (snapshot.state() == State.INIT) {
} else if (snapshotEntry.state() == State.INIT) {
// snapshot hasn't started yet - end it
shards = snapshot.shards();
endSnapshot(snapshot);
shards = snapshotEntry.shards();
endSnapshot(snapshotEntry);
} else {
boolean hasUncompletedShards = false;
// Cleanup in case a node gone missing and snapshot wasn't updated for some reason
for (ObjectCursor<ShardSnapshotStatus> shardStatus : snapshot.shards().values()) {
for (ObjectCursor<ShardSnapshotStatus> shardStatus : snapshotEntry.shards().values()) {
// Check if we still have shard running on existing nodes
if (shardStatus.value.state().completed() == false && shardStatus.value.nodeId() != null
&& currentState.nodes().get(shardStatus.value.nodeId()) != null) {
@ -972,11 +1034,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
} else {
// no shards to wait for - finish the snapshot
logger.debug("trying to delete completed snapshot with no finalizing shards - can delete immediately");
shards = snapshot.shards();
endSnapshot(snapshot);
shards = snapshotEntry.shards();
endSnapshot(snapshotEntry);
}
}
SnapshotsInProgress.Entry newSnapshot = new SnapshotsInProgress.Entry(snapshot, State.ABORTED, shards);
SnapshotsInProgress.Entry newSnapshot = new SnapshotsInProgress.Entry(snapshotEntry, State.ABORTED, shards);
snapshots = new SnapshotsInProgress(newSnapshot);
return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, snapshots).build();
}
@ -993,26 +1055,26 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
logger.trace("adding snapshot completion listener to wait for deleted snapshot to finish");
addListener(new SnapshotCompletionListener() {
@Override
public void onSnapshotCompletion(SnapshotId completedSnapshotId, SnapshotInfo snapshot) {
if (completedSnapshotId.equals(snapshotId)) {
public void onSnapshotCompletion(Snapshot completedSnapshot, SnapshotInfo snapshotInfo) {
if (completedSnapshot.equals(snapshot)) {
logger.trace("deleted snapshot completed - deleting files");
removeListener(this);
deleteSnapshotFromRepository(snapshotId, listener);
deleteSnapshotFromRepository(snapshot, listener);
}
}
@Override
public void onSnapshotFailure(SnapshotId failedSnapshotId, Throwable t) {
if (failedSnapshotId.equals(snapshotId)) {
public void onSnapshotFailure(Snapshot failedSnapshot, Throwable t) {
if (failedSnapshot.equals(snapshot)) {
logger.trace("deleted snapshot failed - deleting files", t);
removeListener(this);
deleteSnapshotFromRepository(snapshotId, listener);
deleteSnapshotFromRepository(snapshot, listener);
}
}
});
} else {
logger.trace("deleted snapshot is not running - deleting files");
deleteSnapshotFromRepository(snapshotId, listener);
deleteSnapshotFromRepository(snapshot, listener);
}
}
});
@ -1029,7 +1091,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
SnapshotsInProgress snapshots = clusterState.custom(SnapshotsInProgress.TYPE);
if (snapshots != null) {
for (SnapshotsInProgress.Entry snapshot : snapshots.entries()) {
if (repository.equals(snapshot.snapshotId().getRepository())) {
if (repository.equals(snapshot.snapshot().getRepository())) {
return true;
}
}
@ -1040,20 +1102,17 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
/**
* Deletes snapshot from repository
*
* @param snapshotId snapshot id
* @param snapshot snapshot
* @param listener listener
*/
private void deleteSnapshotFromRepository(final SnapshotId snapshotId, final DeleteSnapshotListener listener) {
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new Runnable() {
@Override
public void run() {
try {
Repository repository = repositoriesService.repository(snapshotId.getRepository());
repository.deleteSnapshot(snapshotId);
listener.onResponse();
} catch (Throwable t) {
listener.onFailure(t);
}
private void deleteSnapshotFromRepository(final Snapshot snapshot, final DeleteSnapshotListener listener) {
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> {
try {
Repository repository = repositoriesService.repository(snapshot.getRepository());
repository.deleteSnapshot(snapshot.getSnapshotId());
listener.onResponse();
} catch (Throwable t) {
listener.onFailure(t);
}
});
}
@ -1203,7 +1262,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
/**
* Listener for create snapshot operation
*/
public static interface CreateSnapshotListener {
public interface CreateSnapshotListener {
/**
* Called when snapshot has successfully started
@ -1219,7 +1278,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
/**
* Listener for delete snapshot operation
*/
public static interface DeleteSnapshotListener {
public interface DeleteSnapshotListener {
/**
* Called if delete operation was successful
@ -1232,11 +1291,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
void onFailure(Throwable t);
}
public static interface SnapshotCompletionListener {
public interface SnapshotCompletionListener {
void onSnapshotCompletion(SnapshotId snapshotId, SnapshotInfo snapshot);
void onSnapshotCompletion(Snapshot snapshot, SnapshotInfo snapshotInfo);
void onSnapshotFailure(SnapshotId snapshotId, Throwable t);
void onSnapshotFailure(Snapshot snapshot, Throwable t);
}
/**
@ -1244,11 +1303,11 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
*/
public static class SnapshotRequest {
private String cause;
private final String cause;
private String name;
private final String repositoryName;
private String repository;
private final String snapshotName;
private String[] indices;
@ -1265,14 +1324,14 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
/**
* Constructs new snapshot creation request
*
* @param cause cause for snapshot operation
* @param name name of the snapshot
* @param repository name of the repository
* @param repositoryName repository name
* @param snapshotName snapshot name
* @param cause cause for snapshot operation
*/
public SnapshotRequest(String cause, String name, String repository) {
this.cause = cause;
this.name = name;
this.repository = repository;
public SnapshotRequest(final String repositoryName, final String snapshotName, final String cause) {
this.repositoryName = Objects.requireNonNull(repositoryName);
this.snapshotName = Objects.requireNonNull(snapshotName);
this.cause = Objects.requireNonNull(cause);
}
/**
@ -1351,21 +1410,17 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
}
/**
* Returns snapshot name
*
* @return snapshot name
* Returns the repository name
*/
public String name() {
return name;
public String repositoryName() {
return repositoryName;
}
/**
* Returns snapshot repository
*
* @return snapshot repository
* Returns the snapshot name
*/
public String repository() {
return repository;
public String snapshotName() {
return snapshotName;
}
/**

View File

@ -27,13 +27,13 @@ import org.elasticsearch.action.support.replication.ReplicationOperation;
import org.elasticsearch.client.AbstractClientHeadersTestCase;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.IllegalShardRoutingStateException;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
@ -70,7 +70,9 @@ import org.elasticsearch.search.SearchException;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotException;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.TestSearchContext;
import org.elasticsearch.test.VersionUtils;
@ -303,14 +305,16 @@ public class ExceptionSerializationTests extends ESTestCase {
}
public void testSnapshotException() throws IOException {
SnapshotException ex = serialize(
new SnapshotException(new SnapshotId("repo", "snap"), "no such snapshot", new NullPointerException()));
assertEquals(ex.snapshot(), new SnapshotId("repo", "snap"));
assertEquals(ex.getMessage(), "[repo:snap] no such snapshot");
final Snapshot snapshot = new Snapshot("repo", new SnapshotId("snap", UUIDs.randomBase64UUID()));
SnapshotException ex = serialize(new SnapshotException(snapshot, "no such snapshot", new NullPointerException()));
assertEquals(ex.getRepositoryName(), snapshot.getRepository());
assertEquals(ex.getSnapshotName(), snapshot.getSnapshotId().getName());
assertEquals(ex.getMessage(), "[" + snapshot + "] no such snapshot");
assertTrue(ex.getCause() instanceof NullPointerException);
ex = serialize(new SnapshotException(null, "no such snapshot", new NullPointerException()));
assertEquals(ex.snapshot(), null);
assertNull(ex.getRepositoryName());
assertNull(ex.getSnapshotName());
assertEquals(ex.getMessage(), "[_na] no such snapshot");
assertTrue(ex.getCause() instanceof NullPointerException);
}

View File

@ -157,7 +157,7 @@ public class SnapshotBlocksIT extends ESIntegTestCase {
setClusterReadOnly(true);
GetSnapshotsResponse response = client().admin().cluster().prepareGetSnapshots(REPOSITORY_NAME).execute().actionGet();
assertThat(response.getSnapshots(), hasSize(1));
assertThat(response.getSnapshots().get(0).name(), equalTo(SNAPSHOT_NAME));
assertThat(response.getSnapshots().get(0).snapshotId().getName(), equalTo(SNAPSHOT_NAME));
} finally {
setClusterReadOnly(false);
}

View File

@ -0,0 +1,195 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.bwcompat;
import org.elasticsearch.common.io.FileTestUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.snapshots.SnapshotInfo;
import org.elasticsearch.test.ESIntegTestCase;
import java.nio.file.DirectoryStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
/**
* Tests that a repository can handle both snapshots of previous version formats and new version formats,
* as blob names and repository blob formats have changed between the snapshot versions.
*/
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST)
public class RepositoryUpgradabilityIT extends AbstractSnapshotIntegTestCase {
/**
* This tests that a repository can inter-operate with snapshots that both have and don't have a UUID,
* namely when a repository was created in an older version with snapshots created in the old format
* (only snapshot name, no UUID) and then the repository is loaded into newer versions where subsequent
* snapshots have a name and a UUID.
*/
public void testRepositoryWorksWithCrossVersions() throws Exception {
final List<String> repoVersions = listRepoVersions();
// run the test for each supported version
for (final String version : repoVersions) {
final String repoName = "test-repo-" + version;
logger.info("--> creating repository [{}] for version [{}]", repoName, version);
createRepository(version, repoName);
logger.info("--> get the snapshots");
final String originalIndex = "index-" + version;
final Set<String> indices = Sets.newHashSet(originalIndex);
final Set<SnapshotInfo> snapshotInfos = Sets.newHashSet(getSnapshots(repoName));
assertThat(snapshotInfos.size(), equalTo(1));
SnapshotInfo originalSnapshot = snapshotInfos.iterator().next();
assertThat(originalSnapshot.snapshotId(), equalTo(new SnapshotId("test_1", SnapshotId.UNASSIGNED_UUID)));
assertThat(Sets.newHashSet(originalSnapshot.indices()), equalTo(indices));
logger.info("--> restore the original snapshot");
final Set<String> restoredIndices = Sets.newHashSet(
restoreSnapshot(repoName, originalSnapshot.snapshotId().getName())
);
assertThat(restoredIndices, equalTo(indices));
// make sure it has documents
for (final String searchIdx : restoredIndices) {
assertThat(client().prepareSearch(searchIdx).setSize(0).get().getHits().totalHits(), greaterThan(0L));
}
deleteIndices(restoredIndices); // delete so we can restore again later
final String snapshotName2 = "test_2";
logger.info("--> take a new snapshot of the old index");
final int addedDocSize = 10;
for (int i = 0; i < addedDocSize; i++) {
index(originalIndex, "doc", Integer.toString(i), "foo", "new-bar-" + i);
}
refresh();
snapshotInfos.add(createSnapshot(repoName, snapshotName2));
logger.info("--> get the snapshots with the newly created snapshot [{}]", snapshotName2);
Set<SnapshotInfo> snapshotInfosFromRepo = Sets.newHashSet(getSnapshots(repoName));
assertThat(snapshotInfosFromRepo, equalTo(snapshotInfos));
snapshotInfosFromRepo.forEach(snapshotInfo -> {
assertThat(Sets.newHashSet(snapshotInfo.indices()), equalTo(indices));
});
final String snapshotName3 = "test_3";
final String indexName2 = "index2";
logger.info("--> take a new snapshot with a new index");
createIndex(indexName2);
indices.add(indexName2);
for (int i = 0; i < addedDocSize; i++) {
index(indexName2, "doc", Integer.toString(i), "foo", "new-bar-" + i);
}
refresh();
snapshotInfos.add(createSnapshot(repoName, snapshotName3));
logger.info("--> get the snapshots with the newly created snapshot [{}]", snapshotName3);
snapshotInfosFromRepo = Sets.newHashSet(getSnapshots(repoName));
assertThat(snapshotInfosFromRepo, equalTo(snapshotInfos));
snapshotInfosFromRepo.forEach(snapshotInfo -> {
if (snapshotInfo.snapshotId().getName().equals(snapshotName3)) {
// only the last snapshot has all the indices
assertThat(Sets.newHashSet(snapshotInfo.indices()), equalTo(indices));
} else {
assertThat(Sets.newHashSet(snapshotInfo.indices()), equalTo(Sets.newHashSet(originalIndex)));
}
});
deleteIndices(indices); // clean up indices
logger.info("--> restore the old snapshot again");
Set<String> oldRestoredIndices = Sets.newHashSet(restoreSnapshot(repoName, originalSnapshot.snapshotId().getName()));
assertThat(oldRestoredIndices, equalTo(Sets.newHashSet(originalIndex)));
for (final String searchIdx : oldRestoredIndices) {
assertThat(client().prepareSearch(searchIdx).setSize(0).get().getHits().totalHits(),
greaterThanOrEqualTo((long)addedDocSize));
}
deleteIndices(oldRestoredIndices);
logger.info("--> restore the new snapshot");
Set<String> newSnapshotIndices = Sets.newHashSet(restoreSnapshot(repoName, snapshotName3));
assertThat(newSnapshotIndices, equalTo(Sets.newHashSet(originalIndex, indexName2)));
for (final String searchIdx : newSnapshotIndices) {
assertThat(client().prepareSearch(searchIdx).setSize(0).get().getHits().totalHits(),
greaterThanOrEqualTo((long)addedDocSize));
}
deleteIndices(newSnapshotIndices); // clean up indices before starting again
}
}
private List<String> listRepoVersions() throws Exception {
final String prefix = "repo";
final List<String> repoVersions = new ArrayList<>();
final Path repoFiles = getBwcIndicesPath();
try (final DirectoryStream<Path> dirStream = Files.newDirectoryStream(repoFiles, prefix + "-*.zip")) {
for (final Path entry : dirStream) {
final String fileName = entry.getFileName().toString();
String version = fileName.substring(prefix.length() + 1);
version = version.substring(0, version.length() - ".zip".length());
repoVersions.add(version);
}
}
return Collections.unmodifiableList(repoVersions);
}
private void createRepository(final String version, final String repoName) throws Exception {
final String prefix = "repo";
final Path repoFile = getBwcIndicesPath().resolve(prefix + "-" + version + ".zip");
final Path repoPath = randomRepoPath();
FileTestUtils.unzip(repoFile, repoPath, "repo/");
assertAcked(client().admin().cluster().preparePutRepository(repoName)
.setType("fs")
.setSettings(Settings.builder().put("location", repoPath)));
}
private List<SnapshotInfo> getSnapshots(final String repoName) throws Exception {
return client().admin().cluster().prepareGetSnapshots(repoName)
.addSnapshots("_all")
.get()
.getSnapshots();
}
private SnapshotInfo createSnapshot(final String repoName, final String snapshotName) throws Exception {
return client().admin().cluster().prepareCreateSnapshot(repoName, snapshotName)
.setWaitForCompletion(true)
.get()
.getSnapshotInfo();
}
private List<String> restoreSnapshot(final String repoName, final String snapshotName) throws Exception {
return client().admin().cluster().prepareRestoreSnapshot(repoName, snapshotName)
.setWaitForCompletion(true)
.get()
.getRestoreInfo()
.indices();
}
private void deleteIndices(final Set<String> indices) throws Exception {
client().admin().indices().prepareDelete(indices.toArray(new String[indices.size()])).get();
}
}

View File

@ -40,7 +40,6 @@ import org.elasticsearch.test.ESIntegTestCase.Scope;
import org.elasticsearch.test.VersionUtils;
import java.io.IOException;
import java.lang.reflect.Modifier;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.file.DirectoryStream;

View File

@ -30,7 +30,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexTemplateMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.RepositoriesMetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
@ -51,6 +51,7 @@ import org.elasticsearch.gateway.GatewayService;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.test.ESIntegTestCase;
import java.util.Collections;
@ -653,7 +654,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
switch (randomIntBetween(0, 1)) {
case 0:
return new SnapshotsInProgress(new SnapshotsInProgress.Entry(
new SnapshotId(randomName("repo"), randomName("snap")),
new Snapshot(randomName("repo"), new SnapshotId(randomName("snap"), UUIDs.randomBase64UUID())),
randomBoolean(),
randomBoolean(),
SnapshotsInProgress.State.fromValue((byte) randomIntBetween(0, 6)),
@ -662,7 +663,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
ImmutableOpenMap.of()));
case 1:
return new RestoreInProgress(new RestoreInProgress.Entry(
new SnapshotId(randomName("repo"), randomName("snap")),
new Snapshot(randomName("repo"), new SnapshotId(randomName("snap"), UUIDs.randomBase64UUID())),
RestoreInProgress.State.fromValue((byte) randomIntBetween(0, 3)),
emptyList(),
ImmutableOpenMap.of()));

View File

@ -20,18 +20,15 @@
package org.elasticsearch.cluster.routing;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import static org.hamcrest.Matchers.equalTo;
public class ShardRoutingTests extends ESTestCase {
public void testIsSameAllocation() {
@ -155,8 +152,8 @@ public class ShardRoutingTests extends ESTestCase {
case 4:
// change restore source
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
otherRouting.restoreSource() == null ? new RestoreSource(new SnapshotId("test", "s1"), Version.CURRENT, "test") :
new RestoreSource(otherRouting.restoreSource().snapshotId(), Version.CURRENT, otherRouting.index() + "_1"),
otherRouting.restoreSource() == null ? new RestoreSource(new Snapshot("test", new SnapshotId("s1", UUIDs.randomBase64UUID())), Version.CURRENT, "test") :
new RestoreSource(otherRouting.restoreSource().snapshot(), Version.CURRENT, otherRouting.index() + "_1"),
otherRouting.primary(), otherRouting.state(), otherRouting.unassignedInfo());
break;
case 5:

View File

@ -26,7 +26,8 @@ import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
@ -35,6 +36,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.test.ESAllocationTestCase;
import java.util.Collections;
@ -130,7 +132,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
.build();
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsNewRestore(metaData.index("test"), new RestoreSource(new SnapshotId("rep1", "snp1"), Version.CURRENT, "test"), new IntHashSet()).build()).build();
.routingTable(RoutingTable.builder().addAsNewRestore(metaData.index("test"), new RestoreSource(new Snapshot("rep1", new SnapshotId("snp1", UUIDs.randomBase64UUID())), Version.CURRENT, "test"), new IntHashSet()).build()).build();
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.NEW_INDEX_RESTORED));
}
@ -142,7 +144,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
.build();
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsRestore(metaData.index("test"), new RestoreSource(new SnapshotId("rep1", "snp1"), Version.CURRENT, "test")).build()).build();
.routingTable(RoutingTable.builder().addAsRestore(metaData.index("test"), new RestoreSource(new Snapshot("rep1", new SnapshotId("snp1", UUIDs.randomBase64UUID())), Version.CURRENT, "test")).build()).build();
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.EXISTING_INDEX_RESTORED));
}

View File

@ -25,7 +25,8 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.EmptyClusterInfoService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
@ -48,6 +49,7 @@ import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.test.ESAllocationTestCase;
import org.elasticsearch.test.VersionUtils;
import org.elasticsearch.test.gateway.NoopGatewayAllocator;
@ -360,7 +362,8 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
ClusterState state = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsRestore(metaData.index("test"), new RestoreSource(new SnapshotId("rep1", "snp1"),
.routingTable(RoutingTable.builder().addAsRestore(metaData.index("test"),
new RestoreSource(new Snapshot("rep1", new SnapshotId("snp1", UUIDs.randomBase64UUID())),
Version.CURRENT, "test")).build())
.nodes(DiscoveryNodes.builder().put(newNode).put(oldNode1).put(oldNode2)).build();
AllocationDeciders allocationDeciders = new AllocationDeciders(Settings.EMPTY, new AllocationDecider[]{

View File

@ -24,7 +24,7 @@ import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.RestoreSource;
@ -40,6 +40,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardStateMetaData;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.test.ESAllocationTestCase;
import org.junit.Before;
@ -339,8 +340,9 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
.putActiveAllocationIds(0, hasActiveAllocation ? Sets.newHashSet("allocId") : Collections.emptySet()))
.build();
final Snapshot snapshot = new Snapshot("test", new SnapshotId("test", UUIDs.randomBase64UUID()));
RoutingTable routingTable = RoutingTable.builder()
.addAsRestore(metaData.index(shardId.getIndex()), new RestoreSource(new SnapshotId("test", "test"), version, shardId.getIndexName()))
.addAsRestore(metaData.index(shardId.getIndex()), new RestoreSource(snapshot, version, shardId.getIndexName()))
.build();
ClusterState state = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT)
.metaData(metaData)
@ -419,7 +421,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsRestore(metaData.index(shardId.getIndex()), new RestoreSource(new SnapshotId("test", "test"), Version.CURRENT, shardId.getIndexName()))
.addAsRestore(metaData.index(shardId.getIndex()), new RestoreSource(new Snapshot("test", new SnapshotId("test", UUIDs.randomBase64UUID())), Version.CURRENT, shardId.getIndexName()))
.build();
ClusterState state = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT)
.metaData(metaData)

View File

@ -49,7 +49,8 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.InternalClusterInfoService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.AllocationId;
import org.elasticsearch.cluster.routing.RestoreSource;
@ -97,6 +98,7 @@ import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.FieldMaskingReader;
@ -1110,7 +1112,8 @@ public class IndexShardTests extends ESSingleNodeTestCase {
client().admin().indices().prepareFlush("test").get(); // only flush test
final ShardRouting origRouting = test_target.getShardOrNull(0).routingEntry();
ShardRouting routing = ShardRoutingHelper.reinit(origRouting);
routing = ShardRoutingHelper.newWithRestoreSource(routing, new RestoreSource(new SnapshotId("foo", "bar"), Version.CURRENT, "test"));
final Snapshot snapshot = new Snapshot("foo", new SnapshotId("bar", UUIDs.randomBase64UUID()));
routing = ShardRoutingHelper.newWithRestoreSource(routing, new RestoreSource(snapshot, Version.CURRENT, "test"));
test_target.removeShard(0, "just do it man!");
final IndexShard test_target_shard = test_target.createShard(routing);
Store sourceStore = test_shard.store();

View File

@ -0,0 +1,230 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.repositories.blobstore;
import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryResponse;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.repositories.RepositoriesService;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESSingleNodeTestCase;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.blobId;
import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.parseNameUUIDFromBlobName;
import static org.hamcrest.Matchers.equalTo;
/**
* Tests for the {@link BlobStoreRepository} and its subclasses.
*/
public class BlobStoreRepositoryTests extends ESSingleNodeTestCase {
public void testRetrieveSnapshots() throws Exception {
final Client client = client();
final Path location = ESIntegTestCase.randomRepoPath(node().settings());
final String repositoryName = "test-repo";
logger.info("--> creating repository");
PutRepositoryResponse putRepositoryResponse =
client.admin().cluster().preparePutRepository(repositoryName)
.setType("fs")
.setSettings(Settings.builder().put(node().settings()).put("location", location))
.get();
assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true));
logger.info("--> creating an index and indexing documents");
final String indexName = "test-idx";
createIndex(indexName);
ensureGreen();
int numDocs = randomIntBetween(10, 20);
for (int i = 0; i < numDocs; i++) {
String id = Integer.toString(i);
client().prepareIndex(indexName, "type1", id).setSource("text", "sometext").get();
}
client().admin().indices().prepareFlush(indexName).setWaitIfOngoing(true).get();
logger.info("--> create first snapshot");
CreateSnapshotResponse createSnapshotResponse = client.admin()
.cluster()
.prepareCreateSnapshot(repositoryName, "test-snap-1")
.setWaitForCompletion(true)
.setIndices(indexName)
.get();
final SnapshotId snapshotId1 = createSnapshotResponse.getSnapshotInfo().snapshotId();
logger.info("--> create second snapshot");
createSnapshotResponse = client.admin()
.cluster()
.prepareCreateSnapshot(repositoryName, "test-snap-2")
.setWaitForCompletion(true)
.setIndices(indexName)
.get();
final SnapshotId snapshotId2 = createSnapshotResponse.getSnapshotInfo().snapshotId();
logger.info("--> make sure the node's repository can resolve the snapshots");
final RepositoriesService repositoriesService = getInstanceFromNode(RepositoriesService.class);
@SuppressWarnings("unchecked") final BlobStoreRepository repository =
(BlobStoreRepository) repositoriesService.repository(repositoryName);
final List<SnapshotId> originalSnapshots = Arrays.asList(snapshotId1, snapshotId2);
List<SnapshotId> snapshotIds = repository.snapshots().stream()
.sorted((s1, s2) -> s1.getName().compareTo(s2.getName()))
.collect(Collectors.toList());
assertThat(snapshotIds, equalTo(originalSnapshots));
}
public void testSnapshotIndexFile() throws Exception {
final Client client = client();
final Path location = ESIntegTestCase.randomRepoPath(node().settings());
final String repositoryName = "test-repo";
PutRepositoryResponse putRepositoryResponse =
client.admin().cluster().preparePutRepository(repositoryName)
.setType("fs")
.setSettings(Settings.builder().put(node().settings()).put("location", location))
.get();
assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true));
final RepositoriesService repositoriesService = getInstanceFromNode(RepositoriesService.class);
@SuppressWarnings("unchecked") final BlobStoreRepository repository =
(BlobStoreRepository) repositoriesService.repository(repositoryName);
// write to and read from a snapshot file with no entries
repository.writeSnapshotList(Collections.emptyList());
List<SnapshotId> readSnapshotIds = repository.readSnapshotList();
assertThat(readSnapshotIds.size(), equalTo(0));
// write to and read from a snapshot file with a random number of entries
final int numSnapshots = randomIntBetween(1, 1000);
final List<SnapshotId> snapshotIds = new ArrayList<>(numSnapshots);
for (int i = 0; i < numSnapshots; i++) {
snapshotIds.add(new SnapshotId(randomAsciiOfLength(8), UUIDs.randomBase64UUID()));
}
repository.writeSnapshotList(snapshotIds);
readSnapshotIds = repository.readSnapshotList();
assertThat(readSnapshotIds, equalTo(snapshotIds));
}
public void testOldIndexFileFormat() throws Exception {
final Client client = client();
final Path location = ESIntegTestCase.randomRepoPath(node().settings());
final String repositoryName = "test-repo";
PutRepositoryResponse putRepositoryResponse =
client.admin().cluster().preparePutRepository(repositoryName)
.setType("fs")
.setSettings(Settings.builder().put(node().settings()).put("location", location))
.get();
assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true));
final RepositoriesService repositoriesService = getInstanceFromNode(RepositoriesService.class);
@SuppressWarnings("unchecked") final BlobStoreRepository repository =
(BlobStoreRepository) repositoriesService.repository(repositoryName);
// write old index file format
final int numOldSnapshots = randomIntBetween(1, 50);
final List<SnapshotId> snapshotIds = new ArrayList<>();
for (int i = 0; i < numOldSnapshots; i++) {
snapshotIds.add(new SnapshotId(randomAsciiOfLength(8), SnapshotId.UNASSIGNED_UUID));
}
writeOldFormat(repository, snapshotIds.stream().map(SnapshotId::getName).collect(Collectors.toList()));
List<SnapshotId> readSnapshotIds = repository.readSnapshotList();
assertThat(Sets.newHashSet(readSnapshotIds), equalTo(Sets.newHashSet(snapshotIds)));
// write to and read from a snapshot file with a random number of new entries added
final int numSnapshots = randomIntBetween(1, 1000);
for (int i = 0; i < numSnapshots; i++) {
snapshotIds.add(new SnapshotId(randomAsciiOfLength(8), UUIDs.randomBase64UUID()));
}
repository.writeSnapshotList(snapshotIds);
readSnapshotIds = repository.readSnapshotList();
assertThat(Sets.newHashSet(readSnapshotIds), equalTo(Sets.newHashSet(snapshotIds)));
}
public void testParseUUIDFromBlobName() {
String blobStr = "abc123";
Tuple<String, String> pair = parseNameUUIDFromBlobName(blobStr);
assertThat(pair.v1(), equalTo(blobStr)); // snapshot name
assertThat(pair.v2(), equalTo(SnapshotId.UNASSIGNED_UUID)); // snapshot uuid
blobStr = "abcefghijklmnopqrstuvwxyz";
pair = parseNameUUIDFromBlobName(blobStr);
assertThat(pair.v1(), equalTo(blobStr));
assertThat(pair.v2(), equalTo(SnapshotId.UNASSIGNED_UUID));
blobStr = "abc123-xyz"; // not enough characters after '-' to have a uuid
pair = parseNameUUIDFromBlobName(blobStr);
assertThat(pair.v1(), equalTo(blobStr));
assertThat(pair.v2(), equalTo(SnapshotId.UNASSIGNED_UUID));
blobStr = "abc123-a1b2c3d4e5f6g7h8i9j0k1";
pair = parseNameUUIDFromBlobName(blobStr);
assertThat(pair.v1(), equalTo("abc123"));
assertThat(pair.v2(), equalTo("a1b2c3d4e5f6g7h8i9j0k1"));
}
public void testBlobId() {
SnapshotId snapshotId = new SnapshotId("abc123", SnapshotId.UNASSIGNED_UUID);
assertThat(blobId(snapshotId), equalTo("abc123")); // just the snapshot name
snapshotId = new SnapshotId("abc-123", SnapshotId.UNASSIGNED_UUID);
assertThat(blobId(snapshotId), equalTo("abc-123")); // just the snapshot name
String uuid = UUIDs.randomBase64UUID();
snapshotId = new SnapshotId("abc123", uuid);
assertThat(blobId(snapshotId), equalTo("abc123-" + uuid)); // snapshot name + '-' + uuid
uuid = UUIDs.randomBase64UUID();
snapshotId = new SnapshotId("abc-123", uuid);
assertThat(blobId(snapshotId), equalTo("abc-123-" + uuid)); // snapshot name + '-' + uuid
}
private void writeOldFormat(final BlobStoreRepository repository, final List<String> snapshotNames) throws Exception {
final BytesReference bRef;
try (BytesStreamOutput bStream = new BytesStreamOutput()) {
try (StreamOutput stream = new OutputStreamStreamOutput(bStream)) {
XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream);
builder.startObject();
builder.startArray("snapshots");
for (final String snapshotName : snapshotNames) {
builder.value(snapshotName);
}
builder.endArray();
builder.endObject();
builder.close();
}
bRef = bStream.bytes();
}
repository.blobContainer().writeBlob(BlobStoreRepository.SNAPSHOTS_FILE, bRef); // write to index file
}
}

View File

@ -21,7 +21,8 @@ package org.elasticsearch.rest.action.cat;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RestoreSource;
import org.elasticsearch.common.Randomness;
@ -32,6 +33,7 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.test.ESTestCase;
import java.util.ArrayList;
@ -76,10 +78,9 @@ public class RestRecoveryActionTests extends ESTestCase {
final RestoreSource restoreSource = randomBoolean() ? mock(RestoreSource.class) : null;
if (restoreSource != null) {
final SnapshotId snapshotId = mock(SnapshotId.class);
when(snapshotId.getRepository()).thenReturn(randomAsciiOfLength(8));
when(snapshotId.getSnapshot()).thenReturn(randomAsciiOfLength(8));
when(restoreSource.snapshotId()).thenReturn(snapshotId);
final Snapshot snapshot = new Snapshot(randomAsciiOfLength(8),
new SnapshotId(randomAsciiOfLength(8), UUIDs.randomBase64UUID()));
when(restoreSource.snapshot()).thenReturn(snapshot);
}
RecoveryState.Index index = mock(RecoveryState.Index.class);
@ -166,10 +167,10 @@ public class RestRecoveryActionTests extends ESTestCase {
assertThat(cells.get(8).value, equalTo(state.getTargetNode().getName()));
assertThat(
cells.get(9).value,
equalTo(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshotId().getRepository()));
equalTo(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshot().getRepository()));
assertThat(
cells.get(10).value,
equalTo(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshotId().getSnapshot()));
equalTo(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshot().getSnapshotId().getName()));
assertThat(cells.get(11).value, equalTo(state.getIndex().totalRecoverFiles()));
assertThat(cells.get(12).value, equalTo(state.getIndex().recoveredFileCount()));
assertThat(cells.get(13).value, equalTo(percent(state.getIndex().recoveredFilesPercent())));

View File

@ -25,7 +25,6 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.cluster.service.PendingClusterTask;
@ -108,18 +107,29 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
fail("Timeout!!!");
}
public SnapshotInfo waitForCompletion(String repository, String snapshot, TimeValue timeout) throws InterruptedException {
public SnapshotInfo waitForCompletion(String repository, String snapshotName, TimeValue timeout) throws InterruptedException {
long start = System.currentTimeMillis();
SnapshotId snapshotId = new SnapshotId(repository, snapshot);
while (System.currentTimeMillis() - start < timeout.millis()) {
List<SnapshotInfo> snapshotInfos = client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshot).get().getSnapshots();
List<SnapshotInfo> snapshotInfos = client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshotName).get().getSnapshots();
assertThat(snapshotInfos.size(), equalTo(1));
if (snapshotInfos.get(0).state().completed()) {
// Make sure that snapshot clean up operations are finished
ClusterStateResponse stateResponse = client().admin().cluster().prepareState().get();
SnapshotsInProgress snapshotsInProgress = stateResponse.getState().custom(SnapshotsInProgress.TYPE);
if (snapshotsInProgress == null || snapshotsInProgress.snapshot(snapshotId) == null) {
if (snapshotsInProgress == null) {
return snapshotInfos.get(0);
} else {
boolean found = false;
for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) {
final Snapshot curr = entry.snapshot();
if (curr.getRepository().equals(repository) && curr.getSnapshotId().getName().equals(snapshotName)) {
found = true;
break;
}
}
if (found == false) {
return snapshotInfos.get(0);
}
}
}
Thread.sleep(100);
@ -128,12 +138,13 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
return null;
}
public static String blockNodeWithIndex(String index) {
for(String node : internalCluster().nodesInclude("test-idx")) {
((MockRepository)internalCluster().getInstance(RepositoriesService.class, node).repository("test-repo")).blockOnDataFiles(true);
public static String blockNodeWithIndex(final String repositoryName, final String indexName) {
for(String node : internalCluster().nodesInclude(indexName)) {
((MockRepository)internalCluster().getInstance(RepositoriesService.class, node).repository(repositoryName))
.blockOnDataFiles(true);
return node;
}
fail("No nodes for the index " + index + " found");
fail("No nodes for the index " + indexName + " found");
return null;
}
@ -163,8 +174,8 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
}
}
public static void unblockNode(String node) {
((MockRepository)internalCluster().getInstance(RepositoriesService.class, node).repository("test-repo")).unblock();
public static void unblockNode(final String repository, final String node) {
((MockRepository)internalCluster().getInstance(RepositoriesService.class, node).repository(repository)).unblock();
}
protected void assertBusyPendingTasks(final String taskPrefix, final int expectedCount) throws Exception {

View File

@ -313,7 +313,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true));
// Pick one node and block it
String blockedNode = blockNodeWithIndex("test-idx");
String blockedNode = blockNodeWithIndex("test-repo", "test-idx");
logger.info("--> snapshot");
client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get();
@ -322,7 +322,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
waitForBlock(blockedNode, "test-repo", TimeValue.timeValueSeconds(60));
logger.info("--> execution was blocked on node [{}], shutting it down", blockedNode);
unblockNode(blockedNode);
unblockNode("test-repo", blockedNode);
logger.info("--> stopping node [{}]", blockedNode);
stopNode(blockedNode);
@ -361,7 +361,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true));
// Pick one node and block it
String blockedNode = blockNodeWithIndex("test-idx");
String blockedNode = blockNodeWithIndex("test-repo", "test-idx");
// Remove it from the list of available nodes
nodes.remove(blockedNode);
@ -377,7 +377,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
ListenableActionFuture<DeleteSnapshotResponse> deleteSnapshotResponseFuture = internalCluster().client(nodes.get(0)).admin().cluster().prepareDeleteSnapshot("test-repo", "test-snap").execute();
// Make sure that abort makes some progress
Thread.sleep(100);
unblockNode(blockedNode);
unblockNode("test-repo", blockedNode);
logger.info("--> stopping node [{}]", blockedNode);
stopNode(blockedNode);
try {

View File

@ -50,7 +50,6 @@ import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.collect.ImmutableOpenMap;
@ -73,9 +72,11 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
@ -917,7 +918,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
logger.info("--> truncate snapshot file to make it unreadable");
Path snapshotPath = repo.resolve("snap-test-snap-1.dat");
Path snapshotPath = repo.resolve("snap-test-snap-1-" + createSnapshotResponse.getSnapshotInfo().snapshotId().getUUID() + ".dat");
try(SeekableByteChannel outChan = Files.newByteChannel(snapshotPath, StandardOpenOption.WRITE)) {
outChan.truncate(randomInt(10));
}
@ -1120,7 +1121,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L));
// Pick one node and block it
String blockedNode = blockNodeWithIndex("test-idx");
String blockedNode = blockNodeWithIndex("test-repo", "test-idx");
logger.info("--> snapshot");
client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get();
@ -1133,7 +1134,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
client().admin().indices().prepareUpdateSettings("test-idx").setSettings(excludeSettings).get();
logger.info("--> unblocking blocked node");
unblockNode(blockedNode);
unblockNode("test-repo", blockedNode);
logger.info("--> waiting for completion");
SnapshotInfo snapshotInfo = waitForCompletion("test-repo", "test-snap", TimeValue.timeValueSeconds(600));
logger.info("Number of failed shards [{}]", snapshotInfo.shardFailures().size());
@ -1183,7 +1184,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L));
// Pick one node and block it
String blockedNode = blockNodeWithIndex("test-idx");
String blockedNode = blockNodeWithIndex("test-repo", "test-idx");
logger.info("--> snapshot");
client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get();
@ -1215,7 +1216,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
.setType("fs").setSettings(Settings.builder().put("location", repositoryLocation.resolve("test"))));
logger.info("--> unblocking blocked node");
unblockNode(blockedNode);
unblockNode("test-repo", blockedNode);
logger.info("--> waiting for completion");
SnapshotInfo snapshotInfo = waitForCompletion("test-repo", "test-snap", TimeValue.timeValueSeconds(600));
logger.info("Number of failed shards [{}]", snapshotInfo.shardFailures().size());
@ -1436,7 +1437,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
assertThat(client.prepareSearch("test-idx").setSize(0).get().getHits().totalHits(), equalTo(100L));
// Pick one node and block it
String blockedNode = blockNodeWithIndex("test-idx");
String blockedNode = blockNodeWithIndex("test-repo", "test-idx");
logger.info("--> snapshot");
client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get();
@ -1477,7 +1478,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
assertThat(snapshotInfo.state(), equalTo(SnapshotState.IN_PROGRESS));
logger.info("--> unblocking blocked node");
unblockNode(blockedNode);
unblockNode("test-repo", blockedNode);
snapshotInfo = waitForCompletion("test-repo", "test-snap", TimeValue.timeValueSeconds(600));
logger.info("Number of failed shards [{}]", snapshotInfo.shardFailures().size());
@ -1904,7 +1905,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
} finally {
if (initBlocking) {
logger.info("--> unblock running master node");
unblockNode(internalCluster().getMasterName());
unblockNode("test-repo", internalCluster().getMasterName());
} else {
logger.info("--> unblock all data nodes");
unblockAllDataNodes("test-repo");
@ -1993,14 +1994,17 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
Client client = client();
logger.info("--> creating repository");
assertAcked(client.admin().cluster().preparePutRepository("test-repo")
final String repositoryName = "test-repo";
assertAcked(client.admin().cluster().preparePutRepository(repositoryName)
.setType("mock").setSettings(Settings.builder()
.put("location", randomRepoPath())
.put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
));
createIndex("test-idx");
logger.info("--> create the index");
final String idxName = "test-idx";
createIndex(idxName);
ensureGreen();
ClusterService clusterService = internalCluster().getInstance(ClusterService.class, internalCluster().getMasterName());
@ -2008,7 +2012,8 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
final CountDownLatch countDownLatch = new CountDownLatch(1);
logger.info("--> snapshot");
CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").get();
final String snapshotName = "test-snap";
CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot(repositoryName, snapshotName).setWaitForCompletion(true).setIndices(idxName).get();
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
@ -2020,11 +2025,18 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
public ClusterState execute(ClusterState currentState) {
// Simulate orphan snapshot
ImmutableOpenMap.Builder<ShardId, ShardSnapshotStatus> shards = ImmutableOpenMap.builder();
shards.put(new ShardId("test-idx", "_na_", 0), new ShardSnapshotStatus("unknown-node", State.ABORTED));
shards.put(new ShardId("test-idx", "_na_", 1), new ShardSnapshotStatus("unknown-node", State.ABORTED));
shards.put(new ShardId("test-idx", "_na_", 2), new ShardSnapshotStatus("unknown-node", State.ABORTED));
shards.put(new ShardId(idxName, "_na_", 0), new ShardSnapshotStatus("unknown-node", State.ABORTED));
shards.put(new ShardId(idxName, "_na_", 1), new ShardSnapshotStatus("unknown-node", State.ABORTED));
shards.put(new ShardId(idxName, "_na_", 2), new ShardSnapshotStatus("unknown-node", State.ABORTED));
List<Entry> entries = new ArrayList<>();
entries.add(new Entry(new SnapshotId("test-repo", "test-snap"), true, false, State.ABORTED, Collections.singletonList("test-idx"), System.currentTimeMillis(), shards.build()));
entries.add(new Entry(new Snapshot(repositoryName,
createSnapshotResponse.getSnapshotInfo().snapshotId()),
true,
false,
State.ABORTED,
Collections.singletonList(idxName),
System.currentTimeMillis(),
shards.build()));
return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(Collections.unmodifiableList(entries))).build();
}
@ -2042,8 +2054,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
countDownLatch.await();
logger.info("--> try deleting the orphan snapshot");
assertAcked(client.admin().cluster().prepareDeleteSnapshot("test-repo", "test-snap").get("10s"));
assertAcked(client.admin().cluster().prepareDeleteSnapshot(repositoryName, snapshotName).get("10s"));
}
private boolean waitForIndex(final String index, TimeValue timeout) throws InterruptedException {
@ -2143,33 +2154,14 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
.put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
try {
client.admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("_foo").get();
fail("shouldn't be here");
} catch (InvalidSnapshotNameException ex) {
assertThat(ex.getMessage(), containsString("Invalid snapshot name"));
}
try {
client.admin().cluster().prepareCreateSnapshot("test-repo", "_foo").get();
fail("shouldn't be here");
} catch (InvalidSnapshotNameException ex) {
assertThat(ex.getMessage(), containsString("Invalid snapshot name"));
}
try {
client.admin().cluster().prepareDeleteSnapshot("test-repo", "_foo").get();
fail("shouldn't be here");
} catch (InvalidSnapshotNameException ex) {
assertThat(ex.getMessage(), containsString("Invalid snapshot name"));
}
try {
client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("_foo").get();
fail("shouldn't be here");
} catch (InvalidSnapshotNameException ex) {
assertThat(ex.getMessage(), containsString("Invalid snapshot name"));
}
expectThrows(InvalidSnapshotNameException.class,
() -> client.admin().cluster().prepareCreateSnapshot("test-repo", "_foo").get());
expectThrows(SnapshotMissingException.class,
() -> client.admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("_foo").get());
expectThrows(SnapshotMissingException.class,
() -> client.admin().cluster().prepareDeleteSnapshot("test-repo", "_foo").get());
expectThrows(SnapshotMissingException.class,
() -> client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("_foo").get());
}
public void testListCorruptedSnapshot() throws Exception {
@ -2199,7 +2191,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
logger.info("--> truncate snapshot file to make it unreadable");
Path snapshotPath = repo.resolve("snap-test-snap-2.dat");
Path snapshotPath = repo.resolve("snap-test-snap-2-" + createSnapshotResponse.getSnapshotInfo().snapshotId().getUUID() + ".dat");
try(SeekableByteChannel outChan = Files.newByteChannel(snapshotPath, StandardOpenOption.WRITE)) {
outChan.truncate(randomInt(10));
}
@ -2211,13 +2203,194 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
assertThat(snapshotInfos.size(), equalTo(1));
assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS));
assertThat(snapshotInfos.get(0).name(), equalTo("test-snap-1"));
assertThat(snapshotInfos.get(0).snapshotId().getName(), equalTo("test-snap-1"));
try {
client.admin().cluster().prepareGetSnapshots("test-repo").setIgnoreUnavailable(false).get().getSnapshots();
} catch (SnapshotException ex) {
assertThat(ex.snapshot().getRepository(), equalTo("test-repo"));
assertThat(ex.snapshot().getSnapshot(), equalTo("test-snap-2"));
assertThat(ex.getRepositoryName(), equalTo("test-repo"));
assertThat(ex.getSnapshotName(), equalTo("test-snap-2"));
}
}
public void testCannotCreateSnapshotsWithSameName() throws Exception {
final String repositoryName = "test-repo";
final String snapshotName = "test-snap";
final String indexName = "test-idx";
final Client client = client();
final Path repo = randomRepoPath();
logger.info("--> creating repository at {}", repo.toAbsolutePath());
assertAcked(client.admin().cluster().preparePutRepository(repositoryName)
.setType("fs").setSettings(Settings.builder()
.put("location", repo)
.put("compress", false)
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
logger.info("--> creating an index and indexing documents");
createIndex(indexName);
ensureGreen();
for (int i = 0; i < 10; i++) {
index(indexName, "doc", Integer.toString(i), "foo", "bar" + i);
}
refresh();
logger.info("--> take first snapshot");
CreateSnapshotResponse createSnapshotResponse = client.admin()
.cluster()
.prepareCreateSnapshot(repositoryName, snapshotName)
.setWaitForCompletion(true)
.setIndices(indexName)
.get();
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
logger.info("--> index more documents");
for (int i = 10; i < 20; i++) {
index(indexName, "doc", Integer.toString(i), "foo", "bar" + i);
}
refresh();
logger.info("--> second snapshot of the same name should fail");
try {
createSnapshotResponse = client.admin()
.cluster()
.prepareCreateSnapshot(repositoryName, snapshotName)
.setWaitForCompletion(true)
.setIndices(indexName)
.get();
fail("should not be allowed to create a snapshot with the same name as an already existing snapshot: " +
createSnapshotResponse.getSnapshotInfo().snapshotId());
} catch (SnapshotCreationException e) {
assertThat(e.getMessage(), containsString("snapshot with the same name already exists"));
}
logger.info("--> delete the first snapshot");
client.admin().cluster().prepareDeleteSnapshot(repositoryName, snapshotName).get();
logger.info("--> try creating a snapshot with the same name, now it should work because the first one was deleted");
createSnapshotResponse = client.admin()
.cluster()
.prepareCreateSnapshot(repositoryName, snapshotName)
.setWaitForCompletion(true)
.setIndices(indexName)
.get();
assertThat(createSnapshotResponse.getSnapshotInfo().snapshotId().getName(), equalTo(snapshotName));
}
public void testGetSnapshotsRequest() throws Exception {
final String repositoryName = "test-repo";
final String indexName = "test-idx";
final Client client = client();
final Path repo = randomRepoPath();
logger.info("--> creating repository at {}", repo.toAbsolutePath());
assertAcked(client.admin().cluster().preparePutRepository(repositoryName)
.setType("mock").setSettings(Settings.builder()
.put("location", repo)
.put("compress", false)
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
.put("wait_after_unblock", 200)));
logger.info("--> get snapshots on an empty repository");
expectThrows(SnapshotMissingException.class, () -> client.admin()
.cluster()
.prepareGetSnapshots(repositoryName)
.addSnapshots("non-existent-snapshot")
.get());
// with ignore unavailable set to true, should not throw an exception
GetSnapshotsResponse getSnapshotsResponse = client.admin()
.cluster()
.prepareGetSnapshots(repositoryName)
.setIgnoreUnavailable(true)
.addSnapshots("non-existent-snapshot")
.get();
assertThat(getSnapshotsResponse.getSnapshots().size(), equalTo(0));
logger.info("--> creating an index and indexing documents");
// Create index on 2 nodes and make sure each node has a primary by setting no replicas
assertAcked(prepareCreate(indexName, 1, Settings.builder().put("number_of_replicas", 0)));
ensureGreen();
for (int i = 0; i < 10; i++) {
index(indexName, "doc", Integer.toString(i), "foo", "bar" + i);
}
refresh();
final int numSnapshots = randomIntBetween(1, 3) + 1;
logger.info("--> take {} snapshot(s)", numSnapshots);
final String[] snapshotNames = new String[numSnapshots];
for (int i = 0; i < numSnapshots - 1; i++) {
final String snapshotName = randomAsciiOfLength(8).toLowerCase(Locale.ROOT);
CreateSnapshotResponse createSnapshotResponse = client.admin()
.cluster()
.prepareCreateSnapshot(repositoryName, snapshotName)
.setWaitForCompletion(true)
.setIndices(indexName)
.get();
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
snapshotNames[i] = snapshotName;
}
logger.info("--> take another snapshot to be in-progress");
// add documents so there are data files to block on
for (int i = 10; i < 20; i++) {
index(indexName, "doc", Integer.toString(i), "foo", "bar" + i);
}
refresh();
final String inProgressSnapshot = randomAsciiOfLength(8).toLowerCase(Locale.ROOT);
snapshotNames[numSnapshots - 1] = inProgressSnapshot;
// block a node so the create snapshot operation can remain in progress
final String blockedNode = blockNodeWithIndex(repositoryName, indexName);
client.admin().cluster().prepareCreateSnapshot(repositoryName, inProgressSnapshot)
.setWaitForCompletion(false)
.setIndices(indexName)
.get();
waitForBlock(blockedNode, repositoryName, TimeValue.timeValueSeconds(60)); // wait for block to kick in
logger.info("--> get all snapshots with a current in-progress");
// with ignore unavailable set to true, should not throw an exception
getSnapshotsResponse = client.admin().cluster()
.prepareGetSnapshots(repositoryName)
.addSnapshots("_all")
.get();
List<String> sortedNames = Arrays.asList(snapshotNames);
Collections.sort(sortedNames);
assertThat(getSnapshotsResponse.getSnapshots().size(), equalTo(numSnapshots));
assertThat(getSnapshotsResponse.getSnapshots().stream()
.map(s -> s.snapshotId().getName())
.sorted()
.collect(Collectors.toList()), equalTo(sortedNames));
getSnapshotsResponse = client.admin().cluster()
.prepareGetSnapshots(repositoryName)
.addSnapshots(snapshotNames)
.get();
sortedNames = Arrays.asList(snapshotNames);
Collections.sort(sortedNames);
assertThat(getSnapshotsResponse.getSnapshots().size(), equalTo(numSnapshots));
assertThat(getSnapshotsResponse.getSnapshots().stream()
.map(s -> s.snapshotId().getName())
.sorted()
.collect(Collectors.toList()), equalTo(sortedNames));
logger.info("--> make sure duplicates are not returned in the response");
String regexName = snapshotNames[randomIntBetween(0, numSnapshots - 1)];
final int splitPos = regexName.length() / 2;
final String firstRegex = regexName.substring(0, splitPos) + "*";
final String secondRegex = "*" + regexName.substring(splitPos);
getSnapshotsResponse = client.admin().cluster()
.prepareGetSnapshots(repositoryName)
.addSnapshots(snapshotNames)
.addSnapshots(firstRegex, secondRegex)
.get();
assertThat(getSnapshotsResponse.getSnapshots().size(), equalTo(numSnapshots));
assertThat(getSnapshotsResponse.getSnapshots().stream()
.map(s -> s.snapshotId().getName())
.sorted()
.collect(Collectors.toList()), equalTo(sortedNames));
unblockNode(repositoryName, blockedNode); // unblock node
waitForCompletion(repositoryName, inProgressSnapshot, TimeValue.timeValueSeconds(60));
}
}

View File

@ -86,7 +86,7 @@ public class SnapshotBackwardsCompatibilityIT extends ESBackcompatTestCase {
counts[i] = client().prepareSearch(indices[i]).setSize(0).get().getHits().totalHits();
}
logger.info("--> snapshot subset of indices before upgrage");
logger.info("--> snapshot subset of indices before upgrade");
CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1").setWaitForCompletion(true).setIndices("index_before_*").get();
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));

View File

@ -0,0 +1,57 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.snapshots;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.io.stream.ByteBufferStreamInput;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.nio.ByteBuffer;
import static org.hamcrest.CoreMatchers.equalTo;
/**
* Tests for the {@link Snapshot} class.
*/
public class SnapshotTests extends ESTestCase {
public void testSnapshotEquals() {
final SnapshotId snapshotId = new SnapshotId("snap", UUIDs.randomBase64UUID());
final Snapshot original = new Snapshot("repo", snapshotId);
final Snapshot expected = new Snapshot(original.getRepository(), original.getSnapshotId());
assertThat(expected, equalTo(original));
assertThat(expected.getRepository(), equalTo(original.getRepository()));
assertThat(expected.getSnapshotId(), equalTo(original.getSnapshotId()));
assertThat(expected.getSnapshotId().getName(), equalTo(original.getSnapshotId().getName()));
assertThat(expected.getSnapshotId().getUUID(), equalTo(original.getSnapshotId().getUUID()));
}
public void testSerialization() throws IOException {
final SnapshotId snapshotId = new SnapshotId(randomAsciiOfLength(randomIntBetween(2, 8)), UUIDs.randomBase64UUID());
final Snapshot original = new Snapshot(randomAsciiOfLength(randomIntBetween(2, 8)), snapshotId);
final BytesStreamOutput out = new BytesStreamOutput();
original.writeTo(out);
final ByteBufferStreamInput in = new ByteBufferStreamInput(ByteBuffer.wrap(out.bytes().toBytes()));
assertThat(new Snapshot(in), equalTo(original));
}
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.snapshots.mockstore;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.blobstore.BlobContainer;
import org.elasticsearch.common.blobstore.BlobMetaData;
@ -174,6 +174,10 @@ public class MockRepository extends FsRepository {
blockOnControlFiles = blocked;
}
public boolean blockOnDataFiles() {
return blockOnDataFiles;
}
public synchronized void unblockExecution() {
blocked = false;
// Clean blocking flags, so we wouldn't try to block again

View File

@ -24,7 +24,7 @@ import com.microsoft.azure.storage.StorageException;
import org.elasticsearch.cloud.azure.blobstore.AzureBlobStore;
import org.elasticsearch.cloud.azure.storage.AzureStorageService.Storage;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.blobstore.BlobPath;
import org.elasticsearch.common.blobstore.BlobStore;
@ -166,7 +166,7 @@ public class AzureRepository extends BlobStoreRepository {
super.initializeSnapshot(snapshotId, indices, metaData);
} catch (StorageException | URISyntaxException e) {
logger.warn("can not initialize container [{}]: [{}]", blobStore.container(), e.getMessage());
throw new SnapshotCreationException(snapshotId, e);
throw new SnapshotCreationException(repositoryName, snapshotId, e);
}
}

View File

@ -94,8 +94,15 @@ public class AzureStorageServiceMock extends AbstractLifecycleComponent<AzureSto
public Map<String, BlobMetaData> listBlobsByPrefix(String account, LocationMode mode, String container, String keyPath, String prefix) {
MapBuilder<String, BlobMetaData> blobsBuilder = MapBuilder.newMapBuilder();
for (String blobName : blobs.keySet()) {
if (startsWithIgnoreCase(blobName, prefix)) {
blobsBuilder.put(blobName, new PlainBlobMetaData(blobName, blobs.get(blobName).size()));
final String checkBlob;
if (keyPath != null) {
// strip off key path from the beginning of the blob name
checkBlob = blobName.replace(keyPath, "");
} else {
checkBlob = blobName;
}
if (startsWithIgnoreCase(checkBlob, prefix)) {
blobsBuilder.put(blobName, new PlainBlobMetaData(checkBlob, blobs.get(blobName).size()));
}
}
return blobsBuilder.immutableMap();

View File

@ -19,11 +19,15 @@
package org.elasticsearch.common.io;
import org.elasticsearch.common.Nullable;
import org.junit.Assert;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.zip.ZipEntry;
import java.util.zip.ZipInputStream;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileExists;
import static org.hamcrest.CoreMatchers.equalTo;
@ -50,4 +54,44 @@ public class FileTestUtils {
Assert.assertThat(fileContent.trim(), equalTo(expected.trim()));
}
}
/**
* Unzip a zip file to a destination directory. If the zip file does not exist, an IOException is thrown.
* If the destination directory does not exist, it will be created.
*
* @param zip zip file to unzip
* @param destDir directory to unzip the file to
* @param prefixToRemove the (optional) prefix in the zip file path to remove when writing to the destination directory
* @throws IOException if zip file does not exist, or there was an error reading from the zip file or
* writing to the destination directory
*/
public static void unzip(final Path zip, final Path destDir, @Nullable final String prefixToRemove) throws IOException {
if (Files.notExists(zip)) {
throw new IOException("[" + zip + "] zip file must exist");
}
Files.createDirectories(destDir);
try (final ZipInputStream zipInput = new ZipInputStream(Files.newInputStream(zip))) {
ZipEntry entry;
while ((entry = zipInput.getNextEntry()) != null) {
final String entryPath;
if (prefixToRemove != null) {
if (entry.getName().startsWith(prefixToRemove)) {
entryPath = entry.getName().substring(prefixToRemove.length());
} else {
throw new IOException("prefix not found: " + prefixToRemove);
}
} else {
entryPath = entry.getName();
}
final Path path = Paths.get(destDir.toString(), entryPath);
if (entry.isDirectory()) {
Files.createDirectories(path);
} else {
Files.copy(zipInput, path);
}
zipInput.closeEntry();
}
}
}
}

View File

@ -54,6 +54,7 @@ import org.junit.Before;
import org.junit.BeforeClass;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@ -171,14 +172,15 @@ public abstract class ESSingleNodeTestCase extends ESTestCase {
}
private Node newNode() {
final Path tempDir = createTempDir();
Settings settings = Settings.builder()
.put(ClusterName.CLUSTER_NAME_SETTING.getKey(), InternalTestCluster.clusterName("single-node-cluster", randomLong()))
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
.put(Environment.PATH_HOME_SETTING.getKey(), tempDir)
.put(Environment.PATH_REPO_SETTING.getKey(), tempDir.resolve("repo"))
// TODO: use a consistent data path for custom paths
// This needs to tie into the ESIntegTestCase#indexSettings() method
.put(Environment.PATH_SHARED_DATA_SETTING.getKey(), createTempDir().getParent())
.put("node.name", nodeName())
.put("script.inline", "true")
.put("script.stored", "true")
.put(EsExecutors.PROCESSORS_SETTING.getKey(), 1) // limit the number of threads created