Skip to content

Commit 9ef9f1e

Browse files
committed
Avoid stack overflow in auto-follow coordinator (#44421)
This commit avoids a situation where we might stack overflow in the auto-follower coordinator. In the face of repeated failures to get the remote cluster state, we would previously be called back on the same thread and then recurse to try again. If this failure persists, the repeated callbacks on the same thread would lead to a stack overflow. The repeated failures can occur, for example, if the connect queue is full when we attempt to make a connection to the remote cluster. This commit avoids this by truncating the call stack if we are called back on the same thread as the initial request was made on.
1 parent 14a9c22 commit 9ef9f1e

File tree

3 files changed

+153
-35
lines changed

3 files changed

+153
-35
lines changed

x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -188,16 +188,17 @@ public Collection<Object> createComponents(
188188
CcrRestoreSourceService restoreSourceService = new CcrRestoreSourceService(threadPool, ccrSettings);
189189
this.restoreSourceService.set(restoreSourceService);
190190
return Arrays.asList(
191+
ccrLicenseChecker,
192+
restoreSourceService,
193+
new CcrRepositoryManager(settings, clusterService, (NodeClient) client),
194+
new AutoFollowCoordinator(
195+
settings,
196+
client,
197+
clusterService,
191198
ccrLicenseChecker,
192-
restoreSourceService,
193-
new CcrRepositoryManager(settings, clusterService, (NodeClient) client),
194-
new AutoFollowCoordinator(
195-
settings,
196-
client,
197-
clusterService,
198-
ccrLicenseChecker,
199-
threadPool::relativeTimeInMillis,
200-
threadPool::absoluteTimeInMillis));
199+
threadPool::relativeTimeInMillis,
200+
threadPool::absoluteTimeInMillis,
201+
threadPool.executor(Ccr.CCR_THREAD_POOL_NAME)));
201202
}
202203

203204
@Override

x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java

Lines changed: 35 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,7 @@
5454
import java.util.Objects;
5555
import java.util.Set;
5656
import java.util.TreeMap;
57+
import java.util.concurrent.Executor;
5758
import java.util.function.BiConsumer;
5859
import java.util.function.Consumer;
5960
import java.util.function.Function;
@@ -77,6 +78,7 @@ public class AutoFollowCoordinator extends AbstractLifecycleComponent implements
7778
private final CcrLicenseChecker ccrLicenseChecker;
7879
private final LongSupplier relativeMillisTimeProvider;
7980
private final LongSupplier absoluteMillisTimeProvider;
81+
private final Executor executor;
8082

8183
private volatile TimeValue waitForMetadataTimeOut;
8284
private volatile Map<String, AutoFollower> autoFollowers = Collections.emptyMap();
@@ -88,18 +90,20 @@ public class AutoFollowCoordinator extends AbstractLifecycleComponent implements
8890
private final LinkedHashMap<String, Tuple<Long, ElasticsearchException>> recentAutoFollowErrors;
8991

