|
@@ -27,6 +27,7 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
|
@@ -63,12 +64,14 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequ
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerResponse;
|
|
|
+import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
|
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
|
|
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
|
|
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
@@ -118,6 +121,8 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
private boolean isDelegatedCentralizedNodeLabelsConf;
|
|
|
private DynamicResourceConfiguration drConf;
|
|
|
|
|
|
+ private final AtomicLong timelineCollectorVersion = new AtomicLong(0);
|
|
|
+
|
|
|
public ResourceTrackerService(RMContext rmContext,
|
|
|
NodesListManager nodesListManager,
|
|
|
NMLivelinessMonitor nmLivelinessMonitor,
|
|
@@ -525,9 +530,6 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
YarnConfiguration.timelineServiceV2Enabled(getConfig());
|
|
|
if (timelineV2Enabled) {
|
|
|
// Check & update collectors info from request.
|
|
|
- // TODO make sure it won't have race condition issue for AM failed over
|
|
|
- // case that the older registration could possible override the newer
|
|
|
- // one.
|
|
|
updateAppCollectorsMap(request);
|
|
|
}
|
|
|
|
|
@@ -613,14 +615,14 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
|
|
|
private void setAppCollectorsMapToResponse(
|
|
|
List<ApplicationId> runningApps, NodeHeartbeatResponse response) {
|
|
|
- Map<ApplicationId, String> liveAppCollectorsMap = new
|
|
|
- HashMap<ApplicationId, String>();
|
|
|
+ Map<ApplicationId, AppCollectorData> liveAppCollectorsMap = new
|
|
|
+ HashMap<>();
|
|
|
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
|
|
|
// Set collectors for all running apps on this node.
|
|
|
for (ApplicationId appId : runningApps) {
|
|
|
- String appCollectorAddr = rmApps.get(appId).getCollectorAddr();
|
|
|
- if (appCollectorAddr != null) {
|
|
|
- liveAppCollectorsMap.put(appId, appCollectorAddr);
|
|
|
+ AppCollectorData appCollectorData = rmApps.get(appId).getCollectorData();
|
|
|
+ if (appCollectorData != null) {
|
|
|
+ liveAppCollectorsMap.put(appId, appCollectorData);
|
|
|
} else {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Collector for applicaton: " + appId +
|
|
@@ -628,29 +630,43 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- response.setAppCollectorsMap(liveAppCollectorsMap);
|
|
|
+ response.setAppCollectors(liveAppCollectorsMap);
|
|
|
}
|
|
|
|
|
|
private void updateAppCollectorsMap(NodeHeartbeatRequest request) {
|
|
|
- Map<ApplicationId, String> registeredCollectorsMap =
|
|
|
- request.getRegisteredCollectors();
|
|
|
- if (registeredCollectorsMap != null
|
|
|
- && !registeredCollectorsMap.isEmpty()) {
|
|
|
+ Map<ApplicationId, AppCollectorData> registeringCollectorsMap =
|
|
|
+ request.getRegisteringCollectors();
|
|
|
+ if (registeringCollectorsMap != null
|
|
|
+ && !registeringCollectorsMap.isEmpty()) {
|
|
|
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
|
|
|
- for (Map.Entry<ApplicationId, String> entry:
|
|
|
- registeredCollectorsMap.entrySet()) {
|
|
|
+ for (Map.Entry<ApplicationId, AppCollectorData> entry:
|
|
|
+ registeringCollectorsMap.entrySet()) {
|
|
|
ApplicationId appId = entry.getKey();
|
|
|
- String collectorAddr = entry.getValue();
|
|
|
- if (collectorAddr != null && !collectorAddr.isEmpty()) {
|
|
|
+ AppCollectorData collectorData = entry.getValue();
|
|
|
+ if (collectorData != null) {
|
|
|
+ if (!collectorData.isStamped()) {
|
|
|
+ // Stamp the collector if we have not done so
|
|
|
+ collectorData.setRMIdentifier(
|
|
|
+ ResourceManager.getClusterTimeStamp());
|
|
|
+ collectorData.setVersion(
|
|
|
+ timelineCollectorVersion.getAndIncrement());
|
|
|
+ }
|
|
|
RMApp rmApp = rmApps.get(appId);
|
|
|
if (rmApp == null) {
|
|
|
LOG.warn("Cannot update collector info because application ID: " +
|
|
|
appId + " is not found in RMContext!");
|
|
|
} else {
|
|
|
- String previousCollectorAddr = rmApp.getCollectorAddr();
|
|
|
- if (previousCollectorAddr == null
|
|
|
- || !previousCollectorAddr.equals(collectorAddr)) {
|
|
|
- rmApp.setCollectorAddr(collectorAddr);
|
|
|
+ AppCollectorData previousCollectorData = rmApp.getCollectorData();
|
|
|
+ if (AppCollectorData.happensBefore(previousCollectorData,
|
|
|
+ collectorData)) {
|
|
|
+ // Sending collector update event.
|
|
|
+ // Note: RM has to store the newly received collector data
|
|
|
+ // synchronously. Otherwise, the RM may send out stale collector
|
|
|
+ // data before this update is done, and the RM then crashes, the
|
|
|
+ // newly updated collector data will get lost.
|
|
|
+ LOG.info("Update collector information for application " + appId
|
|
|
+ + " with new address: " + collectorData.getCollectorAddr());
|
|
|
+ ((RMAppImpl) rmApp).setCollectorData(collectorData);
|
|
|
}
|
|
|
}
|
|
|
}
|