Skip to content

Commit e087f3d

Browse files
authored
Connecting the reindex data stream persistent task to ReindexDataStreamIndexAction (#118978)
1 parent b813076 commit e087f3d

File tree

2 files changed

+269
-12
lines changed

2 files changed

+269
-12
lines changed

x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/task/ReindexDataStreamPersistentTaskExecutor.java

Lines changed: 113 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -9,8 +9,15 @@
99

1010
import org.elasticsearch.ElasticsearchException;
1111
import org.elasticsearch.action.ActionListener;
12+
import org.elasticsearch.action.admin.indices.rollover.RolloverAction;
13+
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
1214
import org.elasticsearch.action.datastreams.GetDataStreamAction;
15+
import org.elasticsearch.action.datastreams.ModifyDataStreamsAction;
16+
import org.elasticsearch.action.support.CountDownActionListener;
17+
import org.elasticsearch.action.support.master.AcknowledgedResponse;
1318
import org.elasticsearch.client.internal.Client;
19+
import org.elasticsearch.cluster.metadata.DataStream;
20+
import org.elasticsearch.cluster.metadata.DataStreamAction;
1421
import org.elasticsearch.cluster.service.ClusterService;
1522
import org.elasticsearch.core.TimeValue;
1623
import org.elasticsearch.index.Index;
@@ -20,9 +27,13 @@
2027
import org.elasticsearch.persistent.PersistentTasksExecutor;
2128
import org.elasticsearch.tasks.TaskId;
2229
import org.elasticsearch.threadpool.ThreadPool;
30+
import org.elasticsearch.xpack.migrate.action.ReindexDataStreamIndexAction;
2331

32+
import java.util.ArrayList;
33+
import java.util.Collections;
2434
import java.util.List;
2535
import java.util.Map;
36+
import java.util.NoSuchElementException;
2637

2738
import static org.elasticsearch.xpack.migrate.action.ReindexDataStreamAction.getOldIndexVersionPredicate;
2839

@@ -72,22 +83,109 @@ protected void nodeOperation(AllocatedPersistentTask task, ReindexDataStreamTask
7283
reindexClient.execute(GetDataStreamAction.INSTANCE, request, ActionListener.wrap(response -> {
7384
List<GetDataStreamAction.Response.DataStreamInfo> dataStreamInfos = response.getDataStreams();
7485
if (dataStreamInfos.size() == 1) {
75-
List<Index> indices = dataStreamInfos.getFirst().getDataStream().getIndices();
76-
List<Index> indicesToBeReindexed = indices.stream()
77-
.filter(getOldIndexVersionPredicate(clusterService.state().metadata()))
78-
.toList();
79-
reindexDataStreamTask.setPendingIndicesCount(indicesToBeReindexed.size());
80-
for (Index index : indicesToBeReindexed) {
81-
reindexDataStreamTask.incrementInProgressIndicesCount(index.getName());
82-
// TODO This is just a placeholder. This is where the real data stream reindex logic will go
83-
reindexDataStreamTask.reindexSucceeded(index.getName());
86+
DataStream dataStream = dataStreamInfos.getFirst().getDataStream();
87+
if (getOldIndexVersionPredicate(clusterService.state().metadata()).test(dataStream.getWriteIndex())) {
88+
reindexClient.execute(
89+
RolloverAction.INSTANCE,
90+
new RolloverRequest(sourceDataStream, null),
91+
ActionListener.wrap(
92+
rolloverResponse -> reindexIndices(dataStream, reindexDataStreamTask, reindexClient, sourceDataStream),
93+
e -> completeFailedPersistentTask(reindexDataStreamTask, e)
94+
)
95+
);
96+
} else {
97+
reindexIndices(dataStream, reindexDataStreamTask, reindexClient, sourceDataStream);
8498
}
85-
86-
completeSuccessfulPersistentTask(reindexDataStreamTask);
8799
} else {
88100
completeFailedPersistentTask(reindexDataStreamTask, new ElasticsearchException("data stream does not exist"));
89101
}
90-
}, reindexDataStreamTask::markAsFailed));
102+
}, exception -> completeFailedPersistentTask(reindexDataStreamTask, exception)));
103+
}
104+
105+
private void reindexIndices(
106+
DataStream dataStream,
107+
ReindexDataStreamTask reindexDataStreamTask,
108+
ExecuteWithHeadersClient reindexClient,
109+
String sourceDataStream
110+
) {
111+
List<Index> indices = dataStream.getIndices();
112+
List<Index> indicesToBeReindexed = indices.stream().filter(getOldIndexVersionPredicate(clusterService.state().metadata())).toList();
113+
reindexDataStreamTask.setPendingIndicesCount(indicesToBeReindexed.size());
114+
// The CountDownActionListener is 1 more than the number of indices so that the count is not 0 if we have no indices
115+
CountDownActionListener listener = new CountDownActionListener(indicesToBeReindexed.size() + 1, ActionListener.wrap(response1 -> {
116+
completeSuccessfulPersistentTask(reindexDataStreamTask);
117+
}, exception -> { completeFailedPersistentTask(reindexDataStreamTask, exception); }));
118+
List<Index> indicesRemaining = Collections.synchronizedList(new ArrayList<>(indicesToBeReindexed));
119+
final int maxConcurrentIndices = 1;
120+
for (int i = 0; i < maxConcurrentIndices; i++) {
121+
maybeProcessNextIndex(indicesRemaining, reindexDataStreamTask, reindexClient, sourceDataStream, listener);
122+
}
123+
// This takes care of the additional latch count referenced above:
124+
listener.onResponse(null);
125+
}
126+
127+
private void maybeProcessNextIndex(
128+
List<Index> indicesRemaining,
129+
ReindexDataStreamTask reindexDataStreamTask,
130+
ExecuteWithHeadersClient reindexClient,
131+
String sourceDataStream,
132+
CountDownActionListener listener
133+
) {
134+
if (indicesRemaining.isEmpty()) {
135+
return;
136+
}
137+
Index index;
138+
try {
139+
index = indicesRemaining.removeFirst();
140+
} catch (NoSuchElementException e) {
141+
return;
142+
}
143+
reindexDataStreamTask.incrementInProgressIndicesCount(index.getName());
144+
reindexClient.execute(
145+
ReindexDataStreamIndexAction.INSTANCE,
146+
new ReindexDataStreamIndexAction.Request(index.getName()),
147+
ActionListener.wrap(response1 -> {
148+
updateDataStream(sourceDataStream, index.getName(), response1.getDestIndex(), ActionListener.wrap(unused -> {
149+
reindexDataStreamTask.reindexSucceeded(index.getName());
150+
listener.onResponse(null);
151+
maybeProcessNextIndex(indicesRemaining, reindexDataStreamTask, reindexClient, sourceDataStream, listener);
152+
}, exception -> {
153+
reindexDataStreamTask.reindexFailed(index.getName(), exception);
154+
listener.onResponse(null);
155+
}), reindexClient);
156+
}, exception -> {
157+
reindexDataStreamTask.reindexFailed(index.getName(), exception);
158+
listener.onResponse(null);
159+
})
160+
);
161+
}
162+
163+
private void updateDataStream(
164+
String dataStream,
165+
String oldIndex,
166+
String newIndex,
167+
ActionListener<Void> listener,
168+
ExecuteWithHeadersClient reindexClient
169+
) {
170+
reindexClient.execute(
171+
ModifyDataStreamsAction.INSTANCE,
172+
new ModifyDataStreamsAction.Request(
173+
TimeValue.MAX_VALUE,
174+
TimeValue.MAX_VALUE,
175+
List.of(DataStreamAction.removeBackingIndex(dataStream, oldIndex), DataStreamAction.addBackingIndex(dataStream, newIndex))
176+
),
177+
new ActionListener<>() {
178+
@Override
179+
public void onResponse(AcknowledgedResponse response) {
180+
listener.onResponse(null);
181+
}
182+
183+
@Override
184+
public void onFailure(Exception e) {
185+
listener.onFailure(e);
186+
}
187+
}
188+
);
91189
}
92190

