|
@@ -21,12 +21,15 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
|
|
import java.io.IOException;
|
|
|
import java.util.*;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.ConcurrentMap;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.service.AbstractService;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
@@ -67,6 +70,8 @@ import com.google.common.util.concurrent.SettableFuture;
|
|
|
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
|
+@Private
|
|
|
+@Unstable
|
|
|
public abstract class AbstractYarnScheduler
|
|
|
<T extends SchedulerApplicationAttempt, N extends SchedulerNode>
|
|
|
extends AbstractService implements ResourceScheduler {
|
|
@@ -91,7 +96,12 @@ public abstract class AbstractYarnScheduler
|
|
|
private long configuredMaximumAllocationWaitTime;
|
|
|
|
|
|
protected RMContext rmContext;
|
|
|
- protected Map<ApplicationId, SchedulerApplication<T>> applications;
|
|
|
+
|
|
|
+ /*
|
|
|
+ * All schedulers which are inheriting AbstractYarnScheduler should use
|
|
|
+ * concurrent version of 'applications' map.
|
|
|
+ */
|
|
|
+ protected ConcurrentMap<ApplicationId, SchedulerApplication<T>> applications;
|
|
|
protected int nmExpireInterval;
|
|
|
|
|
|
protected final static List<Container> EMPTY_CONTAINER_LIST =
|
|
@@ -123,7 +133,7 @@ public abstract class AbstractYarnScheduler
|
|
|
super.serviceInit(conf);
|
|
|
}
|
|
|
|
|
|
- public synchronized List<Container> getTransferredContainers(
|
|
|
+ public List<Container> getTransferredContainers(
|
|
|
ApplicationAttemptId currentAttempt) {
|
|
|
ApplicationId appId = currentAttempt.getApplicationId();
|
|
|
SchedulerApplication<T> app = applications.get(appId);
|
|
@@ -132,6 +142,9 @@ public abstract class AbstractYarnScheduler
|
|
|
if (appImpl.getApplicationSubmissionContext().getUnmanagedAM()) {
|
|
|
return containerList;
|
|
|
}
|
|
|
+ if (app == null) {
|
|
|
+ return containerList;
|
|
|
+ }
|
|
|
Collection<RMContainer> liveContainers =
|
|
|
app.getCurrentAppAttempt().getLiveContainers();
|
|
|
ContainerId amContainerId =
|