Skip to content

Commit 9f7e951

Browse files
committed
Wait indefinitely, do not retry
1 parent 150f47b commit 9f7e951

File tree

6 files changed

+67
-79
lines changed

6 files changed

+67
-79
lines changed

server/src/main/java/org/elasticsearch/action/admin/cluster/bootstrap/GetDiscoveredNodesRequest.java

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020

2121
import org.elasticsearch.action.ActionRequest;
2222
import org.elasticsearch.action.ActionRequestValidationException;
23+
import org.elasticsearch.common.Nullable;
2324
import org.elasticsearch.common.io.stream.StreamInput;
2425
import org.elasticsearch.common.io.stream.StreamOutput;
2526
import org.elasticsearch.common.unit.TimeValue;
@@ -33,6 +34,8 @@
3334
public class GetDiscoveredNodesRequest extends ActionRequest {
3435

3536
private int waitForNodes = 1;
37+
38+
@Nullable // if the request should wait indefinitely
3639
private TimeValue timeout = TimeValue.timeValueSeconds(30);
3740

3841
public GetDiscoveredNodesRequest() {
@@ -41,7 +44,7 @@ public GetDiscoveredNodesRequest() {
4144
public GetDiscoveredNodesRequest(StreamInput in) throws IOException {
4245
super(in);
4346
waitForNodes = in.readInt();
44-
timeout = in.readTimeValue();
47+
timeout = in.readOptionalTimeValue();
4548
}
4649

4750
/**
@@ -74,8 +77,8 @@ public int getWaitForNodes() {
7477
*
7578
* @param timeout how long to wait to discover sufficiently many nodes to respond successfully.
7679
*/
77-
public void setTimeout(TimeValue timeout) {
78-
if (timeout.compareTo(TimeValue.ZERO) < 0) {
80+
public void setTimeout(@Nullable TimeValue timeout) {
81+
if (timeout != null && timeout.compareTo(TimeValue.ZERO) < 0) {
7982
throw new IllegalArgumentException("negative timeout of [" + timeout + "] is not allowed");
8083
}
8184
this.timeout = timeout;
@@ -87,6 +90,7 @@ public void setTimeout(TimeValue timeout) {
8790
*
8891
* @return how long to wait to discover sufficiently many nodes to respond successfully.
8992
*/
93+
@Nullable
9094
public TimeValue getTimeout() {
9195
return timeout;
9296
}
@@ -105,7 +109,7 @@ public void readFrom(StreamInput in) throws IOException {
105109
public void writeTo(StreamOutput out) throws IOException {
106110
super.writeTo(out);
107111
out.writeInt(waitForNodes);
108-
out.writeTimeValue(timeout);
112+
out.writeOptionalTimeValue(timeout);
109113
}
110114

111115
@Override

server/src/main/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportGetDiscoveredNodesAction.java

Lines changed: 13 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -108,18 +108,20 @@ public String toString() {
108108
listenableFuture.addListener(ActionListener.wrap(releasable::close), directExecutor, threadPool.getThreadContext());
109109
respondIfRequestSatisfied.accept(coordinator.getFoundPeers());
110110

111-
threadPool.schedule(request.getTimeout(), Names.SAME, new Runnable() {
112-
@Override
113-
public void run() {
114-
if (listenerNotified.compareAndSet(false, true)) {
115-
listenableFuture.onFailure(new ElasticsearchTimeoutException("timed out while waiting for " + request));
111+
if (request.getTimeout() != null) {
112+
threadPool.schedule(request.getTimeout(), Names.SAME, new Runnable() {
113+
@Override
114+
public void run() {
115+
if (listenerNotified.compareAndSet(false, true)) {
116+
listenableFuture.onFailure(new ElasticsearchTimeoutException("timed out while waiting for " + request));
117+
}
116118
}
117-
}
118119

119-
@Override
120-
public String toString() {
121-
return "timeout handler for " + request;
122-
}
123-
});
120+
@Override
121+
public String toString() {
122+
return "timeout handler for " + request;
123+
}
124+
});
125+
}
124126
}
125127
}

server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java

Lines changed: 2 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -89,6 +89,7 @@ private void awaitDiscovery() {
8989

9090
final GetDiscoveredNodesRequest request = new GetDiscoveredNodesRequest();
9191
request.setWaitForNodes(initialMasterNodeCount);
92+
request.setTimeout(null);
9293
logger.trace("sending {}", request);
9394
transportService.sendRequest(transportService.getLocalNode(), GetDiscoveredNodesAction.NAME, request,
9495
new TransportResponseHandler<GetDiscoveredNodesResponse>() {
@@ -103,13 +104,7 @@ public void handleResponse(GetDiscoveredNodesResponse response) {
103104

104105
@Override
105106
public void handleException(TransportException exp) {
106-
if (exp.getRootCause() instanceof ElasticsearchTimeoutException) {
107-
logger.debug(new ParameterizedMessage("discovery attempt timed out, retrying, request={}", request), exp);
108-
awaitDiscovery();
109-
} else {
110-
// exceptions other than a timeout are fatal
111-
logger.warn("discovery attempt failed, not retrying", exp);
112-
}
107+
logger.warn("discovery attempt failed", exp);
113108
}
114109

115110
@Override

server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/GetDiscoveredNodesRequestTests.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525

2626
import static org.hamcrest.Matchers.endsWith;
2727
import static org.hamcrest.Matchers.equalTo;
28+
import static org.hamcrest.Matchers.nullValue;
2829
import static org.hamcrest.Matchers.startsWith;
2930
import static org.hamcrest.core.Is.is;
3031

@@ -56,6 +57,9 @@ public void testTimeoutValidation() {
5657
() -> getDiscoveredNodesRequest.setTimeout(TimeValue.timeValueNanos(randomLongBetween(-10, -1))));
5758
assertThat(exception.getMessage(), startsWith("negative timeout of "));
5859
assertThat(exception.getMessage(), endsWith(" is not allowed"));
60+
61+
getDiscoveredNodesRequest.setTimeout(null);
62+
assertThat("value updated", getDiscoveredNodesRequest.getTimeout(), nullValue());
5963
}
6064

6165
public void testSerialization() throws IOException {
@@ -67,6 +71,8 @@ public void testSerialization() throws IOException {
6771

6872
if (randomBoolean()) {
6973
originalRequest.setTimeout(TimeValue.parseTimeValue(randomTimeValue(), "timeout"));
74+
} else if (randomBoolean()) {
75+
originalRequest.setTimeout(null);
7076
}
7177

7278
final GetDiscoveredNodesRequest deserialized = copyWriteable(originalRequest, writableRegistry(), GetDiscoveredNodesRequest::new);

server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportGetDiscoveredNodesActionTests.java

Lines changed: 38 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -171,34 +171,53 @@ public void handleException(TransportException exp) {
171171
assertTrue(countDownLatch.await(10, TimeUnit.SECONDS));
172172
}
173173

174-
public void testFailsQuicklyWithZeroTimeout() throws InterruptedException {
174+
public void testFailsQuicklyWithZeroTimeoutAndAcceptsNullTimeout() throws InterruptedException {
175175
new TransportGetDiscoveredNodesAction(Settings.EMPTY, EMPTY_FILTERS, transportService, coordinator); // registers action
176176
transportService.start();
177177
transportService.acceptIncomingRequests();
178178
coordinator.start();
179179
coordinator.startInitialJoin();
180180

181-
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
182-
getDiscoveredNodesRequest.setWaitForNodes(2);
183-
getDiscoveredNodesRequest.setTimeout(TimeValue.ZERO);
181+
{
182+
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
183+
getDiscoveredNodesRequest.setWaitForNodes(2);
184+
getDiscoveredNodesRequest.setTimeout(null);
185+
transportService.sendRequest(localNode, GetDiscoveredNodesAction.NAME, getDiscoveredNodesRequest, new ResponseHandler() {
186+
@Override
187+
public void handleResponse(GetDiscoveredNodesResponse response) {
188+
throw new AssertionError("should not be called");
189+
}
184190

185-
final CountDownLatch countDownLatch = new CountDownLatch(1);
186-
transportService.sendRequest(localNode, GetDiscoveredNodesAction.NAME, getDiscoveredNodesRequest, new ResponseHandler() {
187-
@Override
188-
public void handleResponse(GetDiscoveredNodesResponse response) {
189-
throw new AssertionError("should not be called");
190-
}
191+
@Override
192+
public void handleException(TransportException exp) {
193+
throw new AssertionError("should not be called", exp);
194+
}
195+
});
196+
}
191197

192-
@Override
193-
public void handleException(TransportException exp) {
194-
final Throwable rootCause = exp.getRootCause();
195-
assertThat(rootCause, instanceOf(ElasticsearchTimeoutException.class));
196-
assertThat(rootCause.getMessage(), startsWith("timed out while waiting for GetDiscoveredNodesRequest{"));
197-
countDownLatch.countDown();
198-
}
199-
});
198+
{
199+
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
200+
getDiscoveredNodesRequest.setWaitForNodes(2);
201+
getDiscoveredNodesRequest.setTimeout(TimeValue.ZERO);
200202

201-
assertTrue(countDownLatch.await(10, TimeUnit.SECONDS));
203+
final CountDownLatch countDownLatch = new CountDownLatch(1);
204+
transportService.sendRequest(localNode, GetDiscoveredNodesAction.NAME, getDiscoveredNodesRequest, new ResponseHandler() {
205+
@Override
206+
public void handleResponse(GetDiscoveredNodesResponse response) {
207+
throw new AssertionError("should not be called");
208+
}
209+
210+
@Override
211+
public void handleException(TransportException exp) {
212+
final Throwable rootCause = exp.getRootCause();
213+
assertThat(rootCause, instanceOf(ElasticsearchTimeoutException.class));
214+
assertThat(rootCause.getMessage(), startsWith("timed out while waiting for GetDiscoveredNodesRequest{"));
215+
countDownLatch.countDown();
216+
}
217+
});
218+
219+
assertTrue(countDownLatch.await(10, TimeUnit.SECONDS));
220+
}
202221
}
203222

204223
public void testGetsDiscoveredNodes() throws InterruptedException {

server/src/test/java/org/elasticsearch/cluster/coordination/ClusterBootstrapServiceTests.java

Lines changed: 0 additions & 38 deletions
Original file line numberDiff line numberDiff line change
@@ -146,44 +146,6 @@ public void messageReceived(GetDiscoveredNodesRequest request, TransportChannel
146146
deterministicTaskQueue.runAllTasks();
147147
}
148148

149-
public void testRetriesOnDiscoveryTimeout() {
150-
AtomicLong callCount = new AtomicLong();
151-
transportService.registerRequestHandler(GetDiscoveredNodesAction.NAME, Names.SAME, GetDiscoveredNodesRequest::new,
152-
(request, channel, task) -> deterministicTaskQueue.scheduleAt(deterministicTaskQueue.getCurrentTimeMillis() + 30000, () -> {
153-
callCount.incrementAndGet();
154-
try {
155-
channel.sendResponse(new ElasticsearchTimeoutException("simulated timeout"));
156-
} catch (IOException e) {
157-
throw new AssertionError("unexpected", e);
158-
}
159-
}));
160-
161-
startServices();
162-
while (callCount.get() < 5) {
163-
if (deterministicTaskQueue.hasDeferredTasks()) {
164-
deterministicTaskQueue.advanceTime();
165-
}
166-
deterministicTaskQueue.runAllRunnableTasks();
167-
}
168-
}
169-
170-
public void testStopsRetryingDiscoveryWhenStopped() {
171-
transportService.registerRequestHandler(GetDiscoveredNodesAction.NAME, Names.SAME, GetDiscoveredNodesRequest::new,
172-
(request, channel, task) -> deterministicTaskQueue.scheduleAt(deterministicTaskQueue.getCurrentTimeMillis() + 30000, () -> {
173-
try {
174-
channel.sendResponse(new ElasticsearchTimeoutException("simulated timeout"));
175-
} catch (IOException e) {
176-
throw new AssertionError("unexpected", e);
177-
}
178-
}));
179-
180-
scheduleStopAfter(150000);
181-
182-
startServices();
183-
deterministicTaskQueue.runAllTasks();
184-
// termination means success
185-
}
186-
187149
public void testBootstrapsOnDiscoverySuccess() {
188150
final AtomicBoolean discoveryAttempted = new AtomicBoolean();
189151
final Set<DiscoveryNode> discoveredNodes = Stream.of(localNode, otherNode1, otherNode2).collect(Collectors.toSet());

0 commit comments

Comments
 (0)