Skip to content

Commit

Permalink
[7.x] Add support for snapshot and restore to data streams (#57675) (#…
Browse files Browse the repository at this point in the history
…58371)

* Add support for snapshot and restore to data streams (#57675)

This change adds support for including data streams in snapshots.
Names are provided in indices field (the same way as in other APIs), wildcards are supported.
If rename pattern is specified it renames both data streams and backing indices.
It also adds test to make sure SLM works correctly.

Closes #57127

Relates to #53100

* version fix

* compilation fix

* compilation fix

* remove unused changes

* compilation fix

* test fix
  • Loading branch information
probakowski committed Jun 19, 2020
1 parent bf8641a commit a44dad9
Show file tree
Hide file tree
Showing 24 changed files with 813 additions and 82 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -98,11 +98,11 @@ public void deleteAllComposableTemplates() {
}

public void testBasicScenario() throws Exception {
createIndexTemplate("id1", "metrics-foo*", "@timestamp1");
createIndexTemplate("id1", "@timestamp1", "metrics-foo*");
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request("metrics-foo");
client().admin().indices().createDataStream(createDataStreamRequest).get();

createIndexTemplate("id2", "metrics-bar*", "@timestamp2");
createIndexTemplate("id2", "@timestamp2", "metrics-bar*");
createDataStreamRequest = new CreateDataStreamAction.Request("metrics-bar");
client().admin().indices().createDataStream(createDataStreamRequest).get();

Expand Down Expand Up @@ -186,7 +186,7 @@ public void testBasicScenario() throws Exception {
}

public void testOtherWriteOps() throws Exception {
createIndexTemplate("id", "metrics-foobar*", "@timestamp1");
createIndexTemplate("id", "@timestamp1", "metrics-foobar*");
String dataStreamName = "metrics-foobar";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
Expand Down Expand Up @@ -351,8 +351,8 @@ public void testTimeStampValidationInvalidFieldMapping() throws Exception {
"[date, date_nanos], but instead found type [keyword]"));
}

public void testDataStreamsResolvability() throws Exception {
createIndexTemplate("id", "logs-*", "ts");
public void testResolvabilityOfDataStreamsInAPIs() throws Exception {
createIndexTemplate("id", "ts", "logs-*");
String dataStreamName = "logs-foobar";
CreateDataStreamAction.Request request = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(request).actionGet();
Expand Down Expand Up @@ -413,7 +413,7 @@ public void testDataStreamsResolvability() throws Exception {
}

public void testCannotDeleteComposableTemplateUsedByDataStream() throws Exception {
createIndexTemplate("id", "metrics-foobar*", "@timestamp1");
createIndexTemplate("id", "@timestamp1", "metrics-foobar*");
String dataStreamName = "metrics-foobar-baz";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
Expand All @@ -436,7 +436,7 @@ public void testCannotDeleteComposableTemplateUsedByDataStream() throws Exceptio
}

public void testAliasActionsFailOnDataStreams() throws Exception {
createIndexTemplate("id1", "metrics-foo*", "@timestamp1");
createIndexTemplate("id1", "@timestamp1", "metrics-foo*");
String dataStreamName = "metrics-foo";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
Expand All @@ -449,7 +449,7 @@ public void testAliasActionsFailOnDataStreams() throws Exception {
}

public void testAliasActionsFailOnDataStreamBackingIndices() throws Exception {
createIndexTemplate("id1", "metrics-foo*", "@timestamp1");
createIndexTemplate("id1", "@timestamp1", "metrics-foo*");
String dataStreamName = "metrics-foo";
CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName);
client().admin().indices().createDataStream(createDataStreamRequest).get();
Expand Down Expand Up @@ -539,11 +539,11 @@ private static void expectFailure(String dataStreamName, ThrowingRunnable runnab
"] matches a data stream, specify the corresponding concrete indices instead."));
}

static void createIndexTemplate(String id, String pattern, String timestampFieldName) throws IOException {
public static void createIndexTemplate(String id, String timestampFieldName, String... pattern) throws IOException {
PutComposableIndexTemplateAction.Request request = new PutComposableIndexTemplateAction.Request(id);
request.indexTemplate(
new ComposableIndexTemplate(
Collections.singletonList(pattern),
Arrays.asList(pattern),
new Template(null,
new CompressedXContent(MetadataCreateDataStreamServiceTests.generateMapping(timestampFieldName)), null),
null, null, null, null,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,218 @@
/*
* 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.ResourceNotFoundException;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse;
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
import org.elasticsearch.action.admin.indices.datastream.CreateDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.datastream.GetDataStreamAction;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.indices.DataStreamIT;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.SearchHit;
import org.junit.Before;

import java.nio.file.Path;
import java.util.Collections;
import java.util.List;
import java.util.Map;

public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {

private static final String DS_BACKING_INDEX_NAME = DataStream.getDefaultBackingIndexName("ds", 1);
private static final String DS2_BACKING_INDEX_NAME = DataStream.getDefaultBackingIndexName("ds2", 1);
private static final Map<String, Integer> DOCUMENT_SOURCE = Collections.singletonMap("@timestamp", 123);
public static final String REPO = "repo";
public static final String SNAPSHOT = "snap";
private Client client;

private String id;

@Before
public void setup() throws Exception {
client = client();
Path location = randomRepoPath();
createRepository(REPO, "fs", location);

DataStreamIT.createIndexTemplate("t1", "@timestamp", "ds", "other-ds");

CreateDataStreamAction.Request request = new CreateDataStreamAction.Request("ds");
AcknowledgedResponse response = client.admin().indices().createDataStream(request).get();
assertTrue(response.isAcknowledged());

request = new CreateDataStreamAction.Request("other-ds");
response = client.admin().indices().createDataStream(request).get();
assertTrue(response.isAcknowledged());

IndexResponse indexResponse = client.prepareIndex("ds", "_doc")
.setOpType(DocWriteRequest.OpType.CREATE)
.setSource(DOCUMENT_SOURCE)
.get();
assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult());
id = indexResponse.getId();
}

public void testSnapshotAndRestore() throws Exception {
CreateSnapshotResponse createSnapshotResponse = client.admin().cluster()
.prepareCreateSnapshot(REPO, SNAPSHOT)
.setWaitForCompletion(true)
.setIndices("ds")
.setIncludeGlobalState(false)
.get();

RestStatus status = createSnapshotResponse.getSnapshotInfo().status();
assertEquals(RestStatus.OK, status);

GetSnapshotsResponse snapshot = client.admin().cluster().prepareGetSnapshots(REPO).setSnapshots(SNAPSHOT).get();
List<SnapshotInfo> snap = snapshot.getSnapshots();
assertEquals(1, snap.size());
assertEquals(Collections.singletonList(DS_BACKING_INDEX_NAME), snap.get(0).indices());

assertTrue(client.admin().indices().deleteDataStream(new DeleteDataStreamAction.Request("ds")).get().isAcknowledged());

RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster()
.prepareRestoreSnapshot(REPO, SNAPSHOT)
.setWaitForCompletion(true)
.setIndices("ds")
.get();

assertEquals(1, restoreSnapshotResponse.getRestoreInfo().successfulShards());

assertEquals(DOCUMENT_SOURCE, client.prepareGet(DS_BACKING_INDEX_NAME, "_doc", id).get().getSourceAsMap());
SearchHit[] hits = client.prepareSearch("ds").get().getHits().getHits();
assertEquals(1, hits.length);
assertEquals(DOCUMENT_SOURCE, hits[0].getSourceAsMap());

GetDataStreamAction.Response ds = client.admin().indices().getDataStreams(new GetDataStreamAction.Request("ds")).get();
assertEquals(1, ds.getDataStreams().size());
assertEquals(1, ds.getDataStreams().get(0).getIndices().size());
assertEquals(DS_BACKING_INDEX_NAME, ds.getDataStreams().get(0).getIndices().get(0).getName());
}

public void testRename() throws Exception {
CreateSnapshotResponse createSnapshotResponse = client.admin().cluster()
.prepareCreateSnapshot(REPO, SNAPSHOT)
.setWaitForCompletion(true)
.setIndices("ds")
.setIncludeGlobalState(false)
.get();

RestStatus status = createSnapshotResponse.getSnapshotInfo().status();
assertEquals(RestStatus.OK, status);

expectThrows(SnapshotRestoreException.class, () -> client.admin().cluster()
.prepareRestoreSnapshot(REPO, SNAPSHOT)
.setWaitForCompletion(true)
.setIndices("ds")
.get());

client.admin().cluster()
.prepareRestoreSnapshot(REPO, SNAPSHOT)
.setWaitForCompletion(true)
.setIndices("ds")
.setRenamePattern("ds")
.setRenameReplacement("ds2")
.get();

GetDataStreamAction.Response ds = client.admin().indices().getDataStreams(new GetDataStreamAction.Request("ds2")).get();
assertEquals(1, ds.getDataStreams().size());
assertEquals(1, ds.getDataStreams().get(0).getIndices().size());
assertEquals(DS2_BACKING_INDEX_NAME, ds.getDataStreams().get(0).getIndices().get(0).getName());
assertEquals(DOCUMENT_SOURCE, client.prepareSearch("ds2").get().getHits().getHits()[0].getSourceAsMap());
assertEquals(DOCUMENT_SOURCE, client.prepareGet(DS2_BACKING_INDEX_NAME, "_doc", id).get().getSourceAsMap());
}

public void testWildcards() throws Exception {
CreateSnapshotResponse createSnapshotResponse = client.admin().cluster()
.prepareCreateSnapshot(REPO, "snap2")
.setWaitForCompletion(true)
.setIndices("d*")
.setIncludeGlobalState(false)
.get();

RestStatus status = createSnapshotResponse.getSnapshotInfo().status();
assertEquals(RestStatus.OK, status);

RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster()
.prepareRestoreSnapshot(REPO, "snap2")
.setWaitForCompletion(true)
.setIndices("d*")
.setRenamePattern("ds")
.setRenameReplacement("ds2")
.get();

assertEquals(RestStatus.OK, restoreSnapshotResponse.status());

GetDataStreamAction.Response ds = client.admin().indices().getDataStreams(new GetDataStreamAction.Request("ds2")).get();
assertEquals(1, ds.getDataStreams().size());
assertEquals(1, ds.getDataStreams().get(0).getIndices().size());
assertEquals(DS2_BACKING_INDEX_NAME, ds.getDataStreams().get(0).getIndices().get(0).getName());
}

public void testDataStreamNotStoredWhenIndexRequested() throws Exception {
CreateSnapshotResponse createSnapshotResponse = client.admin().cluster()
.prepareCreateSnapshot(REPO, "snap2")
.setWaitForCompletion(true)
.setIndices(DS_BACKING_INDEX_NAME)
.setIncludeGlobalState(false)
.get();

RestStatus status = createSnapshotResponse.getSnapshotInfo().status();
assertEquals(RestStatus.OK, status);
expectThrows(Exception.class, () -> client.admin().cluster()
.prepareRestoreSnapshot(REPO, "snap2")
.setWaitForCompletion(true)
.setIndices("ds")
.get());
}

public void testDataStreamNotRestoredWhenIndexRequested() throws Exception {
CreateSnapshotResponse createSnapshotResponse = client.admin().cluster()
.prepareCreateSnapshot(REPO, "snap2")
.setWaitForCompletion(true)
.setIndices("ds")
.setIncludeGlobalState(false)
.get();

RestStatus status = createSnapshotResponse.getSnapshotInfo().status();
assertEquals(RestStatus.OK, status);

assertTrue(client.admin().indices().deleteDataStream(new DeleteDataStreamAction.Request("ds")).get().isAcknowledged());

RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster()
.prepareRestoreSnapshot(REPO, "snap2")
.setWaitForCompletion(true)
.setIndices(".ds-ds-*")
.get();

assertEquals(RestStatus.OK, restoreSnapshotResponse.status());

GetDataStreamAction.Request getRequest = new GetDataStreamAction.Request("ds");
expectThrows(ResourceNotFoundException.class, () -> client.admin().indices().getDataStreams(getRequest).actionGet());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse;
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
Expand All @@ -34,6 +35,7 @@
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptResponse;
import org.elasticsearch.action.admin.indices.datastream.DeleteDataStreamAction;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
Expand All @@ -52,6 +54,7 @@
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.SnapshotsInProgress.State;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.MappingMetadata;
import org.elasticsearch.cluster.metadata.MetadataIndexStateService;
Expand All @@ -78,6 +81,7 @@
import org.elasticsearch.index.engine.EngineTestCase;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.DataStreamIT;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.InvalidIndexNameException;
import org.elasticsearch.ingest.IngestTestPlugin;
Expand Down Expand Up @@ -132,6 +136,7 @@
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertRequestBuilderThrows;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty;
Expand Down Expand Up @@ -2408,6 +2413,61 @@ public void testRecreateBlocksOnRestore() throws Exception {
}
}

public void testDeleteDataStreamDuringSnapshot() throws Exception {
Client client = client();

logger.info("--> creating repository");

assertAcked(client.admin().cluster().preparePutRepository("test-repo")
.setType("mock").setSettings(Settings.builder()
.put("location", randomRepoPath())
.put("compress", randomBoolean())
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
.put("block_on_data", true)));


String dataStream = "test-ds";
DataStreamIT.createIndexTemplate("dst", "@timestamp", dataStream);

logger.info("--> indexing some data");
for (int i = 0; i < 100; i++) {
client.prepareIndex(dataStream, "_doc")
.setOpType(DocWriteRequest.OpType.CREATE)
.setId(Integer.toString(i))
.setSource(Collections.singletonMap("k", "v"))
.execute().actionGet();
}
refresh();

assertThat(client.prepareSearch(dataStream).setSize(0).get().getHits().getTotalHits().value, equalTo(100L));

logger.info("--> snapshot");
ActionFuture<CreateSnapshotResponse> future = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap")
.setIndices(dataStream).setWaitForCompletion(true).setPartial(false).execute();
logger.info("--> wait for block to kick in");
waitForBlockOnAnyDataNode("test-repo", TimeValue.timeValueMinutes(1));

// non-partial snapshots do not allow delete operations on data streams where snapshot has not been completed
try {
logger.info("--> delete index while non-partial snapshot is running");
client.admin().indices().deleteDataStream(new DeleteDataStreamAction.Request(dataStream)).actionGet();
fail("Expected deleting index to fail during snapshot");
} catch (SnapshotInProgressException e) {
assertThat(e.getMessage(), containsString("Cannot delete data streams that are being snapshotted: [test-ds"));
} finally {
logger.info("--> unblock all data nodes");
unblockAllDataNodes("test-repo");
}
logger.info("--> waiting for snapshot to finish");
CreateSnapshotResponse createSnapshotResponse = future.get();

logger.info("Snapshot successfully completed");
SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo();
assertThat(snapshotInfo.state(), equalTo((SnapshotState.SUCCESS)));
assertThat(snapshotInfo.dataStreams(), contains(dataStream));
assertThat(snapshotInfo.indices(), contains(DataStream.getDefaultBackingIndexName(dataStream, 1)));
}

public void testCloseOrDeleteIndexDuringSnapshot() throws Exception {
disableRepoConsistencyCheck("This test intentionally leaves a broken repository");

Expand Down

0 comments on commit a44dad9

Please sign in to comment.