93191
private void completeSuccessfulPersistentTask(ReindexDataStreamTask persistentTask) {
@@ -105,6 +203,9 @@ private TimeValue getTimeToLive(ReindexDataStreamTask reindexDataStreamTask) {
105203
PersistentTasksCustomMetadata.PersistentTask<?> persistentTask = persistentTasksCustomMetadata.getTask(
106204
reindexDataStreamTask.getPersistentTaskId()
107205
);
206+
if (persistentTask == null) {
207+
return TimeValue.timeValueMillis(0);
208+
}
108209
PersistentTaskState state = persistentTask.getState();
109210
final long completionTime;
110211
if (state == null) {

x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataStreamsUpgradeIT.java

Lines changed: 156 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -11,15 +11,23 @@
1111
import org.elasticsearch.client.Response;
1212
import org.elasticsearch.cluster.metadata.DataStream;
1313
import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
14+
import org.elasticsearch.common.time.DateFormatter;
15+
import org.elasticsearch.common.time.FormatNames;
16+
import org.elasticsearch.common.xcontent.XContentHelper;
1417
import org.elasticsearch.core.Booleans;
1518
import org.elasticsearch.core.Strings;
19+
import org.elasticsearch.xcontent.json.JsonXContent;
1620
import org.hamcrest.Matchers;
1721

1822
import java.io.IOException;
1923
import java.nio.charset.StandardCharsets;
24+
import java.time.Instant;
2025
import java.util.List;
26+
import java.util.Map;
27+
import java.util.concurrent.TimeUnit;
2128

2229
import static org.elasticsearch.upgrades.IndexingIT.assertCount;
30+
import static org.hamcrest.Matchers.equalTo;
2331

2432
public class DataStreamsUpgradeIT extends AbstractUpgradeTestCase {
2533

@@ -164,4 +172,152 @@ public void testDataStreamValidationDoesNotBreakUpgrade() throws Exception {
164172
}
165173
}
166174

175+
public void testUpgradeDataStream() throws Exception {
176+
String dataStreamName = "reindex_test_data_stream";
177+
int numRollovers = 5;
178+
if (CLUSTER_TYPE == ClusterType.OLD) {
179+
createAndRolloverDataStream(dataStreamName, numRollovers);
180+
} else if (CLUSTER_TYPE == ClusterType.UPGRADED) {
181+
upgradeDataStream(dataStreamName, numRollovers);
182+
}
183+
}
184+
185+
private static void createAndRolloverDataStream(String dataStreamName, int numRollovers) throws IOException {
186+
// We want to create a data stream and roll it over several times so that we have several indices to upgrade
187+
final String template = """
188+
{
189+
"settings":{
190+
"index": {
191+
"mode": "time_series"
192+
}
193+
},
194+
"mappings":{
195+
"dynamic_templates": [
196+
{
197+
"labels": {
198+
"path_match": "pod.labels.*",
199+
"mapping": {
200+
"type": "keyword",
201+
"time_series_dimension": true
202+
}
203+
}
204+
}
205+
],
206+
"properties": {
207+
"@timestamp" : {
208+
"type": "date"
209+
},
210+
"metricset": {
211+
"type": "keyword",
212+
"time_series_dimension": true
213+
},
214+
"k8s": {
215+
"properties": {
216+
"pod": {
217+
"properties": {
218+
"name": {
219+
"type": "keyword"
220+
},
221+
"network": {
222+
"properties": {
223+
"tx": {
224+
"type": "long"
225+
},
226+
"rx": {
227+
"type": "long"
228+
}
229+
}
230+
}
231+
}
232+
}
233+
}
234+
}
235+
}
236+
}
237+
}
238+
""";
239+
final String indexTemplate = """
240+
{
241+
"index_patterns": ["$PATTERN"],
242+
"template": $TEMPLATE,
243+
"data_stream": {
244+
}
245+
}""";
246+
var putIndexTemplateRequest = new Request("POST", "/_index_template/reindex_test_data_stream_template");
247+
putIndexTemplateRequest.setJsonEntity(indexTemplate.replace("$TEMPLATE", template).replace("$PATTERN", dataStreamName));
248+
assertOK(client().performRequest(putIndexTemplateRequest));
249+
bulkLoadData(dataStreamName);
250+
for (int i = 0; i < numRollovers; i++) {
251+
rollover(dataStreamName);
252+
bulkLoadData(dataStreamName);
253+
}
254+
}
255+
256+
private void upgradeDataStream(String dataStreamName, int numRollovers) throws Exception {
257+
Request reindexRequest = new Request("POST", "/_migration/reindex");
258+
reindexRequest.setJsonEntity(Strings.format("""
259+
{
260+
"mode": "upgrade",
261+
"source": {
262+
"index": "%s"
263+
}
264+
}""", dataStreamName));
265+
Response reindexResponse = client().performRequest(reindexRequest);
266+
assertOK(reindexResponse);
267+
assertBusy(() -> {
268+
Request statusRequest = new Request("GET", "_migration/reindex/" + dataStreamName + "/_status");
269+
Response statusResponse = client().performRequest(statusRequest);
270+
Map<String, Object> statusResponseMap = XContentHelper.convertToMap(
271+
JsonXContent.jsonXContent,
272+
statusResponse.getEntity().getContent(),
273+
false
274+
);
275+
assertOK(statusResponse);
276+
assertThat(statusResponseMap.get("complete"), equalTo(true));
277+
if (isOriginalClusterCurrent()) {
278+
// If the original cluster was the same as this one, we don't want any indices reindexed:
279+
assertThat(statusResponseMap.get("successes"), equalTo(0));
280+
} else {
281+
assertThat(statusResponseMap.get("successes"), equalTo(numRollovers + 1));
282+
}
283+
}, 60, TimeUnit.SECONDS);
284+
Request cancelRequest = new Request("POST", "_migration/reindex/" + dataStreamName + "/_cancel");
285+
Response cancelResponse = client().performRequest(cancelRequest);
286+
assertOK(cancelResponse);
287+
}
288+
289+
private static void bulkLoadData(String dataStreamName) throws IOException {
290+
final String bulk = """
291+
{"create": {}}
292+
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "cat", "network": {"tx": 2001818691, "rx": 802133794}}}}
293+
{"create": {}}
294+
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "hamster", "network": {"tx": 2005177954, "rx": 801479970}}}}
295+
{"create": {}}
296+
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "cow", "network": {"tx": 2006223737, "rx": 802337279}}}}
297+
{"create": {}}
298+
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "rat", "network": {"tx": 2012916202, "rx": 803685721}}}}
299+
{"create": {}}
300+
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "dog", "network": {"tx": 1434521831, "rx": 530575198}}}}
301+
{"create": {}}
302+
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "tiger", "network": {"tx": 1434577921, "rx": 530600088}}}}
303+
{"create": {}}
304+
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "lion", "network": {"tx": 1434587694, "rx": 530604797}}}}
305+
{"create": {}}
306+
{"@timestamp": "$now", "metricset": "pod", "k8s": {"pod": {"name": "elephant", "network": {"tx": 1434595272, "rx": 530605511}}}}
307+
""";
308+
var bulkRequest = new Request("POST", "/" + dataStreamName + "/_bulk");
309+
bulkRequest.setJsonEntity(bulk.replace("$now", formatInstant(Instant.now())));
310+
var response = client().performRequest(bulkRequest);
311+
assertOK(response);
312+
}
313+
314+
static String formatInstant(Instant instant) {
315+
return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant);
316+
}
317+
318+
private static void rollover(String dataStreamName) throws IOException {
319+
Request rolloverRequest = new Request("POST", "/" + dataStreamName + "/_rollover");
320+
Response rolloverResponse = client().performRequest(rolloverRequest);
321+
assertOK(rolloverResponse);
322+
}
167323
}

0 commit comments

Comments
 (0)