Skip to content

Commit

Permalink
Connecting the reindex data stream persistent task to ReindexDataStre…
Browse files Browse the repository at this point in the history
…amIndexAction (#118978) (#119032)

* Connecting the reindex data stream persistent task to ReindexDataStreamIndexAction (#118978)

* removing java 21 syntax

* [CI] Auto commit changes from spotless

* removing java 21 syntax

* removing code that doesnt work in 7.17

* expecting no upgrades in clusters of the same major versoin

* correcting method name after merge

---------

Co-authored-by: elasticsearchmachine <infra-root+elasticsearchmachine@elastic.co>
  • Loading branch information
masseyke and elasticsearchmachine authored Dec 19, 2024
1 parent 3c0da86 commit aa203af
Show file tree
Hide file tree
Showing 2 changed files with 269 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,15 @@

import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.rollover.RolloverAction;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.datastreams.GetDataStreamAction;
import org.elasticsearch.action.datastreams.ModifyDataStreamsAction;
import org.elasticsearch.action.support.CountDownActionListener;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.DataStreamAction;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.index.Index;
Expand All @@ -20,7 +27,10 @@
import org.elasticsearch.persistent.PersistentTasksExecutor;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.migrate.action.ReindexDataStreamIndexAction;

import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;

Expand Down Expand Up @@ -72,22 +82,109 @@ protected void nodeOperation(AllocatedPersistentTask task, ReindexDataStreamTask
reindexClient.execute(GetDataStreamAction.INSTANCE, request, ActionListener.wrap(response -> {
List<GetDataStreamAction.Response.DataStreamInfo> dataStreamInfos = response.getDataStreams();
if (dataStreamInfos.size() == 1) {
List<Index> indices = dataStreamInfos.get(0).getDataStream().getIndices();
List<Index> indicesToBeReindexed = indices.stream()
.filter(getReindexRequiredPredicate(clusterService.state().metadata()))
.toList();
reindexDataStreamTask.setPendingIndicesCount(indicesToBeReindexed.size());
for (Index index : indicesToBeReindexed) {
reindexDataStreamTask.incrementInProgressIndicesCount(index.getName());
// TODO This is just a placeholder. This is where the real data stream reindex logic will go
reindexDataStreamTask.reindexSucceeded(index.getName());
DataStream dataStream = dataStreamInfos.get(0).getDataStream();
if (getReindexRequiredPredicate(clusterService.state().metadata()).test(dataStream.getWriteIndex())) {
reindexClient.execute(
RolloverAction.INSTANCE,
new RolloverRequest(sourceDataStream, null),
ActionListener.wrap(
rolloverResponse -> reindexIndices(dataStream, reindexDataStreamTask, reindexClient, sourceDataStream),
e -> completeFailedPersistentTask(reindexDataStreamTask, e)
)
);
} else {
reindexIndices(dataStream, reindexDataStreamTask, reindexClient, sourceDataStream);
}

completeSuccessfulPersistentTask(reindexDataStreamTask);
} else {
completeFailedPersistentTask(reindexDataStreamTask, new ElasticsearchException("data stream does not exist"));
}
}, reindexDataStreamTask::markAsFailed));
}, exception -> completeFailedPersistentTask(reindexDataStreamTask, exception)));
}

private void reindexIndices(
DataStream dataStream,
ReindexDataStreamTask reindexDataStreamTask,
ExecuteWithHeadersClient reindexClient,
String sourceDataStream
) {
List<Index> indices = dataStream.getIndices();
List<Index> indicesToBeReindexed = indices.stream().filter(getReindexRequiredPredicate(clusterService.state().metadata())).toList();
reindexDataStreamTask.setPendingIndicesCount(indicesToBeReindexed.size());
// The CountDownActionListener is 1 more than the number of indices so that the count is not 0 if we have no indices
CountDownActionListener listener = new CountDownActionListener(indicesToBeReindexed.size() + 1, ActionListener.wrap(response1 -> {
completeSuccessfulPersistentTask(reindexDataStreamTask);
}, exception -> { completeFailedPersistentTask(reindexDataStreamTask, exception); }));
List<Index> indicesRemaining = Collections.synchronizedList(new ArrayList<>(indicesToBeReindexed));
final int maxConcurrentIndices = 1;
for (int i = 0; i < maxConcurrentIndices; i++) {
maybeProcessNextIndex(indicesRemaining, reindexDataStreamTask, reindexClient, sourceDataStream, listener);
}
// This takes care of the additional latch count referenced above:
listener.onResponse(null);
}

