YARN-2673. Made timeline client put APIs retry if ConnectException happens. Contributed by Li Lu.

This commit is contained in:
Zhijie Shen 2014-10-20 12:20:39 -07:00
parent e4d6a87854
commit 89427419a3
5 changed files with 148 additions and 0 deletions

View File

@ -373,6 +373,9 @@ Release 2.6.0 - UNRELEASED
YARN-2676. Enhanced Timeline auth-filter to support proxy users. (Zhijie Shen
via vinodkv)
YARN-2673. Made timeline client put APIs retry if ConnectException happens.
(Li Lu via zjshen)
OPTIMIZATIONS
BUG FIXES

View File

@ -1317,6 +1317,23 @@ public class YarnConfiguration extends Configuration {
public static final boolean
TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT = false;
/** Timeline client settings */
public static final String TIMELINE_SERVICE_CLIENT_PREFIX =
TIMELINE_SERVICE_PREFIX + "client.";
/** Timeline client call, max retries (-1 means no limit) */
public static final String TIMELINE_SERVICE_CLIENT_MAX_RETRIES =
TIMELINE_SERVICE_CLIENT_PREFIX + "max-retries";
public static final int DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES = 30;
/** Timeline client call, retry interval */
public static final String TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS =
TIMELINE_SERVICE_CLIENT_PREFIX + "retry-interval-ms";
public static final long
DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS = 1000;
// ///////////////////////////////
// Shared Cache Configs
// ///////////////////////////////

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.client.api.impl;
import java.io.File;
import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.net.ConnectException;
import java.net.HttpURLConnection;
import java.net.URI;
import java.net.URL;
@ -67,7 +68,10 @@ import org.codehaus.jackson.map.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.sun.jersey.api.client.Client;
import com.sun.jersey.api.client.filter.ClientFilter;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.api.client.ClientRequest;
import com.sun.jersey.api.client.ClientHandlerException;
import com.sun.jersey.api.client.WebResource;
import com.sun.jersey.api.client.config.ClientConfig;
import com.sun.jersey.api.client.config.DefaultClientConfig;
@ -103,6 +107,80 @@ public class TimelineClientImpl extends TimelineClient {
private URI resURI;
private boolean isEnabled;
private TimelineJerseyRetryFilter retryFilter;
static class TimelineJerseyRetryFilter extends ClientFilter {
// maxRetries < 0 means keep trying
@Private
@VisibleForTesting
public int maxRetries;
@Private
@VisibleForTesting
public long retryInterval;
// Indicates if retries happened last time
@Private
@VisibleForTesting
public boolean retried = false;
// Constructor with default retry settings
public TimelineJerseyRetryFilter(Configuration conf) {
super();
maxRetries = conf.getInt(
YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
retryInterval = conf.getLong(
YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
}
@Override
public ClientResponse handle(ClientRequest cr)
throws ClientHandlerException {
int leftRetries = maxRetries;
retried = false;
// keep trying
while (true) {
try {
// try pass the request on, if fail, keep retrying
return getNext().handle(cr);
} catch (ClientHandlerException e) {
// break if there's no retries left
if (leftRetries == 0) {
break;
}
if(e.getCause() instanceof ConnectException) {
if (leftRetries > 0) {
LOG.info("Connection Timeout (" + cr.getURI() + "), will try "
+ leftRetries + " more time(s).");
} else {
// note that maxRetries may be -1 at the very beginning
// maxRetries = -1 means keep trying
LOG.info("Connection Timeout (" + cr.getURI()
+ "), will keep retrying.");
}
retried = true;
} else {
throw e;
}
}
if (leftRetries > 0) {
leftRetries--;
}
try {
// sleep for the given time interval
Thread.sleep(retryInterval);
} catch (InterruptedException ie) {
LOG.warn("Client retry sleep interrupted! ");
}
}
throw new ClientHandlerException("Failed to connect to timeline server. "
+ "Connection retries limit exceeded. "
+ "The posted timeline event may be missing");
};
}
public TimelineClientImpl() {
super(TimelineClientImpl.class.getName());
}
@ -126,6 +204,8 @@ public class TimelineClientImpl extends TimelineClient {
client = new Client(new URLConnectionClientHandler(
new TimelineURLConnectionFactory()), cc);
token = new DelegationTokenAuthenticatedURL.Token();
retryFilter = new TimelineJerseyRetryFilter(conf);
client.addFilter(retryFilter);
if (YarnConfiguration.useHttps(conf)) {
resURI = URI
@ -228,6 +308,12 @@ public class TimelineClientImpl extends TimelineClient {
}
}
@Private
@VisibleForTesting
public TimelineJerseyRetryFilter getRetryFilter() {
return retryFilter;
}
@Private
@VisibleForTesting
public ClientResponse doPostingObject(Object object, String path) {

View File

@ -1322,6 +1322,22 @@
<value>/etc/krb5.keytab</value>
</property>
<property>
<description>
Default maximum number of retires for timeline servive client.
</description>
<name>yarn.timeline-service.client.max-retries</name>
<value>30</value>
</property>
<property>
<description>
Default retry time interval for timeline servive client.
</description>
<name>yarn.timeline-service.client.retry-interval-ms</name>
<value>1000</value>
</property>
<!-- Shared Cache Configuration -->
<property>
<description>Whether the shared cache is enabled</description>

View File

@ -181,6 +181,32 @@ public class TestTimelineClient {
}
}
@Test
public void testCheckRetryCount() throws Exception {
int newMaxRetries = 1;
long newIntervalMs = 1500;
YarnConfiguration conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
newMaxRetries);
conf.setLong(YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
newIntervalMs);
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
TimelineClientImpl client = createTimelineClient(conf);
try {
// This call should fail because there is no timeline server
client.putEntities(generateEntity());
Assert.fail("Exception expected!"
+ "Timeline server should be off to run this test. ");
} catch (ClientHandlerException ce) {
Assert.assertTrue(
"Handler exception for reason other than retry: " + ce.getMessage(),
ce.getMessage().contains("Connection retries limit exceeded"));
// we would expect this exception here, check if the client has retried
Assert.assertTrue("Retry filter didn't perform any retries! ", client
.getRetryFilter().retried);
}
}
private static ClientResponse mockEntityClientResponse(
TimelineClientImpl client, ClientResponse.Status status,
boolean hasError, boolean hasRuntimeError) {