|
@@ -64,7 +64,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
|
|
-
|
|
|
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredAMRMProxyState;
|
|
import org.apache.hadoop.yarn.server.nodemanager.scheduler.DistributedScheduler;
|
|
import org.apache.hadoop.yarn.server.nodemanager.scheduler.DistributedScheduler;
|
|
import org.apache.hadoop.yarn.server.security.MasterKeyData;
|
|
import org.apache.hadoop.yarn.server.security.MasterKeyData;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
@@ -86,6 +86,10 @@ public class AMRMProxyService extends AbstractService implements
|
|
ApplicationMasterProtocol {
|
|
ApplicationMasterProtocol {
|
|
private static final Logger LOG = LoggerFactory
|
|
private static final Logger LOG = LoggerFactory
|
|
.getLogger(AMRMProxyService.class);
|
|
.getLogger(AMRMProxyService.class);
|
|
|
|
+
|
|
|
|
+ private static final String NMSS_USER_KEY = "user";
|
|
|
|
+ private static final String NMSS_AMRMTOKEN_KEY = "amrmtoken";
|
|
|
|
+
|
|
private Server server;
|
|
private Server server;
|
|
private final Context nmContext;
|
|
private final Context nmContext;
|
|
private final AsyncDispatcher dispatcher;
|
|
private final AsyncDispatcher dispatcher;
|
|
@@ -95,9 +99,9 @@ public class AMRMProxyService extends AbstractService implements
|
|
|
|
|
|
/**
|
|
/**
|
|
* Creates an instance of the service.
|
|
* Creates an instance of the service.
|
|
- *
|
|
|
|
- * @param nmContext
|
|
|
|
- * @param dispatcher
|
|
|
|
|
|
+ *
|
|
|
|
+ * @param nmContext NM context
|
|
|
|
+ * @param dispatcher NM dispatcher
|
|
*/
|
|
*/
|
|
public AMRMProxyService(Context nmContext, AsyncDispatcher dispatcher) {
|
|
public AMRMProxyService(Context nmContext, AsyncDispatcher dispatcher) {
|
|
super(AMRMProxyService.class.getName());
|
|
super(AMRMProxyService.class.getName());
|
|
@@ -112,6 +116,14 @@ public class AMRMProxyService extends AbstractService implements
|
|
new ApplicationEventHandler());
|
|
new ApplicationEventHandler());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ protected void serviceInit(Configuration conf) throws Exception {
|
|
|
|
+ super.serviceInit(conf);
|
|
|
|
+ this.secretManager =
|
|
|
|
+ new AMRMProxyTokenSecretManager(this.nmContext.getNMStateStore());
|
|
|
|
+ this.secretManager.init(conf);
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
protected void serviceStart() throws Exception {
|
|
protected void serviceStart() throws Exception {
|
|
LOG.info("Starting AMRMProxyService");
|
|
LOG.info("Starting AMRMProxyService");
|
|
@@ -134,7 +146,6 @@ public class AMRMProxyService extends AbstractService implements
|
|
YarnConfiguration.AMRM_PROXY_CLIENT_THREAD_COUNT,
|
|
YarnConfiguration.AMRM_PROXY_CLIENT_THREAD_COUNT,
|
|
YarnConfiguration.DEFAULT_AMRM_PROXY_CLIENT_THREAD_COUNT);
|
|
YarnConfiguration.DEFAULT_AMRM_PROXY_CLIENT_THREAD_COUNT);
|
|
|
|
|
|
- this.secretManager = new AMRMProxyTokenSecretManager(serverConf);
|
|
|
|
this.secretManager.start();
|
|
this.secretManager.start();
|
|
|
|
|
|
this.server =
|
|
this.server =
|
|
@@ -160,6 +171,62 @@ public class AMRMProxyService extends AbstractService implements
|
|
super.serviceStop();
|
|
super.serviceStop();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Recover from NM state store. Called after serviceInit before serviceStart.
|
|
|
|
+ *
|
|
|
|
+ * @throws IOException if recover fails
|
|
|
|
+ */
|
|
|
|
+ public void recover() throws IOException {
|
|
|
|
+ LOG.info("Recovering AMRMProxyService");
|
|
|
|
+
|
|
|
|
+ RecoveredAMRMProxyState state =
|
|
|
|
+ this.nmContext.getNMStateStore().loadAMRMProxyState();
|
|
|
|
+
|
|
|
|
+ this.secretManager.recover(state);
|
|
|
|
+
|
|
|
|
+ LOG.info("Recovering {} running applications for AMRMProxy",
|
|
|
|
+ state.getAppContexts().size());
|
|
|
|
+ for (Map.Entry<ApplicationAttemptId, Map<String, byte[]>> entry : state
|
|
|
|
+ .getAppContexts().entrySet()) {
|
|
|
|
+ ApplicationAttemptId attemptId = entry.getKey();
|
|
|
|
+ LOG.info("Recovering app attempt {}", attemptId);
|
|
|
|
+
|
|
|
|
+ // Try recover for the running application attempt
|
|
|
|
+ try {
|
|
|
|
+ String user = null;
|
|
|
|
+ Token<AMRMTokenIdentifier> amrmToken = null;
|
|
|
|
+ for (Map.Entry<String, byte[]> contextEntry : entry.getValue()
|
|
|
|
+ .entrySet()) {
|
|
|
|
+ if (contextEntry.getKey().equals(NMSS_USER_KEY)) {
|
|
|
|
+ user = new String(contextEntry.getValue(), "UTF-8");
|
|
|
|
+ } else if (contextEntry.getKey().equals(NMSS_AMRMTOKEN_KEY)) {
|
|
|
|
+ amrmToken = new Token<>();
|
|
|
|
+ amrmToken.decodeFromUrlString(
|
|
|
|
+ new String(contextEntry.getValue(), "UTF-8"));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (amrmToken == null) {
|
|
|
|
+ throw new IOException(
|
|
|
|
+ "No amrmToken found for app attempt " + attemptId);
|
|
|
|
+ }
|
|
|
|
+ if (user == null) {
|
|
|
|
+ throw new IOException("No user found for app attempt " + attemptId);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Token<AMRMTokenIdentifier> localToken =
|
|
|
|
+ this.secretManager.createAndGetAMRMToken(attemptId);
|
|
|
|
+
|
|
|
|
+ initializePipeline(attemptId, user, amrmToken, localToken,
|
|
|
|
+ entry.getValue(), true);
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.error("Exception when recovering " + attemptId
|
|
|
|
+ + ", removing it from NMStateStore and move on", e);
|
|
|
|
+ this.nmContext.getNMStateStore().removeAMRMProxyAppContext(attemptId);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* This is called by the AMs started on this node to register with the RM.
|
|
* This is called by the AMs started on this node to register with the RM.
|
|
* This method does the initial authorization and then forwards the request to
|
|
* This method does the initial authorization and then forwards the request to
|
|
@@ -221,8 +288,8 @@ public class AMRMProxyService extends AbstractService implements
|
|
* application request processing pipeline.
|
|
* application request processing pipeline.
|
|
*
|
|
*
|
|
* @param request - encapsulates information for starting an AM
|
|
* @param request - encapsulates information for starting an AM
|
|
- * @throws IOException
|
|
|
|
- * @throws YarnException
|
|
|
|
|
|
+ * @throws IOException if fails
|
|
|
|
+ * @throws YarnException if fails
|
|
*/
|
|
*/
|
|
public void processApplicationStartRequest(StartContainerRequest request)
|
|
public void processApplicationStartRequest(StartContainerRequest request)
|
|
throws IOException, YarnException {
|
|
throws IOException, YarnException {
|
|
@@ -257,22 +324,25 @@ public class AMRMProxyService extends AbstractService implements
|
|
request.getContainerLaunchContext().setTokens(
|
|
request.getContainerLaunchContext().setTokens(
|
|
ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
|
|
ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
|
|
|
|
|
|
- initializePipeline(containerTokenIdentifierForKey.getContainerID()
|
|
|
|
- .getApplicationAttemptId(),
|
|
|
|
- containerTokenIdentifierForKey.getApplicationSubmitter(),
|
|
|
|
- amrmToken, localToken);
|
|
|
|
|
|
+ initializePipeline(appAttemptId,
|
|
|
|
+ containerTokenIdentifierForKey.getApplicationSubmitter(), amrmToken,
|
|
|
|
+ localToken, null, false);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Initializes the request intercepter pipeline for the specified application.
|
|
* Initializes the request intercepter pipeline for the specified application.
|
|
- *
|
|
|
|
- * @param applicationAttemptId
|
|
|
|
- * @param user
|
|
|
|
- * @param amrmToken
|
|
|
|
|
|
+ *
|
|
|
|
+ * @param applicationAttemptId attempt id
|
|
|
|
+ * @param user user name
|
|
|
|
+ * @param amrmToken amrmToken issued by RM
|
|
|
|
+ * @param localToken amrmToken issued by AMRMProxy
|
|
|
|
+ * @param recoveredDataMap the recovered states for AMRMProxy from NMSS
|
|
|
|
+ * @param isRecovery whether this is to recover a previously existing pipeline
|
|
*/
|
|
*/
|
|
protected void initializePipeline(ApplicationAttemptId applicationAttemptId,
|
|
protected void initializePipeline(ApplicationAttemptId applicationAttemptId,
|
|
String user, Token<AMRMTokenIdentifier> amrmToken,
|
|
String user, Token<AMRMTokenIdentifier> amrmToken,
|
|
- Token<AMRMTokenIdentifier> localToken) {
|
|
|
|
|
|
+ Token<AMRMTokenIdentifier> localToken,
|
|
|
|
+ Map<String, byte[]> recoveredDataMap, boolean isRecovery) {
|
|
RequestInterceptorChainWrapper chainWrapper = null;
|
|
RequestInterceptorChainWrapper chainWrapper = null;
|
|
synchronized (applPipelineMap) {
|
|
synchronized (applPipelineMap) {
|
|
if (applPipelineMap
|
|
if (applPipelineMap
|
|
@@ -288,11 +358,23 @@ public class AMRMProxyService extends AbstractService implements
|
|
&& chainWrapperBackup.getApplicationAttemptId() != null
|
|
&& chainWrapperBackup.getApplicationAttemptId() != null
|
|
&& !chainWrapperBackup.getApplicationAttemptId()
|
|
&& !chainWrapperBackup.getApplicationAttemptId()
|
|
.equals(applicationAttemptId)) {
|
|
.equals(applicationAttemptId)) {
|
|
|
|
+ // TODO: revisit in AMRMProxy HA in YARN-6128
|
|
// Remove the existing pipeline
|
|
// Remove the existing pipeline
|
|
LOG.info("Remove the previous pipeline for ApplicationId: "
|
|
LOG.info("Remove the previous pipeline for ApplicationId: "
|
|
+ applicationAttemptId.toString());
|
|
+ applicationAttemptId.toString());
|
|
RequestInterceptorChainWrapper pipeline =
|
|
RequestInterceptorChainWrapper pipeline =
|
|
applPipelineMap.remove(applicationAttemptId.getApplicationId());
|
|
applPipelineMap.remove(applicationAttemptId.getApplicationId());
|
|
|
|
+
|
|
|
|
+ if (!isRecovery && this.nmContext.getNMStateStore() != null) {
|
|
|
|
+ try {
|
|
|
|
+ this.nmContext.getNMStateStore()
|
|
|
|
+ .removeAMRMProxyAppContext(applicationAttemptId);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Error removing AMRMProxy application context for "
|
|
|
|
+ + applicationAttemptId, e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
try {
|
|
try {
|
|
pipeline.getRootInterceptor().shutdown();
|
|
pipeline.getRootInterceptor().shutdown();
|
|
} catch (Throwable ex) {
|
|
} catch (Throwable ex) {
|
|
@@ -324,7 +406,27 @@ public class AMRMProxyService extends AbstractService implements
|
|
this.createRequestInterceptorChain();
|
|
this.createRequestInterceptorChain();
|
|
interceptorChain.init(createApplicationMasterContext(this.nmContext,
|
|
interceptorChain.init(createApplicationMasterContext(this.nmContext,
|
|
applicationAttemptId, user, amrmToken, localToken));
|
|
applicationAttemptId, user, amrmToken, localToken));
|
|
|
|
+ if (isRecovery) {
|
|
|
|
+ if (recoveredDataMap == null) {
|
|
|
|
+ throw new YarnRuntimeException(
|
|
|
|
+ "null recoveredDataMap recieved for recover");
|
|
|
|
+ }
|
|
|
|
+ interceptorChain.recover(recoveredDataMap);
|
|
|
|
+ }
|
|
chainWrapper.init(interceptorChain, applicationAttemptId);
|
|
chainWrapper.init(interceptorChain, applicationAttemptId);
|
|
|
|
+
|
|
|
|
+ if (!isRecovery && this.nmContext.getNMStateStore() != null) {
|
|
|
|
+ try {
|
|
|
|
+ this.nmContext.getNMStateStore().storeAMRMProxyAppContextEntry(
|
|
|
|
+ applicationAttemptId, NMSS_USER_KEY, user.getBytes("UTF-8"));
|
|
|
|
+ this.nmContext.getNMStateStore().storeAMRMProxyAppContextEntry(
|
|
|
|
+ applicationAttemptId, NMSS_AMRMTOKEN_KEY,
|
|
|
|
+ amrmToken.encodeToUrlString().getBytes("UTF-8"));
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Error storing AMRMProxy application context entry for "
|
|
|
|
+ + applicationAttemptId, e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
} catch (Exception e) {
|
|
} catch (Exception e) {
|
|
this.applPipelineMap.remove(applicationAttemptId.getApplicationId());
|
|
this.applPipelineMap.remove(applicationAttemptId.getApplicationId());
|
|
throw e;
|
|
throw e;
|
|
@@ -335,7 +437,7 @@ public class AMRMProxyService extends AbstractService implements
|
|
* Shuts down the request processing pipeline for the specified application
|
|
* Shuts down the request processing pipeline for the specified application
|
|
* attempt id.
|
|
* attempt id.
|
|
*
|
|
*
|
|
- * @param applicationId
|
|
|
|
|
|
+ * @param applicationId application id
|
|
*/
|
|
*/
|
|
protected void stopApplication(ApplicationId applicationId) {
|
|
protected void stopApplication(ApplicationId applicationId) {
|
|
Preconditions.checkArgument(applicationId != null,
|
|
Preconditions.checkArgument(applicationId != null,
|
|
@@ -362,6 +464,17 @@ public class AMRMProxyService extends AbstractService implements
|
|
"Failed to shutdown the request processing pipeline for app:"
|
|
"Failed to shutdown the request processing pipeline for app:"
|
|
+ applicationId, ex);
|
|
+ applicationId, ex);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // Remove the app context from NMSS after the interceptors are shutdown
|
|
|
|
+ if (this.nmContext.getNMStateStore() != null) {
|
|
|
|
+ try {
|
|
|
|
+ this.nmContext.getNMStateStore()
|
|
|
|
+ .removeAMRMProxyAppContext(pipeline.getApplicationAttemptId());
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Error removing AMRMProxy application context for "
|
|
|
|
+ + applicationId, e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -383,12 +496,24 @@ public class AMRMProxyService extends AbstractService implements
|
|
// Do not propagate this info back to AM
|
|
// Do not propagate this info back to AM
|
|
allocateResponse.setAMRMToken(null);
|
|
allocateResponse.setAMRMToken(null);
|
|
|
|
|
|
- org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> newTokenId =
|
|
|
|
|
|
+ org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> newToken =
|
|
new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>(
|
|
new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>(
|
|
token.getIdentifier().array(), token.getPassword().array(),
|
|
token.getIdentifier().array(), token.getPassword().array(),
|
|
new Text(token.getKind()), new Text(token.getService()));
|
|
new Text(token.getKind()), new Text(token.getService()));
|
|
|
|
|
|
- context.setAMRMToken(newTokenId);
|
|
|
|
|
|
+ context.setAMRMToken(newToken);
|
|
|
|
+
|
|
|
|
+ // Update the AMRMToken in context map in NM state store
|
|
|
|
+ if (this.nmContext.getNMStateStore() != null) {
|
|
|
|
+ try {
|
|
|
|
+ this.nmContext.getNMStateStore().storeAMRMProxyAppContextEntry(
|
|
|
|
+ context.getApplicationAttemptId(), NMSS_AMRMTOKEN_KEY,
|
|
|
|
+ newToken.encodeToUrlString().getBytes("UTF-8"));
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Error storing AMRMProxy application context entry for "
|
|
|
|
+ + context.getApplicationAttemptId(), e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
// Check if the local AMRMToken is rolled up and update the context and
|
|
// Check if the local AMRMToken is rolled up and update the context and
|
|
@@ -431,7 +556,7 @@ public class AMRMProxyService extends AbstractService implements
|
|
|
|
|
|
/**
|
|
/**
|
|
* Gets the Request intercepter chains for all the applications.
|
|
* Gets the Request intercepter chains for all the applications.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @return the request intercepter chains.
|
|
* @return the request intercepter chains.
|
|
*/
|
|
*/
|
|
protected Map<ApplicationId, RequestInterceptorChainWrapper> getPipelines() {
|
|
protected Map<ApplicationId, RequestInterceptorChainWrapper> getPipelines() {
|
|
@@ -489,7 +614,7 @@ public class AMRMProxyService extends AbstractService implements
|
|
/**
|
|
/**
|
|
* Returns the comma separated intercepter class names from the configuration.
|
|
* Returns the comma separated intercepter class names from the configuration.
|
|
*
|
|
*
|
|
- * @param conf
|
|
|
|
|
|
+ * @param conf configuration
|
|
* @return the intercepter class names as an instance of ArrayList
|
|
* @return the intercepter class names as an instance of ArrayList
|
|
*/
|
|
*/
|
|
private List<String> getInterceptorClassNames(Configuration conf) {
|
|
private List<String> getInterceptorClassNames(Configuration conf) {
|
|
@@ -518,7 +643,7 @@ public class AMRMProxyService extends AbstractService implements
|
|
* processing pipeline.
|
|
* processing pipeline.
|
|
*
|
|
*
|
|
* @return the the intercepter wrapper instance
|
|
* @return the the intercepter wrapper instance
|
|
- * @throws YarnException
|
|
|
|
|
|
+ * @throws YarnException if fails
|
|
*/
|
|
*/
|
|
private RequestInterceptorChainWrapper authorizeAndGetInterceptorChain()
|
|
private RequestInterceptorChainWrapper authorizeAndGetInterceptorChain()
|
|
throws YarnException {
|
|
throws YarnException {
|
|
@@ -612,8 +737,8 @@ public class AMRMProxyService extends AbstractService implements
|
|
/**
|
|
/**
|
|
* Initializes the wrapper with the specified parameters.
|
|
* Initializes the wrapper with the specified parameters.
|
|
*
|
|
*
|
|
- * @param rootInterceptor
|
|
|
|
- * @param applicationAttemptId
|
|
|
|
|
|
+ * @param rootInterceptor the root request intercepter
|
|
|
|
+ * @param applicationAttemptId attempt id
|
|
*/
|
|
*/
|
|
public synchronized void init(RequestInterceptor rootInterceptor,
|
|
public synchronized void init(RequestInterceptor rootInterceptor,
|
|
ApplicationAttemptId applicationAttemptId) {
|
|
ApplicationAttemptId applicationAttemptId) {
|
|
@@ -623,7 +748,7 @@ public class AMRMProxyService extends AbstractService implements
|
|
|
|
|
|
/**
|
|
/**
|
|
* Gets the root request intercepter.
|
|
* Gets the root request intercepter.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @return the root request intercepter
|
|
* @return the root request intercepter
|
|
*/
|
|
*/
|
|
public synchronized RequestInterceptor getRootInterceptor() {
|
|
public synchronized RequestInterceptor getRootInterceptor() {
|
|
@@ -632,7 +757,7 @@ public class AMRMProxyService extends AbstractService implements
|
|
|
|
|
|
/**
|
|
/**
|
|
* Gets the application attempt identifier.
|
|
* Gets the application attempt identifier.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @return the application attempt identifier
|
|
* @return the application attempt identifier
|
|
*/
|
|
*/
|
|
public synchronized ApplicationAttemptId getApplicationAttemptId() {
|
|
public synchronized ApplicationAttemptId getApplicationAttemptId() {
|
|
@@ -641,7 +766,7 @@ public class AMRMProxyService extends AbstractService implements
|
|
|
|
|
|
/**
|
|
/**
|
|
* Gets the application identifier.
|
|
* Gets the application identifier.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @return the application identifier
|
|
* @return the application identifier
|
|
*/
|
|
*/
|
|
public synchronized ApplicationId getApplicationId() {
|
|
public synchronized ApplicationId getApplicationId() {
|