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

    https://github.com/apache/spark/pull/1083#discussion_r13779767
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheManager.scala ---
    @@ -128,4 +76,89 @@ private[spark] class CacheManager(blockManager: 
BlockManager) extends Logging {
             }
         }
       }
    +
    +  /**
    +   * Acquire a loading lock for the partition identified by the given 
block ID.
    +   *
    +   * If the lock is free, just acquire it and return None. Otherwise, 
another thread is already
    +   * loading the partition, so we wait for it to finish and return the 
values loaded by the thread.
    +   */
    +  private def acquireLockForPartition(id: RDDBlockId): 
Option[Iterator[Any]] = {
    +    loading.synchronized {
    +      if (!loading.contains(id)) {
    +        // If the partition is free, acquire its lock and begin computing 
its value
    +        loading.add(id)
    +        None
    +      } else {
    +        // Otherwise, wait for another thread to finish and return its 
result
    +        logInfo(s"Another thread is loading $id, waiting for it to 
finish...")
    +        while (loading.contains(id)) {
    +          try {
    +            loading.wait()
    +          } catch {
    +            case e: Exception =>
    +              logWarning(s"Exception while waiting for another thread to 
load $id", e)
    +          }
    +        }
    +        logInfo(s"Finished waiting for $id")
    +        /* See whether someone else has successfully loaded it. The main 
way this would fail
    +         * is for the RDD-level cache eviction policy if someone else has 
loaded the same RDD
    +         * partition but we didn't want to make space for it. However, 
that case is unlikely
    +         * because it's unlikely that two threads would work on the same 
RDD partition. One
    +         * downside of the current code is that threads wait serially if 
this does happen. */
    +        val values = blockManager.get(id)
    +        if (!values.isDefined) {
    +          logInfo(s"Whoever was loading $id failed; we'll try it 
ourselves")
    +          loading.add(id)
    +        }
    +        values
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Cache the values of a partition, keeping track of any updates in the 
storage statuses
    +   * of other blocks along the way.
    +   */
    +  private def cacheValues[T](
    +      key: BlockId,
    +      value: Iterator[T],
    +      storageLevel: StorageLevel,
    +      updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)]): Iterator[T] = {
    +
    +    if (!storageLevel.useMemory) {
    +      /* This RDD is not to be cached in memory, so we can just pass the 
computed values
    +       * as an iterator directly to the BlockManager, rather than first 
fully unrolling
    +       * it in memory. The latter option potentially uses much more memory 
and risks OOM
    +       * exceptions that can be avoided. */
    +      assume(storageLevel.useDisk || storageLevel.useOffHeap, s"Empty 
storage level for $key!")
    --- End diff --
    
    Might make sense to remove this assume; in case we add a new storage level 
in the future, this won't hold any more and because this code is so far away 
from the storage level code, we will likely forget to update this location. 


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to