|
@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.yarn.api.ApplicationConstants;
|
|
import org.apache.hadoop.yarn.api.ApplicationConstants;
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
import org.apache.hadoop.yarn.service.api.records.Service;
|
|
import org.apache.hadoop.yarn.service.api.records.Service;
|
|
|
|
+import org.apache.hadoop.yarn.service.component.ComponentRestartPolicy;
|
|
import org.apache.hadoop.yarn.service.conf.YarnServiceConf;
|
|
import org.apache.hadoop.yarn.service.conf.YarnServiceConf;
|
|
import org.apache.hadoop.yarn.service.conf.YarnServiceConstants;
|
|
import org.apache.hadoop.yarn.service.conf.YarnServiceConstants;
|
|
import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService;
|
|
import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService;
|
|
@@ -116,18 +117,22 @@ public abstract class AbstractProviderService implements ProviderService,
|
|
|
|
|
|
public void buildContainerRetry(AbstractLauncher launcher,
|
|
public void buildContainerRetry(AbstractLauncher launcher,
|
|
Configuration yarnConf,
|
|
Configuration yarnConf,
|
|
- ContainerLaunchService.ComponentLaunchContext compLaunchContext) {
|
|
|
|
|
|
+ ContainerLaunchService.ComponentLaunchContext compLaunchContext,
|
|
|
|
+ ComponentInstance instance) {
|
|
// By default retry forever every 30 seconds
|
|
// By default retry forever every 30 seconds
|
|
- launcher.setRetryContext(
|
|
|
|
- YarnServiceConf.getInt(CONTAINER_RETRY_MAX,
|
|
|
|
- DEFAULT_CONTAINER_RETRY_MAX,
|
|
|
|
- compLaunchContext.getConfiguration(), yarnConf),
|
|
|
|
- YarnServiceConf.getInt(CONTAINER_RETRY_INTERVAL,
|
|
|
|
- DEFAULT_CONTAINER_RETRY_INTERVAL,
|
|
|
|
- compLaunchContext.getConfiguration(), yarnConf),
|
|
|
|
- YarnServiceConf.getLong(CONTAINER_FAILURES_VALIDITY_INTERVAL,
|
|
|
|
- DEFAULT_CONTAINER_FAILURES_VALIDITY_INTERVAL,
|
|
|
|
- compLaunchContext.getConfiguration(), yarnConf));
|
|
|
|
|
|
+
|
|
|
|
+ ComponentRestartPolicy restartPolicy = instance.getComponent()
|
|
|
|
+ .getRestartPolicyHandler();
|
|
|
|
+ if (restartPolicy.allowContainerRetriesForInstance(instance)) {
|
|
|
|
+ launcher.setRetryContext(YarnServiceConf
|
|
|
|
+ .getInt(CONTAINER_RETRY_MAX, DEFAULT_CONTAINER_RETRY_MAX,
|
|
|
|
+ compLaunchContext.getConfiguration(), yarnConf), YarnServiceConf
|
|
|
|
+ .getInt(CONTAINER_RETRY_INTERVAL, DEFAULT_CONTAINER_RETRY_INTERVAL,
|
|
|
|
+ compLaunchContext.getConfiguration(), yarnConf), YarnServiceConf
|
|
|
|
+ .getLong(CONTAINER_FAILURES_VALIDITY_INTERVAL,
|
|
|
|
+ DEFAULT_CONTAINER_FAILURES_VALIDITY_INTERVAL,
|
|
|
|
+ compLaunchContext.getConfiguration(), yarnConf));
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
public void buildContainerLaunchContext(AbstractLauncher launcher,
|
|
public void buildContainerLaunchContext(AbstractLauncher launcher,
|
|
@@ -161,6 +166,6 @@ public abstract class AbstractProviderService implements ProviderService,
|
|
yarnConf, container, compLaunchContext, tokensForSubstitution);
|
|
yarnConf, container, compLaunchContext, tokensForSubstitution);
|
|
|
|
|
|
// Setup container retry settings
|
|
// Setup container retry settings
|
|
- buildContainerRetry(launcher, yarnConf, compLaunchContext);
|
|
|
|
|
|
+ buildContainerRetry(launcher, yarnConf, compLaunchContext, instance);
|
|
}
|
|
}
|
|
}
|
|
}
|