You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by tillrohrmann <gi...@git.apache.org> on 2015/02/04 10:45:54 UTC

[GitHub] flink pull request: [FLINK-1442] Reduce memory consumption of arch...

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

    https://github.com/apache/flink/pull/344#discussion_r24072128
  
    --- Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala ---
    @@ -25,48 +25,82 @@ import org.apache.flink.runtime.jobgraph.JobID
     import org.apache.flink.runtime.messages.ArchiveMessages._
     import org.apache.flink.runtime.messages.JobManagerMessages._
     
    +import scala.collection.mutable.LinkedHashMap
    +import scala.ref.SoftReference
    +
     class MemoryArchivist(private val max_entries: Int) extends Actor with ActorLogMessages with
     ActorLogging {
       /**
        * Map of execution graphs belonging to recently started jobs with the time stamp of the last
    -   * received job event.
    +   * received job event. The insert order is preserved through a LinkedHashMap.
        */
    -  val graphs = collection.mutable.HashMap[JobID, ExecutionGraph]()
    -  val lru = collection.mutable.Queue[JobID]()
    +  val graphs = LinkedHashMap[JobID, SoftReference[ExecutionGraph]]()
     
       override def receiveWithLogMessages: Receive = {
    +    /* Receive Execution Graph to archive */
         case ArchiveExecutionGraph(jobID, graph) => {
    -      graphs.update(jobID, graph)
    +      // wrap graph inside a soft reference
    +      graphs.update(jobID, new SoftReference(graph))
    +
    +      // clear all execution edges of the graph
    +      val iter = graph.getAllExecutionVertices().iterator()
    +      while (iter.hasNext) {
    +        iter.next().clearExecutionEdges()
    +      }
    +
           cleanup(jobID)
         }
     
         case RequestArchivedJobs => {
    -      sender ! ArchivedJobs(graphs.values)
    +      sender ! ArchivedJobs(getAllGraphs())
         }
     
         case RequestJob(jobID) => {
    -      graphs.get(jobID) match {
    -        case Some(graph) => sender ! JobFound(jobID, graph)
    -        case None => sender ! JobNotFound(jobID)
    +      getGraph(jobID) match {
    +        case graph: ExecutionGraph => sender ! JobFound(jobID, graph)
    +        case _ => sender ! JobNotFound(jobID)
           }
         }
     
         case RequestJobStatus(jobID) => {
    -      graphs.get(jobID) match {
    -        case Some(eg) => sender ! CurrentJobStatus(jobID, eg.getState)
    -        case None => sender ! JobNotFound(jobID)
    +      getGraph(jobID) match {
    +        case graph: ExecutionGraph => sender ! CurrentJobStatus(jobID, graph.getState)
    +        case _ => sender ! JobNotFound(jobID)
           }
         }
       }
     
    -  def cleanup(jobID: JobID): Unit = {
    -    if (!lru.contains(jobID)) {
    -      lru.enqueue(jobID)
    +  /**
    +   * Gets all graphs that have not been garbage collected.
    +   * @return An iterable with all valid ExecutionGraphs
    +   */
    +  def getAllGraphs() = graphs.values.flatMap(ref => ref.get match {
    +    case Some(graph) => Seq(graph)
    +    case _ => Seq()
    +  })
    +
    +  /**
    +   * Gets a graph with a jobID if it has not been garbage collected.
    +   * @param jobID
    +   * @return ExecutionGraph or null
    +   */
    +  def getGraph(jobID: JobID) = graphs.get(jobID) match {
    --- End diff --
    
    Why are we doing that? Why not working on the Option type? I don't like null.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---