private void maybeProcessNextIndex(
List<Index> indicesRemaining,
ReindexDataStreamTask reindexDataStreamTask,
ExecuteWithHeadersClient reindexClient,
String sourceDataStream,
CountDownActionListener listener
) {
if (indicesRemaining.isEmpty()) {
return;
}
Index index;
try {
index = indicesRemaining.remove(0);
} catch (IndexOutOfBoundsException e) {
return;
}
reindexDataStreamTask.incrementInProgressIndicesCount(index.getName());
reindexClient.execute(
ReindexDataStreamIndexAction.INSTANCE,
new ReindexDataStreamIndexAction.Request(index.getName()),
ActionListener.wrap(response1 -> {
updateDataStream(sourceDataStream, index.getName(), response1.getDestIndex(), ActionListener.wrap(unused -> {
reindexDataStreamTask.reindexSucceeded(index.getName());
listener.onResponse(null);
maybeProcessNextIndex(indicesRemaining, reindexDataStreamTask, reindexClient, sourceDataStream, listener);
}, exception -> {
reindexDataStreamTask.reindexFailed(index.getName(), exception);
listener.onResponse(null);
}), reindexClient);
}, exception -> {
reindexDataStreamTask.reindexFailed(index.getName(), exception);
listener.onResponse(null);
})
);
}

private void updateDataStream(
String dataStream,
String oldIndex,
String newIndex,
ActionListener<Void> listener,
ExecuteWithHeadersClient reindexClient
) {
reindexClient.execute(
ModifyDataStreamsAction.INSTANCE,
new ModifyDataStreamsAction.Request(
TimeValue.MAX_VALUE,
TimeValue.MAX_VALUE,
List.of(DataStreamAction.removeBackingIndex(dataStream, oldIndex), DataStreamAction.addBackingIndex(dataStream, newIndex))
),
new ActionListener<>() {
@Override
public void onResponse(AcknowledgedResponse response) {
listener.onResponse(null);
}

@Override
public void onFailure(Exception e) {
listener.onFailure(e);
}
}
);
}

