Uploaded image for project: 'Hadoop YARN'
  1. Hadoop YARN
  2. YARN-4597

Introduce ContainerScheduler and a SCHEDULED state to NodeManager container lifecycle

    Details

      Description

      Currently, the NM immediately launches containers after resource localization. Several features could be more cleanly implemented if the NM included a separate stage for reserving resources.

      1. YARN-4597.001.patch
        159 kB
        Arun Suresh
      2. YARN-4597.002.patch
        180 kB
        Arun Suresh
      3. YARN-4597.003.patch
        190 kB
        Arun Suresh
      4. YARN-4597.004.patch
        195 kB
        Arun Suresh
      5. YARN-4597.005.patch
        199 kB
        Arun Suresh
      6. YARN-4597.006.patch
        208 kB
        Arun Suresh
      7. YARN-4597.007.patch
        206 kB
        Arun Suresh
      8. YARN-4597.008.patch
        209 kB
        Arun Suresh
      9. YARN-4597.009.patch
        209 kB
        Arun Suresh
      10. YARN-4597.010.patch
        220 kB
        Arun Suresh
      11. YARN-4597.011.patch
        224 kB
        Arun Suresh
      12. YARN-4597.012.patch
        227 kB
        Arun Suresh
      13. YARN-4597.013.patch
        229 kB
        Arun Suresh

        Issue Links

          Activity

          Hide
          jlowe Jason Lowe added a comment -

          Sorry to arrive late here, but this has backwards-compatibility issues. This change introduces a new container state enumeration and sends that new state to clients that may not be aware of that state since they're not running the new version. That can cause those clients to either crash or make incorrect assumptions, e.g.: state != RUNNING must be COMPLETED, etc.

          Ran across this because YARN-6570 has that same sort of issue. That one has the RM not expecting the new value in 2.8, but any AM asking the NM for container status via NMClient could have the same kind of issue. The NM should not be sending new enumeration values to old clients. Filed YARN-7248 to track the compatibility issue.

          Show
          jlowe Jason Lowe added a comment - Sorry to arrive late here, but this has backwards-compatibility issues. This change introduces a new container state enumeration and sends that new state to clients that may not be aware of that state since they're not running the new version. That can cause those clients to either crash or make incorrect assumptions, e.g.: state != RUNNING must be COMPLETED, etc. Ran across this because YARN-6570 has that same sort of issue. That one has the RM not expecting the new value in 2.8, but any AM asking the NM for container status via NMClient could have the same kind of issue. The NM should not be sending new enumeration values to old clients. Filed YARN-7248 to track the compatibility issue.
          Hide
          asuresh Arun Suresh added a comment -

          Committing this to branch-2

          Show
          asuresh Arun Suresh added a comment - Committing this to branch-2
          Hide
          brahmareddy Brahma Reddy Battula added a comment -

          It's worked after clean compile..Sorry for false alarm.

          Show
          brahmareddy Brahma Reddy Battula added a comment - It's worked after clean compile..Sorry for false alarm.
          Hide
          asuresh Arun Suresh added a comment -
          Show
          asuresh Arun Suresh added a comment - Looks like the build passed too : https://builds.apache.org/job/Hadoop-trunk-Commit/10838/
          Hide
          hudson Hudson added a comment -

          SUCCESS: Integrated in Jenkins build Hadoop-trunk-Commit #10838 (See https://builds.apache.org/job/Hadoop-trunk-Commit/10838/)
          YARN-4597. Introduce ContainerScheduler and a SCHEDULED state to (arun suresh: rev 3219b7b4ac7d12aee343f6ab2980b3357fc618b6)

          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitor.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManager.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
          • (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
          • (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/package-info.java
          • (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
          • (delete) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
          • (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEvent.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerExitStatus.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
          • (edit) hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMROpportunisticMaps.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java
          • (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/AllocationBasedResourceUtilizationTracker.java
          • (delete) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRegression.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
          • (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
          • (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationTracker.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
          • (delete) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/package-info.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
          • (delete) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
          • (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in Jenkins build Hadoop-trunk-Commit #10838 (See https://builds.apache.org/job/Hadoop-trunk-Commit/10838/ ) YARN-4597 . Introduce ContainerScheduler and a SCHEDULED state to (arun suresh: rev 3219b7b4ac7d12aee343f6ab2980b3357fc618b6) (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitor.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManager.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/package-info.java (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java (delete) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEvent.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerExitStatus.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto (edit) hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMROpportunisticMaps.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/AllocationBasedResourceUtilizationTracker.java (delete) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRegression.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java (add) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationTracker.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java (delete) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/package-info.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java (delete) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java (edit) hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
          Hide
          asuresh Arun Suresh added a comment -

          Brahma Reddy Battula, can you try a mvn clean compile ?
          It compiles fine for me... and looks like jenkins is also ok with it..

          Show
          asuresh Arun Suresh added a comment - Brahma Reddy Battula , can you try a mvn clean compile ? It compiles fine for me... and looks like jenkins is also ok with it..
          Hide
          brahmareddy Brahma Reddy Battula added a comment -

          It's broken compilation.can you please check..?

          [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:3.1:compile (default-compile) on project hadoop-yarn-server-nodemanager: Compilation failure: Compilation failure:
          [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[107,26] cannot find symbol
          [ERROR] symbol:   variable NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH
          [ERROR] location: class org.apache.hadoop.yarn.conf.YarnConfiguration
          [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[108,26] cannot find symbol
          [ERROR] symbol:   variable NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT
          [ERROR] location: class org.apache.hadoop.yarn.conf.YarnConfiguration
          [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[263,34] cannot find symbol
          [ERROR] symbol:   variable KILLED_BY_CONTAINER_SCHEDULER
          [ERROR] location: class org.apache.hadoop.yarn.api.records.ContainerExitStatus
          [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[285,30] cannot find symbol
          [ERROR] symbol:   variable KILLED_BY_CONTAINER_SCHEDULER
          [ERROR] location: class org.apache.hadoop.yarn.api.records.ContainerExitStatus
          [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[405,32] cannot find symbol
          [ERROR] symbol:   variable KILLED_BY_CONTAINER_SCHEDULER
          [ERROR] location: class org.apache.hadoop.yarn.api.records.ContainerExitStatus
          [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java:[528,63] cannot find symbol
          [ERROR] symbol:   variable SCHEDULED
          [ERROR] location: class org.apache.hadoop.yarn.api.records.ContainerState
          [ERROR]
          
          
          Show
          brahmareddy Brahma Reddy Battula added a comment - It's broken compilation.can you please check..? [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:3.1:compile (default-compile) on project hadoop-yarn-server-nodemanager: Compilation failure: Compilation failure: [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[107,26] cannot find symbol [ERROR] symbol: variable NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH [ERROR] location: class org.apache.hadoop.yarn.conf.YarnConfiguration [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[108,26] cannot find symbol [ERROR] symbol: variable NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT [ERROR] location: class org.apache.hadoop.yarn.conf.YarnConfiguration [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[263,34] cannot find symbol [ERROR] symbol: variable KILLED_BY_CONTAINER_SCHEDULER [ERROR] location: class org.apache.hadoop.yarn.api.records.ContainerExitStatus [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[285,30] cannot find symbol [ERROR] symbol: variable KILLED_BY_CONTAINER_SCHEDULER [ERROR] location: class org.apache.hadoop.yarn.api.records.ContainerExitStatus [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java:[405,32] cannot find symbol [ERROR] symbol: variable KILLED_BY_CONTAINER_SCHEDULER [ERROR] location: class org.apache.hadoop.yarn.api.records.ContainerExitStatus [ERROR] /F:/Trunk/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java:[528,63] cannot find symbol [ERROR] symbol: variable SCHEDULED [ERROR] location: class org.apache.hadoop.yarn.api.records.ContainerState [ERROR]
          Hide
          asuresh Arun Suresh added a comment -

          Committed this to trunk

          Show
          asuresh Arun Suresh added a comment - Committed this to trunk
          Hide
          asuresh Arun Suresh added a comment -

          The testcase failures are not related to the patch.
          The checkstyle unused import warning refers to a class that has been renamed. This is probably since the github patch records that I had created the file.
          Not sure why I get this findBugs warning since it refers to findBugs files that don't exist.

          Committing this. Thanks for all the reviews Jian He, Karthik Kambatla, Konstantinos Karanasos and Varun Vasudev.

          Show
          asuresh Arun Suresh added a comment - The testcase failures are not related to the patch. The checkstyle unused import warning refers to a class that has been renamed. This is probably since the github patch records that I had created the file. Not sure why I get this findBugs warning since it refers to findBugs files that don't exist. Committing this. Thanks for all the reviews Jian He , Karthik Kambatla , Konstantinos Karanasos and Varun Vasudev .
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 0m 22s Docker mode activated.
          +1 @author 0m 0s The patch does not contain any @author tags.
          +1 test4tests 0m 0s The patch appears to include 19 new or modified test files.
          0 mvndep 0m 16s Maven dependency ordering for branch
          +1 mvninstall 7m 20s trunk passed
          +1 compile 9m 45s trunk passed
          +1 checkstyle 1m 47s trunk passed
          +1 mvnsite 3m 56s trunk passed
          +1 mvneclipse 2m 24s trunk passed
          -1 findbugs 0m 22s branch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml)
          +1 javadoc 2m 56s trunk passed
          0 mvndep 0m 17s Maven dependency ordering for patch
          +1 mvninstall 2m 57s the patch passed
          +1 compile 9m 44s the patch passed
          +1 cc 9m 44s the patch passed
          +1 javac 9m 44s the patch passed
          -0 checkstyle 1m 59s root: The patch generated 10 new + 1070 unchanged - 17 fixed = 1080 total (was 1087)
          +1 mvnsite 4m 39s the patch passed
          +1 mvneclipse 2m 54s the patch passed
          +1 whitespace 0m 0s The patch has no whitespace issues.
          +1 xml 0m 1s The patch has no ill-formed XML file.
          -1 findbugs 0m 26s patch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml)
          +1 javadoc 0m 30s hadoop-yarn-api in the patch passed.
          +1 javadoc 0m 39s hadoop-yarn-common in the patch passed.
          +1 javadoc 0m 27s hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager generated 0 new + 235 unchanged - 1 fixed = 235 total (was 236)
          +1 javadoc 0m 33s hadoop-yarn-server-resourcemanager in the patch passed.
          +1 javadoc 0m 23s hadoop-yarn-server-tests in the patch passed.
          +1 javadoc 0m 28s hadoop-yarn-client in the patch passed.
          +1 javadoc 0m 27s hadoop-mapreduce-client-jobclient in the patch passed.
          +1 unit 0m 36s hadoop-yarn-api in the patch passed.
          +1 unit 2m 35s hadoop-yarn-common in the patch passed.
          +1 unit 13m 34s hadoop-yarn-server-nodemanager in the patch passed.
          -1 unit 40m 55s hadoop-yarn-server-resourcemanager in the patch failed.
          -1 unit 4m 41s hadoop-yarn-server-tests in the patch failed.
          +1 unit 16m 26s hadoop-yarn-client in the patch passed.
          -1 unit 103m 30s hadoop-mapreduce-client-jobclient in the patch failed.
          +1 asflicense 0m 48s The patch does not generate ASF License warnings.
          275m 8s



          Reason Tests
          Failed junit tests hadoop.yarn.server.resourcemanager.TestTokenClientRMService
            hadoop.yarn.server.TestContainerManagerSecurity
            hadoop.yarn.server.TestMiniYarnClusterNodeUtilization
            hadoop.mapred.pipes.TestPipeApplication



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:a9ad5d6
          JIRA Issue YARN-4597
          GITHUB PR https://github.com/apache/hadoop/pull/143
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle cc xml
          uname Linux b311de11307e 3.13.0-95-generic #142-Ubuntu SMP Fri Aug 12 17:00:09 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision trunk / 43aef30
          Default Java 1.8.0_101
          findbugs v3.0.0
          findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/branch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          checkstyle https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/diff-checkstyle-root.txt
          findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/patch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-jobclient.txt
          Test Results https://builds.apache.org/job/PreCommit-YARN-Build/13915/testReport/
          modules C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient U: .
          Console output https://builds.apache.org/job/PreCommit-YARN-Build/13915/console
          Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org

          This message was automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall Vote Subsystem Runtime Comment 0 reexec 0m 22s Docker mode activated. +1 @author 0m 0s The patch does not contain any @author tags. +1 test4tests 0m 0s The patch appears to include 19 new or modified test files. 0 mvndep 0m 16s Maven dependency ordering for branch +1 mvninstall 7m 20s trunk passed +1 compile 9m 45s trunk passed +1 checkstyle 1m 47s trunk passed +1 mvnsite 3m 56s trunk passed +1 mvneclipse 2m 24s trunk passed -1 findbugs 0m 22s branch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml) +1 javadoc 2m 56s trunk passed 0 mvndep 0m 17s Maven dependency ordering for patch +1 mvninstall 2m 57s the patch passed +1 compile 9m 44s the patch passed +1 cc 9m 44s the patch passed +1 javac 9m 44s the patch passed -0 checkstyle 1m 59s root: The patch generated 10 new + 1070 unchanged - 17 fixed = 1080 total (was 1087) +1 mvnsite 4m 39s the patch passed +1 mvneclipse 2m 54s the patch passed +1 whitespace 0m 0s The patch has no whitespace issues. +1 xml 0m 1s The patch has no ill-formed XML file. -1 findbugs 0m 26s patch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml) +1 javadoc 0m 30s hadoop-yarn-api in the patch passed. +1 javadoc 0m 39s hadoop-yarn-common in the patch passed. +1 javadoc 0m 27s hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager generated 0 new + 235 unchanged - 1 fixed = 235 total (was 236) +1 javadoc 0m 33s hadoop-yarn-server-resourcemanager in the patch passed. +1 javadoc 0m 23s hadoop-yarn-server-tests in the patch passed. +1 javadoc 0m 28s hadoop-yarn-client in the patch passed. +1 javadoc 0m 27s hadoop-mapreduce-client-jobclient in the patch passed. +1 unit 0m 36s hadoop-yarn-api in the patch passed. +1 unit 2m 35s hadoop-yarn-common in the patch passed. +1 unit 13m 34s hadoop-yarn-server-nodemanager in the patch passed. -1 unit 40m 55s hadoop-yarn-server-resourcemanager in the patch failed. -1 unit 4m 41s hadoop-yarn-server-tests in the patch failed. +1 unit 16m 26s hadoop-yarn-client in the patch passed. -1 unit 103m 30s hadoop-mapreduce-client-jobclient in the patch failed. +1 asflicense 0m 48s The patch does not generate ASF License warnings. 275m 8s Reason Tests Failed junit tests hadoop.yarn.server.resourcemanager.TestTokenClientRMService   hadoop.yarn.server.TestContainerManagerSecurity   hadoop.yarn.server.TestMiniYarnClusterNodeUtilization   hadoop.mapred.pipes.TestPipeApplication Subsystem Report/Notes Docker Image:yetus/hadoop:a9ad5d6 JIRA Issue YARN-4597 GITHUB PR https://github.com/apache/hadoop/pull/143 Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle cc xml uname Linux b311de11307e 3.13.0-95-generic #142-Ubuntu SMP Fri Aug 12 17:00:09 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision trunk / 43aef30 Default Java 1.8.0_101 findbugs v3.0.0 findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/branch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt checkstyle https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/diff-checkstyle-root.txt findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/patch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13915/artifact/patchprocess/patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-jobclient.txt Test Results https://builds.apache.org/job/PreCommit-YARN-Build/13915/testReport/ modules C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient U: . Console output https://builds.apache.org/job/PreCommit-YARN-Build/13915/console Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          asuresh Arun Suresh added a comment - - edited

          Thanks again for the review Konstantinos Karanasos,
          I've updated the patch based on your suggestions except the following:

          1.

          ..the fields of the OpportunisticContainersStatus() can still be updated during the getOpportunisticContainersStatus(). To avoid synchronization, we could set the fields using an event, and then in the getOpportunisticContainersStatus() we would just return the object.

          Given that the opportunisticContainerStatus is meant for reporting, I think we can live with minor temporary inconsistencies. An event is probably an overkill.

          2.

          In the SchedulerNode, I still suggest to put the ++numContainers and the --numContainers inside the if statements. If I remember well, these fields are used for the web UI, so there will be a disconnect between the resources used (referring only to guaranteed containers) and the number of containers (referring to both guaranteed and opportunistic at the moment). The stats for the opportunistic containers are carried by the opportunisticContainersStatus, so we are good with reporting them too.

          I still feel numContainers SHOULD include the opportunistic containers. Else, we should change the 'numContainers' field name to 'numGuaranteedContainers' - but I am less inclined to make anymore changes in the RM for this patch. I am actually of the opinion that since users can see the actual opp. container count, by virtue of YARN-2995, the total count would be more valuable than just the guaranteed container count. Also in the future, we might have other container execution types, and I would not want a specific counter on the SchedulerNode for each of these.
          Let's re-visit this if we do find the UI jarring.

          Show
          asuresh Arun Suresh added a comment - - edited Thanks again for the review Konstantinos Karanasos , I've updated the patch based on your suggestions except the following: 1. ..the fields of the OpportunisticContainersStatus() can still be updated during the getOpportunisticContainersStatus(). To avoid synchronization, we could set the fields using an event, and then in the getOpportunisticContainersStatus() we would just return the object. Given that the opportunisticContainerStatus is meant for reporting, I think we can live with minor temporary inconsistencies. An event is probably an overkill. 2. In the SchedulerNode, I still suggest to put the ++numContainers and the --numContainers inside the if statements. If I remember well, these fields are used for the web UI, so there will be a disconnect between the resources used (referring only to guaranteed containers) and the number of containers (referring to both guaranteed and opportunistic at the moment). The stats for the opportunistic containers are carried by the opportunisticContainersStatus, so we are good with reporting them too. I still feel numContainers SHOULD include the opportunistic containers. Else, we should change the 'numContainers' field name to 'numGuaranteedContainers' - but I am less inclined to make anymore changes in the RM for this patch. I am actually of the opinion that since users can see the actual opp. container count, by virtue of YARN-2995 , the total count would be more valuable than just the guaranteed container count. Also in the future, we might have other container execution types, and I would not want a specific counter on the SchedulerNode for each of these. Let's re-visit this if we do find the UI jarring.
          Hide
          kkaranasos Konstantinos Karanasos added a comment -

          Thanks for the updated patch, Arun Suresh. Looks good to me.
          Some final comments below... All are minor, so up to you to address (I would only "insist" about the last one).

          • In the ContainerScheduler:
            • In the comment for the runningContainers, let's mention that these are the running containers, including the containers that are in the process of transitioning from the SCHEDULED to the RUNNING state. I think the rest are details that might be confusing.
            • In the updateQueuingLimit, you can do an extra check of the form if (this.queuingLimit.getMaxQueueLength() < queuedOpportunisticContainers.size()) to avoid calling the shedding if the queue is not long enough. This might often be the case if the NM has imposed a small queue size.
            • I was thinking that, although less likely than before, the fields of the OpportunisticContainersStatus() can still be updated during the getOpportunisticContainersStatus(). To avoid synchronization, we could set the fields using an event, and then in the getOpportunisticContainersStatus() we would just return the object. But if you think it is too much, we can leave it as is.
            • In the onContainerCompleted, a container can belong either to queued guaranteed, to queued opportunistic or to running. So, you could avoid doing the remove from all lists once found in one of them.
          • In the YarnConfiguration, let's include in a comment that the max queue length coming from the RM is the globally max queue length.
          • In the SchedulerNode, I still suggest to put the ++numContainers and the --numContainers inside the if statements. If I remember well, these fields are used for the web UI, so there will be a disconnect between the resources used (referring only to guaranteed containers) and the number of containers (referring to both guaranteed and opportunistic at the moment). The stats for the opportunistic containers are carried by the opportunisticContainersStatus, so we are good with reporting them too.

          Again, all comments are minor. +1 for the patch and thanks for all the work!

          Show
          kkaranasos Konstantinos Karanasos added a comment - Thanks for the updated patch, Arun Suresh . Looks good to me. Some final comments below... All are minor, so up to you to address (I would only "insist" about the last one). In the ContainerScheduler : In the comment for the runningContainers, let's mention that these are the running containers, including the containers that are in the process of transitioning from the SCHEDULED to the RUNNING state. I think the rest are details that might be confusing. In the updateQueuingLimit , you can do an extra check of the form if (this.queuingLimit.getMaxQueueLength() < queuedOpportunisticContainers.size()) to avoid calling the shedding if the queue is not long enough. This might often be the case if the NM has imposed a small queue size. I was thinking that, although less likely than before, the fields of the OpportunisticContainersStatus() can still be updated during the getOpportunisticContainersStatus() . To avoid synchronization, we could set the fields using an event, and then in the getOpportunisticContainersStatus() we would just return the object. But if you think it is too much, we can leave it as is. In the onContainerCompleted , a container can belong either to queued guaranteed, to queued opportunistic or to running. So, you could avoid doing the remove from all lists once found in one of them. In the YarnConfiguration , let's include in a comment that the max queue length coming from the RM is the globally max queue length. In the SchedulerNode , I still suggest to put the ++numContainers and the --numContainers inside the if statements. If I remember well, these fields are used for the web UI, so there will be a disconnect between the resources used (referring only to guaranteed containers) and the number of containers (referring to both guaranteed and opportunistic at the moment). The stats for the opportunistic containers are carried by the opportunisticContainersStatus, so we are good with reporting them too. Again, all comments are minor. +1 for the patch and thanks for all the work!
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 0m 20s Docker mode activated.
          +1 @author 0m 0s The patch does not contain any @author tags.
          +1 test4tests 0m 0s The patch appears to include 19 new or modified test files.
          0 mvndep 1m 34s Maven dependency ordering for branch
          +1 mvninstall 6m 44s trunk passed
          +1 compile 9m 32s trunk passed
          +1 checkstyle 1m 47s trunk passed
          +1 mvnsite 3m 57s trunk passed
          +1 mvneclipse 2m 24s trunk passed
          -1 findbugs 0m 23s branch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml)
          +1 javadoc 3m 3s trunk passed
          0 mvndep 0m 17s Maven dependency ordering for patch
          +1 mvninstall 3m 5s the patch passed
          +1 compile 10m 50s the patch passed
          +1 cc 10m 50s the patch passed
          +1 javac 10m 50s the patch passed
          -0 checkstyle 1m 50s root: The patch generated 10 new + 1070 unchanged - 17 fixed = 1080 total (was 1087)
          +1 mvnsite 4m 25s the patch passed
          +1 mvneclipse 2m 44s the patch passed
          +1 whitespace 0m 0s The patch has no whitespace issues.
          +1 xml 0m 1s The patch has no ill-formed XML file.
          -1 findbugs 0m 24s patch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml)
          +1 javadoc 0m 29s hadoop-yarn-api in the patch passed.
          +1 javadoc 0m 37s hadoop-yarn-common in the patch passed.
          +1 javadoc 0m 28s hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager generated 0 new + 235 unchanged - 1 fixed = 235 total (was 236)
          +1 javadoc 0m 31s hadoop-yarn-server-resourcemanager in the patch passed.
          +1 javadoc 0m 21s hadoop-yarn-server-tests in the patch passed.
          +1 javadoc 0m 23s hadoop-yarn-client in the patch passed.
          +1 javadoc 0m 23s hadoop-mapreduce-client-jobclient in the patch passed.
          +1 unit 0m 38s hadoop-yarn-api in the patch passed.
          +1 unit 2m 31s hadoop-yarn-common in the patch passed.
          +1 unit 13m 34s hadoop-yarn-server-nodemanager in the patch passed.
          -1 unit 42m 50s hadoop-yarn-server-resourcemanager in the patch failed.
          -1 unit 4m 39s hadoop-yarn-server-tests in the patch failed.
          +1 unit 16m 39s hadoop-yarn-client in the patch passed.
          -1 unit 104m 20s hadoop-mapreduce-client-jobclient in the patch failed.
          +1 asflicense 0m 48s The patch does not generate ASF License warnings.
          279m 53s



          Reason Tests
          Failed junit tests hadoop.yarn.server.resourcemanager.TestTokenClientRMService
            hadoop.yarn.server.TestContainerManagerSecurity
            hadoop.yarn.server.TestMiniYarnClusterNodeUtilization
            hadoop.mapred.pipes.TestPipeApplication



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:a9ad5d6
          JIRA Issue YARN-4597
          GITHUB PR https://github.com/apache/hadoop/pull/143
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle cc xml
          uname Linux 6921a08a0b4a 3.13.0-95-generic #142-Ubuntu SMP Fri Aug 12 17:00:09 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision trunk / 6efb8c9
          Default Java 1.8.0_101
          findbugs v3.0.0
          findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/branch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          checkstyle https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/diff-checkstyle-root.txt
          findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/patch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-jobclient.txt
          Test Results https://builds.apache.org/job/PreCommit-YARN-Build/13898/testReport/
          modules C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient U: .
          Console output https://builds.apache.org/job/PreCommit-YARN-Build/13898/console
          Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org

          This message was automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall Vote Subsystem Runtime Comment 0 reexec 0m 20s Docker mode activated. +1 @author 0m 0s The patch does not contain any @author tags. +1 test4tests 0m 0s The patch appears to include 19 new or modified test files. 0 mvndep 1m 34s Maven dependency ordering for branch +1 mvninstall 6m 44s trunk passed +1 compile 9m 32s trunk passed +1 checkstyle 1m 47s trunk passed +1 mvnsite 3m 57s trunk passed +1 mvneclipse 2m 24s trunk passed -1 findbugs 0m 23s branch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml) +1 javadoc 3m 3s trunk passed 0 mvndep 0m 17s Maven dependency ordering for patch +1 mvninstall 3m 5s the patch passed +1 compile 10m 50s the patch passed +1 cc 10m 50s the patch passed +1 javac 10m 50s the patch passed -0 checkstyle 1m 50s root: The patch generated 10 new + 1070 unchanged - 17 fixed = 1080 total (was 1087) +1 mvnsite 4m 25s the patch passed +1 mvneclipse 2m 44s the patch passed +1 whitespace 0m 0s The patch has no whitespace issues. +1 xml 0m 1s The patch has no ill-formed XML file. -1 findbugs 0m 24s patch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml) +1 javadoc 0m 29s hadoop-yarn-api in the patch passed. +1 javadoc 0m 37s hadoop-yarn-common in the patch passed. +1 javadoc 0m 28s hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager generated 0 new + 235 unchanged - 1 fixed = 235 total (was 236) +1 javadoc 0m 31s hadoop-yarn-server-resourcemanager in the patch passed. +1 javadoc 0m 21s hadoop-yarn-server-tests in the patch passed. +1 javadoc 0m 23s hadoop-yarn-client in the patch passed. +1 javadoc 0m 23s hadoop-mapreduce-client-jobclient in the patch passed. +1 unit 0m 38s hadoop-yarn-api in the patch passed. +1 unit 2m 31s hadoop-yarn-common in the patch passed. +1 unit 13m 34s hadoop-yarn-server-nodemanager in the patch passed. -1 unit 42m 50s hadoop-yarn-server-resourcemanager in the patch failed. -1 unit 4m 39s hadoop-yarn-server-tests in the patch failed. +1 unit 16m 39s hadoop-yarn-client in the patch passed. -1 unit 104m 20s hadoop-mapreduce-client-jobclient in the patch failed. +1 asflicense 0m 48s The patch does not generate ASF License warnings. 279m 53s Reason Tests Failed junit tests hadoop.yarn.server.resourcemanager.TestTokenClientRMService   hadoop.yarn.server.TestContainerManagerSecurity   hadoop.yarn.server.TestMiniYarnClusterNodeUtilization   hadoop.mapred.pipes.TestPipeApplication Subsystem Report/Notes Docker Image:yetus/hadoop:a9ad5d6 JIRA Issue YARN-4597 GITHUB PR https://github.com/apache/hadoop/pull/143 Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle cc xml uname Linux 6921a08a0b4a 3.13.0-95-generic #142-Ubuntu SMP Fri Aug 12 17:00:09 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision trunk / 6efb8c9 Default Java 1.8.0_101 findbugs v3.0.0 findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/branch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt checkstyle https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/diff-checkstyle-root.txt findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/patch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13898/artifact/patchprocess/patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-jobclient.txt Test Results https://builds.apache.org/job/PreCommit-YARN-Build/13898/testReport/ modules C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient U: . Console output https://builds.apache.org/job/PreCommit-YARN-Build/13898/console Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          kasha Karthik Kambatla added a comment -

          The latest patch looks good to me.

          Show
          kasha Karthik Kambatla added a comment - The latest patch looks good to me.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87836296

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,411 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.AsyncDispatcher;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +
          +import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all its launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersToKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> runningContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Tracker that decides how utilization of the cluster
          + // increases / decreases based on container start / finish
          + private ResourceUtilizationTracker utilizationTracker;
          +
          + private final AsyncDispatcher dispatcher;
          + private final NodeManagerMetrics metrics;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + * @param dispatcher AsyncDispatcher.
          + * @param metrics NodeManagerMetrics.
          + */
          + public ContainerScheduler(Context context, AsyncDispatcher dispatcher,
          + NodeManagerMetrics metrics)

          { + this(context, dispatcher, metrics, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration. + NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, AsyncDispatcher dispatcher,
          + NodeManagerMetrics metrics, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.dispatcher = dispatcher; + this.metrics = metrics; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationTracker = new ResourceUtilizationTracker(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + case SHED_QUEUED_CONTAINERS: + shedQueuedOpportunisticContainers(); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + this.opportunisticContainersStatus.setOpportMemoryUsed( + metrics.getOpportMemoryUsed()); + this.opportunisticContainersStatus.setOpportCoresUsed( + metrics.getOpportCoresUsed()); + this.opportunisticContainersStatus.setRunningOpportContainers( + metrics.getRunningOpportContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + oppContainersToKill.remove(container.getContainerId());
          +
          + // This could be killed externally for eg. by the ContainerManager,
          + // in which case, the container might still be queued.
          + queuedOpportunisticContainers.remove(container.getContainerId());
          + queuedGuaranteedContainers.remove(container.getContainerId());
          +
          + // decrement only if it was a running container
          + Container completedContainer = runningContainers.remove(container
          + .getContainerId());
          + if (completedContainer != null) {
          + this.utilizationTracker.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.metrics.opportunisticContainerCompleted(container); + }

          + startPendingContainers();
          + }
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationTracker.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationTracker.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container [" + container.getContainerId() + + "] state. The Container has been queued.", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersToKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + runningContainers.put(container.getContainerId(), container);
          + this.utilizationTracker.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.metrics.opportunisticContainerStarted(container); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          — End diff –

          Done.. fixed in latest commit

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87836296 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,411 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + + +import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all its launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersToKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> runningContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Tracker that decides how utilization of the cluster + // increases / decreases based on container start / finish + private ResourceUtilizationTracker utilizationTracker; + + private final AsyncDispatcher dispatcher; + private final NodeManagerMetrics metrics; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + * @param dispatcher AsyncDispatcher. + * @param metrics NodeManagerMetrics. + */ + public ContainerScheduler(Context context, AsyncDispatcher dispatcher, + NodeManagerMetrics metrics) { + this(context, dispatcher, metrics, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration. + NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, AsyncDispatcher dispatcher, + NodeManagerMetrics metrics, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.dispatcher = dispatcher; + this.metrics = metrics; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationTracker = new ResourceUtilizationTracker(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + case SHED_QUEUED_CONTAINERS: + shedQueuedOpportunisticContainers(); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + this.opportunisticContainersStatus.setOpportMemoryUsed( + metrics.getOpportMemoryUsed()); + this.opportunisticContainersStatus.setOpportCoresUsed( + metrics.getOpportCoresUsed()); + this.opportunisticContainersStatus.setRunningOpportContainers( + metrics.getRunningOpportContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + oppContainersToKill.remove(container.getContainerId()); + + // This could be killed externally for eg. by the ContainerManager, + // in which case, the container might still be queued. + queuedOpportunisticContainers.remove(container.getContainerId()); + queuedGuaranteedContainers.remove(container.getContainerId()); + + // decrement only if it was a running container + Container completedContainer = runningContainers.remove(container + .getContainerId()); + if (completedContainer != null) { + this.utilizationTracker.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.metrics.opportunisticContainerCompleted(container); + } + startPendingContainers(); + } + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationTracker.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationTracker.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container [" + container.getContainerId() + + "] state. The Container has been queued.", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersToKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + runningContainers.put(container.getContainerId(), container); + this.utilizationTracker.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.metrics.opportunisticContainerStarted(container); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = — End diff – Done.. fixed in latest commit
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87836252

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationTracker.java —
          @@ -0,0 +1,161 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.api.records.Resource;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +/**
          + * This class abstracts out how a container contributes to Resource Utilization.
          + * It is used by the

          {@link ContainerScheduler}

          to determine which
          + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED
          + * container.
          + * It currently equates resource utilization with the total resource allocated
          + * to the container. Another implementation might choose to use the actual
          + * resource utilization.
          + */
          +
          +public class ResourceUtilizationTracker {
          — End diff –

          Done.... fixed in latest commit

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87836252 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationTracker.java — @@ -0,0 +1,161 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class abstracts out how a container contributes to Resource Utilization. + * It is used by the {@link ContainerScheduler} to determine which + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED + * container. + * It currently equates resource utilization with the total resource allocated + * to the container. Another implementation might choose to use the actual + * resource utilization. + */ + +public class ResourceUtilizationTracker { — End diff – Done.... fixed in latest commit
          Hide
          asuresh Arun Suresh added a comment -

          Updated patch (v012)
          Moved ResourceUtilizationTracker to an interface and a minor variable name-change based on Karthik's comments.

          Show
          asuresh Arun Suresh added a comment - Updated patch (v012) Moved ResourceUtilizationTracker to an interface and a minor variable name-change based on Karthik's comments.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87826671

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationTracker.java —
          @@ -0,0 +1,161 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.api.records.Resource;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +/**
          + * This class abstracts out how a container contributes to Resource Utilization.
          + * It is used by the

          {@link ContainerScheduler}

          to determine which
          + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED
          + * container.
          + * It currently equates resource utilization with the total resource allocated
          + * to the container. Another implementation might choose to use the actual
          + * resource utilization.
          + */
          +
          +public class ResourceUtilizationTracker {
          — End diff –

          Ah yes... missed this... will fix

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87826671 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationTracker.java — @@ -0,0 +1,161 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class abstracts out how a container contributes to Resource Utilization. + * It is used by the {@link ContainerScheduler} to determine which + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED + * container. + * It currently equates resource utilization with the total resource allocated + * to the container. Another implementation might choose to use the actual + * resource utilization. + */ + +public class ResourceUtilizationTracker { — End diff – Ah yes... missed this... will fix
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87823624

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          — End diff –

          Sounds reasonable.

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87823624 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> — End diff – Sounds reasonable.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87823047

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationTracker.java —
          @@ -0,0 +1,161 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.api.records.Resource;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +/**
          + * This class abstracts out how a container contributes to Resource Utilization.
          + * It is used by the

          {@link ContainerScheduler}

          to determine which
          + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED
          + * container.
          + * It currently equates resource utilization with the total resource allocated
          + * to the container. Another implementation might choose to use the actual
          + * resource utilization.
          + */
          +
          +public class ResourceUtilizationTracker {
          — End diff –

          Didn't we think this should be an interface with the default implementation being the one that considers allocation - a.k.a AllocationBased

          {ResourceUtilization}

          Tracker

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87823047 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationTracker.java — @@ -0,0 +1,161 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class abstracts out how a container contributes to Resource Utilization. + * It is used by the {@link ContainerScheduler} to determine which + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED + * container. + * It currently equates resource utilization with the total resource allocated + * to the container. Another implementation might choose to use the actual + * resource utilization. + */ + +public class ResourceUtilizationTracker { — End diff – Didn't we think this should be an interface with the default implementation being the one that considers allocation - a.k.a AllocationBased {ResourceUtilization} Tracker
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87823436

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,411 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.AsyncDispatcher;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +
          +import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all its launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersToKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> runningContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Tracker that decides how utilization of the cluster
          + // increases / decreases based on container start / finish
          + private ResourceUtilizationTracker utilizationTracker;
          +
          + private final AsyncDispatcher dispatcher;
          + private final NodeManagerMetrics metrics;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + * @param dispatcher AsyncDispatcher.
          + * @param metrics NodeManagerMetrics.
          + */
          + public ContainerScheduler(Context context, AsyncDispatcher dispatcher,
          + NodeManagerMetrics metrics)

          { + this(context, dispatcher, metrics, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration. + NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, AsyncDispatcher dispatcher,
          + NodeManagerMetrics metrics, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.dispatcher = dispatcher; + this.metrics = metrics; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationTracker = new ResourceUtilizationTracker(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + case SHED_QUEUED_CONTAINERS: + shedQueuedOpportunisticContainers(); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + this.opportunisticContainersStatus.setOpportMemoryUsed( + metrics.getOpportMemoryUsed()); + this.opportunisticContainersStatus.setOpportCoresUsed( + metrics.getOpportCoresUsed()); + this.opportunisticContainersStatus.setRunningOpportContainers( + metrics.getRunningOpportContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + oppContainersToKill.remove(container.getContainerId());
          +
          + // This could be killed externally for eg. by the ContainerManager,
          + // in which case, the container might still be queued.
          + queuedOpportunisticContainers.remove(container.getContainerId());
          + queuedGuaranteedContainers.remove(container.getContainerId());
          +
          + // decrement only if it was a running container
          + Container completedContainer = runningContainers.remove(container
          + .getContainerId());
          + if (completedContainer != null) {
          + this.utilizationTracker.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.metrics.opportunisticContainerCompleted(container); + }

          + startPendingContainers();
          + }
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationTracker.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationTracker.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container [" + container.getContainerId() + + "] state. The Container has been queued.", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersToKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + runningContainers.put(container.getContainerId(), container);
          + this.utilizationTracker.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.metrics.opportunisticContainerStarted(container); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          — End diff –

          Nit: should we call this lifoIterator so it is easier to understand?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87823436 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,411 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + + +import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all its launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersToKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> runningContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Tracker that decides how utilization of the cluster + // increases / decreases based on container start / finish + private ResourceUtilizationTracker utilizationTracker; + + private final AsyncDispatcher dispatcher; + private final NodeManagerMetrics metrics; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + * @param dispatcher AsyncDispatcher. + * @param metrics NodeManagerMetrics. + */ + public ContainerScheduler(Context context, AsyncDispatcher dispatcher, + NodeManagerMetrics metrics) { + this(context, dispatcher, metrics, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration. + NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, AsyncDispatcher dispatcher, + NodeManagerMetrics metrics, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.dispatcher = dispatcher; + this.metrics = metrics; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationTracker = new ResourceUtilizationTracker(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + case SHED_QUEUED_CONTAINERS: + shedQueuedOpportunisticContainers(); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + this.opportunisticContainersStatus.setOpportMemoryUsed( + metrics.getOpportMemoryUsed()); + this.opportunisticContainersStatus.setOpportCoresUsed( + metrics.getOpportCoresUsed()); + this.opportunisticContainersStatus.setRunningOpportContainers( + metrics.getRunningOpportContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + oppContainersToKill.remove(container.getContainerId()); + + // This could be killed externally for eg. by the ContainerManager, + // in which case, the container might still be queued. + queuedOpportunisticContainers.remove(container.getContainerId()); + queuedGuaranteedContainers.remove(container.getContainerId()); + + // decrement only if it was a running container + Container completedContainer = runningContainers.remove(container + .getContainerId()); + if (completedContainer != null) { + this.utilizationTracker.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.metrics.opportunisticContainerCompleted(container); + } + startPendingContainers(); + } + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationTracker.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationTracker.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container [" + container.getContainerId() + + "] state. The Container has been queued.", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersToKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + runningContainers.put(container.getContainerId(), container); + this.utilizationTracker.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.metrics.opportunisticContainerStarted(container); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = — End diff – Nit: should we call this lifoIterator so it is easier to understand?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87822206

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -295,29 +306,29 @@ private void startAllocatedContainer(Container container) {
          // Go over the running opportunistic containers.
          // Use a descending iterator to kill more recently started containers.
          Iterator<Container> reverseContainerIterator =
          — End diff –

          Nit: Should we call this lifoIterator?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87822206 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -295,29 +306,29 @@ private void startAllocatedContainer(Container container) { // Go over the running opportunistic containers. // Use a descending iterator to kill more recently started containers. Iterator<Container> reverseContainerIterator = — End diff – Nit: Should we call this lifoIterator?
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 0m 23s Docker mode activated.
          +1 @author 0m 0s The patch does not contain any @author tags.
          +1 test4tests 0m 0s The patch appears to include 19 new or modified test files.
          0 mvndep 0m 43s Maven dependency ordering for branch
          +1 mvninstall 6m 46s trunk passed
          +1 compile 9m 36s trunk passed
          +1 checkstyle 1m 46s trunk passed
          +1 mvnsite 3m 54s trunk passed
          +1 mvneclipse 2m 25s trunk passed
          -1 findbugs 0m 22s branch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml)
          +1 javadoc 2m 55s trunk passed
          0 mvndep 0m 18s Maven dependency ordering for patch
          +1 mvninstall 2m 56s the patch passed
          +1 compile 9m 12s the patch passed
          +1 cc 9m 12s the patch passed
          +1 javac 9m 12s the patch passed
          -0 checkstyle 1m 52s root: The patch generated 10 new + 1070 unchanged - 17 fixed = 1080 total (was 1087)
          +1 mvnsite 4m 20s the patch passed
          +1 mvneclipse 2m 46s the patch passed
          +1 whitespace 0m 0s The patch has no whitespace issues.
          +1 xml 0m 1s The patch has no ill-formed XML file.
          -1 findbugs 0m 25s patch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml)
          +1 javadoc 0m 28s hadoop-yarn-api in the patch passed.
          +1 javadoc 0m 38s hadoop-yarn-common in the patch passed.
          +1 javadoc 0m 27s hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager generated 0 new + 235 unchanged - 1 fixed = 235 total (was 236)
          +1 javadoc 0m 33s hadoop-yarn-server-resourcemanager in the patch passed.
          +1 javadoc 0m 22s hadoop-yarn-server-tests in the patch passed.
          +1 javadoc 0m 26s hadoop-yarn-client in the patch passed.
          +1 javadoc 0m 23s hadoop-mapreduce-client-jobclient in the patch passed.
          +1 unit 0m 36s hadoop-yarn-api in the patch passed.
          +1 unit 2m 28s hadoop-yarn-common in the patch passed.
          +1 unit 13m 32s hadoop-yarn-server-nodemanager in the patch passed.
          -1 unit 40m 5s hadoop-yarn-server-resourcemanager in the patch failed.
          -1 unit 4m 40s hadoop-yarn-server-tests in the patch failed.
          +1 unit 16m 16s hadoop-yarn-client in the patch passed.
          -1 unit 106m 16s hadoop-mapreduce-client-jobclient in the patch failed.
          +1 asflicense 0m 47s The patch does not generate ASF License warnings.
          275m 8s



          Reason Tests
          Failed junit tests hadoop.yarn.server.resourcemanager.TestRMRestart
            hadoop.yarn.server.resourcemanager.TestTokenClientRMService
            hadoop.yarn.server.TestContainerManagerSecurity
            hadoop.yarn.server.TestMiniYarnClusterNodeUtilization
            hadoop.mapred.pipes.TestPipeApplication



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:a9ad5d6
          JIRA Issue YARN-4597
          GITHUB PR https://github.com/apache/hadoop/pull/143
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle cc xml
          uname Linux a35c44e112ec 3.13.0-95-generic #142-Ubuntu SMP Fri Aug 12 17:00:09 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision trunk / 96f4392
          Default Java 1.8.0_101
          findbugs v3.0.0
          findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/branch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          checkstyle https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/diff-checkstyle-root.txt
          findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/patch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-jobclient.txt
          Test Results https://builds.apache.org/job/PreCommit-YARN-Build/13887/testReport/
          modules C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient U: .
          Console output https://builds.apache.org/job/PreCommit-YARN-Build/13887/console
          Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org

          This message was automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall Vote Subsystem Runtime Comment 0 reexec 0m 23s Docker mode activated. +1 @author 0m 0s The patch does not contain any @author tags. +1 test4tests 0m 0s The patch appears to include 19 new or modified test files. 0 mvndep 0m 43s Maven dependency ordering for branch +1 mvninstall 6m 46s trunk passed +1 compile 9m 36s trunk passed +1 checkstyle 1m 46s trunk passed +1 mvnsite 3m 54s trunk passed +1 mvneclipse 2m 25s trunk passed -1 findbugs 0m 22s branch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml) +1 javadoc 2m 55s trunk passed 0 mvndep 0m 18s Maven dependency ordering for patch +1 mvninstall 2m 56s the patch passed +1 compile 9m 12s the patch passed +1 cc 9m 12s the patch passed +1 javac 9m 12s the patch passed -0 checkstyle 1m 52s root: The patch generated 10 new + 1070 unchanged - 17 fixed = 1080 total (was 1087) +1 mvnsite 4m 20s the patch passed +1 mvneclipse 2m 46s the patch passed +1 whitespace 0m 0s The patch has no whitespace issues. +1 xml 0m 1s The patch has no ill-formed XML file. -1 findbugs 0m 25s patch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml) +1 javadoc 0m 28s hadoop-yarn-api in the patch passed. +1 javadoc 0m 38s hadoop-yarn-common in the patch passed. +1 javadoc 0m 27s hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager generated 0 new + 235 unchanged - 1 fixed = 235 total (was 236) +1 javadoc 0m 33s hadoop-yarn-server-resourcemanager in the patch passed. +1 javadoc 0m 22s hadoop-yarn-server-tests in the patch passed. +1 javadoc 0m 26s hadoop-yarn-client in the patch passed. +1 javadoc 0m 23s hadoop-mapreduce-client-jobclient in the patch passed. +1 unit 0m 36s hadoop-yarn-api in the patch passed. +1 unit 2m 28s hadoop-yarn-common in the patch passed. +1 unit 13m 32s hadoop-yarn-server-nodemanager in the patch passed. -1 unit 40m 5s hadoop-yarn-server-resourcemanager in the patch failed. -1 unit 4m 40s hadoop-yarn-server-tests in the patch failed. +1 unit 16m 16s hadoop-yarn-client in the patch passed. -1 unit 106m 16s hadoop-mapreduce-client-jobclient in the patch failed. +1 asflicense 0m 47s The patch does not generate ASF License warnings. 275m 8s Reason Tests Failed junit tests hadoop.yarn.server.resourcemanager.TestRMRestart   hadoop.yarn.server.resourcemanager.TestTokenClientRMService   hadoop.yarn.server.TestContainerManagerSecurity   hadoop.yarn.server.TestMiniYarnClusterNodeUtilization   hadoop.mapred.pipes.TestPipeApplication Subsystem Report/Notes Docker Image:yetus/hadoop:a9ad5d6 JIRA Issue YARN-4597 GITHUB PR https://github.com/apache/hadoop/pull/143 Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle cc xml uname Linux a35c44e112ec 3.13.0-95-generic #142-Ubuntu SMP Fri Aug 12 17:00:09 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision trunk / 96f4392 Default Java 1.8.0_101 findbugs v3.0.0 findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/branch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt checkstyle https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/diff-checkstyle-root.txt findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/patch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13887/artifact/patchprocess/patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-jobclient.txt Test Results https://builds.apache.org/job/PreCommit-YARN-Build/13887/testReport/ modules C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient U: . Console output https://builds.apache.org/job/PreCommit-YARN-Build/13887/console Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          asuresh Arun Suresh added a comment -

          Updating patch (v011):

          1. Fixing some checkstyles and javadocs (I cannot figure out what the findbugs are)
          2. Added one more test case to verify that if an AM requests to stop a container that is currently queued, it is correctly dequeued from the Scheduler
          Show
          asuresh Arun Suresh added a comment - Updating patch (v011): Fixing some checkstyles and javadocs (I cannot figure out what the findbugs are) Added one more test case to verify that if an AM requests to stop a container that is currently queued, it is correctly dequeued from the Scheduler
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 0m 25s Docker mode activated.
          +1 @author 0m 0s The patch does not contain any @author tags.
          +1 test4tests 0m 0s The patch appears to include 19 new or modified test files.
          0 mvndep 1m 40s Maven dependency ordering for branch
          +1 mvninstall 9m 1s trunk passed
          +1 compile 11m 44s trunk passed
          +1 checkstyle 1m 59s trunk passed
          +1 mvnsite 4m 26s trunk passed
          +1 mvneclipse 2m 30s trunk passed
          -1 findbugs 0m 22s branch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml)
          +1 javadoc 2m 56s trunk passed
          0 mvndep 0m 17s Maven dependency ordering for patch
          +1 mvninstall 2m 56s the patch passed
          +1 compile 9m 17s the patch passed
          +1 cc 9m 17s the patch passed
          -1 javac 9m 17s root generated 1 new + 690 unchanged - 1 fixed = 691 total (was 691)
          -0 checkstyle 1m 51s root: The patch generated 15 new + 1070 unchanged - 17 fixed = 1085 total (was 1087)
          +1 mvnsite 4m 18s the patch passed
          +1 mvneclipse 2m 50s the patch passed
          +1 whitespace 0m 0s The patch has no whitespace issues.
          +1 xml 0m 1s The patch has no ill-formed XML file.
          -1 findbugs 0m 25s patch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml)
          -1 javadoc 0m 28s hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager generated 2 new + 235 unchanged - 1 fixed = 237 total (was 236)
          +1 unit 0m 36s hadoop-yarn-api in the patch passed.
          +1 unit 2m 28s hadoop-yarn-common in the patch passed.
          +1 unit 13m 27s hadoop-yarn-server-nodemanager in the patch passed.
          -1 unit 40m 11s hadoop-yarn-server-resourcemanager in the patch failed.
          -1 unit 4m 39s hadoop-yarn-server-tests in the patch failed.
          +1 unit 16m 18s hadoop-yarn-client in the patch passed.
          -1 unit 105m 10s hadoop-mapreduce-client-jobclient in the patch failed.
          +1 asflicense 0m 48s The patch does not generate ASF License warnings.
          280m 20s



          Reason Tests
          Failed junit tests hadoop.yarn.server.resourcemanager.TestTokenClientRMService
            hadoop.yarn.server.TestContainerManagerSecurity
            hadoop.yarn.server.TestMiniYarnClusterNodeUtilization
            hadoop.mapred.pipes.TestPipeApplication



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:a9ad5d6
          JIRA Issue YARN-4597
          GITHUB PR https://github.com/apache/hadoop/pull/143
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle cc xml
          uname Linux b42363efd879 3.13.0-95-generic #142-Ubuntu SMP Fri Aug 12 17:00:09 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision trunk / 96f4392
          Default Java 1.8.0_101
          findbugs v3.0.0
          findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/branch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          javac https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/diff-compile-javac-root.txt
          checkstyle https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/diff-checkstyle-root.txt
          findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/patch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          javadoc https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/diff-javadoc-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt
          unit https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-jobclient.txt
          Test Results https://builds.apache.org/job/PreCommit-YARN-Build/13886/testReport/
          modules C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient U: .
          Console output https://builds.apache.org/job/PreCommit-YARN-Build/13886/console
          Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org

          This message was automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall Vote Subsystem Runtime Comment 0 reexec 0m 25s Docker mode activated. +1 @author 0m 0s The patch does not contain any @author tags. +1 test4tests 0m 0s The patch appears to include 19 new or modified test files. 0 mvndep 1m 40s Maven dependency ordering for branch +1 mvninstall 9m 1s trunk passed +1 compile 11m 44s trunk passed +1 checkstyle 1m 59s trunk passed +1 mvnsite 4m 26s trunk passed +1 mvneclipse 2m 30s trunk passed -1 findbugs 0m 22s branch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml) +1 javadoc 2m 56s trunk passed 0 mvndep 0m 17s Maven dependency ordering for patch +1 mvninstall 2m 56s the patch passed +1 compile 9m 17s the patch passed +1 cc 9m 17s the patch passed -1 javac 9m 17s root generated 1 new + 690 unchanged - 1 fixed = 691 total (was 691) -0 checkstyle 1m 51s root: The patch generated 15 new + 1070 unchanged - 17 fixed = 1085 total (was 1087) +1 mvnsite 4m 18s the patch passed +1 mvneclipse 2m 50s the patch passed +1 whitespace 0m 0s The patch has no whitespace issues. +1 xml 0m 1s The patch has no ill-formed XML file. -1 findbugs 0m 25s patch/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests no findbugs output file (hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target/findbugsXml.xml) -1 javadoc 0m 28s hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager generated 2 new + 235 unchanged - 1 fixed = 237 total (was 236) +1 unit 0m 36s hadoop-yarn-api in the patch passed. +1 unit 2m 28s hadoop-yarn-common in the patch passed. +1 unit 13m 27s hadoop-yarn-server-nodemanager in the patch passed. -1 unit 40m 11s hadoop-yarn-server-resourcemanager in the patch failed. -1 unit 4m 39s hadoop-yarn-server-tests in the patch failed. +1 unit 16m 18s hadoop-yarn-client in the patch passed. -1 unit 105m 10s hadoop-mapreduce-client-jobclient in the patch failed. +1 asflicense 0m 48s The patch does not generate ASF License warnings. 280m 20s Reason Tests Failed junit tests hadoop.yarn.server.resourcemanager.TestTokenClientRMService   hadoop.yarn.server.TestContainerManagerSecurity   hadoop.yarn.server.TestMiniYarnClusterNodeUtilization   hadoop.mapred.pipes.TestPipeApplication Subsystem Report/Notes Docker Image:yetus/hadoop:a9ad5d6 JIRA Issue YARN-4597 GITHUB PR https://github.com/apache/hadoop/pull/143 Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle cc xml uname Linux b42363efd879 3.13.0-95-generic #142-Ubuntu SMP Fri Aug 12 17:00:09 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision trunk / 96f4392 Default Java 1.8.0_101 findbugs v3.0.0 findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/branch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt javac https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/diff-compile-javac-root.txt checkstyle https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/diff-checkstyle-root.txt findbugs https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/patch-findbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt javadoc https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/diff-javadoc-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-nodemanager.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-tests.txt unit https://builds.apache.org/job/PreCommit-YARN-Build/13886/artifact/patchprocess/patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-jobclient.txt Test Results https://builds.apache.org/job/PreCommit-YARN-Build/13886/testReport/ modules C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient U: . Console output https://builds.apache.org/job/PreCommit-YARN-Build/13886/console Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          asuresh Arun Suresh added a comment - - edited

          Updating patch addressing Konstantinos Karanasos's and Karthik Kambatla's comments.

          Thanks both of you for the detailed reviews.

          I've responded to Karthik's comments directly on github.
          Konstantinos Karanasos, I've address your comments except the following.

          1.

          queuedOpportunisticContainers will have concurrency issues. We are updating it when containers arrive but also in the shedQueuedOpportunisticContainers.

          Good catch. Ive fixed it in the latest patch by sending a 'SHED_QUEUED_CONTAINERS' event to the ContainerScheduler when the Node HB response from the RM asks to shed queued containers. In addition to preserving the fact that ContainerScheduler operations are serialized, it also ensures that the Node HB thread is not blocked.

          2.

          shedQueuedOpportunisticContainers: numAllowed is the number of allowed containers in the queue. Instead, we are killing numAllowed containers. In other words, we should not kill numAllowed, but queuedOpportunisticContainers.size() - numAllowed.

          Even though I agreed with you offline, I took a look again, and actually the logic is correct. The numAllowed counter is initialized to maxAllowed and the decremented in the loop. Containers are killed only AFTER it's value goes <= 0. In any case, I've added a testcase in 'TestContainerSchedulerQueuing' to verify that this actually works.

          3.

          line 252, indeed we can now do extraOpportContainersToKill -> opportContainersToKill, as Karthik mentioned at a comment.

          I think 'extra' is still apt. Since (as I mentioned to Karthik), these are 'extra' opp containers over and above what is already present in the 'oppContainersToKill'.

          4.

          queuedGuaranteedContainers and queuedOpportunisticContainers: I think we should use queues. I don't think we retrieve the container by the key anywhere either ways.

          Karthik mentioned this in his comments too. LinkedHashMaps are essentially a indexed queue. Additionally, there is actually 1 case where we need to retrieve by the key: When the AM asks to kill a container that is queued. Furthermore, queue re-ordering etc. might be easier with a map... Lets keep it as a LinkedHashMap unless we find it is detrimental in some way.

          5.

          oppContainersMarkedForKill: could be a Set, right?

          Technically yes, but I would have to modify Container to add equals and hashcode too, which I felt was too much of a hastle.. I prefer to keep it as it is.

          6.

          fields of the opportunisticContainersStatus are set in different places. Due to that, when we call getOpportunisticContainersStatus() we may see an inconsistent object. Let's set the fields only in the getOpportunisticContainersStatus().

          Agreed... I've also added the oppMem, oppCores & numOpp values in the NodeManagerMetrics.

          7.

          There seem to be two redundant parameters at YarnConfiguration at the moment: NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH and NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH. If I am not missing something, we should keep one of the two.

          Actually they are bit different. NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH and the corresponding max value is used by the RM to calculate a limit value for the Queue. It is possible that the Queue can momentarily go above that. While NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH is used in the NM to prevent queuing beyond that value. It is a configuration hard limit (Karthik Kambatla had requested it)

          Show
          asuresh Arun Suresh added a comment - - edited Updating patch addressing Konstantinos Karanasos 's and Karthik Kambatla 's comments. Thanks both of you for the detailed reviews. I've responded to Karthik's comments directly on github. Konstantinos Karanasos , I've address your comments except the following. 1. queuedOpportunisticContainers will have concurrency issues. We are updating it when containers arrive but also in the shedQueuedOpportunisticContainers. Good catch. Ive fixed it in the latest patch by sending a 'SHED_QUEUED_CONTAINERS' event to the ContainerScheduler when the Node HB response from the RM asks to shed queued containers. In addition to preserving the fact that ContainerScheduler operations are serialized, it also ensures that the Node HB thread is not blocked. 2. shedQueuedOpportunisticContainers: numAllowed is the number of allowed containers in the queue. Instead, we are killing numAllowed containers. In other words, we should not kill numAllowed, but queuedOpportunisticContainers.size() - numAllowed. Even though I agreed with you offline, I took a look again, and actually the logic is correct. The numAllowed counter is initialized to maxAllowed and the decremented in the loop. Containers are killed only AFTER it's value goes <= 0. In any case, I've added a testcase in 'TestContainerSchedulerQueuing' to verify that this actually works. 3. line 252, indeed we can now do extraOpportContainersToKill -> opportContainersToKill, as Karthik mentioned at a comment. I think 'extra' is still apt. Since (as I mentioned to Karthik), these are 'extra' opp containers over and above what is already present in the 'oppContainersToKill'. 4. queuedGuaranteedContainers and queuedOpportunisticContainers: I think we should use queues. I don't think we retrieve the container by the key anywhere either ways. Karthik mentioned this in his comments too. LinkedHashMaps are essentially a indexed queue. Additionally, there is actually 1 case where we need to retrieve by the key: When the AM asks to kill a container that is queued. Furthermore, queue re-ordering etc. might be easier with a map... Lets keep it as a LinkedHashMap unless we find it is detrimental in some way. 5. oppContainersMarkedForKill: could be a Set, right? Technically yes, but I would have to modify Container to add equals and hashcode too, which I felt was too much of a hastle.. I prefer to keep it as it is. 6. fields of the opportunisticContainersStatus are set in different places. Due to that, when we call getOpportunisticContainersStatus() we may see an inconsistent object. Let's set the fields only in the getOpportunisticContainersStatus(). Agreed... I've also added the oppMem, oppCores & numOpp values in the NodeManagerMetrics. 7. There seem to be two redundant parameters at YarnConfiguration at the moment: NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH and NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH. If I am not missing something, we should keep one of the two. Actually they are bit different. NM_CONTAINER_QUEUING_MIN_QUEUE_LENGTH and the corresponding max value is used by the RM to calculate a limit value for the Queue. It is possible that the Queue can momentarily go above that. While NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH is used in the NM to prevent queuing beyond that value. It is a configuration hard limit ( Karthik Kambatla had requested it)
          Hide
          kkaranasos Konstantinos Karanasos added a comment -

          Here are some comments on the ContainerScheduler:

          • queuedOpportunisticContainers will have concurrency issues. We are updating it when containers arrive but also in the shedQueuedOpportunisticContainers.
          • queuedGuaranteedContainers and queuedOpportunisticContainers: I think we should use queues. I don't think we retrieve the container by the key anywhere either ways.
          • oppContainersMarkedForKill: could be a Set, right?
          • scheduledToRunContainers are containers that are either already running or are going to run very soon (transitioning from SCHEDULED to RUNNING state). Name is a bit misleading, because it sounds like they are only the ones belonging to the second category. I would rather say runningContainers and specify in a comment that they might not be running at this very moment but will be running very soon.
          • In the onContainerCompleted(), the scheduledToRunContainers.remove(container.getContainerId()) and the startPendingContainers() can go inside the if statement above. If the container was not running and no resources were freed up, we don't need to call the startPendingContainers().
          • fields of the opportunisticContainersStatus are set in different places. Due to that, when we call getOpportunisticContainersStatus() we may see an inconsistent object. Let's set the fields only in the getOpportunisticContainersStatus().
          • line 252, indeed we can now do extraOpportContainersToKill -> opportContainersToKill, as Karthik mentioned at a comment.
          • line 87: increase -> increases
          • shedQueuedOpportunisticContainers:
            • numAllowed is the number of allowed containers in the queue. Instead, we are killing numAllowed containers. In other words, we should not kill numAllowed, but queuedOpportunisticContainers.size() - numAllowed.
            • "Container Killed to make room for Guaranteed Container." -> "Container killed to meet NM queuing limits". Instead of kill, you can also say de-queued.
          Show
          kkaranasos Konstantinos Karanasos added a comment - Here are some comments on the ContainerScheduler : queuedOpportunisticContainers will have concurrency issues. We are updating it when containers arrive but also in the shedQueuedOpportunisticContainers . queuedGuaranteedContainers and queuedOpportunisticContainers : I think we should use queues. I don't think we retrieve the container by the key anywhere either ways. oppContainersMarkedForKill : could be a Set, right? scheduledToRunContainers are containers that are either already running or are going to run very soon (transitioning from SCHEDULED to RUNNING state). Name is a bit misleading, because it sounds like they are only the ones belonging to the second category. I would rather say runningContainers and specify in a comment that they might not be running at this very moment but will be running very soon. In the onContainerCompleted() , the scheduledToRunContainers.remove(container.getContainerId()) and the startPendingContainers() can go inside the if statement above. If the container was not running and no resources were freed up, we don't need to call the startPendingContainers() . fields of the opportunisticContainersStatus are set in different places. Due to that, when we call getOpportunisticContainersStatus() we may see an inconsistent object. Let's set the fields only in the getOpportunisticContainersStatus() . line 252, indeed we can now do extraOpportContainersToKill -> opportContainersToKill, as Karthik mentioned at a comment. line 87: increase -> increases shedQueuedOpportunisticContainers : numAllowed is the number of allowed containers in the queue. Instead, we are killing numAllowed containers. In other words, we should not kill numAllowed, but queuedOpportunisticContainers.size() - numAllowed . "Container Killed to make room for Guaranteed Container." -> "Container killed to meet NM queuing limits". Instead of kill, you can also say de-queued.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87670013

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java —
          @@ -0,0 +1,163 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.Resource;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +/**
          + * This class abstracts out how a container contributes to Resource Utilization.
          + * It is used by the

          {@link ContainerScheduler}

          to determine which
          + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED
          + * container.
          + * It currently equates resource utilization with the total resource allocated
          + * to the container. Another implementation might choose to use the actual
          + * resource utilization.
          + */
          +
          +public class ResourceUtilizationManager {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ResourceUtilizationManager.class);
          +
          + private ResourceUtilization containersAllocation;
          + private ContainerScheduler scheduler;
          +
          + ResourceUtilizationManager(ContainerScheduler scheduler)

          { + this.containersAllocation = ResourceUtilization.newInstance(0, 0, 0.0f); + this.scheduler = scheduler; + }

          +
          + /**
          + * Get the current accumulated utilization. Currently it is the accumulation
          + * of totally allocated resources to a container.
          + * @return ResourceUtilization Resource Utilization.
          + */
          + public ResourceUtilization getCurrentUtilization()

          { + return this.containersAllocation; + }

          +
          + /**
          + * Add Container's resources to the accumulated Utilization.
          + * @param container Container.
          + */
          + public void addContainerResources(Container container)

          { + increaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + }

          +
          + /**
          + * Subtract Container's resources to the accumulated Utilization.
          + * @param container Container.
          + */
          + public void subtractContainerResource(Container container)

          { + decreaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + }

          +
          + /**
          + * Check if NM has resources available currently to run the container.
          + * @param container Container.
          + * @return True, if NM has resources available currently to run the container.
          + */
          + public boolean hasResourcesAvailable(Container container)

          { + long pMemBytes = container.getResource().getMemorySize() * 1024 * 1024L; + return hasResourcesAvailable(pMemBytes, + (long) (getContainersMonitor().getVmemRatio()* pMemBytes), + container.getResource().getVirtualCores()); + }

          +
          + private boolean hasResourcesAvailable(long pMemBytes, long vMemBytes,
          + int cpuVcores) {
          + // Check physical memory.
          + if (LOG.isDebugEnabled()) {
          + LOG.debug("pMemCheck [current={} + asked={} > allowed={}]",
          + this.containersAllocation.getPhysicalMemory(),
          + (pMemBytes >> 20),
          + (getContainersMonitor().getPmemAllocatedForContainers() >> 20));
          + }
          + if (this.containersAllocation.getPhysicalMemory() +
          + (int) (pMemBytes >> 20) >
          + (int) (getContainersMonitor()
          + .getPmemAllocatedForContainers() >> 20))

          { + return false; + }
          +
          + if (LOG.isDebugEnabled()) {
          + LOG.debug("before vMemCheck" +
          + "[isEnabled={}, current={} + asked={} > allowed={}]",
          + getContainersMonitor().isVmemCheckEnabled(),
          + this.containersAllocation.getVirtualMemory(), (vMemBytes >> 20),
          + (getContainersMonitor().getVmemAllocatedForContainers() >> 20));
          + }
          + // Check virtual memory.
          + if (getContainersMonitor().isVmemCheckEnabled() &&
          + this.containersAllocation.getVirtualMemory() +
          + (int) (vMemBytes >> 20) >
          + (int) (getContainersMonitor()
          + .getVmemAllocatedForContainers() >> 20)) { + return false; + }

          +
          + float vCores = (float) cpuVcores /
          + getContainersMonitor().getVCoresAllocatedForContainers();
          + if (LOG.isDebugEnabled()) {
          + LOG.debug("before cpuCheck [asked={} > allowed={}]",
          + this.containersAllocation.getCPU(), vCores);
          + }
          + // Check CPU.
          + if (this.containersAllocation.getCPU() + vCores > 1.0f)

          { + return false; + }

          + return true;
          + }
          +
          + public ContainersMonitor getContainersMonitor()

          { + return this.scheduler.getContainersMonitor(); + }

          +
          + public static void increaseResourceUtilization(
          + ContainersMonitor containersMonitor, ResourceUtilization resourceAlloc,
          + Resource resource)

          { + float vCores = (float) resource.getVirtualCores() / + containersMonitor.getVCoresAllocatedForContainers(); + int vmem = (int) (resource.getMemorySize() + * containersMonitor.getVmemRatio()); + resourceAlloc.addTo((int)resource.getMemorySize(), vmem, vCores); + }

          +
          + public static void decreaseResourceUtilization(
          — End diff –

          I sort of agree, but since it depends certain information from the ContainersMonitor (which we have to pass in to the method), I thought we should maybe keep it as static.
          If you don't mind, lets keep it a it is for this patch.. and refactor when we get a more concrete argument.

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87670013 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java — @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class abstracts out how a container contributes to Resource Utilization. + * It is used by the {@link ContainerScheduler} to determine which + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED + * container. + * It currently equates resource utilization with the total resource allocated + * to the container. Another implementation might choose to use the actual + * resource utilization. + */ + +public class ResourceUtilizationManager { + + private static final Logger LOG = + LoggerFactory.getLogger(ResourceUtilizationManager.class); + + private ResourceUtilization containersAllocation; + private ContainerScheduler scheduler; + + ResourceUtilizationManager(ContainerScheduler scheduler) { + this.containersAllocation = ResourceUtilization.newInstance(0, 0, 0.0f); + this.scheduler = scheduler; + } + + /** + * Get the current accumulated utilization. Currently it is the accumulation + * of totally allocated resources to a container. + * @return ResourceUtilization Resource Utilization. + */ + public ResourceUtilization getCurrentUtilization() { + return this.containersAllocation; + } + + /** + * Add Container's resources to the accumulated Utilization. + * @param container Container. + */ + public void addContainerResources(Container container) { + increaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + } + + /** + * Subtract Container's resources to the accumulated Utilization. + * @param container Container. + */ + public void subtractContainerResource(Container container) { + decreaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + } + + /** + * Check if NM has resources available currently to run the container. + * @param container Container. + * @return True, if NM has resources available currently to run the container. + */ + public boolean hasResourcesAvailable(Container container) { + long pMemBytes = container.getResource().getMemorySize() * 1024 * 1024L; + return hasResourcesAvailable(pMemBytes, + (long) (getContainersMonitor().getVmemRatio()* pMemBytes), + container.getResource().getVirtualCores()); + } + + private boolean hasResourcesAvailable(long pMemBytes, long vMemBytes, + int cpuVcores) { + // Check physical memory. + if (LOG.isDebugEnabled()) { + LOG.debug("pMemCheck [current={} + asked={} > allowed={}] ", + this.containersAllocation.getPhysicalMemory(), + (pMemBytes >> 20), + (getContainersMonitor().getPmemAllocatedForContainers() >> 20)); + } + if (this.containersAllocation.getPhysicalMemory() + + (int) (pMemBytes >> 20) > + (int) (getContainersMonitor() + .getPmemAllocatedForContainers() >> 20)) { + return false; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("before vMemCheck" + + " [isEnabled={}, current={} + asked={} > allowed={}] ", + getContainersMonitor().isVmemCheckEnabled(), + this.containersAllocation.getVirtualMemory(), (vMemBytes >> 20), + (getContainersMonitor().getVmemAllocatedForContainers() >> 20)); + } + // Check virtual memory. + if (getContainersMonitor().isVmemCheckEnabled() && + this.containersAllocation.getVirtualMemory() + + (int) (vMemBytes >> 20) > + (int) (getContainersMonitor() + .getVmemAllocatedForContainers() >> 20)) { + return false; + } + + float vCores = (float) cpuVcores / + getContainersMonitor().getVCoresAllocatedForContainers(); + if (LOG.isDebugEnabled()) { + LOG.debug("before cpuCheck [asked={} > allowed={}] ", + this.containersAllocation.getCPU(), vCores); + } + // Check CPU. + if (this.containersAllocation.getCPU() + vCores > 1.0f) { + return false; + } + return true; + } + + public ContainersMonitor getContainersMonitor() { + return this.scheduler.getContainersMonitor(); + } + + public static void increaseResourceUtilization( + ContainersMonitor containersMonitor, ResourceUtilization resourceAlloc, + Resource resource) { + float vCores = (float) resource.getVirtualCores() / + containersMonitor.getVCoresAllocatedForContainers(); + int vmem = (int) (resource.getMemorySize() + * containersMonitor.getVmemRatio()); + resourceAlloc.addTo((int)resource.getMemorySize(), vmem, vCores); + } + + public static void decreaseResourceUtilization( — End diff – I sort of agree, but since it depends certain information from the ContainersMonitor (which we have to pass in to the method), I thought we should maybe keep it as static. If you don't mind, lets keep it a it is for this patch.. and refactor when we get a more concrete argument.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87669163

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java —
          @@ -0,0 +1,163 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          — End diff –

          will do..

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87669163 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java — @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.api.records.ExecutionType; — End diff – will do..
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87669098

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java —
          @@ -743,6 +739,8 @@ private void changeContainerResource(
          LOG.warn("Container " + containerId.toString() + "does not exist");
          return;
          }
          + // TODO: Route this through the ContainerScheduler to
          — End diff –

          will do: YARN-5860

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87669098 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java — @@ -743,6 +739,8 @@ private void changeContainerResource( LOG.warn("Container " + containerId.toString() + "does not exist"); return; } + // TODO: Route this through the ContainerScheduler to — End diff – will do: YARN-5860
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87668870

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManager.java —
          @@ -26,6 +26,8 @@
          import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent;
          import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
          .ContainersMonitor;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler
          + .ContainerScheduler;
          — End diff –

          as mentioned earlier.. can we leave it as it is, since it is ignored by checkstyle ?

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87668870 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManager.java — @@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor .ContainersMonitor; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler + .ContainerScheduler; — End diff – as mentioned earlier.. can we leave it as it is, since it is ignored by checkstyle ?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87668767

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml —
          @@ -1000,10 +1000,10 @@
          </property>

          <property>

          • <description>Enable Queuing of OPPORTUNISTIC containers on the
            + <description>Max numbed of OPPORTUNISTIC containers to queue at the
              • End diff –

          done..

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87668767 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml — @@ -1000,10 +1000,10 @@ </property> <property> <description>Enable Queuing of OPPORTUNISTIC containers on the + <description>Max numbed of OPPORTUNISTIC containers to queue at the End diff – done..
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87668681

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEvent.java —
          @@ -0,0 +1,51 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.event.AbstractEvent;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
          + .Container;
          — End diff –

          My IDE is very obstinate about it Can we leave it as it is.. especially since the checkstyle anyway ignores it ?

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87668681 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEvent.java — @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container + .Container; — End diff – My IDE is very obstinate about it Can we leave it as it is.. especially since the checkstyle anyway ignores it ?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87668561

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator();
          + while(reverseContainerIterator.hasNext() &&
          + !hasSufficientResources(resourcesToFreeUp)) {
          + Container runningCont = reverseContainerIterator.next();
          + if (runningCont.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC) {
          +
          + if (oppContainersMarkedForKill.containsKey(
          + runningCont.getContainerId()))

          { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + }

          + extraOpportContainersToKill.add(runningCont);
          + ResourceUtilizationManager.decreaseResourceUtilization(
          + getContainersMonitor(), resourcesToFreeUp,
          + runningCont.getResource());
          + }
          + }
          + if (!hasSufficientResources(resourcesToFreeUp)) {
          + LOG.warn("There are no sufficient resources to start guaranteed [{}]" +
          + "even after attempting to kill all running" +
          + "opportunistic containers.", containerToStartId);
          + }
          + return extraOpportContainersToKill;
          + }
          +
          + private boolean hasSufficientResources(
          + ResourceUtilization resourcesToFreeUp)

          { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + }

          +
          + private ResourceUtilization resourcesToFreeUp(
          + ContainerId containerToStartId) {
          + // Get allocation of currently allocated containers.
          + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization
          + .newInstance(this.utilizationManager.getCurrentUtilization());
          +
          + // Add to the allocation the allocation of the pending guaranteed
          + // containers that will start before the current container will be started.
          + for (Container container : queuedGuaranteedContainers.values()) {
          + ResourceUtilizationManager.increaseResourceUtilization(
          + getContainersMonitor(), resourceAllocationToFreeUp,
          + container.getResource());
          + if (container.getContainerId().equals(containerToStartId))

          { + break; + }

          + }
          +
          + // These Resources have already been freed, due to demand from an
          + // earlier Guaranteed container.
          + for (Container container : oppContainersMarkedForKill.values())

          { + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + }

          +
          + // Subtract the overall node resources.
          + getContainersMonitor().subtractNodeResourcesFromResourceUtilization(
          + resourceAllocationToFreeUp);
          + return resourceAllocationToFreeUp;
          + }
          +
          + public void updateQueuingLimit(ContainerQueuingLimit limit) {
          + this.queuingLimit.setMaxQueueLength(limit.getMaxQueueLength());
          + // TODO: Include wait time as well once it is implemented
          + if (this.queuingLimit.getMaxQueueLength() > -1)

          { + shedQueuedOpportunisticContainers(); + }

          + }
          +
          + private void shedQueuedOpportunisticContainers() {
          + int numAllowed = this.queuingLimit.getMaxQueueLength();
          + Iterator<Container> containerIter =
          + queuedOpportunisticContainers.values().iterator();
          + while (containerIter.hasNext()) {
          + Container container = containerIter.next();
          + if (numAllowed <= 0) {
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          — End diff –

          The Log.info line is printed immediately, which is why it has 'will be killed'.
          The diagnostic line is part of the Container status message and is in past-tense and does not need the containerid.

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87668561 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); + this.utilizationManager.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The additional opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator(); + while(reverseContainerIterator.hasNext() && + !hasSufficientResources(resourcesToFreeUp)) { + Container runningCont = reverseContainerIterator.next(); + if (runningCont.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + + if (oppContainersMarkedForKill.containsKey( + runningCont.getContainerId())) { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + } + extraOpportContainersToKill.add(runningCont); + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourcesToFreeUp, + runningCont.getResource()); + } + } + if (!hasSufficientResources(resourcesToFreeUp)) { + LOG.warn("There are no sufficient resources to start guaranteed [{}] " + + "even after attempting to kill all running" + + "opportunistic containers.", containerToStartId); + } + return extraOpportContainersToKill; + } + + private boolean hasSufficientResources( + ResourceUtilization resourcesToFreeUp) { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + } + + private ResourceUtilization resourcesToFreeUp( + ContainerId containerToStartId) { + // Get allocation of currently allocated containers. + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization + .newInstance(this.utilizationManager.getCurrentUtilization()); + + // Add to the allocation the allocation of the pending guaranteed + // containers that will start before the current container will be started. + for (Container container : queuedGuaranteedContainers.values()) { + ResourceUtilizationManager.increaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + if (container.getContainerId().equals(containerToStartId)) { + break; + } + } + + // These Resources have already been freed, due to demand from an + // earlier Guaranteed container. + for (Container container : oppContainersMarkedForKill.values()) { + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + } + + // Subtract the overall node resources. + getContainersMonitor().subtractNodeResourcesFromResourceUtilization( + resourceAllocationToFreeUp); + return resourceAllocationToFreeUp; + } + + public void updateQueuingLimit(ContainerQueuingLimit limit) { + this.queuingLimit.setMaxQueueLength(limit.getMaxQueueLength()); + // TODO: Include wait time as well once it is implemented + if (this.queuingLimit.getMaxQueueLength() > -1) { + shedQueuedOpportunisticContainers(); + } + } + + private void shedQueuedOpportunisticContainers() { + int numAllowed = this.queuingLimit.getMaxQueueLength(); + Iterator<Container> containerIter = + queuedOpportunisticContainers.values().iterator(); + while (containerIter.hasNext()) { + Container container = containerIter.next(); + if (numAllowed <= 0) { + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); — End diff – The Log.info line is printed immediately, which is why it has 'will be killed'. The diagnostic line is part of the Container status message and is in past-tense and does not need the containerid.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87668393

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator();
          + while(reverseContainerIterator.hasNext() &&
          + !hasSufficientResources(resourcesToFreeUp)) {
          + Container runningCont = reverseContainerIterator.next();
          + if (runningCont.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC) {
          +
          + if (oppContainersMarkedForKill.containsKey(
          + runningCont.getContainerId()))

          { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + }

          + extraOpportContainersToKill.add(runningCont);
          + ResourceUtilizationManager.decreaseResourceUtilization(
          + getContainersMonitor(), resourcesToFreeUp,
          + runningCont.getResource());
          + }
          + }
          + if (!hasSufficientResources(resourcesToFreeUp)) {
          + LOG.warn("There are no sufficient resources to start guaranteed [{}]" +
          + "even after attempting to kill all running" +
          + "opportunistic containers.", containerToStartId);
          + }
          + return extraOpportContainersToKill;
          + }
          +
          + private boolean hasSufficientResources(
          + ResourceUtilization resourcesToFreeUp)

          { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + }

          +
          + private ResourceUtilization resourcesToFreeUp(
          + ContainerId containerToStartId) {
          + // Get allocation of currently allocated containers.
          + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization
          + .newInstance(this.utilizationManager.getCurrentUtilization());
          +
          + // Add to the allocation the allocation of the pending guaranteed
          + // containers that will start before the current container will be started.
          + for (Container container : queuedGuaranteedContainers.values()) {
          + ResourceUtilizationManager.increaseResourceUtilization(
          + getContainersMonitor(), resourceAllocationToFreeUp,
          + container.getResource());
          + if (container.getContainerId().equals(containerToStartId))

          { + break; + }

          + }
          +
          + // These Resources have already been freed, due to demand from an
          + // earlier Guaranteed container.
          + for (Container container : oppContainersMarkedForKill.values())

          { + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + }

          +
          + // Subtract the overall node resources.
          + getContainersMonitor().subtractNodeResourcesFromResourceUtilization(
          + resourceAllocationToFreeUp);
          + return resourceAllocationToFreeUp;
          + }
          +
          + public void updateQueuingLimit(ContainerQueuingLimit limit) {
          + this.queuingLimit.setMaxQueueLength(limit.getMaxQueueLength());
          + // TODO: Include wait time as well once it is implemented
          — End diff –

          DOne..

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87668393 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); + this.utilizationManager.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The additional opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator(); + while(reverseContainerIterator.hasNext() && + !hasSufficientResources(resourcesToFreeUp)) { + Container runningCont = reverseContainerIterator.next(); + if (runningCont.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + + if (oppContainersMarkedForKill.containsKey( + runningCont.getContainerId())) { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + } + extraOpportContainersToKill.add(runningCont); + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourcesToFreeUp, + runningCont.getResource()); + } + } + if (!hasSufficientResources(resourcesToFreeUp)) { + LOG.warn("There are no sufficient resources to start guaranteed [{}] " + + "even after attempting to kill all running" + + "opportunistic containers.", containerToStartId); + } + return extraOpportContainersToKill; + } + + private boolean hasSufficientResources( + ResourceUtilization resourcesToFreeUp) { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + } + + private ResourceUtilization resourcesToFreeUp( + ContainerId containerToStartId) { + // Get allocation of currently allocated containers. + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization + .newInstance(this.utilizationManager.getCurrentUtilization()); + + // Add to the allocation the allocation of the pending guaranteed + // containers that will start before the current container will be started. + for (Container container : queuedGuaranteedContainers.values()) { + ResourceUtilizationManager.increaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + if (container.getContainerId().equals(containerToStartId)) { + break; + } + } + + // These Resources have already been freed, due to demand from an + // earlier Guaranteed container. + for (Container container : oppContainersMarkedForKill.values()) { + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + } + + // Subtract the overall node resources. + getContainersMonitor().subtractNodeResourcesFromResourceUtilization( + resourceAllocationToFreeUp); + return resourceAllocationToFreeUp; + } + + public void updateQueuingLimit(ContainerQueuingLimit limit) { + this.queuingLimit.setMaxQueueLength(limit.getMaxQueueLength()); + // TODO: Include wait time as well once it is implemented — End diff – DOne..
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87668067

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator();
          + while(reverseContainerIterator.hasNext() &&
          + !hasSufficientResources(resourcesToFreeUp)) {
          + Container runningCont = reverseContainerIterator.next();
          + if (runningCont.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC) {
          +
          + if (oppContainersMarkedForKill.containsKey(
          + runningCont.getContainerId()))

          { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + }

          + extraOpportContainersToKill.add(runningCont);
          + ResourceUtilizationManager.decreaseResourceUtilization(
          + getContainersMonitor(), resourcesToFreeUp,
          + runningCont.getResource());
          + }
          + }
          + if (!hasSufficientResources(resourcesToFreeUp)) {
          + LOG.warn("There are no sufficient resources to start guaranteed [{}]" +
          — End diff –

          Actually that message might be a bit misleading. It should be re-worded to 'There are no sufficient resources at the moment to start guaranteed...'. You can only reach here if kill signals have been sent to Opp. containers, but the completed container event hasn't been received by the Scheduler yet. At which point the guaranteed container will start.
          It is never possible to allocate a guaranteed container on node that cannot start it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87668067 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); + this.utilizationManager.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The additional opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator(); + while(reverseContainerIterator.hasNext() && + !hasSufficientResources(resourcesToFreeUp)) { + Container runningCont = reverseContainerIterator.next(); + if (runningCont.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + + if (oppContainersMarkedForKill.containsKey( + runningCont.getContainerId())) { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + } + extraOpportContainersToKill.add(runningCont); + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourcesToFreeUp, + runningCont.getResource()); + } + } + if (!hasSufficientResources(resourcesToFreeUp)) { + LOG.warn("There are no sufficient resources to start guaranteed [{}] " + — End diff – Actually that message might be a bit misleading. It should be re-worded to 'There are no sufficient resources at the moment to start guaranteed...'. You can only reach here if kill signals have been sent to Opp. containers, but the completed container event hasn't been received by the Scheduler yet. At which point the guaranteed container will start. It is never possible to allocate a guaranteed container on node that cannot start it.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87667155

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          — End diff –

          In this case, the 'extra' is needed. It signifies all the container above the already marked 'oppContainersToKill' that need to be killed.

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87667155 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); + this.utilizationManager.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The additional opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); — End diff – In this case, the 'extra' is needed. It signifies all the container above the already marked 'oppContainersToKill' that need to be killed.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87666512

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          — End diff –

          I agree it is not technically needed. But, being a scheduler, I felt it would come in handy to have a data-structure of running containers.
          One other use-case I forsee is if the container is updated (if we implement container promotion, for instance), we would have to move it from one list to another, which is cumbersome.
          I vote to keep it as it is for the time-being and move it if required.

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87666512 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); — End diff – I agree it is not technically needed. But, being a scheduler, I felt it would come in handy to have a data-structure of running containers. One other use-case I forsee is if the container is updated (if we implement container promotion, for instance), we would have to move it from one list to another, which is cumbersome. I vote to keep it as it is for the time-being and move it if required.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87666090

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e) {
          + LOG.warn("Could not store container state into store..", e);
          — End diff –

          Will modify the error message...

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87666090 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); — End diff – Will modify the error message...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87665464

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          — End diff –

          Ditto.. as above.

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87665464 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + — End diff – Ditto.. as above.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87665434

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          — End diff –

          Same as above...

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87665434 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", — End diff – Same as above...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87665076

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          — End diff –

          I agree with @kkaranasos, lets keep it as it is... Regarding the LIFO order of killing, I am using a descending iterator so candidates are examined in LIFO order.

          It was earlier 'runningContainers'. But based of @jian-he's feedback, we felt this is better, since the containers need not be immediately running at the point it is added to this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87665076 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = — End diff – I agree with @kkaranasos, lets keep it as it is... Regarding the LIFO order of killing, I am using a descending iterator so candidates are examined in LIFO order. It was earlier 'runningContainers'. But based of @jian-he's feedback, we felt this is better, since the containers need not be immediately running at the point it is added to this.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87664657

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          — End diff –

          Will do

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87664657 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = — End diff – Will do
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87664536

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          — End diff –

          Ditto

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87664536 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> — End diff – Ditto
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user xslogic commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87664283

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          — End diff –

          I chose a LinkedHashMap, since it is essentially an indexed Queue. We might need to check if a container exists in the queue or remove a specific container from the queue (for eg. when AM asks to kill a queued container).
          Also, like you noted, we would also need to implement hashCode/equals for Container...

          Show
          githubbot ASF GitHub Bot added a comment - Github user xslogic commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87664283 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> — End diff – I chose a LinkedHashMap, since it is essentially an indexed Queue. We might need to check if a container exists in the queue or remove a specific container from the queue (for eg. when AM asks to kill a queued container). Also, like you noted, we would also need to implement hashCode/equals for Container...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kkaranasos commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87647259

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          — End diff –

          I had named that extra because some containers might already have been marked for killing, and here you will keep the additional ones.

          Show
          githubbot ASF GitHub Bot added a comment - Github user kkaranasos commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87647259 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = — End diff – I had named that extra because some containers might already have been marked for killing, and here you will keep the additional ones.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87640967

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator();
          + while(reverseContainerIterator.hasNext() &&
          + !hasSufficientResources(resourcesToFreeUp)) {
          + Container runningCont = reverseContainerIterator.next();
          + if (runningCont.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC) {
          +
          + if (oppContainersMarkedForKill.containsKey(
          — End diff –

          How about removing all containers that are already marked before we get to this..

          If runningOppContainers were a stack and oppContainersToKill were a queue, you could just do
          `
          for (Container container : runningOppContainers.removeAll(oppContainersToKill)) {
          killContainer(container);
          if (hasSufficientResources())

          { break; }

          }`

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87640967 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); + this.utilizationManager.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The additional opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator(); + while(reverseContainerIterator.hasNext() && + !hasSufficientResources(resourcesToFreeUp)) { + Container runningCont = reverseContainerIterator.next(); + if (runningCont.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + + if (oppContainersMarkedForKill.containsKey( — End diff – How about removing all containers that are already marked before we get to this.. If runningOppContainers were a stack and oppContainersToKill were a queue, you could just do ` for (Container container : runningOppContainers.removeAll(oppContainersToKill)) { killContainer(container); if (hasSufficientResources()) { break; } }`
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87643141

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml —
          @@ -1000,10 +1000,10 @@
          </property>

          <property>

          • <description>Enable Queuing of OPPORTUNISTIC containers on the
            + <description>Max numbed of OPPORTUNISTIC containers to queue at the
              • End diff –

          s/numbed/number

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87643141 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml — @@ -1000,10 +1000,10 @@ </property> <property> <description>Enable Queuing of OPPORTUNISTIC containers on the + <description>Max numbed of OPPORTUNISTIC containers to queue at the End diff – s/numbed/number
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87645682

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java —
          @@ -0,0 +1,163 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          — End diff –

          Some unused imports.

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87645682 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java — @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.api.records.ExecutionType; — End diff – Some unused imports.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87645560

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java —
          @@ -0,0 +1,163 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.Resource;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +/**
          + * This class abstracts out how a container contributes to Resource Utilization.
          + * It is used by the

          {@link ContainerScheduler}

          to determine which
          + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED
          + * container.
          + * It currently equates resource utilization with the total resource allocated
          + * to the container. Another implementation might choose to use the actual
          + * resource utilization.
          + */
          +
          +public class ResourceUtilizationManager {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ResourceUtilizationManager.class);
          +
          + private ResourceUtilization containersAllocation;
          + private ContainerScheduler scheduler;
          +
          + ResourceUtilizationManager(ContainerScheduler scheduler)

          { + this.containersAllocation = ResourceUtilization.newInstance(0, 0, 0.0f); + this.scheduler = scheduler; + }

          +
          + /**
          + * Get the current accumulated utilization. Currently it is the accumulation
          + * of totally allocated resources to a container.
          + * @return ResourceUtilization Resource Utilization.
          + */
          + public ResourceUtilization getCurrentUtilization()

          { + return this.containersAllocation; + }

          +
          + /**
          + * Add Container's resources to the accumulated Utilization.
          + * @param container Container.
          + */
          + public void addContainerResources(Container container)

          { + increaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + }

          +
          + /**
          + * Subtract Container's resources to the accumulated Utilization.
          + * @param container Container.
          + */
          + public void subtractContainerResource(Container container)

          { + decreaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + }

          +
          + /**
          + * Check if NM has resources available currently to run the container.
          + * @param container Container.
          + * @return True, if NM has resources available currently to run the container.
          + */
          + public boolean hasResourcesAvailable(Container container)

          { + long pMemBytes = container.getResource().getMemorySize() * 1024 * 1024L; + return hasResourcesAvailable(pMemBytes, + (long) (getContainersMonitor().getVmemRatio()* pMemBytes), + container.getResource().getVirtualCores()); + }

          +
          + private boolean hasResourcesAvailable(long pMemBytes, long vMemBytes,
          + int cpuVcores) {
          + // Check physical memory.
          + if (LOG.isDebugEnabled()) {
          + LOG.debug("pMemCheck [current={} + asked={} > allowed={}]",
          + this.containersAllocation.getPhysicalMemory(),
          + (pMemBytes >> 20),
          + (getContainersMonitor().getPmemAllocatedForContainers() >> 20));
          + }
          + if (this.containersAllocation.getPhysicalMemory() +
          + (int) (pMemBytes >> 20) >
          + (int) (getContainersMonitor()
          + .getPmemAllocatedForContainers() >> 20))

          { + return false; + }
          +
          + if (LOG.isDebugEnabled()) {
          + LOG.debug("before vMemCheck" +
          + "[isEnabled={}, current={} + asked={} > allowed={}]",
          + getContainersMonitor().isVmemCheckEnabled(),
          + this.containersAllocation.getVirtualMemory(), (vMemBytes >> 20),
          + (getContainersMonitor().getVmemAllocatedForContainers() >> 20));
          + }
          + // Check virtual memory.
          + if (getContainersMonitor().isVmemCheckEnabled() &&
          + this.containersAllocation.getVirtualMemory() +
          + (int) (vMemBytes >> 20) >
          + (int) (getContainersMonitor()
          + .getVmemAllocatedForContainers() >> 20)) { + return false; + }

          +
          + float vCores = (float) cpuVcores /
          + getContainersMonitor().getVCoresAllocatedForContainers();
          + if (LOG.isDebugEnabled()) {
          + LOG.debug("before cpuCheck [asked={} > allowed={}]",
          + this.containersAllocation.getCPU(), vCores);
          + }
          + // Check CPU.
          + if (this.containersAllocation.getCPU() + vCores > 1.0f)

          { + return false; + }

          + return true;
          + }
          +
          + public ContainersMonitor getContainersMonitor()

          { + return this.scheduler.getContainersMonitor(); + }

          +
          + public static void increaseResourceUtilization(
          — End diff –

          Shouldn't this be a non-static method on ResourceUtilization instead?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87645560 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java — @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class abstracts out how a container contributes to Resource Utilization. + * It is used by the {@link ContainerScheduler} to determine which + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED + * container. + * It currently equates resource utilization with the total resource allocated + * to the container. Another implementation might choose to use the actual + * resource utilization. + */ + +public class ResourceUtilizationManager { + + private static final Logger LOG = + LoggerFactory.getLogger(ResourceUtilizationManager.class); + + private ResourceUtilization containersAllocation; + private ContainerScheduler scheduler; + + ResourceUtilizationManager(ContainerScheduler scheduler) { + this.containersAllocation = ResourceUtilization.newInstance(0, 0, 0.0f); + this.scheduler = scheduler; + } + + /** + * Get the current accumulated utilization. Currently it is the accumulation + * of totally allocated resources to a container. + * @return ResourceUtilization Resource Utilization. + */ + public ResourceUtilization getCurrentUtilization() { + return this.containersAllocation; + } + + /** + * Add Container's resources to the accumulated Utilization. + * @param container Container. + */ + public void addContainerResources(Container container) { + increaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + } + + /** + * Subtract Container's resources to the accumulated Utilization. + * @param container Container. + */ + public void subtractContainerResource(Container container) { + decreaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + } + + /** + * Check if NM has resources available currently to run the container. + * @param container Container. + * @return True, if NM has resources available currently to run the container. + */ + public boolean hasResourcesAvailable(Container container) { + long pMemBytes = container.getResource().getMemorySize() * 1024 * 1024L; + return hasResourcesAvailable(pMemBytes, + (long) (getContainersMonitor().getVmemRatio()* pMemBytes), + container.getResource().getVirtualCores()); + } + + private boolean hasResourcesAvailable(long pMemBytes, long vMemBytes, + int cpuVcores) { + // Check physical memory. + if (LOG.isDebugEnabled()) { + LOG.debug("pMemCheck [current={} + asked={} > allowed={}] ", + this.containersAllocation.getPhysicalMemory(), + (pMemBytes >> 20), + (getContainersMonitor().getPmemAllocatedForContainers() >> 20)); + } + if (this.containersAllocation.getPhysicalMemory() + + (int) (pMemBytes >> 20) > + (int) (getContainersMonitor() + .getPmemAllocatedForContainers() >> 20)) { + return false; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("before vMemCheck" + + " [isEnabled={}, current={} + asked={} > allowed={}] ", + getContainersMonitor().isVmemCheckEnabled(), + this.containersAllocation.getVirtualMemory(), (vMemBytes >> 20), + (getContainersMonitor().getVmemAllocatedForContainers() >> 20)); + } + // Check virtual memory. + if (getContainersMonitor().isVmemCheckEnabled() && + this.containersAllocation.getVirtualMemory() + + (int) (vMemBytes >> 20) > + (int) (getContainersMonitor() + .getVmemAllocatedForContainers() >> 20)) { + return false; + } + + float vCores = (float) cpuVcores / + getContainersMonitor().getVCoresAllocatedForContainers(); + if (LOG.isDebugEnabled()) { + LOG.debug("before cpuCheck [asked={} > allowed={}] ", + this.containersAllocation.getCPU(), vCores); + } + // Check CPU. + if (this.containersAllocation.getCPU() + vCores > 1.0f) { + return false; + } + return true; + } + + public ContainersMonitor getContainersMonitor() { + return this.scheduler.getContainersMonitor(); + } + + public static void increaseResourceUtilization( — End diff – Shouldn't this be a non-static method on ResourceUtilization instead?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87641199

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator();
          + while(reverseContainerIterator.hasNext() &&
          + !hasSufficientResources(resourcesToFreeUp)) {
          + Container runningCont = reverseContainerIterator.next();
          + if (runningCont.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC) {
          +
          + if (oppContainersMarkedForKill.containsKey(
          + runningCont.getContainerId()))

          { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + }

          + extraOpportContainersToKill.add(runningCont);
          + ResourceUtilizationManager.decreaseResourceUtilization(
          + getContainersMonitor(), resourcesToFreeUp,
          + runningCont.getResource());
          + }
          + }
          + if (!hasSufficientResources(resourcesToFreeUp)) {
          + LOG.warn("There are no sufficient resources to start guaranteed [{}]" +
          — End diff –

          Shouldn't be this higher than WARN?

          This seems like a pretty bad case that we should never run into. Should we take more drastic measures? Crash the NM especially if failfast is true?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87641199 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); + this.utilizationManager.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The additional opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator(); + while(reverseContainerIterator.hasNext() && + !hasSufficientResources(resourcesToFreeUp)) { + Container runningCont = reverseContainerIterator.next(); + if (runningCont.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + + if (oppContainersMarkedForKill.containsKey( + runningCont.getContainerId())) { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + } + extraOpportContainersToKill.add(runningCont); + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourcesToFreeUp, + runningCont.getResource()); + } + } + if (!hasSufficientResources(resourcesToFreeUp)) { + LOG.warn("There are no sufficient resources to start guaranteed [{}] " + — End diff – Shouldn't be this higher than WARN? This seems like a pretty bad case that we should never run into. Should we take more drastic measures? Crash the NM especially if failfast is true?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87644035

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java —
          @@ -743,6 +739,8 @@ private void changeContainerResource(
          LOG.warn("Container " + containerId.toString() + "does not exist");
          return;
          }
          + // TODO: Route this through the ContainerScheduler to
          — End diff –

          Annotate TODO with JIRA?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87644035 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java — @@ -743,6 +739,8 @@ private void changeContainerResource( LOG.warn("Container " + containerId.toString() + "does not exist"); return; } + // TODO: Route this through the ContainerScheduler to — End diff – Annotate TODO with JIRA?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87645581

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java —
          @@ -0,0 +1,163 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.Resource;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +/**
          + * This class abstracts out how a container contributes to Resource Utilization.
          + * It is used by the

          {@link ContainerScheduler}

          to determine which
          + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED
          + * container.
          + * It currently equates resource utilization with the total resource allocated
          + * to the container. Another implementation might choose to use the actual
          + * resource utilization.
          + */
          +
          +public class ResourceUtilizationManager {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ResourceUtilizationManager.class);
          +
          + private ResourceUtilization containersAllocation;
          + private ContainerScheduler scheduler;
          +
          + ResourceUtilizationManager(ContainerScheduler scheduler)

          { + this.containersAllocation = ResourceUtilization.newInstance(0, 0, 0.0f); + this.scheduler = scheduler; + }

          +
          + /**
          + * Get the current accumulated utilization. Currently it is the accumulation
          + * of totally allocated resources to a container.
          + * @return ResourceUtilization Resource Utilization.
          + */
          + public ResourceUtilization getCurrentUtilization()

          { + return this.containersAllocation; + }

          +
          + /**
          + * Add Container's resources to the accumulated Utilization.
          + * @param container Container.
          + */
          + public void addContainerResources(Container container)

          { + increaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + }

          +
          + /**
          + * Subtract Container's resources to the accumulated Utilization.
          + * @param container Container.
          + */
          + public void subtractContainerResource(Container container)

          { + decreaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + }

          +
          + /**
          + * Check if NM has resources available currently to run the container.
          + * @param container Container.
          + * @return True, if NM has resources available currently to run the container.
          + */
          + public boolean hasResourcesAvailable(Container container)

          { + long pMemBytes = container.getResource().getMemorySize() * 1024 * 1024L; + return hasResourcesAvailable(pMemBytes, + (long) (getContainersMonitor().getVmemRatio()* pMemBytes), + container.getResource().getVirtualCores()); + }

          +
          + private boolean hasResourcesAvailable(long pMemBytes, long vMemBytes,
          + int cpuVcores) {
          + // Check physical memory.
          + if (LOG.isDebugEnabled()) {
          + LOG.debug("pMemCheck [current={} + asked={} > allowed={}]",
          + this.containersAllocation.getPhysicalMemory(),
          + (pMemBytes >> 20),
          + (getContainersMonitor().getPmemAllocatedForContainers() >> 20));
          + }
          + if (this.containersAllocation.getPhysicalMemory() +
          + (int) (pMemBytes >> 20) >
          + (int) (getContainersMonitor()
          + .getPmemAllocatedForContainers() >> 20))

          { + return false; + }
          +
          + if (LOG.isDebugEnabled()) {
          + LOG.debug("before vMemCheck" +
          + "[isEnabled={}, current={} + asked={} > allowed={}]",
          + getContainersMonitor().isVmemCheckEnabled(),
          + this.containersAllocation.getVirtualMemory(), (vMemBytes >> 20),
          + (getContainersMonitor().getVmemAllocatedForContainers() >> 20));
          + }
          + // Check virtual memory.
          + if (getContainersMonitor().isVmemCheckEnabled() &&
          + this.containersAllocation.getVirtualMemory() +
          + (int) (vMemBytes >> 20) >
          + (int) (getContainersMonitor()
          + .getVmemAllocatedForContainers() >> 20)) { + return false; + }

          +
          + float vCores = (float) cpuVcores /
          + getContainersMonitor().getVCoresAllocatedForContainers();
          + if (LOG.isDebugEnabled()) {
          + LOG.debug("before cpuCheck [asked={} > allowed={}]",
          + this.containersAllocation.getCPU(), vCores);
          + }
          + // Check CPU.
          + if (this.containersAllocation.getCPU() + vCores > 1.0f)

          { + return false; + }

          + return true;
          + }
          +
          + public ContainersMonitor getContainersMonitor()

          { + return this.scheduler.getContainersMonitor(); + }

          +
          + public static void increaseResourceUtilization(
          + ContainersMonitor containersMonitor, ResourceUtilization resourceAlloc,
          + Resource resource)

          { + float vCores = (float) resource.getVirtualCores() / + containersMonitor.getVCoresAllocatedForContainers(); + int vmem = (int) (resource.getMemorySize() + * containersMonitor.getVmemRatio()); + resourceAlloc.addTo((int)resource.getMemorySize(), vmem, vCores); + }

          +
          + public static void decreaseResourceUtilization(
          — End diff –

          Shouldn't this be a non-static method on ResourceUtilization instead?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87645581 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java — @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class abstracts out how a container contributes to Resource Utilization. + * It is used by the {@link ContainerScheduler} to determine which + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED + * container. + * It currently equates resource utilization with the total resource allocated + * to the container. Another implementation might choose to use the actual + * resource utilization. + */ + +public class ResourceUtilizationManager { + + private static final Logger LOG = + LoggerFactory.getLogger(ResourceUtilizationManager.class); + + private ResourceUtilization containersAllocation; + private ContainerScheduler scheduler; + + ResourceUtilizationManager(ContainerScheduler scheduler) { + this.containersAllocation = ResourceUtilization.newInstance(0, 0, 0.0f); + this.scheduler = scheduler; + } + + /** + * Get the current accumulated utilization. Currently it is the accumulation + * of totally allocated resources to a container. + * @return ResourceUtilization Resource Utilization. + */ + public ResourceUtilization getCurrentUtilization() { + return this.containersAllocation; + } + + /** + * Add Container's resources to the accumulated Utilization. + * @param container Container. + */ + public void addContainerResources(Container container) { + increaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + } + + /** + * Subtract Container's resources to the accumulated Utilization. + * @param container Container. + */ + public void subtractContainerResource(Container container) { + decreaseResourceUtilization( + getContainersMonitor(), this.containersAllocation, + container.getResource()); + } + + /** + * Check if NM has resources available currently to run the container. + * @param container Container. + * @return True, if NM has resources available currently to run the container. + */ + public boolean hasResourcesAvailable(Container container) { + long pMemBytes = container.getResource().getMemorySize() * 1024 * 1024L; + return hasResourcesAvailable(pMemBytes, + (long) (getContainersMonitor().getVmemRatio()* pMemBytes), + container.getResource().getVirtualCores()); + } + + private boolean hasResourcesAvailable(long pMemBytes, long vMemBytes, + int cpuVcores) { + // Check physical memory. + if (LOG.isDebugEnabled()) { + LOG.debug("pMemCheck [current={} + asked={} > allowed={}] ", + this.containersAllocation.getPhysicalMemory(), + (pMemBytes >> 20), + (getContainersMonitor().getPmemAllocatedForContainers() >> 20)); + } + if (this.containersAllocation.getPhysicalMemory() + + (int) (pMemBytes >> 20) > + (int) (getContainersMonitor() + .getPmemAllocatedForContainers() >> 20)) { + return false; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("before vMemCheck" + + " [isEnabled={}, current={} + asked={} > allowed={}] ", + getContainersMonitor().isVmemCheckEnabled(), + this.containersAllocation.getVirtualMemory(), (vMemBytes >> 20), + (getContainersMonitor().getVmemAllocatedForContainers() >> 20)); + } + // Check virtual memory. + if (getContainersMonitor().isVmemCheckEnabled() && + this.containersAllocation.getVirtualMemory() + + (int) (vMemBytes >> 20) > + (int) (getContainersMonitor() + .getVmemAllocatedForContainers() >> 20)) { + return false; + } + + float vCores = (float) cpuVcores / + getContainersMonitor().getVCoresAllocatedForContainers(); + if (LOG.isDebugEnabled()) { + LOG.debug("before cpuCheck [asked={} > allowed={}] ", + this.containersAllocation.getCPU(), vCores); + } + // Check CPU. + if (this.containersAllocation.getCPU() + vCores > 1.0f) { + return false; + } + return true; + } + + public ContainersMonitor getContainersMonitor() { + return this.scheduler.getContainersMonitor(); + } + + public static void increaseResourceUtilization( + ContainersMonitor containersMonitor, ResourceUtilization resourceAlloc, + Resource resource) { + float vCores = (float) resource.getVirtualCores() / + containersMonitor.getVCoresAllocatedForContainers(); + int vmem = (int) (resource.getMemorySize() + * containersMonitor.getVmemRatio()); + resourceAlloc.addTo((int)resource.getMemorySize(), vmem, vCores); + } + + public static void decreaseResourceUtilization( — End diff – Shouldn't this be a non-static method on ResourceUtilization instead?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87642982

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEvent.java —
          @@ -0,0 +1,51 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.event.AbstractEvent;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
          + .Container;
          — End diff –

          One line for imports?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87642982 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEvent.java — @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container + .Container; — End diff – One line for imports?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87643223

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManager.java —
          @@ -26,6 +26,8 @@
          import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent;
          import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
          .ContainersMonitor;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler
          + .ContainerScheduler;
          — End diff –

          imports should be on a single line?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87643223 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManager.java — @@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor .ContainersMonitor; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler + .ContainerScheduler; — End diff – imports should be on a single line?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87642663

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator();
          + while(reverseContainerIterator.hasNext() &&
          + !hasSufficientResources(resourcesToFreeUp)) {
          + Container runningCont = reverseContainerIterator.next();
          + if (runningCont.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC) {
          +
          + if (oppContainersMarkedForKill.containsKey(
          + runningCont.getContainerId()))

          { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + }

          + extraOpportContainersToKill.add(runningCont);
          + ResourceUtilizationManager.decreaseResourceUtilization(
          + getContainersMonitor(), resourcesToFreeUp,
          + runningCont.getResource());
          + }
          + }
          + if (!hasSufficientResources(resourcesToFreeUp)) {
          + LOG.warn("There are no sufficient resources to start guaranteed [{}]" +
          + "even after attempting to kill all running" +
          + "opportunistic containers.", containerToStartId);
          + }
          + return extraOpportContainersToKill;
          + }
          +
          + private boolean hasSufficientResources(
          + ResourceUtilization resourcesToFreeUp)

          { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + }

          +
          + private ResourceUtilization resourcesToFreeUp(
          + ContainerId containerToStartId) {
          + // Get allocation of currently allocated containers.
          + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization
          + .newInstance(this.utilizationManager.getCurrentUtilization());
          +
          + // Add to the allocation the allocation of the pending guaranteed
          + // containers that will start before the current container will be started.
          + for (Container container : queuedGuaranteedContainers.values()) {
          + ResourceUtilizationManager.increaseResourceUtilization(
          + getContainersMonitor(), resourceAllocationToFreeUp,
          + container.getResource());
          + if (container.getContainerId().equals(containerToStartId))

          { + break; + }

          + }
          +
          + // These Resources have already been freed, due to demand from an
          + // earlier Guaranteed container.
          + for (Container container : oppContainersMarkedForKill.values())

          { + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + }

          +
          + // Subtract the overall node resources.
          + getContainersMonitor().subtractNodeResourcesFromResourceUtilization(
          + resourceAllocationToFreeUp);
          + return resourceAllocationToFreeUp;
          + }
          +
          + public void updateQueuingLimit(ContainerQueuingLimit limit) {
          + this.queuingLimit.setMaxQueueLength(limit.getMaxQueueLength());
          + // TODO: Include wait time as well once it is implemented
          + if (this.queuingLimit.getMaxQueueLength() > -1)

          { + shedQueuedOpportunisticContainers(); + }

          + }
          +
          + private void shedQueuedOpportunisticContainers() {
          + int numAllowed = this.queuingLimit.getMaxQueueLength();
          + Iterator<Container> containerIter =
          + queuedOpportunisticContainers.values().iterator();
          + while (containerIter.hasNext()) {
          + Container container = containerIter.next();
          + if (numAllowed <= 0) {
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          — End diff –

          Shouldn't this diagnostic message be similar to the log below?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87642663 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); + this.utilizationManager.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The additional opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator(); + while(reverseContainerIterator.hasNext() && + !hasSufficientResources(resourcesToFreeUp)) { + Container runningCont = reverseContainerIterator.next(); + if (runningCont.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + + if (oppContainersMarkedForKill.containsKey( + runningCont.getContainerId())) { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + } + extraOpportContainersToKill.add(runningCont); + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourcesToFreeUp, + runningCont.getResource()); + } + } + if (!hasSufficientResources(resourcesToFreeUp)) { + LOG.warn("There are no sufficient resources to start guaranteed [{}] " + + "even after attempting to kill all running" + + "opportunistic containers.", containerToStartId); + } + return extraOpportContainersToKill; + } + + private boolean hasSufficientResources( + ResourceUtilization resourcesToFreeUp) { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + } + + private ResourceUtilization resourcesToFreeUp( + ContainerId containerToStartId) { + // Get allocation of currently allocated containers. + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization + .newInstance(this.utilizationManager.getCurrentUtilization()); + + // Add to the allocation the allocation of the pending guaranteed + // containers that will start before the current container will be started. + for (Container container : queuedGuaranteedContainers.values()) { + ResourceUtilizationManager.increaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + if (container.getContainerId().equals(containerToStartId)) { + break; + } + } + + // These Resources have already been freed, due to demand from an + // earlier Guaranteed container. + for (Container container : oppContainersMarkedForKill.values()) { + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + } + + // Subtract the overall node resources. + getContainersMonitor().subtractNodeResourcesFromResourceUtilization( + resourceAllocationToFreeUp); + return resourceAllocationToFreeUp; + } + + public void updateQueuingLimit(ContainerQueuingLimit limit) { + this.queuingLimit.setMaxQueueLength(limit.getMaxQueueLength()); + // TODO: Include wait time as well once it is implemented + if (this.queuingLimit.getMaxQueueLength() > -1) { + shedQueuedOpportunisticContainers(); + } + } + + private void shedQueuedOpportunisticContainers() { + int numAllowed = this.queuingLimit.getMaxQueueLength(); + Iterator<Container> containerIter = + queuedOpportunisticContainers.values().iterator(); + while (containerIter.hasNext()) { + Container container = containerIter.next(); + if (numAllowed <= 0) { + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); — End diff – Shouldn't this diagnostic message be similar to the log below?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87645111

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java —
          @@ -0,0 +1,163 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.Resource;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +/**
          + * This class abstracts out how a container contributes to Resource Utilization.
          + * It is used by the

          {@link ContainerScheduler}

          to determine which
          + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED
          + * container.
          + * It currently equates resource utilization with the total resource allocated
          + * to the container. Another implementation might choose to use the actual
          + * resource utilization.
          + */
          +
          +public class ResourceUtilizationManager {
          — End diff –

          Should this be called ResourceUtilizationTracker?

          Also, would it make more sense for this to be an interface with an allocation-based implementation? Later, one could add a usage-based implementation and may be integrate tightly with ContainersMonitor?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87645111 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ResourceUtilizationManager.java — @@ -0,0 +1,163 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class abstracts out how a container contributes to Resource Utilization. + * It is used by the {@link ContainerScheduler} to determine which + * OPPORTUNISTIC containers to be killed to make room for a GUARANTEED + * container. + * It currently equates resource utilization with the total resource allocated + * to the container. Another implementation might choose to use the actual + * resource utilization. + */ + +public class ResourceUtilizationManager { — End diff – Should this be called ResourceUtilizationTracker? Also, would it make more sense for this to be an interface with an allocation-based implementation? Later, one could add a usage-based implementation and may be integrate tightly with ContainersMonitor?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87643346

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java —
          @@ -311,6 +318,10 @@ protected void createAMRMProxyService(Configuration conf) {
          }
          }

          + protected ContainerScheduler createContainerScheduler(Context cntxt) {
          — End diff –

          Is this method exposed for testing? If yes, should it be marked @VisibleForTesting?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87643346 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java — @@ -311,6 +318,10 @@ protected void createAMRMProxyService(Configuration conf) { } } + protected ContainerScheduler createContainerScheduler(Context cntxt) { — End diff – Is this method exposed for testing? If yes, should it be marked @VisibleForTesting?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87641844

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator();
          + while(reverseContainerIterator.hasNext() &&
          + !hasSufficientResources(resourcesToFreeUp)) {
          + Container runningCont = reverseContainerIterator.next();
          + if (runningCont.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC) {
          +
          + if (oppContainersMarkedForKill.containsKey(
          + runningCont.getContainerId()))

          { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + }

          + extraOpportContainersToKill.add(runningCont);
          + ResourceUtilizationManager.decreaseResourceUtilization(
          + getContainersMonitor(), resourcesToFreeUp,
          + runningCont.getResource());
          + }
          + }
          + if (!hasSufficientResources(resourcesToFreeUp)) {
          + LOG.warn("There are no sufficient resources to start guaranteed [{}]" +
          + "even after attempting to kill all running" +
          + "opportunistic containers.", containerToStartId);
          + }
          + return extraOpportContainersToKill;
          + }
          +
          + private boolean hasSufficientResources(
          + ResourceUtilization resourcesToFreeUp)

          { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + }

          +
          + private ResourceUtilization resourcesToFreeUp(
          + ContainerId containerToStartId) {
          + // Get allocation of currently allocated containers.
          + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization
          + .newInstance(this.utilizationManager.getCurrentUtilization());
          +
          + // Add to the allocation the allocation of the pending guaranteed
          + // containers that will start before the current container will be started.
          + for (Container container : queuedGuaranteedContainers.values()) {
          + ResourceUtilizationManager.increaseResourceUtilization(
          + getContainersMonitor(), resourceAllocationToFreeUp,
          + container.getResource());
          + if (container.getContainerId().equals(containerToStartId))

          { + break; + }

          + }
          +
          + // These Resources have already been freed, due to demand from an
          + // earlier Guaranteed container.
          + for (Container container : oppContainersMarkedForKill.values())

          { + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + }

          +
          + // Subtract the overall node resources.
          + getContainersMonitor().subtractNodeResourcesFromResourceUtilization(
          + resourceAllocationToFreeUp);
          + return resourceAllocationToFreeUp;
          + }
          +
          + public void updateQueuingLimit(ContainerQueuingLimit limit) {
          + this.queuingLimit.setMaxQueueLength(limit.getMaxQueueLength());
          + // TODO: Include wait time as well once it is implemented
          — End diff –

          Annotate the TODO with a JIRA?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87641844 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); + this.utilizationManager.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The additional opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator(); + while(reverseContainerIterator.hasNext() && + !hasSufficientResources(resourcesToFreeUp)) { + Container runningCont = reverseContainerIterator.next(); + if (runningCont.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + + if (oppContainersMarkedForKill.containsKey( + runningCont.getContainerId())) { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + } + extraOpportContainersToKill.add(runningCont); + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourcesToFreeUp, + runningCont.getResource()); + } + } + if (!hasSufficientResources(resourcesToFreeUp)) { + LOG.warn("There are no sufficient resources to start guaranteed [{}] " + + "even after attempting to kill all running" + + "opportunistic containers.", containerToStartId); + } + return extraOpportContainersToKill; + } + + private boolean hasSufficientResources( + ResourceUtilization resourcesToFreeUp) { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + } + + private ResourceUtilization resourcesToFreeUp( + ContainerId containerToStartId) { + // Get allocation of currently allocated containers. + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization + .newInstance(this.utilizationManager.getCurrentUtilization()); + + // Add to the allocation the allocation of the pending guaranteed + // containers that will start before the current container will be started. + for (Container container : queuedGuaranteedContainers.values()) { + ResourceUtilizationManager.increaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + if (container.getContainerId().equals(containerToStartId)) { + break; + } + } + + // These Resources have already been freed, due to demand from an + // earlier Guaranteed container. + for (Container container : oppContainersMarkedForKill.values()) { + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + } + + // Subtract the overall node resources. + getContainersMonitor().subtractNodeResourcesFromResourceUtilization( + resourceAllocationToFreeUp); + return resourceAllocationToFreeUp; + } + + public void updateQueuingLimit(ContainerQueuingLimit limit) { + this.queuingLimit.setMaxQueueLength(limit.getMaxQueueLength()); + // TODO: Include wait time as well once it is implemented — End diff – Annotate the TODO with a JIRA?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87641557

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          + // Track resources that need to be freed.
          + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp(
          + containerToStartId);
          +
          + // Go over the running opportunistic containers.
          + // Use a descending iterator to kill more recently started containers.
          + Iterator<Container> reverseContainerIterator =
          + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator();
          + while(reverseContainerIterator.hasNext() &&
          + !hasSufficientResources(resourcesToFreeUp)) {
          + Container runningCont = reverseContainerIterator.next();
          + if (runningCont.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC) {
          +
          + if (oppContainersMarkedForKill.containsKey(
          + runningCont.getContainerId()))

          { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + }

          + extraOpportContainersToKill.add(runningCont);
          + ResourceUtilizationManager.decreaseResourceUtilization(
          + getContainersMonitor(), resourcesToFreeUp,
          + runningCont.getResource());
          + }
          + }
          + if (!hasSufficientResources(resourcesToFreeUp)) {
          + LOG.warn("There are no sufficient resources to start guaranteed [{}]" +
          + "even after attempting to kill all running" +
          + "opportunistic containers.", containerToStartId);
          + }
          + return extraOpportContainersToKill;
          + }
          +
          + private boolean hasSufficientResources(
          + ResourceUtilization resourcesToFreeUp)

          { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + }

          +
          + private ResourceUtilization resourcesToFreeUp(
          + ContainerId containerToStartId) {
          + // Get allocation of currently allocated containers.
          + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization
          + .newInstance(this.utilizationManager.getCurrentUtilization());
          +
          + // Add to the allocation the allocation of the pending guaranteed
          + // containers that will start before the current container will be started.
          + for (Container container : queuedGuaranteedContainers.values()) {
          + ResourceUtilizationManager.increaseResourceUtilization(
          + getContainersMonitor(), resourceAllocationToFreeUp,
          + container.getResource());
          + if (container.getContainerId().equals(containerToStartId))

          { + break; + }

          + }
          +
          + // These Resources have already been freed, due to demand from an
          — End diff –

          Rephrase? "These resources are being freed, likely at the behest of another guaranteed container."?

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87641557 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed containers. + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers = + new LinkedHashMap<>(); + + private final ContainerQueuingLimit queuingLimit = + ContainerQueuingLimit.newInstance(); + + private final OpportunisticContainersStatus opportunisticContainersStatus; + + // Resource Utilization Manager that decides how utilization of the cluster + // increase / decreases based on container start / finish + private ResourceUtilizationManager utilizationManager; + + /** + * Instantiate a Container Scheduler. + * @param context NodeManager Context. + */ + public ContainerScheduler(Context context) { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + } + + @VisibleForTesting + public ContainerScheduler(Context context, int qLength) { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + } + + /** + * Handle ContainerSchedulerEvents. + * @param event ContainerSchedulerEvent. + */ + @Override + public void handle(ContainerSchedulerEvent event) { + switch (event.getType()) { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + } + } + + /** + * Return number of queued containers. + * @return Number of queued containers. + */ + public int getNumQueuedContainers() { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedGuaranteedContainers() { + return this.queuedGuaranteedContainers.size(); + } + + @VisibleForTesting + public int getNumQueuedOpportunisticContainers() { + return this.queuedOpportunisticContainers.size(); + } + + public OpportunisticContainersStatus getOpportunisticContainersStatus() { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + } + + private void onContainerCompleted(Container container) { + // decrement only if it was a running container + if (scheduledToRunContainers.containsKey(container.getContainerId())) { + this.utilizationManager.subtractContainerResource(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + } + } + scheduledToRunContainers.remove(container.getContainerId()); + oppContainersMarkedForKill.remove(container.getContainerId()); + startPendingContainers(); + } + + private void startPendingContainers() { + // Start pending guaranteed containers, if resources available. + boolean resourcesAvailable = + startContainersFromQueue(queuedGuaranteedContainers.values()); + // Start opportunistic containers, if resources available. + if (resourcesAvailable) { + startContainersFromQueue(queuedOpportunisticContainers.values()); + } + } + + private boolean startContainersFromQueue( + Collection<Container> queuedContainers) { + Iterator<Container> cIter = queuedContainers.iterator(); + boolean resourcesAvailable = true; + while (cIter.hasNext() && resourcesAvailable) { + Container container = cIter.next(); + if (this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + cIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + @VisibleForTesting + protected void scheduleContainer(Container container) { + if (maxOppQueueLength <= 0) { + startAllocatedContainer(container); + return; + } + if (queuedGuaranteedContainers.isEmpty() && + queuedOpportunisticContainers.isEmpty() && + this.utilizationManager.hasResourcesAvailable(container)) { + startAllocatedContainer(container); + } else { + LOG.info("No available resources for container {} to start its execution " + + "immediately.", container.getContainerId()); + boolean isQueued = true; + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.GUARANTEED) { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + } else { + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) { + LOG.info("Opportunistic container {} will be queued at the NM.", + container.getContainerId()); + queuedOpportunisticContainers.put( + container.getContainerId(), container); + } else { + isQueued = false; + LOG.info("Opportunistic container [{}] will not be queued at the NM" + + "since max queue length [{}] has been reached", + container.getContainerId(), maxOppQueueLength); + container.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Opportunistic container queue is full."); + } + } + if (isQueued) { + try { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + } catch (IOException e) { + LOG.warn("Could not store container state into store..", e); + } + } + } + } + + private void killOpportunisticContainers(Container container) { + List<Container> extraOpportContainersToKill = + pickOpportunisticContainersToKill(container.getContainerId()); + // Kill the opportunistic containers that were chosen. + for (Container contToKill : extraOpportContainersToKill) { + contToKill.sendKillEvent( + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER, + "Container Killed to make room for Guaranteed Container."); + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + contToKill.getContainerId(), container.getContainerId()); + } + } + + private void startAllocatedContainer(Container container) { + LOG.info("Starting container [" + container.getContainerId()+ "] "); + scheduledToRunContainers.put(container.getContainerId(), container); + this.utilizationManager.addContainerResources(container); + if (container.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + } + container.sendLaunchEvent(); + } + + private List<Container> pickOpportunisticContainersToKill( + ContainerId containerToStartId) { + // The additional opportunistic containers that need to be killed for the + // given container to start. + List<Container> extraOpportContainersToKill = new ArrayList<>(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. + // Use a descending iterator to kill more recently started containers. + Iterator<Container> reverseContainerIterator = + new LinkedList<>(scheduledToRunContainers.values()).descendingIterator(); + while(reverseContainerIterator.hasNext() && + !hasSufficientResources(resourcesToFreeUp)) { + Container runningCont = reverseContainerIterator.next(); + if (runningCont.getContainerTokenIdentifier().getExecutionType() == + ExecutionType.OPPORTUNISTIC) { + + if (oppContainersMarkedForKill.containsKey( + runningCont.getContainerId())) { + // These containers have already been marked to be killed. + // So exclude them.. + continue; + } + extraOpportContainersToKill.add(runningCont); + ResourceUtilizationManager.decreaseResourceUtilization( + getContainersMonitor(), resourcesToFreeUp, + runningCont.getResource()); + } + } + if (!hasSufficientResources(resourcesToFreeUp)) { + LOG.warn("There are no sufficient resources to start guaranteed [{}] " + + "even after attempting to kill all running" + + "opportunistic containers.", containerToStartId); + } + return extraOpportContainersToKill; + } + + private boolean hasSufficientResources( + ResourceUtilization resourcesToFreeUp) { + return resourcesToFreeUp.getPhysicalMemory() <= 0 && + resourcesToFreeUp.getVirtualMemory() <= 0 && + resourcesToFreeUp.getCPU() <= 0.0f; + } + + private ResourceUtilization resourcesToFreeUp( + ContainerId containerToStartId) { + // Get allocation of currently allocated containers. + ResourceUtilization resourceAllocationToFreeUp = ResourceUtilization + .newInstance(this.utilizationManager.getCurrentUtilization()); + + // Add to the allocation the allocation of the pending guaranteed + // containers that will start before the current container will be started. + for (Container container : queuedGuaranteedContainers.values()) { + ResourceUtilizationManager.increaseResourceUtilization( + getContainersMonitor(), resourceAllocationToFreeUp, + container.getResource()); + if (container.getContainerId().equals(containerToStartId)) { + break; + } + } + + // These Resources have already been freed, due to demand from an — End diff – Rephrase? "These resources are being freed, likely at the behest of another guaranteed container."?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user kambatla commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/143#discussion_r87640015

          — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java —
          @@ -0,0 +1,393 @@
          +/**
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
          +
          +import com.google.common.annotations.VisibleForTesting;
          +import org.apache.hadoop.service.AbstractService;
          +import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
          +import org.apache.hadoop.yarn.api.records.ContainerId;
          +import org.apache.hadoop.yarn.api.records.ExecutionType;
          +import org.apache.hadoop.yarn.api.records.ResourceUtilization;
          +import org.apache.hadoop.yarn.conf.YarnConfiguration;
          +import org.apache.hadoop.yarn.event.EventHandler;
          +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
          +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
          +import org.apache.hadoop.yarn.server.nodemanager.Context;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
          +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
          +
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.HashMap;
          +import java.util.Iterator;
          +import java.util.LinkedHashMap;
          +import java.util.LinkedList;
          +import java.util.List;
          +import java.util.Map;
          +
          +/**
          + * The ContainerScheduler manages a collection of runnable containers. It
          + * ensures that a container is launched only if all it launch criteria are
          + * met. It also ensures that OPPORTUNISTIC containers are killed to make
          + * room for GUARANTEED containers.
          + */
          +public class ContainerScheduler extends AbstractService implements
          + EventHandler<ContainerSchedulerEvent> {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(ContainerScheduler.class);
          +
          + private final Context context;
          + private final int maxOppQueueLength;
          +
          + // Queue of Guaranteed Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedGuaranteedContainers = new LinkedHashMap<>();
          + // Queue of Opportunistic Containers waiting for resources to run
          + private final LinkedHashMap<ContainerId, Container>
          + queuedOpportunisticContainers = new LinkedHashMap<>();
          +
          + // Used to keep track of containers that have been marked to be killed
          + // to make room for a guaranteed container.
          + private final Map<ContainerId, Container> oppContainersMarkedForKill =
          + new HashMap<>();
          +
          + // Containers launched by the Scheduler will take a while to actually
          + // move to the RUNNING state, but should still be fair game for killing
          + // by the scheduler to make room for guaranteed containers.
          + private final LinkedHashMap<ContainerId, Container> scheduledToRunContainers =
          + new LinkedHashMap<>();
          +
          + private final ContainerQueuingLimit queuingLimit =
          + ContainerQueuingLimit.newInstance();
          +
          + private final OpportunisticContainersStatus opportunisticContainersStatus;
          +
          + // Resource Utilization Manager that decides how utilization of the cluster
          + // increase / decreases based on container start / finish
          + private ResourceUtilizationManager utilizationManager;
          +
          + /**
          + * Instantiate a Container Scheduler.
          + * @param context NodeManager Context.
          + */
          + public ContainerScheduler(Context context)

          { + this(context, context.getConf().getInt( + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, + YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH_DEFAULT)); + }

          +
          + @VisibleForTesting
          + public ContainerScheduler(Context context, int qLength)

          { + super(ContainerScheduler.class.getName()); + this.context = context; + this.maxOppQueueLength = (qLength <= 0) ? 0 : qLength; + this.utilizationManager = new ResourceUtilizationManager(this); + this.opportunisticContainersStatus = + OpportunisticContainersStatus.newInstance(); + }

          +
          + /**
          + * Handle ContainerSchedulerEvents.
          + * @param event ContainerSchedulerEvent.
          + */
          + @Override
          + public void handle(ContainerSchedulerEvent event) {
          + switch (event.getType())

          { + case SCHEDULE_CONTAINER: + scheduleContainer(event.getContainer()); + break; + case CONTAINER_COMPLETED: + onContainerCompleted(event.getContainer()); + break; + default: + LOG.error("Unknown event arrived at ContainerScheduler: " + + event.toString()); + }

          + }
          +
          + /**
          + * Return number of queued containers.
          + * @return Number of queued containers.
          + */
          + public int getNumQueuedContainers()

          { + return this.queuedGuaranteedContainers.size() + + this.queuedOpportunisticContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedGuaranteedContainers()

          { + return this.queuedGuaranteedContainers.size(); + }

          +
          + @VisibleForTesting
          + public int getNumQueuedOpportunisticContainers()

          { + return this.queuedOpportunisticContainers.size(); + }

          +
          + public OpportunisticContainersStatus getOpportunisticContainersStatus()

          { + this.opportunisticContainersStatus.setQueuedOpportContainers( + getNumQueuedOpportunisticContainers()); + this.opportunisticContainersStatus.setWaitQueueLength( + getNumQueuedContainers()); + return this.opportunisticContainersStatus; + }

          +
          + private void onContainerCompleted(Container container) {
          + // decrement only if it was a running container
          + if (scheduledToRunContainers.containsKey(container.getContainerId())) {
          + this.utilizationManager.subtractContainerResource(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + - container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + - container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + - 1); + }

          + }
          + scheduledToRunContainers.remove(container.getContainerId());
          + oppContainersMarkedForKill.remove(container.getContainerId());
          + startPendingContainers();
          + }
          +
          + private void startPendingContainers() {
          + // Start pending guaranteed containers, if resources available.
          + boolean resourcesAvailable =
          + startContainersFromQueue(queuedGuaranteedContainers.values());
          + // Start opportunistic containers, if resources available.
          + if (resourcesAvailable)

          { + startContainersFromQueue(queuedOpportunisticContainers.values()); + }

          + }
          +
          + private boolean startContainersFromQueue(
          + Collection<Container> queuedContainers) {
          + Iterator<Container> cIter = queuedContainers.iterator();
          + boolean resourcesAvailable = true;
          + while (cIter.hasNext() && resourcesAvailable) {
          + Container container = cIter.next();
          + if (this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + cIter.remove(); + }

          else

          { + resourcesAvailable = false; + }

          + }
          + return resourcesAvailable;
          + }
          +
          + @VisibleForTesting
          + protected void scheduleContainer(Container container) {
          + if (maxOppQueueLength <= 0)

          { + startAllocatedContainer(container); + return; + }

          + if (queuedGuaranteedContainers.isEmpty() &&
          + queuedOpportunisticContainers.isEmpty() &&
          + this.utilizationManager.hasResourcesAvailable(container))

          { + startAllocatedContainer(container); + }

          else {
          + LOG.info("No available resources for container {} to start its execution "
          + + "immediately.", container.getContainerId());
          + boolean isQueued = true;
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.GUARANTEED)

          { + queuedGuaranteedContainers.put(container.getContainerId(), container); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportunisticContainers(container); + }

          else {
          + if (queuedOpportunisticContainers.size() <= maxOppQueueLength) {
          + LOG.info("Opportunistic container {} will be queued at the NM.",
          + container.getContainerId());
          + queuedOpportunisticContainers.put(
          + container.getContainerId(), container);
          + } else {
          + isQueued = false;
          + LOG.info("Opportunistic container [{}] will not be queued at the NM" +
          + "since max queue length [{}] has been reached",
          + container.getContainerId(), maxOppQueueLength);
          + container.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Opportunistic container queue is full.");
          + }
          + }
          + if (isQueued) {
          + try

          { + this.context.getNMStateStore().storeContainerQueued( + container.getContainerId()); + }

          catch (IOException e)

          { + LOG.warn("Could not store container state into store..", e); + }

          + }
          + }
          + }
          +
          + private void killOpportunisticContainers(Container container) {
          + List<Container> extraOpportContainersToKill =
          + pickOpportunisticContainersToKill(container.getContainerId());
          + // Kill the opportunistic containers that were chosen.
          + for (Container contToKill : extraOpportContainersToKill) {
          + contToKill.sendKillEvent(
          + ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
          + "Container Killed to make room for Guaranteed Container.");
          + oppContainersMarkedForKill.put(contToKill.getContainerId(), contToKill);
          + LOG.info(
          + "Opportunistic container {} will be killed in order to start the "
          + + "execution of guaranteed container {}.",
          + contToKill.getContainerId(), container.getContainerId());
          + }
          + }
          +
          + private void startAllocatedContainer(Container container) {
          + LOG.info("Starting container [" + container.getContainerId()+ "]");
          + scheduledToRunContainers.put(container.getContainerId(), container);
          + this.utilizationManager.addContainerResources(container);
          + if (container.getContainerTokenIdentifier().getExecutionType() ==
          + ExecutionType.OPPORTUNISTIC)

          { + this.opportunisticContainersStatus.setOpportMemoryUsed( + this.opportunisticContainersStatus.getOpportMemoryUsed() + + container.getResource().getMemorySize()); + this.opportunisticContainersStatus.setOpportCoresUsed( + this.opportunisticContainersStatus.getOpportCoresUsed() + + container.getResource().getVirtualCores()); + this.opportunisticContainersStatus.setRunningOpportContainers( + this.opportunisticContainersStatus.getRunningOpportContainers() + + 1); + }

          + container.sendLaunchEvent();
          + }
          +
          + private List<Container> pickOpportunisticContainersToKill(
          + ContainerId containerToStartId) {
          + // The additional opportunistic containers that need to be killed for the
          + // given container to start.
          + List<Container> extraOpportContainersToKill = new ArrayList<>();
          — End diff –

          Same comment about extra...

          Show
          githubbot ASF GitHub Bot added a comment - Github user kambatla commented on a diff in the pull request: https://github.com/apache/hadoop/pull/143#discussion_r87640015 — Diff: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java — @@ -0,0 +1,393 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; +import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * The ContainerScheduler manages a collection of runnable containers. It + * ensures that a container is launched only if all it launch criteria are + * met. It also ensures that OPPORTUNISTIC containers are killed to make + * room for GUARANTEED containers. + */ +public class ContainerScheduler extends AbstractService implements + EventHandler<ContainerSchedulerEvent> { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerScheduler.class); + + private final Context context; + private final int maxOppQueueLength; + + // Queue of Guaranteed Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedGuaranteedContainers = new LinkedHashMap<>(); + // Queue of Opportunistic Containers waiting for resources to run + private final LinkedHashMap<ContainerId, Container> + queuedOpportunisticContainers = new LinkedHashMap<>(); + + // Used to keep track of containers that have been marked to be killed + // to make room for a guaranteed container. + private final Map<ContainerId, Container> oppContainersMarkedForKill = + new HashMap<>(); + + // Containers launched by the Scheduler will take a while to actually + // move to the RUNNING state, but should still be fair game for killing + // by the scheduler to make room for guaranteed