|
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
|
|
@@ -34,6 +35,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInf
|
|
|
import org.apache.hadoop.yarn.util.SystemClock;
|
|
|
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.ConcurrentLinkedQueue;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Set;
|
|
@@ -57,9 +59,10 @@ public class ActivitiesManager extends AbstractService {
|
|
|
private Set<NodeId> activeRecordedNodes;
|
|
|
private ConcurrentMap<ApplicationId, Long>
|
|
|
recordingAppActivitiesUntilSpecifiedTime;
|
|
|
- private ConcurrentMap<ApplicationId, AppAllocation> appsAllocation;
|
|
|
- private ConcurrentMap<ApplicationId, List<AppAllocation>>
|
|
|
- completedAppAllocations;
|
|
|
+ private ThreadLocal<Map<ApplicationId, AppAllocation>>
|
|
|
+ appsAllocation;
|
|
|
+ @VisibleForTesting
|
|
|
+ ConcurrentMap<ApplicationId, Queue<AppAllocation>> completedAppAllocations;
|
|
|
private boolean recordNextAvailableNode = false;
|
|
|
private List<NodeAllocation> lastAvailableNodeActivities = null;
|
|
|
private Thread cleanUpThread;
|
|
@@ -71,7 +74,7 @@ public class ActivitiesManager extends AbstractService {
|
|
|
super(ActivitiesManager.class.getName());
|
|
|
recordingNodesAllocation = ThreadLocal.withInitial(() -> new HashMap());
|
|
|
completedNodeAllocations = new ConcurrentHashMap<>();
|
|
|
- appsAllocation = new ConcurrentHashMap<>();
|
|
|
+ appsAllocation = ThreadLocal.withInitial(() -> new HashMap());
|
|
|
completedAppAllocations = new ConcurrentHashMap<>();
|
|
|
activeRecordedNodes = Collections.newSetFromMap(new ConcurrentHashMap<>());
|
|
|
recordingAppActivitiesUntilSpecifiedTime = new ConcurrentHashMap<>();
|
|
@@ -79,11 +82,15 @@ public class ActivitiesManager extends AbstractService {
|
|
|
}
|
|
|
|
|
|
public AppActivitiesInfo getAppActivitiesInfo(ApplicationId applicationId) {
|
|
|
- if (rmContext.getRMApps().get(applicationId).getFinalApplicationStatus()
|
|
|
+ RMApp app = rmContext.getRMApps().get(applicationId);
|
|
|
+ if (app != null && app.getFinalApplicationStatus()
|
|
|
== FinalApplicationStatus.UNDEFINED) {
|
|
|
- List<AppAllocation> allocations = completedAppAllocations.get(
|
|
|
- applicationId);
|
|
|
-
|
|
|
+ Queue<AppAllocation> curAllocations =
|
|
|
+ completedAppAllocations.get(applicationId);
|
|
|
+ List<AppAllocation> allocations = null;
|
|
|
+ if (curAllocations != null) {
|
|
|
+ allocations = new ArrayList(curAllocations);
|
|
|
+ }
|
|
|
return new AppActivitiesInfo(allocations, applicationId);
|
|
|
} else {
|
|
|
return new AppActivitiesInfo(
|
|
@@ -135,13 +142,13 @@ public class ActivitiesManager extends AbstractService {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- Iterator<Map.Entry<ApplicationId, List<AppAllocation>>> iteApp =
|
|
|
+ Iterator<Map.Entry<ApplicationId, Queue<AppAllocation>>> iteApp =
|
|
|
completedAppAllocations.entrySet().iterator();
|
|
|
while (iteApp.hasNext()) {
|
|
|
- Map.Entry<ApplicationId, List<AppAllocation>> appAllocation =
|
|
|
+ Map.Entry<ApplicationId, Queue<AppAllocation>> appAllocation =
|
|
|
iteApp.next();
|
|
|
- if (rmContext.getRMApps().get(appAllocation.getKey())
|
|
|
- .getFinalApplicationStatus()
|
|
|
+ RMApp rmApp = rmContext.getRMApps().get(appAllocation.getKey());
|
|
|
+ if (rmApp == null || rmApp.getFinalApplicationStatus()
|
|
|
!= FinalApplicationStatus.UNDEFINED) {
|
|
|
iteApp.remove();
|
|
|
}
|
|
@@ -191,18 +198,16 @@ public class ActivitiesManager extends AbstractService {
|
|
|
SchedulerApplicationAttempt application) {
|
|
|
ApplicationId applicationId = application.getApplicationId();
|
|
|
|
|
|
- if (recordingAppActivitiesUntilSpecifiedTime.containsKey(applicationId)
|
|
|
- && recordingAppActivitiesUntilSpecifiedTime.get(applicationId)
|
|
|
- > currTS) {
|
|
|
- appsAllocation.put(applicationId,
|
|
|
- new AppAllocation(application.getPriority(), nodeID,
|
|
|
- application.getQueueName()));
|
|
|
- }
|
|
|
-
|
|
|
- if (recordingAppActivitiesUntilSpecifiedTime.containsKey(applicationId)
|
|
|
- && recordingAppActivitiesUntilSpecifiedTime.get(applicationId)
|
|
|
- <= currTS) {
|
|
|
- turnOffActivityMonitoringForApp(applicationId);
|
|
|
+ Long turnOffTimestamp =
|
|
|
+ recordingAppActivitiesUntilSpecifiedTime.get(applicationId);
|
|
|
+ if (turnOffTimestamp != null) {
|
|
|
+ if (turnOffTimestamp > currTS) {
|
|
|
+ appsAllocation.get().put(applicationId,
|
|
|
+ new AppAllocation(application.getPriority(), nodeID,
|
|
|
+ application.getQueueName()));
|
|
|
+ } else {
|
|
|
+ turnOffActivityMonitoringForApp(applicationId);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -223,7 +228,7 @@ public class ActivitiesManager extends AbstractService {
|
|
|
ContainerId containerId, String priority, ActivityState state,
|
|
|
String diagnostic, String type) {
|
|
|
if (shouldRecordThisApp(applicationId)) {
|
|
|
- AppAllocation appAllocation = appsAllocation.get(applicationId);
|
|
|
+ AppAllocation appAllocation = appsAllocation.get().get(applicationId);
|
|
|
appAllocation.addAppAllocationActivity(containerId == null ?
|
|
|
"Container-Id-Not-Assigned" :
|
|
|
containerId.toString(), priority, state, diagnostic, type);
|
|
@@ -245,24 +250,27 @@ public class ActivitiesManager extends AbstractService {
|
|
|
ContainerId containerId, ActivityState appState, String diagnostic) {
|
|
|
if (shouldRecordThisApp(applicationId)) {
|
|
|
long currTS = SystemClock.getInstance().getTime();
|
|
|
- AppAllocation appAllocation = appsAllocation.remove(applicationId);
|
|
|
+ AppAllocation appAllocation = appsAllocation.get().remove(applicationId);
|
|
|
appAllocation.updateAppContainerStateAndTime(containerId, appState,
|
|
|
currTS, diagnostic);
|
|
|
|
|
|
- List<AppAllocation> appAllocations;
|
|
|
- if (completedAppAllocations.containsKey(applicationId)) {
|
|
|
- appAllocations = completedAppAllocations.get(applicationId);
|
|
|
- } else {
|
|
|
- appAllocations = new ArrayList<>();
|
|
|
- completedAppAllocations.put(applicationId, appAllocations);
|
|
|
+ Queue<AppAllocation> appAllocations =
|
|
|
+ completedAppAllocations.get(applicationId);
|
|
|
+ if (appAllocations == null) {
|
|
|
+ appAllocations = new ConcurrentLinkedQueue<>();
|
|
|
+ Queue<AppAllocation> curAppAllocations =
|
|
|
+ completedAppAllocations.putIfAbsent(applicationId, appAllocations);
|
|
|
+ if (curAppAllocations != null) {
|
|
|
+ appAllocations = curAppAllocations;
|
|
|
+ }
|
|
|
}
|
|
|
if (appAllocations.size() == 1000) {
|
|
|
- appAllocations.remove(0);
|
|
|
+ appAllocations.poll();
|
|
|
}
|
|
|
appAllocations.add(appAllocation);
|
|
|
-
|
|
|
- if (recordingAppActivitiesUntilSpecifiedTime.get(applicationId)
|
|
|
- <= currTS) {
|
|
|
+ Long stopTime =
|
|
|
+ recordingAppActivitiesUntilSpecifiedTime.get(applicationId);
|
|
|
+ if (stopTime != null && stopTime <= currTS) {
|
|
|
turnOffActivityMonitoringForApp(applicationId);
|
|
|
}
|
|
|
}
|
|
@@ -292,8 +300,12 @@ public class ActivitiesManager extends AbstractService {
|
|
|
}
|
|
|
|
|
|
boolean shouldRecordThisApp(ApplicationId applicationId) {
|
|
|
+ if (recordingAppActivitiesUntilSpecifiedTime.isEmpty()
|
|
|
+ || appsAllocation.get().isEmpty()) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
return recordingAppActivitiesUntilSpecifiedTime.containsKey(applicationId)
|
|
|
- && appsAllocation.containsKey(applicationId);
|
|
|
+ && appsAllocation.get().containsKey(applicationId);
|
|
|
}
|
|
|
|
|
|
boolean shouldRecordThisNode(NodeId nodeID) {
|