|
@@ -21,6 +21,10 @@ package org.apache.hadoop.yarn.server.resourcemanager;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.curator.framework.AuthInfo;
|
|
|
+import org.apache.curator.framework.CuratorFramework;
|
|
|
+import org.apache.curator.framework.CuratorFrameworkFactory;
|
|
|
+import org.apache.curator.retry.RetryNTimes;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol;
|
|
@@ -28,7 +32,11 @@ import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
import org.apache.hadoop.http.lib.StaticUserWebFilter;
|
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
|
import org.apache.hadoop.metrics2.source.JvmMetrics;
|
|
|
-import org.apache.hadoop.security.*;
|
|
|
+import org.apache.hadoop.security.AuthenticationFilterInitializer;
|
|
|
+import org.apache.hadoop.security.Groups;
|
|
|
+import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
|
|
|
+import org.apache.hadoop.security.SecurityUtil;
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
|
|
|
import org.apache.hadoop.security.authorize.ProxyUsers;
|
|
|
import org.apache.hadoop.service.AbstractService;
|
|
@@ -40,6 +48,7 @@ import org.apache.hadoop.util.JvmPauseMonitor;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
+import org.apache.hadoop.util.ZKUtil;
|
|
|
import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
@@ -58,8 +67,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMaste
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
|
@@ -78,7 +87,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAlloca
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
|
|
@@ -96,12 +107,15 @@ import org.apache.hadoop.yarn.webapp.WebApp;
|
|
|
import org.apache.hadoop.yarn.webapp.WebApps;
|
|
|
import org.apache.hadoop.yarn.webapp.WebApps.Builder;
|
|
|
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
|
|
|
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
import java.io.PrintStream;
|
|
|
import java.net.InetSocketAddress;
|
|
|
+import java.nio.charset.Charset;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
+import java.security.SecureRandom;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.BlockingQueue;
|
|
@@ -158,6 +172,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|
|
protected ResourceTrackerService resourceTracker;
|
|
|
private JvmPauseMonitor pauseMonitor;
|
|
|
private boolean curatorEnabled = false;
|
|
|
+ private CuratorFramework curator;
|
|
|
+ private final String zkRootNodePassword =
|
|
|
+ Long.toString(new SecureRandom().nextLong());
|
|
|
+ private boolean recoveryEnabled;
|
|
|
|
|
|
@VisibleForTesting
|
|
|
protected String webAppAddress;
|
|
@@ -237,7 +255,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|
|
curatorEnabled = conf.getBoolean(YarnConfiguration.CURATOR_LEADER_ELECTOR,
|
|
|
YarnConfiguration.DEFAULT_CURATOR_LEADER_ELECTOR_ENABLED);
|
|
|
if (curatorEnabled) {
|
|
|
- LeaderElectorService elector = new LeaderElectorService(rmContext);
|
|
|
+ this.curator = createAndStartCurator(conf);
|
|
|
+ LeaderElectorService elector = new LeaderElectorService(rmContext, this);
|
|
|
addService(elector);
|
|
|
rmContext.setLeaderElectorService(elector);
|
|
|
}
|
|
@@ -281,7 +300,58 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|
|
|
|
|
super.serviceInit(this.conf);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ public CuratorFramework createAndStartCurator(Configuration conf)
|
|
|
+ throws Exception {
|
|
|
+ String zkHostPort = conf.get(YarnConfiguration.RM_ZK_ADDRESS);
|
|
|
+ if (zkHostPort == null) {
|
|
|
+ throw new YarnRuntimeException(
|
|
|
+ YarnConfiguration.RM_ZK_ADDRESS + " is not configured.");
|
|
|
+ }
|
|
|
+ int numRetries = conf.getInt(YarnConfiguration.RM_ZK_NUM_RETRIES,
|
|
|
+ YarnConfiguration.DEFAULT_ZK_RM_NUM_RETRIES);
|
|
|
+ int zkSessionTimeout = conf.getInt(YarnConfiguration.RM_ZK_TIMEOUT_MS,
|
|
|
+ YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
|
|
|
+ int zkRetryInterval = conf.getInt(YarnConfiguration.RM_ZK_RETRY_INTERVAL_MS,
|
|
|
+ YarnConfiguration.DEFAULT_RM_ZK_RETRY_INTERVAL_MS);
|
|
|
+
|
|
|
+ // set up zk auths
|
|
|
+ List<ZKUtil.ZKAuthInfo> zkAuths = RMZKUtils.getZKAuths(conf);
|
|
|
+ List<AuthInfo> authInfos = new ArrayList<>();
|
|
|
+ for (ZKUtil.ZKAuthInfo zkAuth : zkAuths) {
|
|
|
+ authInfos.add(new AuthInfo(zkAuth.getScheme(), zkAuth.getAuth()));
|
|
|
+ }
|
|
|
+
|
|
|
+ if (HAUtil.isHAEnabled(conf) && HAUtil.getConfValueForRMInstance(
|
|
|
+ YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf) == null) {
|
|
|
+ String zkRootNodeUsername = HAUtil
|
|
|
+ .getConfValueForRMInstance(YarnConfiguration.RM_ADDRESS,
|
|
|
+ YarnConfiguration.DEFAULT_RM_ADDRESS, conf);
|
|
|
+ byte[] defaultFencingAuth =
|
|
|
+ (zkRootNodeUsername + ":" + zkRootNodePassword)
|
|
|
+ .getBytes(Charset.forName("UTF-8"));
|
|
|
+ authInfos.add(new AuthInfo(new DigestAuthenticationProvider().getScheme(),
|
|
|
+ defaultFencingAuth));
|
|
|
+ }
|
|
|
+
|
|
|
+ CuratorFramework client = CuratorFrameworkFactory.builder()
|
|
|
+ .connectString(zkHostPort)
|
|
|
+ .sessionTimeoutMs(zkSessionTimeout)
|
|
|
+ .retryPolicy(new RetryNTimes(numRetries, zkRetryInterval))
|
|
|
+ .authorization(authInfos).build();
|
|
|
+ client.start();
|
|
|
+ return client;
|
|
|
+ }
|
|
|
+
|
|
|
+ public CuratorFramework getCurator() {
|
|
|
+ return this.curator;
|
|
|
+ }
|
|
|
+
|
|
|
+ public String getZkRootNodePassword() {
|
|
|
+ return this.zkRootNodePassword;
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
protected QueueACLsManager createQueueACLsManager(ResourceScheduler scheduler,
|
|
|
Configuration conf) {
|
|
|
return new QueueACLsManager(scheduler, conf);
|
|
@@ -417,7 +487,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|
|
private ApplicationMasterLauncher applicationMasterLauncher;
|
|
|
private ContainerAllocationExpirer containerAllocationExpirer;
|
|
|
private ResourceManager rm;
|
|
|
- private boolean recoveryEnabled;
|
|
|
private RMActiveServiceContext activeServiceContext;
|
|
|
|
|
|
RMActiveServices(ResourceManager rm) {
|
|
@@ -458,29 +527,26 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|
|
rmContext.setRMDelegatedNodeLabelsUpdater(delegatedNodeLabelsUpdater);
|
|
|
}
|
|
|
|
|
|
- boolean isRecoveryEnabled = conf.getBoolean(
|
|
|
- YarnConfiguration.RECOVERY_ENABLED,
|
|
|
+ recoveryEnabled = conf.getBoolean(YarnConfiguration.RECOVERY_ENABLED,
|
|
|
YarnConfiguration.DEFAULT_RM_RECOVERY_ENABLED);
|
|
|
|
|
|
RMStateStore rmStore = null;
|
|
|
- if (isRecoveryEnabled) {
|
|
|
- recoveryEnabled = true;
|
|
|
+ if (recoveryEnabled) {
|
|
|
rmStore = RMStateStoreFactory.getStore(conf);
|
|
|
boolean isWorkPreservingRecoveryEnabled =
|
|
|
conf.getBoolean(
|
|
|
YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED,
|
|
|
YarnConfiguration.DEFAULT_RM_WORK_PRESERVING_RECOVERY_ENABLED);
|
|
|
rmContext
|
|
|
- .setWorkPreservingRecoveryEnabled(isWorkPreservingRecoveryEnabled);
|
|
|
+ .setWorkPreservingRecoveryEnabled(isWorkPreservingRecoveryEnabled);
|
|
|
} else {
|
|
|
- recoveryEnabled = false;
|
|
|
rmStore = new NullRMStateStore();
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
+ rmStore.setResourceManager(rm);
|
|
|
rmStore.init(conf);
|
|
|
rmStore.setRMDispatcher(rmDispatcher);
|
|
|
- rmStore.setResourceManager(rm);
|
|
|
} catch (Exception e) {
|
|
|
// the Exception from stateStore.init() needs to be handled for
|
|
|
// HA and we need to give up master status if we got fenced
|
|
@@ -1117,6 +1183,9 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|
|
configurationProvider.close();
|
|
|
}
|
|
|
super.serviceStop();
|
|
|
+ if (curator != null) {
|
|
|
+ curator.close();
|
|
|
+ }
|
|
|
transitionToStandby(false);
|
|
|
rmContext.setHAServiceState(HAServiceState.STOPPING);
|
|
|
}
|
|
@@ -1164,7 +1233,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|
|
public ClientRMService getClientRMService() {
|
|
|
return this.clientRM;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* return the scheduler.
|
|
|
* @return the scheduler for the Resource Manager.
|
|
@@ -1335,5 +1404,4 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|
|
out.println(" "
|
|
|
+ "[-remove-application-from-state-store <appId>]" + "\n");
|
|
|
}
|
|
|
-
|
|
|
}
|