private void completeSuccessfulPersistentTask(ReindexDataStreamTask persistentTask) {
Expand All @@ -105,6 +202,9 @@ private TimeValue getTimeToLive(ReindexDataStreamTask reindexDataStreamTask) {
PersistentTasksCustomMetadata.PersistentTask<?> persistentTask = persistentTasksCustomMetadata.getTask(
reindexDataStreamTask.getPersistentTaskId()
);
if (persistentTask == null) {
return TimeValue.timeValueMillis(0);
}
PersistentTaskState state = persistentTask.getState();
final long completionTime;
if (state == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,19 +7,29 @@
package org.elasticsearch.upgrades;

import org.apache.http.util.EntityUtils;
import org.elasticsearch.Build;
import org.elasticsearch.Version;
import org.elasticsearch.client.Request;
import org.elasticsearch.client.Response;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
import org.elasticsearch.common.time.DateFormatter;
import org.elasticsearch.common.time.FormatNames;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.core.Booleans;
import org.elasticsearch.core.Strings;
import org.elasticsearch.xcontent.json.JsonXContent;
import org.hamcrest.Matchers;

import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.time.Instant;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;

import static org.elasticsearch.upgrades.IndexingIT.assertCount;
import static org.hamcrest.Matchers.equalTo;

public class DataStreamsUpgradeIT extends AbstractUpgradeTestCase {

Expand Down Expand Up @@ -164,4 +174,151 @@ public void testDataStreamValidationDoesNotBreakUpgrade() throws Exception {
}
}

public void testUpgradeDataStream() throws Exception {
String dataStreamName = "reindex_test_data_stream";
int numRollovers = 5;
if (CLUSTER_TYPE == ClusterType.OLD) {
createAndRolloverDataStream(dataStreamName, numRollovers);
} else if (CLUSTER_TYPE == ClusterType.UPGRADED) {
upgradeDataStream(dataStreamName, numRollovers);
}
}

private static void createAndRolloverDataStream(String dataStreamName, int numRollovers) throws IOException {
// We want to create a data stream and roll it over several times so that we have several indices to upgrade
final String template = """
{
"mappings":{
"dynamic_templates": [
{
"labels": {
"path_match": "pod.labels.*",
"mapping": {
"type": "keyword",
"time_series_dimension": true
}
}
}
],
"properties": {
"@timestamp" : {
"type": "date"
},
"metricset": {
"type": "keyword",
"time_series_dimension": true
},
"k8s": {
"properties": {
"pod": {
"properties": {
"name": {
"type": "keyword"
},
"network": {
"properties": {
"tx": {
"type": "long"
},
"rx": {
"type": "long"
}
}
}
}
}
}
}
}
}
}
""";
final String indexTemplate = """
{
"index_patterns": ["$PATTERN"],
"template": $TEMPLATE,
"data_stream": {
}
}""";
var putIndexTemplateRequest = new Request("POST", "/_index_template/reindex_test_data_stream_template");
putIndexTemplateRequest.setJsonEntity(indexTemplate.replace("$TEMPLATE", template).replace("$PATTERN", dataStreamName));
assertOK(client().performRequest(putIndexTemplateRequest));
bulkLoadData(dataStreamName);
for (int i = 0; i < numRollovers; i++) {
rollover(dataStreamName);
bulkLoadData(dataStreamName);
}
}

private void upgradeDataStream(String dataStreamName, int numRollovers) throws Exception {
Request reindexRequest = new Request("POST", "/_migration/reindex");
reindexRequest.setJsonEntity(Strings.format("""
{
"mode": "upgrade",
"source": {
"index": "%s"
}
}""", dataStreamName));
Response reindexResponse = client().performRequest(reindexRequest);
assertOK(reindexResponse);
assertBusy(() -> {
Request statusRequest = new Request("GET", "_migration/reindex/" + dataStreamName + "/_status");
Response statusResponse = client().performRequest(statusRequest);
Map<String, Object> statusResponseMap = XContentHelper.convertToMap(
JsonXContent.jsonXContent,
statusResponse.getEntity().getContent(),
false
);
assertOK(statusResponse);
assertThat(statusResponseMap.get("complete"), equalTo(true));
if (isOriginalClusterSameMajorVersionAsCurrent()) {
// If the original cluster was the same as this one, we don't want any indices reindexed:
assertThat(statusResponseMap.get("successes"), equalTo(0));
} else {
assertThat(statusResponseMap.get("successes"), equalTo(numRollovers + 1));
}
}, 60, TimeUnit.SECONDS);
Request cancelRequest = new Request("POST", "_migration/reindex/" + dataStreamName + "/_cancel");
Response cancelResponse = client().performRequest(cancelRequest);
assertOK(cancelResponse);
}

private boolean isOriginalClusterSameMajorVersionAsCurrent() {
return Version.fromString(UPGRADE_FROM_VERSION).major == Version.fromString(Build.current().version()).major;
}

private static void bulkLoadData(String dataStreamName) throws IOException {
final String bulk = """
{"create": {}}
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "cat", "network": {"tx": 2001818691, "rx": 802133794}}}}
{"create": {}}
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "hamster", "network": {"tx": 2005177954, "rx": 801479970}}}}
{"create": {}}
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "cow", "network": {"tx": 2006223737, "rx": 802337279}}}}
{"create": {}}
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "rat", "network": {"tx": 2012916202, "rx": 803685721}}}}
{"create": {}}
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "dog", "network": {"tx": 1434521831, "rx": 530575198}}}}
{"create": {}}
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "tiger", "network": {"tx": 1434577921, "rx": 530600088}}}}
{"create": {}}
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "lion", "network": {"tx": 1434587694, "rx": 530604797}}}}
{"create": {}}
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "elephant", "network": {"tx": 1434595272, "rx": 530605511}}}}
""";
var bulkRequest = new Request("POST", "/" + dataStreamName + "/_bulk");
bulkRequest.setJsonEntity(bulk.replace("$now", formatInstant(Instant.now())));
var response = client().performRequest(bulkRequest);
assertOK(response);
}

static String formatInstant(Instant instant) {
return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant);
}

private static void rollover(String dataStreamName) throws IOException {
Request rolloverRequest = new Request("POST", "/" + dataStreamName + "/_rollover");
Response rolloverResponse = client().performRequest(rolloverRequest);
assertOK(rolloverResponse);
}
}

0 comments on commit aa203af

Please sign in to comment.