|
@@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
-import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
|
|
|
import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl;
|
|
|
|
|
@@ -77,7 +76,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretMan
|
|
|
|
|
|
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
|
|
|
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
|
|
-import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
@@ -103,7 +101,7 @@ public class OpportunisticContainerAllocatorAMService
|
|
|
private final NodeQueueLoadMonitor nodeMonitor;
|
|
|
private final OpportunisticContainerAllocator oppContainerAllocator;
|
|
|
|
|
|
- private final int k;
|
|
|
+ private final int numNodes;
|
|
|
|
|
|
private final long cacheRefreshInterval;
|
|
|
private volatile List<RemoteNode> cachedNodes;
|
|
@@ -176,7 +174,7 @@ public class OpportunisticContainerAllocatorAMService
|
|
|
|
|
|
if (!appAttempt.getApplicationAttemptId().equals(appAttemptId)){
|
|
|
LOG.error("Calling allocate on previous or removed or non "
|
|
|
- + "existent application attempt " + appAttemptId);
|
|
|
+ + "existent application attempt {}", appAttemptId);
|
|
|
return;
|
|
|
}
|
|
|
|
|
@@ -238,7 +236,7 @@ public class OpportunisticContainerAllocatorAMService
|
|
|
new DistributedOpportunisticContainerAllocator(
|
|
|
rmContext.getContainerTokenSecretManager(),
|
|
|
maxAllocationsPerAMHeartbeat);
|
|
|
- this.k = rmContext.getYarnConfiguration().getInt(
|
|
|
+ this.numNodes = rmContext.getYarnConfiguration().getInt(
|
|
|
YarnConfiguration.OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED,
|
|
|
YarnConfiguration.DEFAULT_OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED);
|
|
|
long nodeSortInterval = rmContext.getYarnConfiguration().getLong(
|
|
@@ -440,12 +438,12 @@ public class OpportunisticContainerAllocatorAMService
|
|
|
// <-- IGNORED EVENTS : END -->
|
|
|
default:
|
|
|
LOG.error("Unknown event arrived at" +
|
|
|
- "OpportunisticContainerAllocatorAMService: " + event.toString());
|
|
|
+ "OpportunisticContainerAllocatorAMService: {}", event);
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
- public QueueLimitCalculator getNodeManagerQueueLimitCalculator() {
|
|
|
+ QueueLimitCalculator getNodeManagerQueueLimitCalculator() {
|
|
|
return nodeMonitor.getThresholdCalculator();
|
|
|
}
|
|
|
|
|
@@ -455,7 +453,7 @@ public class OpportunisticContainerAllocatorAMService
|
|
|
if ((currTime - lastCacheUpdateTime > cacheRefreshInterval)
|
|
|
|| (cachedNodes == null)) {
|
|
|
cachedNodes = convertToRemoteNodes(
|
|
|
- this.nodeMonitor.selectLeastLoadedNodes(this.k));
|
|
|
+ this.nodeMonitor.selectLeastLoadedNodes(this.numNodes));
|
|
|
if (cachedNodes.size() > 0) {
|
|
|
lastCacheUpdateTime = currTime;
|
|
|
}
|
|
@@ -485,12 +483,4 @@ public class OpportunisticContainerAllocatorAMService
|
|
|
}
|
|
|
return null;
|
|
|
}
|
|
|
-
|
|
|
- private static ApplicationAttemptId getAppAttemptId() throws YarnException {
|
|
|
- AMRMTokenIdentifier amrmTokenIdentifier =
|
|
|
- YarnServerSecurityUtils.authorizeRequest();
|
|
|
- ApplicationAttemptId applicationAttemptId =
|
|
|
- amrmTokenIdentifier.getApplicationAttemptId();
|
|
|
- return applicationAttemptId;
|
|
|
- }
|
|
|
}
|