@InterfaceAudience.Public @InterfaceStability.Stable public abstract class AMRMClientAsync<T extends org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest> extends AbstractService
AMRMClientAsync
handles communication with the ResourceManager
and provides asynchronous updates on events such as container allocations and
completions. It contains a thread that sends periodic heartbeats to the
ResourceManager.
It should be used by implementing a CallbackHandler:
class MyCallbackHandler implements AMRMClientAsync.CallbackHandler {
public void onContainersAllocated(List<Container> containers) {
[run tasks on the containers]
}
public void onContainersCompleted(List<ContainerStatus> statuses) {
[update progress, check whether app is done]
}
public void onNodesUpdated(List<NodeReport> updated) {}
public void onReboot() {}
}
The client's lifecycle should be managed similarly to the following:
AMRMClientAsync asyncClient =
createAMRMClientAsync(appAttId, 1000, new MyCallbackhandler());
asyncClient.init(conf);
asyncClient.start();
RegisterApplicationMasterResponse response = asyncClient
.registerApplicationMaster(appMasterHostname, appMasterRpcPort,
appMasterTrackingUrl);
asyncClient.addContainerRequest(containerRequest);
[... wait for application to complete]
asyncClient.unregisterApplicationMaster(status, appMsg, trackingUrl);
asyncClient.stop();
Modifier and Type | Field and Description |
---|---|
protected AMRMClient<T> |
client |
protected org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler |
handler |
protected AtomicInteger |
heartbeatIntervalMs |
Modifier | Constructor and Description |
---|---|
protected |
AMRMClientAsync(AMRMClient<T> client,
int intervalMs,
org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler callbackHandler) |
protected |
AMRMClientAsync(int intervalMs,
org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler callbackHandler) |
Modifier and Type | Method and Description |
---|---|
abstract void |
addContainerRequest(T req)
Request containers for resources before calling
allocate |
static <T extends org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest> |
createAMRMClientAsync(AMRMClient<T> client,
int intervalMs,
org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler callbackHandler) |
static <T extends org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest> |
createAMRMClientAsync(int intervalMs,
org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler callbackHandler) |
abstract Resource |
getAvailableResources()
Get the currently available resources in the cluster.
|
abstract int |
getClusterNodeCount()
Get the current number of nodes in the cluster.
|
abstract List<? extends Collection<T>> |
getMatchingRequests(Priority priority,
String resourceName,
Resource capability) |
abstract RegisterApplicationMasterResponse |
registerApplicationMaster(String appHostName,
int appHostPort,
String appTrackingUrl)
Registers this application master with the resource manager.
|
abstract void |
releaseAssignedContainer(ContainerId containerId)
Release containers assigned by the Resource Manager.
|
abstract void |
removeContainerRequest(T req)
Remove previous container request.
|
void |
setHeartbeatInterval(int interval) |
abstract void |
unregisterApplicationMaster(FinalApplicationStatus appStatus,
String appMessage,
String appTrackingUrl)
Unregister the application master.
|
void |
waitFor(com.google.common.base.Supplier<Boolean> check)
Wait for
check to return true for each 1000 ms. |
void |
waitFor(com.google.common.base.Supplier<Boolean> check,
int checkEveryMillis)
Wait for
check to return true for each
checkEveryMillis ms. |
void |
waitFor(com.google.common.base.Supplier<Boolean> check,
int checkEveryMillis,
int logInterval)
Wait for
check to return true for each
checkEveryMillis ms. |
close, getBlockers, getConfig, getFailureCause, getFailureState, getLifecycleHistory, getName, getServiceState, getStartTime, init, isInState, noteFailure, putBlocker, registerGlobalListener, registerServiceListener, removeBlocker, serviceInit, serviceStart, serviceStop, setConfig, start, stop, toString, unregisterGlobalListener, unregisterServiceListener, waitForServiceToStop
protected final AMRMClient<T extends org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest> client
protected final org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler handler
protected final AtomicInteger heartbeatIntervalMs
protected AMRMClientAsync(int intervalMs, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler callbackHandler)
@InterfaceAudience.Private protected AMRMClientAsync(AMRMClient<T> client, int intervalMs, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler callbackHandler)
public static <T extends org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest> AMRMClientAsync<T> createAMRMClientAsync(int intervalMs, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler callbackHandler)
public static <T extends org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest> AMRMClientAsync<T> createAMRMClientAsync(AMRMClient<T> client, int intervalMs, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler callbackHandler)
public void setHeartbeatInterval(int interval)
public abstract List<? extends Collection<T>> getMatchingRequests(Priority priority, String resourceName, Resource capability)
public abstract RegisterApplicationMasterResponse registerApplicationMaster(String appHostName, int appHostPort, String appTrackingUrl) throws YarnException, IOException
YarnException
IOException
public abstract void unregisterApplicationMaster(FinalApplicationStatus appStatus, String appMessage, String appTrackingUrl) throws YarnException, IOException
appStatus
- Success/Failure status of the masterappMessage
- Diagnostics message on failureappTrackingUrl
- New URL to get master infoYarnException
IOException
public abstract void addContainerRequest(T req)
allocate
req
- Resource requestpublic abstract void removeContainerRequest(T req)
req
- Resource requestpublic abstract void releaseAssignedContainer(ContainerId containerId)
containerId
- public abstract Resource getAvailableResources()
public abstract int getClusterNodeCount()
public void waitFor(com.google.common.base.Supplier<Boolean> check) throws InterruptedException
check
to return true for each 1000 ms.
See also waitFor(com.google.common.base.Supplier, int)
and waitFor(com.google.common.base.Supplier, int, int)
check
- InterruptedException
public void waitFor(com.google.common.base.Supplier<Boolean> check, int checkEveryMillis) throws InterruptedException
check
to return true for each
checkEveryMillis
ms.
See also waitFor(com.google.common.base.Supplier, int, int)
check
- user defined checkercheckEveryMillis
- interval to call check
InterruptedException
public void waitFor(com.google.common.base.Supplier<Boolean> check, int checkEveryMillis, int logInterval) throws InterruptedException
check
to return true for each
checkEveryMillis
ms. In the main loop, this method will log
the message "waiting in main loop" for each logInterval
times
iteration to confirm the thread is alive.check
- user defined checkercheckEveryMillis
- interval to call check
logInterval
- interval to log for eachInterruptedException
Copyright © 2015 Apache Software Foundation. All Rights Reserved.