9092
public AutoFollowCoordinator(
91-
Settings settings,
92-
Client client,
93-
ClusterService clusterService,
94-
CcrLicenseChecker ccrLicenseChecker,
95-
LongSupplier relativeMillisTimeProvider,
96-
LongSupplier absoluteMillisTimeProvider) {
93+
final Settings settings,
94+
final Client client,
95+
final ClusterService clusterService,
96+
final CcrLicenseChecker ccrLicenseChecker,
97+
final LongSupplier relativeMillisTimeProvider,
98+
final LongSupplier absoluteMillisTimeProvider,
99+
final Executor executor) {
97100

98101
this.client = client;
99102
this.clusterService = clusterService;
100103
this.ccrLicenseChecker = Objects.requireNonNull(ccrLicenseChecker, "ccrLicenseChecker");
101104
this.relativeMillisTimeProvider = relativeMillisTimeProvider;
102105
this.absoluteMillisTimeProvider = absoluteMillisTimeProvider;
106+
this.executor = Objects.requireNonNull(executor);
103107
this.recentAutoFollowErrors = new LinkedHashMap<String, Tuple<Long, ElasticsearchException>>() {
104108
@Override
105109
protected boolean removeEldestEntry(final Map.Entry<String, Tuple<Long, ElasticsearchException>> eldest) {
@@ -209,7 +213,7 @@ void updateAutoFollowers(ClusterState followerClusterState) {
209213
Map<String, AutoFollower> newAutoFollowers = new HashMap<>(newRemoteClusters.size());
210214
for (String remoteCluster : newRemoteClusters) {
211215
AutoFollower autoFollower =
212-
new AutoFollower(remoteCluster, this::updateStats, clusterService::state, relativeMillisTimeProvider) {
216+
new AutoFollower(remoteCluster, this::updateStats, clusterService::state, relativeMillisTimeProvider, executor) {
213217

214218
@Override
215219
void getRemoteClusterState(final String remoteCluster,
@@ -331,6 +335,7 @@ abstract static class AutoFollower {
331335
private final Consumer<List<AutoFollowResult>> statsUpdater;
332336
private final Supplier<ClusterState> followerClusterStateSupplier;
333337
private final LongSupplier relativeTimeProvider;
338+
private final Executor executor;
334339

335340
private volatile long lastAutoFollowTimeInMillis = -1;
336341
private volatile long metadataVersion = 0;
@@ -343,11 +348,13 @@ abstract static class AutoFollower {
343348
AutoFollower(final String remoteCluster,
344349
final Consumer<List<AutoFollowResult>> statsUpdater,
345350
final Supplier<ClusterState> followerClusterStateSupplier,
346-
LongSupplier relativeTimeProvider) {
351+
final LongSupplier relativeTimeProvider,
352+
final Executor executor) {
347353
this.remoteCluster = remoteCluster;
348354
this.statsUpdater = statsUpdater;
349355
this.followerClusterStateSupplier = followerClusterStateSupplier;
350356
this.relativeTimeProvider = relativeTimeProvider;
357+
this.executor = Objects.requireNonNull(executor);
351358
}
352359

353360
void start() {
@@ -386,6 +393,7 @@ void start() {
386393
this.autoFollowPatternsCountDown = new CountDown(patterns.size());
387394
this.autoFollowResults = new AtomicArray<>(patterns.size());
388395

396+
final Thread thread = Thread.currentThread();
389397
getRemoteClusterState(remoteCluster, metadataVersion + 1, (remoteClusterStateResponse, remoteError) -> {
390398
// Also check removed flag here, as it may take a while for this remote cluster state api call to return:
391399
if (removed) {
@@ -402,7 +410,7 @@ void start() {
402410
}
403411
ClusterState remoteClusterState = remoteClusterStateResponse.getState();
404412
metadataVersion = remoteClusterState.metaData().version();
405-
autoFollowIndices(autoFollowMetadata, clusterState, remoteClusterState, patterns);
413+
autoFollowIndices(autoFollowMetadata, clusterState, remoteClusterState, patterns, thread);
406414
} else {
407415
assert remoteError != null;
408416
if (remoteError instanceof NoSuchRemoteClusterException) {
@@ -413,7 +421,7 @@ void start() {
413421

414422
for (int i = 0; i < patterns.size(); i++) {
415423
String autoFollowPatternName = patterns.get(i);
416-
finalise(i, new AutoFollowResult(autoFollowPatternName, remoteError));
424+
finalise(i, new AutoFollowResult(autoFollowPatternName, remoteError), thread);
417425
}
418426
}
419427
});
@@ -427,7 +435,8 @@ void stop() {
427435
private void autoFollowIndices(final AutoFollowMetadata autoFollowMetadata,
428436
final ClusterState clusterState,
429437
final ClusterState remoteClusterState,
430-
final List<String> patterns) {
438+
final List<String> patterns,
439+
final Thread thread) {
431440
int i = 0;
432441
for (String autoFollowPatternName : patterns) {
433442
final int slot = i;
@@ -438,7 +447,7 @@ private void autoFollowIndices(final AutoFollowMetadata autoFollowMetadata,
438447
final List<Index> leaderIndicesToFollow =
439448
getLeaderIndicesToFollow(autoFollowPattern, remoteClusterState, followedIndices);
440449
if (leaderIndicesToFollow.isEmpty()) {
441-
finalise(slot, new AutoFollowResult(autoFollowPatternName));
450+
finalise(slot, new AutoFollowResult(autoFollowPatternName), thread);
442451
} else {
443452
List<Tuple<String, AutoFollowPattern>> patternsForTheSameRemoteCluster = autoFollowMetadata.getPatterns()
444453
.entrySet().stream()
@@ -447,7 +456,7 @@ private void autoFollowIndices(final AutoFollowMetadata autoFollowMetadata,
447456
.map(item -> new Tuple<>(item.getKey(), item.getValue()))
448457
.collect(Collectors.toList());
449458

450-
Consumer<AutoFollowResult> resultHandler = result -> finalise(slot, result);
459+
Consumer<AutoFollowResult> resultHandler = result -> finalise(slot, result, thread);
451460
checkAutoFollowPattern(autoFollowPatternName, remoteCluster, autoFollowPattern, leaderIndicesToFollow, headers,
452461
patternsForTheSameRemoteCluster, remoteClusterState.metaData(), clusterState.metaData(), resultHandler);
453462
}
@@ -575,11 +584,23 @@ private void followLeaderIndex(String autoFollowPattenName,
575584
createAndFollow(headers, request, successHandler, onResult);
576585
}
577586

578-
private void finalise(int slot, AutoFollowResult result) {
587+
private void finalise(int slot, AutoFollowResult result, final Thread thread) {
579588
assert autoFollowResults.get(slot) == null;
580589
autoFollowResults.set(slot, result);
581590
if (autoFollowPatternsCountDown.countDown()) {
582591
statsUpdater.accept(autoFollowResults.asList());
592+
/*
593+
* In the face of a failure, we could be called back on the same thread. That is, it could be that we
594+
* never fired off the asynchronous remote cluster state call, instead failing beforehand. In this case,
595+
* we will recurse on the same thread. If there are repeated failures, we could blow the stack and
596+
* overflow. A real-world scenario in which this can occur is if the local connect queue is full. To
597+
* avoid this, if we are called back on the same thread, then we truncate the stack by forking to
598+
* another thread.
599+
*/
600+
if (thread == Thread.currentThread()) {
601+
executor.execute(this::start);
602+
return;
603+
}
583604
start();
584605
}
585606
}

0 commit comments

Comments
 (0)