|
@@ -26,6 +26,7 @@ import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
import org.apache.commons.lang.StringUtils;
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -77,6 +78,8 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
|
|
|
import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+
|
|
|
public class ClientServiceDelegate {
|
|
|
private static final Log LOG = LogFactory.getLog(ClientServiceDelegate.class);
|
|
|
private static final String UNAVAILABLE = "N/A";
|
|
@@ -93,7 +96,8 @@ public class ClientServiceDelegate {
|
|
|
private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
|
|
|
private static String UNKNOWN_USER = "Unknown User";
|
|
|
private String trackingUrl;
|
|
|
-
|
|
|
+ private AtomicBoolean usingAMProxy = new AtomicBoolean(false);
|
|
|
+ private int maxClientRetry;
|
|
|
private boolean amAclDisabledStatusLogged = false;
|
|
|
|
|
|
public ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm,
|
|
@@ -287,6 +291,7 @@ public class ClientServiceDelegate {
|
|
|
MRClientProtocol proxy =
|
|
|
(MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
|
|
|
serviceAddr, conf);
|
|
|
+ usingAMProxy.set(true);
|
|
|
LOG.trace("Connected to ApplicationMaster at: " + serviceAddr);
|
|
|
return proxy;
|
|
|
}
|
|
@@ -301,13 +306,15 @@ public class ClientServiceDelegate {
|
|
|
} catch (NoSuchMethodException e) {
|
|
|
throw new YarnRuntimeException("Method name mismatch", e);
|
|
|
}
|
|
|
- int maxRetries = this.conf.getInt(
|
|
|
+ maxClientRetry = this.conf.getInt(
|
|
|
MRJobConfig.MR_CLIENT_MAX_RETRIES,
|
|
|
MRJobConfig.DEFAULT_MR_CLIENT_MAX_RETRIES);
|
|
|
IOException lastException = null;
|
|
|
- while (maxRetries > 0) {
|
|
|
+ while (maxClientRetry > 0) {
|
|
|
+ MRClientProtocol MRClientProxy = null;
|
|
|
try {
|
|
|
- return methodOb.invoke(getProxy(), args);
|
|
|
+ MRClientProxy = getProxy();
|
|
|
+ return methodOb.invoke(MRClientProxy, args);
|
|
|
} catch (InvocationTargetException e) {
|
|
|
// Will not throw out YarnException anymore
|
|
|
LOG.debug("Failed to contact AM/History for job " + jobId +
|
|
@@ -315,22 +322,44 @@ public class ClientServiceDelegate {
|
|
|
// Force reconnection by setting the proxy to null.
|
|
|
realProxy = null;
|
|
|
// HS/AMS shut down
|
|
|
- maxRetries--;
|
|
|
+ // if it's AM shut down, do not decrement maxClientRetry as we wait for
|
|
|
+ // AM to be restarted.
|
|
|
+ if (!usingAMProxy.get()) {
|
|
|
+ maxClientRetry--;
|
|
|
+ }
|
|
|
+ usingAMProxy.set(false);
|
|
|
lastException = new IOException(e.getTargetException());
|
|
|
-
|
|
|
+ try {
|
|
|
+ Thread.sleep(100);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ LOG.warn("ClientServiceDelegate invoke call interrupted", ie);
|
|
|
+ throw new YarnRuntimeException(ie);
|
|
|
+ }
|
|
|
} catch (Exception e) {
|
|
|
LOG.debug("Failed to contact AM/History for job " + jobId
|
|
|
+ " Will retry..", e);
|
|
|
// Force reconnection by setting the proxy to null.
|
|
|
realProxy = null;
|
|
|
// RM shutdown
|
|
|
- maxRetries--;
|
|
|
- lastException = new IOException(e.getMessage());
|
|
|
+ maxClientRetry--;
|
|
|
+ lastException = new IOException(e.getMessage());
|
|
|
+ try {
|
|
|
+ Thread.sleep(100);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ LOG.warn("ClientServiceDelegate invoke call interrupted", ie);
|
|
|
+ throw new YarnRuntimeException(ie);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
throw lastException;
|
|
|
}
|
|
|
|
|
|
+ // Only for testing
|
|
|
+ @VisibleForTesting
|
|
|
+ public int getMaxClientRetry() {
|
|
|
+ return this.maxClientRetry;
|
|
|
+ }
|
|
|
+
|
|
|
public org.apache.hadoop.mapreduce.Counters getJobCounters(JobID arg0) throws IOException,
|
|
|
InterruptedException {
|
|
|
org.apache.hadoop.mapreduce.v2.api.records.JobId jobID = TypeConverter.toYarn(arg0);
|