You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by JoshRosen <gi...@git.apache.org> on 2016/01/11 21:39:10 UTC

[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

GitHub user JoshRosen opened a pull request:

    https://github.com/apache/spark/pull/10705

    [SPARK-12757][WIP] Use reference counting to prevent blocks from being evicted during reads

    As a pre-requisite to off-heap caching of blocks, we need a mechanism to prevent pages / blocks from being evicted while they are being read. With on-heap objects, evicting a block while it is being read merely leads to memory-accounting problems (because we assume that an evicted block is a candidate for garbage-collection, which will not be true during a read), but with off-heap memory this will lead to either data corruption or segmentation faults.
    
    To address this, we should add a reference-counting mechanism to track which blocks/pages are being read in order to prevent them from being evicted prematurely. I propose to do this in two phases: first, add a safe, conservative approach in which all BlockManager.get*() calls implicitly increment the reference count of blocks and where tasks' references are automatically freed upon task completion. This will be correct but may have adverse performance impacts because it will prevent legitimate block evictions. In phase two, we should incrementally add release() calls in order to fix the eviction of unreferenced blocks. The latter change may need to touch many different components, which is why I propose to do it separately in order to make the changes easier to reason about and review.
    
    This PR is currently WIP, pending test fixes and a few additional improvements:
    
    - I need to add significantly more debug logging statements. From my experience in working on other memory-management-related things in Spark, I've realized that it's extremely useful to have a set of verbose logging statements that we can enable with a Log4J conf.
    - Guard the "non-zero reference count prevents eviction" check behind a debugging feature-flag to let us disable this feature for testing. This will be a useful debugging aid in phase 2.
    - Get the existing tests to pass.
    - Write API documentation for the `release()` methods.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/JoshRosen/spark pin-pages

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/10705.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #10705
    
----
commit 5d130e44dbb8259588ac1b9006dc41c597c8a4a0
Author: Josh Rosen <jo...@databricks.com>
Date:   2016-01-08T21:11:51Z

    Add block reference counting class.

commit 423faabe3a34c6021a859c93cb97ac7c946529e2
Author: Josh Rosen <jo...@databricks.com>
Date:   2016-01-08T21:46:13Z

    Make the ReferenceCounter generic, since it's not specific to storage in any respect.

commit 1ee665f845addb493c0c822764018d3188aa30d1
Author: Josh Rosen <jo...@databricks.com>
Date:   2016-01-08T21:52:50Z

    Merge remote-tracking branch 'origin/master' into pin-pages

commit 76cfebd15137fb0090f89dbd1791aad9eca09902
Author: Josh Rosen <jo...@databricks.com>
Date:   2016-01-08T23:13:33Z

    Integrate reference counter into storage eviction code.

commit 7265784f821c5ca451322e0a2b1bfdcf8c952af4
Author: Josh Rosen <jo...@databricks.com>
Date:   2016-01-11T20:24:53Z

    Merge remote-tracking branch 'origin/master' into pin-pages

----


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183767434
  
    **[Test build #51248 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51248/consoleFull)** for PR 10705 at commit [`27e98a3`](https://github.com/apache/spark/commit/27e98a36bc343d3bbd1715a5f613f7e7a35b85c5).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188443477
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183058644
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183482989
  
    **[Test build #51200 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51200/consoleFull)** for PR 10705 at commit [`9c8d530`](https://github.com/apache/spark/commit/9c8d5308ae4482c86a381e9a863fa91d3ed63899).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-175192026
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/50114/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53684788
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    --- End diff --
    
    unlock also calls notifyAll. Is this necessary/important?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r50872433
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -403,7 +406,20 @@ private[spark] class BlockManager(
        */
       def getLocal(blockId: BlockId): Option[BlockResult] = {
         logDebug(s"Getting local block $blockId")
    -    doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]]
    +    val res = doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]]
    --- End diff --
    
    A shortened version of this should be a class comment. Describe what pinning means and what are the API semantics.
    
    "we should add a pin-counting mechanism to track which blocks/pages are being read in order to prevent them from being evicted prematurely. I propose to do this in two phases: first, add a safe, conservative approach in which all BlockManager.get*() calls implicitly increment the pin count of blocks and where tasks' pins are automatically freed upon task completion (this PR)"


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188117398
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-172106324
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49477/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188056736
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51821/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53728185
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -805,13 +814,12 @@ private[spark] class BlockManager(
               }
             }
           } finally {
    -        // If we failed in putting the block to memory/disk, notify other possible readers
    -        // that it has failed, and then remove it from the block info map.
    -        if (!marked) {
    -          // Note that the remove must happen before markFailure otherwise another thread
    -          // could've inserted a new BlockInfo before we remove it.
    -          blockInfo.remove(blockId)
    -          putBlockInfo.markFailure()
    +        if (!blockWasSuccessfullyStored) {
    +          // Guard against the fact that MemoryStore might have already removed the block if the
    +          // put() failed and the block could not be dropped to disk.
    +          if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) {
    --- End diff --
    
    See 504986f548e09e3097dbb84d4e50c760c538aaee


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53876274
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  def size: Long = _size
    +  def size_=(s: Long): Unit = {
    +    _size = s
    +    checkInvariants()
    +  }
    +  private[this] var _size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  def readerCount: Int = _readerCount
    +  def readerCount_=(c: Int): Unit = {
    +    _readerCount = c
    +    checkInvariants()
    +  }
    +  private[this] var _readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or
    +   * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or
    +   * [[BlockInfo.NO_WRITER]] if this block is not locked for writing.
    +   */
    +  def writerTask: Long = _writerTask
    +  def writerTask_=(t: Long): Unit = {
    +    _writerTask = t
    +    checkInvariants()
    +  }
    +  private[this] var _writerTask: Long = 0
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  def removed: Boolean = _removed
    +  def removed_=(r: Boolean): Unit = {
    +    _removed = r
    +    checkInvariants()
    +  }
    +  private[this] var _removed: Boolean = false
    +
    +  private def checkInvariants(): Unit = {
    +    // A block's reader count must be non-negative:
    +    assert(_readerCount >= 0)
    +    // A block is either locked for reading or for writing, but not for both at the same time:
    +    assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER))
    +    // If a block is removed then it is not locked:
    +    assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER))
    +  }
    +
    +  checkInvariants()
    +}
    +
    +private[storage] object BlockInfo {
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being unlocked.
    +   */
    +  val NO_WRITER: Long = -1
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being held by
    +   * a non-task thread (e.g. by a driver thread or by unit test code).
    +   */
    +  val NON_TASK_WRITER: Long = -1024
    +}
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant).
    +   */
    +  @GuardedBy("this")
    +  private[this] val readLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]]
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  // Initialization for special task attempt ids:
    +  registerTask(BlockInfo.NON_TASK_WRITER)
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Called at the start of a task in order to register that task with this [[BlockInfoManager]].
    +   * This must be called prior to calling any other BlockInfoManager methods from that task.
    +   */
    +  def registerTask(taskAttemptId: TaskAttemptId): Unit = {
    +    require(!readLocksByTask.contains(taskAttemptId),
    +      s"Task attempt $taskAttemptId is already registered")
    +    readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create()
    --- End diff --
    
    you don't have this in sychronized but do in 339. Any reason?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744479
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    +  }
    +
    +  /**
    +   * Removes the given block and automatically drops all locks on it.
    +   *
    +   * This can only be called while holding a write lock on the given block.
    +   */
    +  def removeBlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId")
    +    infos.get(blockId) match {
    +      case Some(blockInfo) =>
    +        if (blockInfo.writerTask != currentTaskAttemptId) {
    +          throw new IllegalStateException(
    +            s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock")
    +        } else {
    +          infos.remove(blockId)
    +          blockInfo.removed = true
    +        }
    +      case None =>
    +        throw new IllegalArgumentException(
    +          s"Task $currentTaskAttemptId called remove() on non-existent block $blockId")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Delete all state. Called during shutdown.
    +   */
    +  def clear(): Unit = synchronized {
    +    infos.clear()
    +    readLocksByTask.invalidateAll()
    +    writeLocksByTask.clear()
    --- End diff --
    
    Added.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53683536
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    --- End diff --
    
    is this a TODO comment? Did you mean to put it under the TODO comment up there in L71?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184988591
  
    Hmm, it looks like `KryoSerializerDistributedSuite` is flaky. Here's the exception:
    
    ```
    sbt.ForkMain$ForkError: org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 1.0 failed 1 times, most recent failure: Lost task 1.0 in stage 1.0 (TID 4, localhost): java.lang.IllegalStateException: Block rdd_0_1 not found
    	at org.apache.spark.storage.BlockInfoManager$$anonfun$1.apply(BlockInfoManager.scala:217)
    	at org.apache.spark.storage.BlockInfoManager$$anonfun$1.apply(BlockInfoManager.scala:217)
    	at scala.Option.getOrElse(Option.scala:121)
    	at org.apache.spark.storage.BlockInfoManager.releaseLock(BlockInfoManager.scala:216)
    	at org.apache.spark.storage.BlockManager.releaseLock(BlockManager.scala:613)
    	at org.apache.spark.CacheManager$$anonfun$1.apply$mcV$sp(CacheManager.scala:52)
    	at org.apache.spark.util.CompletionIterator$$anon$1.completion(CompletionIterator.scala:46)
    	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:35)
    	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
    	at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:147)
    	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:77)
    	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:45)
    	at org.apache.spark.scheduler.Task.run(Task.scala:81)
    	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
    	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
    	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    	at java.lang.Thread.run(Thread.java:745)
    ```
    
    This looks pretty fishy to me, since we shouldn't ever be trying to remove a block if it's already locked. I still have an outstanding TODO to do better precondition checking for this, so maybe that'll help debug it.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53720679
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheManager.scala ---
    @@ -136,7 +147,13 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
            */
           blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel)
           blockManager.get(key) match {
    -        case Some(v) => v.data.asInstanceOf[Iterator[T]]
    +        case Some(v) =>
    --- End diff --
    
    I believe the lock will be released at the end of the task.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53682337
  
    --- Diff: core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala ---
    @@ -90,22 +90,22 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
     
       /**
        * Divide the object into multiple blocks and put those blocks in the block manager.
    +   *
        * @param value the object to divide
        * @return number of blocks this broadcast variable is divided into
        */
       private def writeBlocks(value: T): Int = {
         // Store a copy of the broadcast variable in the driver so that tasks run on the driver
         // do not create a duplicate copy of the broadcast variable's value.
    -    SparkEnv.get.blockManager.putSingle(broadcastId, value, StorageLevel.MEMORY_AND_DISK,
    -      tellMaster = false)
    +    val blockManager = SparkEnv.get.blockManager
    +    blockManager.putSingle(broadcastId, value, StorageLevel.MEMORY_AND_DISK, tellMaster = false)
    +    blockManager.releaseLock(broadcastId)
    --- End diff --
    
    actually there are many other places in other files too


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744546
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -318,7 +316,11 @@ private[spark] class BlockManager(
        * may not know of).
        */
       def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = {
    -    (blockInfo.asScala.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq
    +    val matches =
    +      (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()).filter(filter)
    +    // The `toArray` is necessary here in order to force the list to be materialized so that we
    +    // don't try to serialize a lazy iterator when responding to client requests.
    +    matches.toArray.toSeq
    --- End diff --
    
    Fixed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53683331
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheManager.scala ---
    @@ -46,7 +48,14 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
             val existingMetrics = context.taskMetrics().registerInputMetrics(blockResult.readMethod)
             existingMetrics.incBytesReadInternal(blockResult.bytes)
     
    -        val iter = blockResult.data.asInstanceOf[Iterator[T]]
    +        val iter = {
    +          val dataIter = blockResult.data.asInstanceOf[Iterator[T]]
    +          if (blockResult.readMethod != DataReadMethod.Network) {
    +            CompletionIterator[T, Iterator[T]](dataIter, blockManager.releaseLock(key))
    --- End diff --
    
    Can we bake this logic in blockResult.data and have it return a CompletionIterator? If not, let's move this as a utility in blockResult.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188114685
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51855/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53700954
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    +  }
    +
    +  /**
    +   * Removes the given block and automatically drops all locks on it.
    +   *
    +   * This can only be called while holding a write lock on the given block.
    +   */
    +  def removeBlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId")
    +    infos.get(blockId) match {
    +      case Some(blockInfo) =>
    +        if (blockInfo.writerTask != currentTaskAttemptId) {
    +          throw new IllegalStateException(
    +            s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock")
    +        } else {
    +          infos.remove(blockId)
    +          blockInfo.removed = true
    --- End diff --
    
    where's the part that `automatically drops all locks on it`? Do we need to set `blockInfo.writerTask = -1` here?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184930364
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51384/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185546067
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51456/
    Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-175357078
  
    **[Test build #50147 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50147/consoleFull)** for PR 10705 at commit [`150c6e1`](https://github.com/apache/spark/commit/150c6e1d390d6d0346817b31ddd0911c82e72baf).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53696226
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    --- End diff --
    
    Atomically create metadata for a block, if it doesn't exist.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185437301
  
    **[Test build #51445 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51445/consoleFull)** for PR 10705 at commit [`25b09d7`](https://github.com/apache/spark/commit/25b09d762273e5b743055e2944734671b3a9197c).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183084115
  
    **[Test build #51132 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51132/consoleFull)** for PR 10705 at commit [`a0c5bb3`](https://github.com/apache/spark/commit/a0c5bb336c0dc06ec9ffdf2ff12cb4f7aae3bc1d).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173431236
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744314
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,335 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +}
    +
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[putAndLockForWritingIfAbsent()]]) and are removed
    +   * by [[remove()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def getAndLockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    --- End diff --
    
    Fixed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187969914
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51815/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187652269
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51748/
    Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183809601
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183766952
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53876330
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  def size: Long = _size
    +  def size_=(s: Long): Unit = {
    +    _size = s
    +    checkInvariants()
    +  }
    +  private[this] var _size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  def readerCount: Int = _readerCount
    +  def readerCount_=(c: Int): Unit = {
    +    _readerCount = c
    +    checkInvariants()
    +  }
    +  private[this] var _readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or
    +   * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or
    +   * [[BlockInfo.NO_WRITER]] if this block is not locked for writing.
    +   */
    +  def writerTask: Long = _writerTask
    +  def writerTask_=(t: Long): Unit = {
    +    _writerTask = t
    +    checkInvariants()
    +  }
    +  private[this] var _writerTask: Long = 0
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  def removed: Boolean = _removed
    +  def removed_=(r: Boolean): Unit = {
    +    _removed = r
    +    checkInvariants()
    +  }
    +  private[this] var _removed: Boolean = false
    +
    +  private def checkInvariants(): Unit = {
    +    // A block's reader count must be non-negative:
    +    assert(_readerCount >= 0)
    +    // A block is either locked for reading or for writing, but not for both at the same time:
    +    assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER))
    +    // If a block is removed then it is not locked:
    +    assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER))
    +  }
    +
    +  checkInvariants()
    +}
    +
    +private[storage] object BlockInfo {
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being unlocked.
    +   */
    +  val NO_WRITER: Long = -1
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being held by
    +   * a non-task thread (e.g. by a driver thread or by unit test code).
    +   */
    +  val NON_TASK_WRITER: Long = -1024
    +}
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant).
    +   */
    +  @GuardedBy("this")
    +  private[this] val readLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]]
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  // Initialization for special task attempt ids:
    +  registerTask(BlockInfo.NON_TASK_WRITER)
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Called at the start of a task in order to register that task with this [[BlockInfoManager]].
    +   * This must be called prior to calling any other BlockInfoManager methods from that task.
    +   */
    +  def registerTask(taskAttemptId: TaskAttemptId): Unit = {
    +    require(!readLocksByTask.contains(taskAttemptId),
    +      s"Task attempt $taskAttemptId is already registered")
    +    readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create()
    +  }
    +
    +  /**
    +   * Returns the current task's task attempt id (which uniquely identifies the task), or
    +   * [[BlockInfo.NON_TASK_WRITER]] if called by a non-task thread.
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(BlockInfo.NON_TASK_WRITER)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * If another task has already locked this block for reading, then the read lock will be
    +   * immediately granted to the calling task and its lock count will be incremented.
    +   *
    +   * If another task has locked this block for reading, then this call will block until the write
    --- End diff --
    
    "locked this block for writing"


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188939377
  
    **[Test build #51985 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51985/consoleFull)** for PR 10705 at commit [`9becde3`](https://github.com/apache/spark/commit/9becde3d94ea41bba9e275b9108c61b91074f035).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188951622
  
    Jenkins, retest this please.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-170711337
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49167/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171840298
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53729151
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheManager.scala ---
    @@ -46,7 +48,14 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
             val existingMetrics = context.taskMetrics().registerInputMetrics(blockResult.readMethod)
             existingMetrics.incBytesReadInternal(blockResult.bytes)
     
    -        val iter = blockResult.data.asInstanceOf[Iterator[T]]
    +        val iter = {
    +          val dataIter = blockResult.data.asInstanceOf[Iterator[T]]
    +          if (blockResult.readMethod != DataReadMethod.Network) {
    +            CompletionIterator[T, Iterator[T]](dataIter, blockManager.releaseLock(key))
    --- End diff --
    
    In 66202f2d7eda4abcf1b315a85f91b43d4424d93e, I updated BlockManager to pass CompletionIterators to BlockResult. This felt a bit cleaner than putting a BlockManager reference into the BlockResult constructor in order to be able to reference it in the completion iterator, although I suppose that I could call SparkEnv.get.blockManager... from the completion iterator's cleanup task instead.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187971574
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by tedyu <gi...@git.apache.org>.
Github user tedyu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r54321396
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,445 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * Instances of this class are _not_ thread-safe and are protected by locks in the
    + * [[BlockInfoManager]].
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  def size: Long = _size
    +  def size_=(s: Long): Unit = {
    +    _size = s
    +    checkInvariants()
    +  }
    +  private[this] var _size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  def readerCount: Int = _readerCount
    +  def readerCount_=(c: Int): Unit = {
    +    _readerCount = c
    +    checkInvariants()
    +  }
    +  private[this] var _readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or
    +   * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or
    +   * [[BlockInfo.NO_WRITER]] if this block is not locked for writing.
    +   */
    +  def writerTask: Long = _writerTask
    +  def writerTask_=(t: Long): Unit = {
    +    _writerTask = t
    +    checkInvariants()
    +  }
    +  private[this] var _writerTask: Long = 0
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  def removed: Boolean = _removed
    +  def removed_=(r: Boolean): Unit = {
    +    _removed = r
    +    checkInvariants()
    +  }
    +  private[this] var _removed: Boolean = false
    +
    +  private def checkInvariants(): Unit = {
    +    // A block's reader count must be non-negative:
    +    assert(_readerCount >= 0)
    +    // A block is either locked for reading or for writing, but not for both at the same time:
    +    assert(_readerCount == 0 || _writerTask == BlockInfo.NO_WRITER)
    +    // If a block is removed then it is not locked:
    +    assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER))
    +  }
    +
    +  checkInvariants()
    +}
    +
    +private[storage] object BlockInfo {
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being unlocked.
    +   */
    +  val NO_WRITER: Long = -1
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being held by
    +   * a non-task thread (e.g. by a driver thread or by unit test code).
    +   */
    +  val NON_TASK_WRITER: Long = -1024
    +}
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant).
    +   */
    +  @GuardedBy("this")
    +  private[this] val readLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]]
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  // Initialization for special task attempt ids:
    +  registerTask(BlockInfo.NON_TASK_WRITER)
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Called at the start of a task in order to register that task with this [[BlockInfoManager]].
    +   * This must be called prior to calling any other BlockInfoManager methods from that task.
    +   */
    +  def registerTask(taskAttemptId: TaskAttemptId): Unit = synchronized {
    +    require(!readLocksByTask.contains(taskAttemptId),
    +      s"Task attempt $taskAttemptId is already registered")
    +    readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create()
    +  }
    +
    +  /**
    +   * Returns the current task's task attempt id (which uniquely identifies the task), or
    +   * [[BlockInfo.NON_TASK_WRITER]] if called by a non-task thread.
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(BlockInfo.NON_TASK_WRITER)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * If another task has already locked this block for reading, then the read lock will be
    +   * immediately granted to the calling task and its lock count will be incremented.
    +   *
    +   * If another task has locked this block for writing, then this call will block until the write
    +   * lock is released or will return immediately if `blocking = false`.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != BlockInfo.NO_WRITER) {
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If another task has already locked this block for either reading or writing, then this call
    +   * will block until the other locks are released or will return immediately if `blocking = false`.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * method will throw an exception.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask == currentTaskAttemptId) {
    +        throw new IllegalStateException(
    +          s"Task $currentTaskAttemptId has already locked $blockId for writing")
    +      } else {
    +        while (info.writerTask != BlockInfo.NO_WRITER || info.readerCount != 0) {
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Throws an exception if the current task does not hold a write lock on the given block.
    +   * Otherwise, returns the block's BlockInfo.
    +   */
    +  def assertBlockIsLockedForWriting(blockId: BlockId): BlockInfo = synchronized {
    +    infos.get(blockId) match {
    +      case Some(info) =>
    +        if (info.writerTask != currentTaskAttemptId) {
    +          throw new SparkException(
    +            s"Task $currentTaskAttemptId has not locked block $blockId for writing")
    +        } else {
    +          info
    +        }
    +      case None =>
    +        throw new SparkException(s"Block $blockId does not exist")
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks. This method is only exposed for use by
    +   * [[BlockManager.getStatus()]] and should not be called by other code outside of this class.
    +   */
    +  private[storage] def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold on" +
    +        s" block $blockId")
    +    unlock(blockId)
    +    val lockOutcome = lockForReading(blockId, blocking = false)
    +    assert(lockOutcome.isDefined)
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != BlockInfo.NO_WRITER) {
    +      info.writerTask = BlockInfo.NO_WRITER
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a block and acquire a write lock for it, if it doesn't already
    +   * exist.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    if (!infos.contains(blockId)) {
    +      infos(blockId) = newBlockInfo
    +      newBlockInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId successfully locked new block $blockId")
    +      true
    +    } else {
    +      logTrace(s"Task $currentTaskAttemptId did not create and lock block $blockId " +
    +        s"because that block already exists")
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def releaseAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +
    +    val readLocks = synchronized {
    +      readLocksByTask.remove(taskAttemptId).get
    +    }
    +    val writeLocks = synchronized {
    +      writeLocksByTask.remove(taskAttemptId).getOrElse(Seq.empty)
    +    }
    +
    +    for (blockId <- writeLocks) {
    +      infos.get(blockId).foreach { info =>
    +        assert(info.writerTask == taskAttemptId)
    +        info.writerTask = BlockInfo.NO_WRITER
    +      }
    +      blocksWithReleasedLocks += blockId
    +    }
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    --- End diff --
    
    Should an exception be thrown here instead ?
    In production, assertion may not be enabled.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53869401
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -521,16 +509,17 @@ private[spark] class BlockManager(
                   }
    --- End diff --
    
    Yeah, I think this is okay since we're not mutating anything which is being concurrent read in this scenario.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r54178354
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -452,7 +440,10 @@ private[spark] class BlockManager(
             if (level.useMemory) {
               logDebug(s"Getting block $blockId from memory")
               val result = if (asBlockResult) {
    -            memoryStore.getValues(blockId).map(new BlockResult(_, DataReadMethod.Memory, info.size))
    +            memoryStore.getValues(blockId).map { iter =>
    +              val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId))
    +              new BlockResult(ci, DataReadMethod.Memory, info.size)
    --- End diff --
    
    I'll do this in a followup.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53727839
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -805,13 +814,12 @@ private[spark] class BlockManager(
               }
             }
           } finally {
    -        // If we failed in putting the block to memory/disk, notify other possible readers
    -        // that it has failed, and then remove it from the block info map.
    -        if (!marked) {
    -          // Note that the remove must happen before markFailure otherwise another thread
    -          // could've inserted a new BlockInfo before we remove it.
    -          blockInfo.remove(blockId)
    -          putBlockInfo.markFailure()
    +        if (!blockWasSuccessfullyStored) {
    +          // Guard against the fact that MemoryStore might have already removed the block if the
    +          // put() failed and the block could not be dropped to disk.
    +          if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) {
    --- End diff --
    
    Err, a new commit. Pushing soon.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53684304
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    --- End diff --
    
    Why are these semantics useful? 


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53687324
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    --- End diff --
    
    For releasing locks upon task completion.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r54177384
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -452,7 +440,10 @@ private[spark] class BlockManager(
             if (level.useMemory) {
               logDebug(s"Getting block $blockId from memory")
               val result = if (asBlockResult) {
    -            memoryStore.getValues(blockId).map(new BlockResult(_, DataReadMethod.Memory, info.size))
    +            memoryStore.getValues(blockId).map { iter =>
    +              val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId))
    +              new BlockResult(ci, DataReadMethod.Memory, info.size)
    --- End diff --
    
    If you push it further then BlockResult needs to hold a reference to the BlockManager.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184930361
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183103714
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51139/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53686068
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    +  }
    +
    +  /**
    +   * Removes the given block and automatically drops all locks on it.
    +   *
    +   * This can only be called while holding a write lock on the given block.
    +   */
    +  def removeBlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId")
    +    infos.get(blockId) match {
    +      case Some(blockInfo) =>
    +        if (blockInfo.writerTask != currentTaskAttemptId) {
    +          throw new IllegalStateException(
    +            s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock")
    +        } else {
    +          infos.remove(blockId)
    --- End diff --
    
    I think you'll catch more asserts if you set writeTask to -1


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188442898
  
    **[Test build #51883 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51883/consoleFull)** for PR 10705 at commit [`9becde3`](https://github.com/apache/spark/commit/9becde3d94ea41bba9e275b9108c61b91074f035).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188560858
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51910/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-178869295
  
    **[Test build #50601 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50601/consoleFull)** for PR 10705 at commit [`76fc9f5`](https://github.com/apache/spark/commit/76fc9f5580b384d6f9fc990078f7f080bf298ada).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r49797633
  
    --- Diff: core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.spark.util.collection
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.TaskContext
    +
    +/**
    + * Thread-safe collection for maintaining both global and per-task reference counts for objects.
    + */
    +private[spark] class ReferenceCounter[T] {
    --- End diff --
    
    Is there any reason you did it this way instead of a counter per object? Not sure how many blocks we have but this seems contention prone.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53877235
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  def size: Long = _size
    +  def size_=(s: Long): Unit = {
    +    _size = s
    +    checkInvariants()
    +  }
    +  private[this] var _size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  def readerCount: Int = _readerCount
    +  def readerCount_=(c: Int): Unit = {
    +    _readerCount = c
    +    checkInvariants()
    +  }
    +  private[this] var _readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or
    +   * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or
    +   * [[BlockInfo.NO_WRITER]] if this block is not locked for writing.
    +   */
    +  def writerTask: Long = _writerTask
    +  def writerTask_=(t: Long): Unit = {
    +    _writerTask = t
    +    checkInvariants()
    +  }
    +  private[this] var _writerTask: Long = 0
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  def removed: Boolean = _removed
    +  def removed_=(r: Boolean): Unit = {
    +    _removed = r
    +    checkInvariants()
    +  }
    +  private[this] var _removed: Boolean = false
    +
    +  private def checkInvariants(): Unit = {
    +    // A block's reader count must be non-negative:
    +    assert(_readerCount >= 0)
    +    // A block is either locked for reading or for writing, but not for both at the same time:
    +    assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER))
    +    // If a block is removed then it is not locked:
    +    assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER))
    +  }
    +
    +  checkInvariants()
    +}
    +
    +private[storage] object BlockInfo {
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being unlocked.
    +   */
    +  val NO_WRITER: Long = -1
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being held by
    +   * a non-task thread (e.g. by a driver thread or by unit test code).
    +   */
    +  val NON_TASK_WRITER: Long = -1024
    +}
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant).
    +   */
    +  @GuardedBy("this")
    +  private[this] val readLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]]
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  // Initialization for special task attempt ids:
    +  registerTask(BlockInfo.NON_TASK_WRITER)
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Called at the start of a task in order to register that task with this [[BlockInfoManager]].
    +   * This must be called prior to calling any other BlockInfoManager methods from that task.
    +   */
    +  def registerTask(taskAttemptId: TaskAttemptId): Unit = {
    +    require(!readLocksByTask.contains(taskAttemptId),
    +      s"Task attempt $taskAttemptId is already registered")
    +    readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create()
    --- End diff --
    
    Whoops, this entire method should be synchronized.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744386
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    --- End diff --
    
    Updated.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171795620
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49411/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-175192021
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184076652
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51287/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185977996
  
    I've updated this with a new description / summary of key changes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53685204
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    --- End diff --
    
    can you make these `-1` and `-1024` constants somewhere, like `val NO_WRITER = -1` or something


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53685793
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    +  }
    +
    +  /**
    +   * Removes the given block and automatically drops all locks on it.
    --- End diff --
    
    These comments are a bit confusing to read together. Drops all locks but it can only have 1 lock.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744607
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/MemoryStore.scala ---
    @@ -418,9 +423,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo
             while (freedMemory < space && iterator.hasNext) {
               val pair = iterator.next()
               val blockId = pair.getKey
    -          if (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) {
    -            selectedBlocks += blockId
    -            freedMemory += pair.getValue.size
    +          if (blockIsEvictable(blockId)) {
    +            if (blockManager.blockInfoManager.lockForWriting(blockId, false).isDefined) {
    --- End diff --
    
    Done.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174397987
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174413670
  
    **[Test build #49971 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49971/consoleFull)** for PR 10705 at commit [`36253df`](https://github.com/apache/spark/commit/36253dfe6879023562e404d9daf0b2c0c364e718).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53728457
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -227,6 +229,17 @@ private[spark] class Executor(
                   logError(errMsg)
                 }
               }
    +
    +          if (releasedLocks.nonEmpty) {
    +            val errMsg =
    +              s"${releasedLocks.size} block locks were not released by TID = $taskId:\n" +
    +                releasedLocks.mkString("[", ", ", "]")
    +            if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false) && !threwException) {
    --- End diff --
    
    We can't because of the `limit()` / `take()` issue.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53722164
  
    --- Diff: core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala ---
    @@ -179,8 +198,13 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
                 blocks, SparkEnv.get.serializer, compressionCodec)
               // Store the merged copy in BlockManager so other tasks on this executor don't
               // need to re-fetch it.
    -          SparkEnv.get.blockManager.putSingle(
    -            broadcastId, obj, StorageLevel.MEMORY_AND_DISK, tellMaster = false)
    +          blockManager.putSingle(broadcastId, obj, StorageLevel.MEMORY_AND_DISK, tellMaster = false)
    +          Option(TaskContext.get()) match {
    +            case Some(taskContext) =>
    +              taskContext.addTaskCompletionListener(_ => blockManager.releaseLock(broadcastId))
    +            case None =>
    +              blockManager.releaseLock(broadcastId)
    --- End diff --
    
    in this case we're just putting a block. Why not just always release the block right after the put (if the put is successful)? I don't see any reason to wait until the end of the task.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53876818
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  def size: Long = _size
    +  def size_=(s: Long): Unit = {
    +    _size = s
    +    checkInvariants()
    +  }
    +  private[this] var _size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  def readerCount: Int = _readerCount
    +  def readerCount_=(c: Int): Unit = {
    +    _readerCount = c
    +    checkInvariants()
    +  }
    +  private[this] var _readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or
    +   * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or
    +   * [[BlockInfo.NO_WRITER]] if this block is not locked for writing.
    +   */
    +  def writerTask: Long = _writerTask
    +  def writerTask_=(t: Long): Unit = {
    +    _writerTask = t
    +    checkInvariants()
    +  }
    +  private[this] var _writerTask: Long = 0
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  def removed: Boolean = _removed
    +  def removed_=(r: Boolean): Unit = {
    +    _removed = r
    +    checkInvariants()
    +  }
    +  private[this] var _removed: Boolean = false
    +
    +  private def checkInvariants(): Unit = {
    +    // A block's reader count must be non-negative:
    +    assert(_readerCount >= 0)
    +    // A block is either locked for reading or for writing, but not for both at the same time:
    +    assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER))
    +    // If a block is removed then it is not locked:
    +    assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER))
    +  }
    +
    +  checkInvariants()
    +}
    +
    +private[storage] object BlockInfo {
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being unlocked.
    +   */
    +  val NO_WRITER: Long = -1
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being held by
    +   * a non-task thread (e.g. by a driver thread or by unit test code).
    +   */
    +  val NON_TASK_WRITER: Long = -1024
    +}
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant).
    +   */
    +  @GuardedBy("this")
    +  private[this] val readLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]]
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  // Initialization for special task attempt ids:
    +  registerTask(BlockInfo.NON_TASK_WRITER)
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Called at the start of a task in order to register that task with this [[BlockInfoManager]].
    +   * This must be called prior to calling any other BlockInfoManager methods from that task.
    +   */
    +  def registerTask(taskAttemptId: TaskAttemptId): Unit = {
    +    require(!readLocksByTask.contains(taskAttemptId),
    +      s"Task attempt $taskAttemptId is already registered")
    +    readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create()
    +  }
    +
    +  /**
    +   * Returns the current task's task attempt id (which uniquely identifies the task), or
    +   * [[BlockInfo.NON_TASK_WRITER]] if called by a non-task thread.
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(BlockInfo.NON_TASK_WRITER)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * If another task has already locked this block for reading, then the read lock will be
    +   * immediately granted to the calling task and its lock count will be incremented.
    +   *
    +   * If another task has locked this block for reading, then this call will block until the write
    +   * lock is released or will return immediately if `blocking = false`.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != BlockInfo.NO_WRITER) {
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If another task has already locked this block for either reading or writing, then this call
    +   * will block until the other locks are released or will return immediately if `blocking = false`.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * method will throw an exception.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask == currentTaskAttemptId) {
    +        throw new IllegalStateException(
    +          s"Task $currentTaskAttemptId has already locked $blockId for writing")
    +      } else {
    +        while (info.writerTask != BlockInfo.NO_WRITER || info.readerCount != 0) {
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Throws an exception if the current task does not hold a write lock on the given block.
    +   * Otherwise, returns the block's BlockInfo.
    +   */
    +  def assertBlockIsLockedForWriting(blockId: BlockId): BlockInfo = synchronized {
    +    infos.get(blockId) match {
    +      case Some(info) =>
    +        if (info.writerTask != currentTaskAttemptId) {
    +          throw new SparkException(
    +            s"Task $currentTaskAttemptId has not locked block $blockId for writing")
    +        } else {
    +          info
    +        }
    +      case None =>
    +        throw new SparkException(s"Block $blockId does not exist")
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks. This method is only exposed for use by
    +   * [[BlockManager.getStatus()]] and should not be called by other code outside of this class.
    +   */
    +  private[storage] def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold on" +
    +        s" block $blockId")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != BlockInfo.NO_WRITER) {
    +      info.writerTask = BlockInfo.NO_WRITER
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a block and acquire a write lock for it, if it doesn't already
    +   * exist.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    if (!infos.contains(blockId)) {
    +      infos(blockId) = newBlockInfo
    +      newBlockInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    --- End diff --
    
    is it interesting to logTrace the result of this?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185517660
  
    **[Test build #51456 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51456/consoleFull)** for PR 10705 at commit [`bcb8318`](https://github.com/apache/spark/commit/bcb8318b558cb9add9b9ddf260c929ec4e268542).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r49797806
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/MemoryStore.scala ---
    @@ -213,6 +213,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo
       }
     
       override def remove(blockId: BlockId): Boolean = memoryManager.synchronized {
    +    val referenceCount = blockManager.getReferenceCount(blockId)
    --- End diff --
    
    What are the semantics here? It seems reasonable for another thread to get this block. Who calls remove?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53687486
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    +  }
    +
    +  /**
    +   * Removes the given block and automatically drops all locks on it.
    +   *
    +   * This can only be called while holding a write lock on the given block.
    +   */
    +  def removeBlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId")
    +    infos.get(blockId) match {
    +      case Some(blockInfo) =>
    +        if (blockInfo.writerTask != currentTaskAttemptId) {
    +          throw new IllegalStateException(
    +            s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock")
    +        } else {
    +          infos.remove(blockId)
    +          blockInfo.removed = true
    +        }
    +      case None =>
    +        throw new IllegalArgumentException(
    +          s"Task $currentTaskAttemptId called remove() on non-existent block $blockId")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Delete all state. Called during shutdown.
    +   */
    +  def clear(): Unit = synchronized {
    +    infos.clear()
    +    readLocksByTask.invalidateAll()
    +    writeLocksByTask.clear()
    --- End diff --
    
    It's only ever called during forced shutdown, so I don't think it matters, but it doesn't hurt either.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-189015891
  
    **[Test build #51987 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51987/consoleFull)** for PR 10705 at commit [`9becde3`](https://github.com/apache/spark/commit/9becde3d94ea41bba9e275b9108c61b91074f035).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171840302
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49427/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174668142
  
    **[Test build #50004 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50004/consoleFull)** for PR 10705 at commit [`77d8c5c`](https://github.com/apache/spark/commit/77d8c5caff50ce462b458bd440474fd5f3234966).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53708381
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -521,16 +509,17 @@ private[spark] class BlockManager(
                   }
                 }
               }
    +        } else {
    +          releaseLock(blockId)
    +          None
             }
    -      }
    -    } else {
    -      logDebug(s"Block $blockId not registered locally")
         }
    -    None
       }
     
       /**
        * Get block from remote block managers.
    +   *
    +   * This does not acquire a local lock on this block.
    --- End diff --
    
    I think "This does not acquire a lock on this block in this JVM" is clearer. The concept of "local lock" is not super intuitive (is there a "remote lock"?)


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173490745
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49861/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183153290
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53683999
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    --- End diff --
    
    What is the purpose of this?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53657976
  
    --- Diff: core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala ---
    @@ -127,16 +127,23 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
           def getRemote: Option[ByteBuffer] = bm.getRemoteBytes(pieceId).map { block =>
             // If we found the block from remote executors/driver's BlockManager, put the block
             // in this executor's BlockManager.
    -        SparkEnv.get.blockManager.putBytes(
    -          pieceId,
    -          block,
    -          StorageLevel.MEMORY_AND_DISK_SER,
    -          tellMaster = true)
    +        bm.putBytes(pieceId, block, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true)
             block
           }
           val block: ByteBuffer = getLocal.orElse(getRemote).getOrElse(
             throw new SparkException(s"Failed to get $pieceId of $broadcastId"))
           blocks(pid) = block
    +      Option(TaskContext.get()) match {
    --- End diff --
    
    Sure, will do.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185359892
  
    I'm going to need to change `BlockManager.doPut()` but that method has gotten too long to be easily understandable, so I'll look into splitting off a separate pull request to see if I can break down that method.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174413776
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49971/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174387538
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49970/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187587361
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51747/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53688782
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -805,13 +814,12 @@ private[spark] class BlockManager(
               }
             }
           } finally {
    -        // If we failed in putting the block to memory/disk, notify other possible readers
    -        // that it has failed, and then remove it from the block info map.
    -        if (!marked) {
    -          // Note that the remove must happen before markFailure otherwise another thread
    -          // could've inserted a new BlockInfo before we remove it.
    -          blockInfo.remove(blockId)
    -          putBlockInfo.markFailure()
    +        if (!blockWasSuccessfullyStored) {
    +          // Guard against the fact that MemoryStore might have already removed the block if the
    +          // put() failed and the block could not be dropped to disk.
    +          if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) {
    --- End diff --
    
    The reason for this is pretty convoluted; I'll explain offline.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-189016298
  
    Merged build finished. Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53726176
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    --- End diff --
    
    It seems to me the purpose of the `eq` check here is that you want to detect the case where we did the "update" part of `getOrElseUpdate`. If so I think it's actually just cleaner to write it out:
    ```
    if (!infos.contains(blockId)) {
      infos(blockId) = newBlockInfo
      actualInfo.writerTask = ...
      writeLocksByTask ...
      true
    } else {
      false
    }
    ```
    Unless you actually intended the behavior where calling `lockNewBlockForWriting` on the *exact* same `BlockInfo` object will re-aquire the write lock, which doesn't seem super useful to me.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184048345
  
    **[Test build #51288 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51288/consoleFull)** for PR 10705 at commit [`e549f2f`](https://github.com/apache/spark/commit/e549f2f82ea6517c6b9fe39a47f3fb19bc950b27).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r51646986
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -218,6 +218,7 @@ private[spark] class Executor(
               threwException = false
               res
             } finally {
    +          env.blockManager.releaseAllPinsForTask(taskId)
    --- End diff --
    
    See comment below; while I've gone ahead and added the machinery for this, we can't turn it on for all tests right now.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188117063
  
    **[Test build #51841 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51841/consoleFull)** for PR 10705 at commit [`b963178`](https://github.com/apache/spark/commit/b963178d5521dbced168c802b22e47eebc6acad2).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53751438
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    --- End diff --
    
    Agreed. Once I refactor things so that we never try to re-acquire a write lock then I'll use your suggested simplification.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171815100
  
    **[Test build #49422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49422/consoleFull)** for PR 10705 at commit [`c1a8d85`](https://github.com/apache/spark/commit/c1a8d85e6404f5db56ced0c2f4dc0d18dbfee955).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174387537
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187606736
  
    @andrewor14 and @nongli, thanks for the detailed review. I've addressed most of your comments and only have a few outstanding changes in order to clean up a few more confusing parts. I'll try to get those wrapped up by mid-day tomorrow and then this should be ready for a final sign-off.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183810965
  
    Looks like this hung in `BlockManagerReplicationSuite`. I'll investigate.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171822752
  
    **[Test build #49427 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49427/consoleFull)** for PR 10705 at commit [`575a47b`](https://github.com/apache/spark/commit/575a47bbd0154a2d9eccc37c86f8e35931bc95d0).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171807381
  
    **[Test build #49419 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49419/consoleFull)** for PR 10705 at commit [`8ae88b0`](https://github.com/apache/spark/commit/8ae88b0a4904e55e3f685c1a6bd1abe2fa0f4f35).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187904589
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174387270
  
    This has been updated and is now ready for further review. I've gone ahead and done the reference -> pin renaming, as discussed previously, have fixed the flaky / failing tests, and have added a bit more documentation.
    
    I'll keep working on trying to complete the remaining checklist items tonight (adding more debug logging, a debugging feature-flag, etc.) but it would be great to get feedback + sign-off on the subset of changes + test changes here, just to make sure there won't be any major new surprise review comments at the last minute.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188560851
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-186411074
  
    @zsxwing, one problem is the avoidance of double-release of locks. Consider the case where you read a block from the block manager, do a bunch of pipelined filters and maps on it, and then do a `take` or `limit`: the `take` or `limit` operator doesn't know the block ids of the blocks that are being read and thus doesn't know which blocks to register with the task completion callback. Even if we did know which blocks to register, we'd have to then guard against double-release of a block in case the block would be freed by CompletionIterator (in the case where the take() ends up consuming the entire iterator).
    
    The more principled approach to handling `take()` and `limit()` is to use an internal iterator interface which supports an explicit `close()` operation to signal the fact that the rest of the iterator will not be consumed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53685356
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    --- End diff --
    
    Should this be public? When should you use this?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-175357238
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183491902
  
    **[Test build #51204 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51204/consoleFull)** for PR 10705 at commit [`f3fc298`](https://github.com/apache/spark/commit/f3fc2987585352d9e5a2dd3083ed65c1f5f30507).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-175131705
  
    Jenkins, retest this please.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187969913
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185545913
  
    **[Test build #51456 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51456/consoleFull)** for PR 10705 at commit [`bcb8318`](https://github.com/apache/spark/commit/bcb8318b558cb9add9b9ddf260c929ec4e268542).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188144514
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51864/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173127879
  
    **[Test build #49773 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49773/consoleFull)** for PR 10705 at commit [`90cf403`](https://github.com/apache/spark/commit/90cf4030ca88838538577c49e78ee5c36f408fff).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183097126
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173490560
  
    **[Test build #49861 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49861/consoleFull)** for PR 10705 at commit [`12ed084`](https://github.com/apache/spark/commit/12ed0841b5d5cf171e9db9325bf9f61f3dd8046b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171839999
  
    **[Test build #49427 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49427/consoleFull)** for PR 10705 at commit [`575a47b`](https://github.com/apache/spark/commit/575a47bbd0154a2d9eccc37c86f8e35931bc95d0).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53876449
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    --- End diff --
    
    Worth noting this is not thread safe and protected by the lock in BlockInfoManager


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185996316
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51501/
    Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183766953
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51247/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173158344
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49773/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173469381
  
    **[Test build #49861 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49861/consoleFull)** for PR 10705 at commit [`12ed084`](https://github.com/apache/spark/commit/12ed0841b5d5cf171e9db9325bf9f61f3dd8046b).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184076992
  
    Woohoo! Looks like the refactored version of this patch is finally passing all tests, so I'll do another pass to address a few minor debugability todos and then hopefully we can get this in early next week to unblock the rest of the block manager refactorings that I have queued up.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53748113
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    --- End diff --
    
    Nope, we don't need it. Removed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53687313
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    --- End diff --
    
    Yep, it's a todo; I'll do it in a followup patch.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53696758
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    --- End diff --
    
    why do we care about reference equality here? What if I just have the `shuffleId` and the `mapId` or something and I want to lock it?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53877424
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -989,77 +1014,62 @@ private[spark] class BlockManager(
        * store reaches its limit and needs to free up space.
        *
        * If `data` is not put on disk, it won't be created.
    +   *
    +   * The caller of this method must hold a write lock on the block before calling this method.
    --- End diff --
    
    Comment on lock status after function returns.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r54180953
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -452,7 +440,10 @@ private[spark] class BlockManager(
             if (level.useMemory) {
               logDebug(s"Getting block $blockId from memory")
               val result = if (asBlockResult) {
    -            memoryStore.getValues(blockId).map(new BlockResult(_, DataReadMethod.Memory, info.size))
    +            memoryStore.getValues(blockId).map { iter =>
    +              val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId))
    +              new BlockResult(ci, DataReadMethod.Memory, info.size)
    --- End diff --
    
    can we pass in an optional completion callback instead?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53723348
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -521,16 +509,17 @@ private[spark] class BlockManager(
                   }
    --- End diff --
    
    In L496 of the new code, where we put the values we read from disk back into memory, do we need to "upgrade" the lock? Will someone else be able to read the same block in the mean time, since we did not acquire the write lock here?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53694268
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    --- End diff --
    
    (also for `lockForWriting`)


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744476
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,357 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold on" +
    +        s" block $blockId")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    +  }
    +
    +  /**
    +   * Removes the given block and automatically drops all locks on it.
    +   *
    +   * This can only be called while holding a write lock on the given block.
    --- End diff --
    
    TODO: enforce this.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184048372
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51288/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53592740
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    --- End diff --
    
    This doesn't match the comment: `this iterator are mutable and thus may reflect blocks that are deleted while the iterator is being traversed.` since they are copied to an array


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174397879
  
    **[Test build #49964 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49964/consoleFull)** for PR 10705 at commit [`43e50ed`](https://github.com/apache/spark/commit/43e50ed39c5dee9c15ae5aeac451dd02647ac8e5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744319
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    --- End diff --
    
    Fixed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r54182055
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -452,7 +440,10 @@ private[spark] class BlockManager(
             if (level.useMemory) {
               logDebug(s"Getting block $blockId from memory")
               val result = if (asBlockResult) {
    -            memoryStore.getValues(blockId).map(new BlockResult(_, DataReadMethod.Memory, info.size))
    +            memoryStore.getValues(blockId).map { iter =>
    +              val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId))
    +              new BlockResult(ci, DataReadMethod.Memory, info.size)
    --- End diff --
    
    The difference is that now the programmer needs to explicitly `completionCallback = None`. If the `completionCallback` is specified then you don't need to do the network check. It's better in that today you have zero reminder that you need to release the lock by the end of the task.
    
    Actually an even better way IMO is to have a `LocalBlockResult` and a `RemoteBlockResult` so there's no way the programmer can forget to release the lock.
    
    By the way, I'm not quite done reviewing yet but feel free to address these in a follow-up patch.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188048616
  
    **[Test build #51841 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51841/consoleFull)** for PR 10705 at commit [`b963178`](https://github.com/apache/spark/commit/b963178d5521dbced168c802b22e47eebc6acad2).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744589
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -521,16 +509,17 @@ private[spark] class BlockManager(
                   }
                 }
               }
    +        } else {
    +          releaseLock(blockId)
    +          None
             }
    -      }
    -    } else {
    -      logDebug(s"Block $blockId not registered locally")
         }
    -    None
       }
     
       /**
        * Get block from remote block managers.
    +   *
    +   * This does not acquire a local lock on this block.
    --- End diff --
    
    I agree; updated.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171815359
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49422/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53682666
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    --- End diff --
    
    I see. Maybe just say `entries` may return out of date blocks? BTW, you can just use `infos.toArray.toIterator`.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53696069
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    --- End diff --
    
    This reads more like `releaseAllLocksForTask` or `unlockForTask`


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744430
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    +  }
    +
    +  /**
    +   * Removes the given block and automatically drops all locks on it.
    +   *
    +   * This can only be called while holding a write lock on the given block.
    +   */
    +  def removeBlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId")
    +    infos.get(blockId) match {
    +      case Some(blockInfo) =>
    +        if (blockInfo.writerTask != currentTaskAttemptId) {
    +          throw new IllegalStateException(
    +            s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock")
    +        } else {
    +          infos.remove(blockId)
    --- End diff --
    
    Done as part of updating invariants.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188522589
  
    **[Test build #51910 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51910/consoleFull)** for PR 10705 at commit [`9becde3`](https://github.com/apache/spark/commit/9becde3d94ea41bba9e275b9108c61b91074f035).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183809615
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51248/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184067383
  
    **[Test build #51291 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51291/consoleFull)** for PR 10705 at commit [`717c476`](https://github.com/apache/spark/commit/717c476668bad9a1516bd462461714108163c496).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53888703
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    --- End diff --
    
    Done.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53695813
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    --- End diff --
    
    what's the reason for using `MultiMap` here? Is the idea that if a task acquired write lock 3 times the task has to release it 3 times? (If so) why not just make the locks re-entrant?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185379120
  
    Actually, forget it: I don't know that there's an obvious way to break down `doPut()` which isn't going to be just as hard to review.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174381575
  
    **[Test build #49964 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49964/consoleFull)** for PR 10705 at commit [`43e50ed`](https://github.com/apache/spark/commit/43e50ed39c5dee9c15ae5aeac451dd02647ac8e5).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53889065
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -989,77 +1014,62 @@ private[spark] class BlockManager(
        * store reaches its limit and needs to free up space.
        *
        * If `data` is not put on disk, it won't be created.
    +   *
    +   * The caller of this method must hold a write lock on the block before calling this method.
    --- End diff --
    
    Done.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188453371
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185479212
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-189052867
  
    LGTM. There are still a few remaining issues about maintainability but they can be addressed in a follow-up patch.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187689047
  
    **[Test build #51753 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51753/consoleFull)** for PR 10705 at commit [`07e0e37`](https://github.com/apache/spark/commit/07e0e377d87762cffde110dc93c51b95c0bc7c3c).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53589538
  
    --- Diff: core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala ---
    @@ -127,16 +127,23 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
           def getRemote: Option[ByteBuffer] = bm.getRemoteBytes(pieceId).map { block =>
             // If we found the block from remote executors/driver's BlockManager, put the block
             // in this executor's BlockManager.
    -        SparkEnv.get.blockManager.putBytes(
    -          pieceId,
    -          block,
    -          StorageLevel.MEMORY_AND_DISK_SER,
    -          tellMaster = true)
    +        bm.putBytes(pieceId, block, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true)
             block
           }
           val block: ByteBuffer = getLocal.orElse(getRemote).getOrElse(
             throw new SparkException(s"Failed to get $pieceId of $broadcastId"))
           blocks(pid) = block
    +      Option(TaskContext.get()) match {
    --- End diff --
    
    Could you add a util method for these codes? I saw they appear multiple times.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183058138
  
    **[Test build #51128 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51128/consoleFull)** for PR 10705 at commit [`77939c2`](https://github.com/apache/spark/commit/77939c2bf60b3425b2df682b7dab8352ef7edfdf).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187991486
  
    LGTM Good work!


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53888668
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  def size: Long = _size
    +  def size_=(s: Long): Unit = {
    +    _size = s
    +    checkInvariants()
    +  }
    +  private[this] var _size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  def readerCount: Int = _readerCount
    +  def readerCount_=(c: Int): Unit = {
    +    _readerCount = c
    +    checkInvariants()
    +  }
    +  private[this] var _readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or
    +   * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or
    +   * [[BlockInfo.NO_WRITER]] if this block is not locked for writing.
    +   */
    +  def writerTask: Long = _writerTask
    +  def writerTask_=(t: Long): Unit = {
    +    _writerTask = t
    +    checkInvariants()
    +  }
    +  private[this] var _writerTask: Long = 0
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  def removed: Boolean = _removed
    +  def removed_=(r: Boolean): Unit = {
    +    _removed = r
    +    checkInvariants()
    +  }
    +  private[this] var _removed: Boolean = false
    +
    +  private def checkInvariants(): Unit = {
    +    // A block's reader count must be non-negative:
    +    assert(_readerCount >= 0)
    +    // A block is either locked for reading or for writing, but not for both at the same time:
    +    assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER))
    +    // If a block is removed then it is not locked:
    +    assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER))
    +  }
    +
    +  checkInvariants()
    +}
    +
    +private[storage] object BlockInfo {
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being unlocked.
    +   */
    +  val NO_WRITER: Long = -1
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being held by
    +   * a non-task thread (e.g. by a driver thread or by unit test code).
    +   */
    +  val NON_TASK_WRITER: Long = -1024
    +}
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant).
    +   */
    +  @GuardedBy("this")
    +  private[this] val readLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]]
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  // Initialization for special task attempt ids:
    +  registerTask(BlockInfo.NON_TASK_WRITER)
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Called at the start of a task in order to register that task with this [[BlockInfoManager]].
    +   * This must be called prior to calling any other BlockInfoManager methods from that task.
    +   */
    +  def registerTask(taskAttemptId: TaskAttemptId): Unit = {
    +    require(!readLocksByTask.contains(taskAttemptId),
    +      s"Task attempt $taskAttemptId is already registered")
    +    readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create()
    +  }
    +
    +  /**
    +   * Returns the current task's task attempt id (which uniquely identifies the task), or
    +   * [[BlockInfo.NON_TASK_WRITER]] if called by a non-task thread.
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(BlockInfo.NON_TASK_WRITER)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * If another task has already locked this block for reading, then the read lock will be
    +   * immediately granted to the calling task and its lock count will be incremented.
    +   *
    +   * If another task has locked this block for reading, then this call will block until the write
    --- End diff --
    
    Whoops, good catch.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-189066700
  
    Merged into master.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53657869
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    --- End diff --
    
    The comment is confusingly-worded; I'll explain more in person, but the idea here is that the elements in the array are mutable even though the array is immutable, so if we call `.entries()`, then `.remove(blockId)`, the removed block might appear in `entries` while the caller is iterating over the returned array.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53683409
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    --- End diff --
    
    if you make these vars private and add public setter methods then this is easy to do


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171795428
  
    **[Test build #49411 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49411/consoleFull)** for PR 10705 at commit [`7cad770`](https://github.com/apache/spark/commit/7cad770025a3ab4f3d2e312482ff7baf20a159cb).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184027813
  
    **[Test build #51288 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51288/consoleFull)** for PR 10705 at commit [`e549f2f`](https://github.com/apache/spark/commit/e549f2f82ea6517c6b9fe39a47f3fb19bc950b27).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173428607
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49841/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188453375
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51898/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184021971
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183486808
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51200/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53717677
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -521,16 +509,17 @@ private[spark] class BlockManager(
                   }
                 }
               }
    +        } else {
    +          releaseLock(blockId)
    --- End diff --
    
    It took me a while to figure out what this is case is. IIUC this happens when there is an entry for this block in `BlockInfoManager`, but it is neither in memory nor marked to use disk. In other words, this is only for `MEMORY_ONLY` and after the block has already been evicted. In that case it would make sense to release the lock as you have done.
    
    I'm actually not sure if we'll ever hit this case (if the block has already been evicted then its metadata won't even be in `BlockInfoManager`). Either way a simple comment to explain what this case represents will be very helpful.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53727541
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -805,13 +814,12 @@ private[spark] class BlockManager(
               }
             }
           } finally {
    -        // If we failed in putting the block to memory/disk, notify other possible readers
    -        // that it has failed, and then remove it from the block info map.
    -        if (!marked) {
    -          // Note that the remove must happen before markFailure otherwise another thread
    -          // could've inserted a new BlockInfo before we remove it.
    -          blockInfo.remove(blockId)
    -          putBlockInfo.markFailure()
    +        if (!blockWasSuccessfullyStored) {
    +          // Guard against the fact that MemoryStore might have already removed the block if the
    +          // put() failed and the block could not be dropped to disk.
    +          if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) {
    --- End diff --
    
    yeah, I also don't follow


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183058638
  
    **[Test build #51128 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51128/consoleFull)** for PR 10705 at commit [`77939c2`](https://github.com/apache/spark/commit/77939c2bf60b3425b2df682b7dab8352ef7edfdf).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53877407
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -989,77 +1014,62 @@ private[spark] class BlockManager(
        * store reaches its limit and needs to free up space.
        *
        * If `data` is not put on disk, it won't be created.
    +   *
    +   * The caller of this method must hold a write lock on the block before calling this method.
    +   *
    +   * @return the block's new effective StorageLevel.
        */
       def dropFromMemory(
           blockId: BlockId,
    -      data: () => Either[Array[Any], ByteBuffer]): Unit = {
    -
    +      data: () => Either[Array[Any], ByteBuffer]): StorageLevel = {
         logInfo(s"Dropping block $blockId from memory")
    -    val info = blockInfo.get(blockId)
    -
    -    // If the block has not already been dropped
    -    if (info != null) {
    -      info.synchronized {
    -        // required ? As of now, this will be invoked only for blocks which are ready
    -        // But in case this changes in future, adding for consistency sake.
    -        if (!info.waitForReady()) {
    -          // If we get here, the block write failed.
    -          logWarning(s"Block $blockId was marked as failure. Nothing to drop")
    -          return
    -        } else if (blockInfo.asScala.get(blockId).isEmpty) {
    -          logWarning(s"Block $blockId was already dropped.")
    -          return
    -        }
    -        var blockIsUpdated = false
    -        val level = info.level
    -
    -        // Drop to disk, if storage level requires
    -        if (level.useDisk && !diskStore.contains(blockId)) {
    -          logInfo(s"Writing block $blockId to disk")
    -          data() match {
    -            case Left(elements) =>
    -              diskStore.putArray(blockId, elements, level, returnValues = false)
    -            case Right(bytes) =>
    -              diskStore.putBytes(blockId, bytes, level)
    -          }
    -          blockIsUpdated = true
    -        }
    +    val info = blockInfoManager.assertBlockIsLockedForWriting(blockId)
    +    var blockIsUpdated = false
    +    val level = info.level
    +
    +    // Drop to disk, if storage level requires
    +    if (level.useDisk && !diskStore.contains(blockId)) {
    +      logInfo(s"Writing block $blockId to disk")
    +      data() match {
    +        case Left(elements) =>
    +          diskStore.putArray(blockId, elements, level, returnValues = false)
    +        case Right(bytes) =>
    +          diskStore.putBytes(blockId, bytes, level)
    +      }
    +      blockIsUpdated = true
    +    }
     
    -        // Actually drop from memory store
    -        val droppedMemorySize =
    -          if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L
    -        val blockIsRemoved = memoryStore.remove(blockId)
    -        if (blockIsRemoved) {
    -          blockIsUpdated = true
    -        } else {
    -          logWarning(s"Block $blockId could not be dropped from memory as it does not exist")
    -        }
    +    // Actually drop from memory store
    +    val droppedMemorySize =
    +      if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L
    +    val blockIsRemoved = memoryStore.remove(blockId)
    +    if (blockIsRemoved) {
    +      blockIsUpdated = true
    +    } else {
    +      logWarning(s"Block $blockId could not be dropped from memory as it does not exist")
    --- End diff --
    
    Is this still possible?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53888342
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala ---
    @@ -65,7 +65,11 @@ class NettyBlockRpcServer(
             val level: StorageLevel =
               serializer.newInstance().deserialize(ByteBuffer.wrap(uploadBlock.metadata))
             val data = new NioManagedBuffer(ByteBuffer.wrap(uploadBlock.blockData))
    -        blockManager.putBlockData(BlockId(uploadBlock.blockId), data, level)
    +        val blockId = BlockId(uploadBlock.blockId)
    +        val putSucceeded = blockManager.putBlockData(blockId, data, level)
    --- End diff --
    
    Yeah, let's revisit this in a followup. Swapping the default might be a good move since I think the current semantics are only really useful in the CacheManager.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53686131
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    +  }
    +
    +  /**
    +   * Removes the given block and automatically drops all locks on it.
    +   *
    +   * This can only be called while holding a write lock on the given block.
    +   */
    +  def removeBlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId")
    +    infos.get(blockId) match {
    +      case Some(blockInfo) =>
    +        if (blockInfo.writerTask != currentTaskAttemptId) {
    +          throw new IllegalStateException(
    +            s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock")
    +        } else {
    +          infos.remove(blockId)
    +          blockInfo.removed = true
    +        }
    +      case None =>
    +        throw new IllegalArgumentException(
    +          s"Task $currentTaskAttemptId called remove() on non-existent block $blockId")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Delete all state. Called during shutdown.
    +   */
    +  def clear(): Unit = synchronized {
    +    infos.clear()
    +    readLocksByTask.invalidateAll()
    +    writeLocksByTask.clear()
    --- End diff --
    
    Does this need notifyALl?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183486800
  
    **[Test build #51200 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51200/consoleFull)** for PR 10705 at commit [`9c8d530`](https://github.com/apache/spark/commit/9c8d5308ae4482c86a381e9a863fa91d3ed63899).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188999121
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51985/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185949053
  
    This should now be ready for more review. Reviewable link, in case you want to use that to manage reviews: https://reviewable.io/reviews/apache/spark/10705


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173158192
  
    **[Test build #49773 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49773/consoleFull)** for PR 10705 at commit [`90cf403`](https://github.com/apache/spark/commit/90cf4030ca88838538577c49e78ee5c36f408fff).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53701132
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    +    readLocks.entrySet().iterator().asScala.foreach { entry =>
    +      val blockId = entry.getElement
    +      val lockCount = entry.getCount
    +      blocksWithReleasedLocks += blockId
    +      synchronized {
    +        get(blockId).foreach { info =>
    +          info.readerCount -= lockCount
    +          assert(info.readerCount >= 0)
    +        }
    +      }
    +    }
    +    synchronized {
    +      notifyAll()
    +    }
    +    blocksWithReleasedLocks
    +  }
    +
    +  /**
    +   * Returns the number of blocks tracked.
    +   */
    +  def size: Int = synchronized {
    +    infos.size
    +  }
    +
    +  /**
    +   * Return the number of map entries in this pin counter's internal data structures.
    +   * This is used in unit tests in order to detect memory leaks.
    +   */
    +  private[storage] def getNumberOfMapEntries: Long = synchronized {
    +    size +
    +      readLocksByTask.size() +
    +      readLocksByTask.asMap().asScala.map(_._2.size()).sum +
    +      writeLocksByTask.size +
    +      writeLocksByTask.map(_._2.size).sum
    +  }
    +
    +  /**
    +   * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries
    +   * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator
    +   * is being traversed.
    +   */
    +  def entries: Iterator[(BlockId, BlockInfo)] = synchronized {
    +    infos.iterator.toArray.toIterator
    +  }
    +
    +  /**
    +   * Removes the given block and automatically drops all locks on it.
    +   *
    +   * This can only be called while holding a write lock on the given block.
    +   */
    +  def removeBlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId")
    +    infos.get(blockId) match {
    +      case Some(blockInfo) =>
    +        if (blockInfo.writerTask != currentTaskAttemptId) {
    +          throw new IllegalStateException(
    +            s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock")
    +        } else {
    +          infos.remove(blockId)
    +          blockInfo.removed = true
    --- End diff --
    
    Setting `blockInfo.removed` covers this, but we'll need to do that if we remove the redundant `if (blockInfo.removed)` check in the `while` loop bodies.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187651889
  
    **[Test build #51748 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51748/consoleFull)** for PR 10705 at commit [`9427576`](https://github.com/apache/spark/commit/9427576e240645dd20f3f65af335eb25a00cff3e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187939399
  
    **[Test build #51806 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51806/consoleFull)** for PR 10705 at commit [`697eba2`](https://github.com/apache/spark/commit/697eba213c11bc92724cfd891f5b86acd60dfd4f).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-178933457
  
    Merged build finished. Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173465206
  
    Jenkins, retest this please.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187984214
  
    **[Test build #51821 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51821/consoleFull)** for PR 10705 at commit [`b9d6e18`](https://github.com/apache/spark/commit/b9d6e181db0c7293b0b00b3a679716f652df936f).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53683591
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheManager.scala ---
    @@ -136,7 +147,13 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
            */
           blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel)
           blockManager.get(key) match {
    -        case Some(v) => v.data.asInstanceOf[Iterator[T]]
    +        case Some(v) =>
    --- End diff --
    
    What's the consequence of forgetting to release? Does it get caught somewhere else and released as a catch all? Similar to the other comments, this makes it seem like a try {} finally{} is needed after the get(key).
    



---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188144403
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187492881
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51714/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171795618
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-170711143
  
    **[Test build #49167 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49167/consoleFull)** for PR 10705 at commit [`7265784`](https://github.com/apache/spark/commit/7265784f821c5ca451322e0a2b1bfdcf8c952af4).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173158342
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184033063
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53720027
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheManager.scala ---
    @@ -46,7 +48,14 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
             val existingMetrics = context.taskMetrics().registerInputMetrics(blockResult.readMethod)
             existingMetrics.incBytesReadInternal(blockResult.bytes)
     
    -        val iter = blockResult.data.asInstanceOf[Iterator[T]]
    +        val iter = {
    +          val dataIter = blockResult.data.asInstanceOf[Iterator[T]]
    +          if (blockResult.readMethod != DataReadMethod.Network) {
    +            CompletionIterator[T, Iterator[T]](dataIter, blockManager.releaseLock(key))
    --- End diff --
    
    +1 putting a `completionIterator` method or something in `BlockResult`


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53722284
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -227,6 +229,17 @@ private[spark] class Executor(
                   logError(errMsg)
                 }
               }
    +
    +          if (releasedLocks.nonEmpty) {
    +            val errMsg =
    +              s"${releasedLocks.size} block locks were not released by TID = $taskId:\n" +
    +                releasedLocks.mkString("[", ", ", "]")
    +            if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false) && !threwException) {
    --- End diff --
    
    should we enable this in tests?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184035747
  
    **[Test build #51291 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51291/consoleFull)** for PR 10705 at commit [`717c476`](https://github.com/apache/spark/commit/717c476668bad9a1516bd462461714108163c496).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183153291
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51132/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183170958
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51146/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184033065
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51289/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-178868305
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/50600/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53701301
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -318,7 +316,11 @@ private[spark] class BlockManager(
        * may not know of).
        */
       def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = {
    -    (blockInfo.asScala.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq
    +    val matches =
    +      (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()).filter(filter)
    +    // The `toArray` is necessary here in order to force the list to be materialized so that we
    +    // don't try to serialize a lazy iterator when responding to client requests.
    +    matches.toArray.toSeq
    --- End diff --
    
    style nit:
    ```
    (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks())
      .filter(filter)
      // ...
      .toArray
      .toSeq
    ```


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/10705


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744366
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    --- End diff --
    
    Good idea; done.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174668701
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188114683
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744601
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.spark.storage
    +
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
    +
    +
    +private[storage] class BlockManagerManagedBuffer(
    --- End diff --
    
    Added.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171825662
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174405795
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49969/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188999118
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187971575
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51819/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187968241
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51806/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53682420
  
    --- Diff: core/src/main/scala/org/apache/spark/network/BlockDataManager.scala ---
    @@ -31,6 +31,14 @@ trait BlockDataManager {
     
       /**
        * Put the block locally, using the given storage level.
    +   *
    +   * Returns true if the block was stored and false if the put operation failed or the block
    +   * aready existed.
    --- End diff --
    
    already


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187587358
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187491138
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51712/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53719624
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheManager.scala ---
    @@ -58,7 +67,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
             // If another thread already holds the lock, wait for it to finish return its results
             val storedValues = acquireLockForPartition[T](key)
             if (storedValues.isDefined) {
    -          return new InterruptibleIterator[T](context, storedValues.get)
    +          val iter =
    +            CompletionIterator[T, Iterator[T]](storedValues.get, blockManager.releaseLock(key))
    --- End diff --
    
    This is mainly a note to self and other reviewers: I spent a long time wrapping my head around `blockManager.get` and I think I finally got it. The confusing part was that `get` sometimes acquires the lock but sometimes doesn't, so in all places where we call `get` we must likewise sometimes release the lock later.
    
    In `CacheManager`, we do this by checking whether the read method is network. If it is, then we read the block from a remote host so we don't have to release any locks. Otherwise, we read it locally (whether from memory or from disk) so we do have to release the lock.
    
    In this line, however, we assume another thread in the same JVM has computed the values for us, so there is no way we got the block's values over the network, so we don't need to do the network check as we did elsewhere.
    
    @JoshRosen is that correct?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185034523
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51408/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174390213
  
    **[Test build #49971 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49971/consoleFull)** for PR 10705 at commit [`36253df`](https://github.com/apache/spark/commit/36253dfe6879023562e404d9daf0b2c0c364e718).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183112573
  
    **[Test build #51146 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51146/consoleFull)** for PR 10705 at commit [`ef7d885`](https://github.com/apache/spark/commit/ef7d88508af04b81d6671fd7ccf55111ca3e7856).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53743845
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    --- End diff --
    
    With a bit of refactoring I think I can simplify it so that we never attempt to acquire a write lock which we already hold (i.e. make that into an error condition). Basically, the idea is that after my planned changes we'll only ever acquire write locks either in `BlockManager.doPut()` or in `MemoryStore.evictBlocksToFreeSpace()`.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185996312
  
    Merged build finished. Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174405316
  
    **[Test build #49969 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49969/consoleFull)** for PR 10705 at commit [`8d45da6`](https://github.com/apache/spark/commit/8d45da608a5b8d72259b2a275271f2740b1ebdec).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53889110
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  def size: Long = _size
    +  def size_=(s: Long): Unit = {
    +    _size = s
    +    checkInvariants()
    +  }
    +  private[this] var _size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  def readerCount: Int = _readerCount
    +  def readerCount_=(c: Int): Unit = {
    +    _readerCount = c
    +    checkInvariants()
    +  }
    +  private[this] var _readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or
    +   * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or
    +   * [[BlockInfo.NO_WRITER]] if this block is not locked for writing.
    +   */
    +  def writerTask: Long = _writerTask
    +  def writerTask_=(t: Long): Unit = {
    +    _writerTask = t
    +    checkInvariants()
    +  }
    +  private[this] var _writerTask: Long = 0
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  def removed: Boolean = _removed
    +  def removed_=(r: Boolean): Unit = {
    +    _removed = r
    +    checkInvariants()
    +  }
    +  private[this] var _removed: Boolean = false
    +
    +  private def checkInvariants(): Unit = {
    +    // A block's reader count must be non-negative:
    +    assert(_readerCount >= 0)
    +    // A block is either locked for reading or for writing, but not for both at the same time:
    +    assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER))
    +    // If a block is removed then it is not locked:
    +    assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER))
    +  }
    +
    +  checkInvariants()
    +}
    +
    +private[storage] object BlockInfo {
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being unlocked.
    +   */
    +  val NO_WRITER: Long = -1
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being held by
    +   * a non-task thread (e.g. by a driver thread or by unit test code).
    +   */
    +  val NON_TASK_WRITER: Long = -1024
    +}
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant).
    +   */
    +  @GuardedBy("this")
    +  private[this] val readLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]]
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  // Initialization for special task attempt ids:
    +  registerTask(BlockInfo.NON_TASK_WRITER)
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Called at the start of a task in order to register that task with this [[BlockInfoManager]].
    +   * This must be called prior to calling any other BlockInfoManager methods from that task.
    +   */
    +  def registerTask(taskAttemptId: TaskAttemptId): Unit = {
    +    require(!readLocksByTask.contains(taskAttemptId),
    +      s"Task attempt $taskAttemptId is already registered")
    +    readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create()
    +  }
    +
    +  /**
    +   * Returns the current task's task attempt id (which uniquely identifies the task), or
    +   * [[BlockInfo.NON_TASK_WRITER]] if called by a non-task thread.
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(BlockInfo.NON_TASK_WRITER)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * If another task has already locked this block for reading, then the read lock will be
    +   * immediately granted to the calling task and its lock count will be incremented.
    +   *
    +   * If another task has locked this block for reading, then this call will block until the write
    +   * lock is released or will return immediately if `blocking = false`.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != BlockInfo.NO_WRITER) {
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If another task has already locked this block for either reading or writing, then this call
    +   * will block until the other locks are released or will return immediately if `blocking = false`.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * method will throw an exception.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask == currentTaskAttemptId) {
    +        throw new IllegalStateException(
    +          s"Task $currentTaskAttemptId has already locked $blockId for writing")
    +      } else {
    +        while (info.writerTask != BlockInfo.NO_WRITER || info.readerCount != 0) {
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Throws an exception if the current task does not hold a write lock on the given block.
    +   * Otherwise, returns the block's BlockInfo.
    +   */
    +  def assertBlockIsLockedForWriting(blockId: BlockId): BlockInfo = synchronized {
    +    infos.get(blockId) match {
    +      case Some(info) =>
    +        if (info.writerTask != currentTaskAttemptId) {
    +          throw new SparkException(
    +            s"Task $currentTaskAttemptId has not locked block $blockId for writing")
    +        } else {
    +          info
    +        }
    +      case None =>
    +        throw new SparkException(s"Block $blockId does not exist")
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks. This method is only exposed for use by
    +   * [[BlockManager.getStatus()]] and should not be called by other code outside of this class.
    +   */
    +  private[storage] def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold on" +
    +        s" block $blockId")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    --- End diff --
    
    Done.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53876641
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  def size: Long = _size
    +  def size_=(s: Long): Unit = {
    +    _size = s
    +    checkInvariants()
    +  }
    +  private[this] var _size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  def readerCount: Int = _readerCount
    +  def readerCount_=(c: Int): Unit = {
    +    _readerCount = c
    +    checkInvariants()
    +  }
    +  private[this] var _readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or
    +   * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or
    +   * [[BlockInfo.NO_WRITER]] if this block is not locked for writing.
    +   */
    +  def writerTask: Long = _writerTask
    +  def writerTask_=(t: Long): Unit = {
    +    _writerTask = t
    +    checkInvariants()
    +  }
    +  private[this] var _writerTask: Long = 0
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  def removed: Boolean = _removed
    +  def removed_=(r: Boolean): Unit = {
    +    _removed = r
    +    checkInvariants()
    +  }
    +  private[this] var _removed: Boolean = false
    +
    +  private def checkInvariants(): Unit = {
    +    // A block's reader count must be non-negative:
    +    assert(_readerCount >= 0)
    +    // A block is either locked for reading or for writing, but not for both at the same time:
    +    assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER))
    +    // If a block is removed then it is not locked:
    +    assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER))
    +  }
    +
    +  checkInvariants()
    +}
    +
    +private[storage] object BlockInfo {
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being unlocked.
    +   */
    +  val NO_WRITER: Long = -1
    +
    +  /**
    +   * Special task attempt id constant used to mark a block's write lock as being held by
    +   * a non-task thread (e.g. by a driver thread or by unit test code).
    +   */
    +  val NON_TASK_WRITER: Long = -1024
    +}
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant).
    +   */
    +  @GuardedBy("this")
    +  private[this] val readLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]]
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  // Initialization for special task attempt ids:
    +  registerTask(BlockInfo.NON_TASK_WRITER)
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Called at the start of a task in order to register that task with this [[BlockInfoManager]].
    +   * This must be called prior to calling any other BlockInfoManager methods from that task.
    +   */
    +  def registerTask(taskAttemptId: TaskAttemptId): Unit = {
    +    require(!readLocksByTask.contains(taskAttemptId),
    +      s"Task attempt $taskAttemptId is already registered")
    +    readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create()
    +  }
    +
    +  /**
    +   * Returns the current task's task attempt id (which uniquely identifies the task), or
    +   * [[BlockInfo.NON_TASK_WRITER]] if called by a non-task thread.
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(BlockInfo.NON_TASK_WRITER)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * If another task has already locked this block for reading, then the read lock will be
    +   * immediately granted to the calling task and its lock count will be incremented.
    +   *
    +   * If another task has locked this block for reading, then this call will block until the write
    +   * lock is released or will return immediately if `blocking = false`.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != BlockInfo.NO_WRITER) {
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If another task has already locked this block for either reading or writing, then this call
    +   * will block until the other locks are released or will return immediately if `blocking = false`.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * method will throw an exception.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask == currentTaskAttemptId) {
    +        throw new IllegalStateException(
    +          s"Task $currentTaskAttemptId has already locked $blockId for writing")
    +      } else {
    +        while (info.writerTask != BlockInfo.NO_WRITER || info.readerCount != 0) {
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Throws an exception if the current task does not hold a write lock on the given block.
    +   * Otherwise, returns the block's BlockInfo.
    +   */
    +  def assertBlockIsLockedForWriting(blockId: BlockId): BlockInfo = synchronized {
    +    infos.get(blockId) match {
    +      case Some(info) =>
    +        if (info.writerTask != currentTaskAttemptId) {
    +          throw new SparkException(
    +            s"Task $currentTaskAttemptId has not locked block $blockId for writing")
    +        } else {
    +          info
    +        }
    +      case None =>
    +        throw new SparkException(s"Block $blockId does not exist")
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks. This method is only exposed for use by
    +   * [[BlockManager.getStatus()]] and should not be called by other code outside of this class.
    +   */
    +  private[storage] def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold on" +
    +        s" block $blockId")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    --- End diff --
    
    Can you assert the return is not none


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171825666
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49419/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r52800233
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,335 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +}
    +
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[putAndLockForWritingIfAbsent()]]) and are removed
    +   * by [[remove()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def getAndLockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    --- End diff --
    
    Do you need to check info.removed after waking up from wait()?
    
    After wait(), can you have infor.writerTask = -1 and readerCount = 0 and then you return a removed block?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53889341
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -989,77 +1014,62 @@ private[spark] class BlockManager(
        * store reaches its limit and needs to free up space.
        *
        * If `data` is not put on disk, it won't be created.
    +   *
    +   * The caller of this method must hold a write lock on the block before calling this method.
    +   *
    +   * @return the block's new effective StorageLevel.
        */
       def dropFromMemory(
           blockId: BlockId,
    -      data: () => Either[Array[Any], ByteBuffer]): Unit = {
    -
    +      data: () => Either[Array[Any], ByteBuffer]): StorageLevel = {
         logInfo(s"Dropping block $blockId from memory")
    -    val info = blockInfo.get(blockId)
    -
    -    // If the block has not already been dropped
    -    if (info != null) {
    -      info.synchronized {
    -        // required ? As of now, this will be invoked only for blocks which are ready
    -        // But in case this changes in future, adding for consistency sake.
    -        if (!info.waitForReady()) {
    -          // If we get here, the block write failed.
    -          logWarning(s"Block $blockId was marked as failure. Nothing to drop")
    -          return
    -        } else if (blockInfo.asScala.get(blockId).isEmpty) {
    -          logWarning(s"Block $blockId was already dropped.")
    -          return
    -        }
    -        var blockIsUpdated = false
    -        val level = info.level
    -
    -        // Drop to disk, if storage level requires
    -        if (level.useDisk && !diskStore.contains(blockId)) {
    -          logInfo(s"Writing block $blockId to disk")
    -          data() match {
    -            case Left(elements) =>
    -              diskStore.putArray(blockId, elements, level, returnValues = false)
    -            case Right(bytes) =>
    -              diskStore.putBytes(blockId, bytes, level)
    -          }
    -          blockIsUpdated = true
    -        }
    +    val info = blockInfoManager.assertBlockIsLockedForWriting(blockId)
    +    var blockIsUpdated = false
    +    val level = info.level
    +
    +    // Drop to disk, if storage level requires
    +    if (level.useDisk && !diskStore.contains(blockId)) {
    +      logInfo(s"Writing block $blockId to disk")
    +      data() match {
    +        case Left(elements) =>
    +          diskStore.putArray(blockId, elements, level, returnValues = false)
    +        case Right(bytes) =>
    +          diskStore.putBytes(blockId, bytes, level)
    +      }
    +      blockIsUpdated = true
    +    }
     
    -        // Actually drop from memory store
    -        val droppedMemorySize =
    -          if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L
    -        val blockIsRemoved = memoryStore.remove(blockId)
    -        if (blockIsRemoved) {
    -          blockIsUpdated = true
    -        } else {
    -          logWarning(s"Block $blockId could not be dropped from memory as it does not exist")
    -        }
    +    // Actually drop from memory store
    +    val droppedMemorySize =
    +      if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L
    +    val blockIsRemoved = memoryStore.remove(blockId)
    +    if (blockIsRemoved) {
    +      blockIsUpdated = true
    +    } else {
    +      logWarning(s"Block $blockId could not be dropped from memory as it does not exist")
    --- End diff --
    
    I think that it still might be. I'd rather deal with this in a followup, since it's cleanup (the same logic existed in the old code prior to this patch).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53728024
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -805,13 +814,12 @@ private[spark] class BlockManager(
               }
             }
           } finally {
    -        // If we failed in putting the block to memory/disk, notify other possible readers
    -        // that it has failed, and then remove it from the block info map.
    -        if (!marked) {
    -          // Note that the remove must happen before markFailure otherwise another thread
    -          // could've inserted a new BlockInfo before we remove it.
    -          blockInfo.remove(blockId)
    -          putBlockInfo.markFailure()
    +        if (!blockWasSuccessfullyStored) {
    +          // Guard against the fact that MemoryStore might have already removed the block if the
    +          // put() failed and the block could not be dropped to disk.
    +          if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) {
    --- End diff --
    
    Ohhhh, I think I got it. Let's say you try to put a block `MEMORY_ONLY` but there is not enough space, in which case we will call `blockManager.dropFromMemory` on that block. This call also acquires the write lock and releases the lock in the end. However, the way this whole locking thing works is that if I acquire the write lock twice, a single call to `unlock` will release both locks, so if that happens we need to acquire the write lock again here in this line.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187689480
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51753/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53722288
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheManager.scala ---
    @@ -58,7 +67,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
             // If another thread already holds the lock, wait for it to finish return its results
             val storedValues = acquireLockForPartition[T](key)
             if (storedValues.isDefined) {
    -          return new InterruptibleIterator[T](context, storedValues.get)
    +          val iter =
    +            CompletionIterator[T, Iterator[T]](storedValues.get, blockManager.releaseLock(key))
    --- End diff --
    
    Yep, that's an accurate summary.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188560682
  
    **[Test build #51910 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51910/consoleFull)** for PR 10705 at commit [`9becde3`](https://github.com/apache/spark/commit/9becde3d94ea41bba9e275b9108c61b91074f035).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173428606
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188448922
  
    Jenkins, retest this please.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-171769630
  
    **[Test build #49411 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/49411/consoleFull)** for PR 10705 at commit [`7cad770`](https://github.com/apache/spark/commit/7cad770025a3ab4f3d2e312482ff7baf20a159cb).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174405789
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53888620
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,438 @@
    +/*
    + * 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.spark.storage
    +
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, SparkException, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  def size: Long = _size
    +  def size_=(s: Long): Unit = {
    +    _size = s
    +    checkInvariants()
    +  }
    +  private[this] var _size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  def readerCount: Int = _readerCount
    +  def readerCount_=(c: Int): Unit = {
    +    _readerCount = c
    +    checkInvariants()
    +  }
    +  private[this] var _readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or
    +   * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or
    +   * [[BlockInfo.NO_WRITER]] if this block is not locked for writing.
    +   */
    +  def writerTask: Long = _writerTask
    +  def writerTask_=(t: Long): Unit = {
    +    _writerTask = t
    +    checkInvariants()
    +  }
    +  private[this] var _writerTask: Long = 0
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  def removed: Boolean = _removed
    +  def removed_=(r: Boolean): Unit = {
    +    _removed = r
    +    checkInvariants()
    +  }
    +  private[this] var _removed: Boolean = false
    +
    +  private def checkInvariants(): Unit = {
    +    // A block's reader count must be non-negative:
    +    assert(_readerCount >= 0)
    +    // A block is either locked for reading or for writing, but not for both at the same time:
    +    assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER))
    --- End diff --
    
    Agreed; done.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188998834
  
    **[Test build #51985 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51985/consoleFull)** for PR 10705 at commit [`9becde3`](https://github.com/apache/spark/commit/9becde3d94ea41bba9e275b9108c61b91074f035).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53699158
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    --- End diff --
    
    I understand that `LoadingCache` is thread-safe, but isn't there a race condition here where:
    1. We get the read locks by this task
    2. Another thread of this task acquires a read lock for an either new or existing block
    3. We go ahead and invalidate this task, so we lose the lock info from (2)


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-178933463
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/50601/
    Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53747428
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    +      }
    +      if (info.removed) return None
    +      info.readerCount += 1
    +      readLocksByTask(currentTaskAttemptId).add(blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Lock a block for writing and return its metadata.
    +   *
    +   * If this is called by a task which already holds the block's exclusive write lock, then this
    +   * will return success but will not further increment any lock counts (so both write-lock
    +   * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for writing).
    +   */
    +  def lockForWriting(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      if (info.writerTask != currentTaskAttemptId) {
    +        while (info.writerTask != -1 || info.readerCount != 0) {
    +          if (info.removed) return None
    +          if (blocking) wait() else return None
    +        }
    +        if (info.removed) return None
    +      }
    +      info.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId")
    +      info
    +    }
    +  }
    +
    +  /**
    +   * Get a block's metadata without acquiring any locks.
    +   */
    +  def get(blockId: BlockId): Option[BlockInfo] = synchronized {
    +    infos.get(blockId)
    +  }
    +
    +  /**
    +   * Downgrades an exclusive write lock to a shared read lock.
    +   */
    +  def downgradeLock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId")
    +    val info = get(blockId).get
    +    require(info.writerTask == currentTaskAttemptId,
    +      s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold")
    +    unlock(blockId)
    +    lockForReading(blockId, blocking = false)
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Release a lock on the given block.
    +   */
    +  def unlock(blockId: BlockId): Unit = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId")
    +    val info = get(blockId).getOrElse {
    +      throw new IllegalStateException(s"Block $blockId not found")
    +    }
    +    if (info.writerTask != -1) {
    +      info.writerTask = -1
    +      writeLocksByTask.removeBinding(currentTaskAttemptId, blockId)
    +    } else {
    +      assert(info.readerCount > 0, s"Block $blockId is not locked for reading")
    +      info.readerCount -= 1
    +      val countsForTask = readLocksByTask.get(currentTaskAttemptId)
    +      val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1
    +      assert(newPinCountForTask >= 0,
    +        s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it")
    +    }
    +    notifyAll()
    +  }
    +
    +  /**
    +   * Atomically create metadata for a non-existent block.
    +   *
    +   * @param blockId the block id.
    +   * @param newBlockInfo the block info for the new block.
    +   * @return true if the block did not already exist, false otherwise. If this returns false, then
    +   *         no new locks are acquired. If this returns true, a write lock on the new block will
    +   *         be held.
    +   */
    +  def lockNewBlockForWriting(
    +      blockId: BlockId,
    +      newBlockInfo: BlockInfo): Boolean = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to put $blockId")
    +    val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo)
    +    if (actualInfo eq newBlockInfo) {
    +      actualInfo.writerTask = currentTaskAttemptId
    +      writeLocksByTask.addBinding(currentTaskAttemptId, blockId)
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Release all lock held by the given task, clearing that task's pin bookkeeping
    +   * structures and updating the global pin counts. This method should be called at the
    +   * end of a task (either by a task completion handler or in `TaskRunner.run()`).
    +   *
    +   * @return the ids of blocks whose pins were released
    +   */
    +  def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = {
    +    val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]()
    +    synchronized {
    +      writeLocksByTask.remove(taskAttemptId).foreach { locks =>
    +        for (blockId <- locks) {
    +          infos.get(blockId).foreach { info =>
    +            assert(info.writerTask == taskAttemptId)
    +            info.writerTask = -1
    +          }
    +          blocksWithReleasedLocks += blockId
    +        }
    +      }
    +      notifyAll()
    +    }
    +    val readLocks = readLocksByTask.get(taskAttemptId)
    +    readLocksByTask.invalidate(taskAttemptId)
    --- End diff --
    
    The assumption here is that this task's threads have finished executing, so it will not attempt to acquire any additional locks after this method has been called. If this assumption were not to hold, then we'd need to guard against new locks occurring after `unlockAllLocksForTask` returns. I think we might be able to handle this by requiring an explicit `registerTask()` to be called in order to initialize the map entries in `readLocksByTask` and `writeLocksByTask`; that would let us detect and reject invalid lock calls that occur after task completion. I'll take a shot at this tomorrow. 


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-173431237
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/49843/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r52832524
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,335 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +}
    +
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[putAndLockForWritingIfAbsent()]]) and are removed
    +   * by [[remove()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def getAndLockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    +        if (info.removed) return None
    +        if (blocking) wait() else return None
    --- End diff --
    
    Good catch. Although the current code can't hit the issue that you describe, it's possible that future refactorings or changes could introduce a bug here, so I'll add extra defensive checks around `removed` after we exit the `while` loop.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53721241
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheManager.scala ---
    @@ -136,7 +147,13 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
            */
           blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel)
           blockManager.get(key) match {
    -        case Some(v) => v.data.asInstanceOf[Iterator[T]]
    +        case Some(v) =>
    --- End diff --
    
    Yep, that's right. Any lock acquired in a task and not explicitly released by that task will be auto-released upon task completion or failure.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-183554174
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53728196
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.spark.storage
    +
    +import java.nio.ByteBuffer
    +
    +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
    +
    +
    +private[storage] class BlockManagerManagedBuffer(
    --- End diff --
    
    can you add a java doc here to explain why this is needed


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53709597
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/MemoryStore.scala ---
    @@ -418,9 +423,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo
             while (freedMemory < space && iterator.hasNext) {
               val pair = iterator.next()
               val blockId = pair.getKey
    -          if (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) {
    -            selectedBlocks += blockId
    -            freedMemory += pair.getValue.size
    +          if (blockIsEvictable(blockId)) {
    +            if (blockManager.blockInfoManager.lockForWriting(blockId, false).isDefined) {
    --- End diff --
    
    name boolean parameter, also add a comment to explain why we don't want to block here.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-187967925
  
    **[Test build #51806 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51806/consoleFull)** for PR 10705 at commit [`697eba2`](https://github.com/apache/spark/commit/697eba213c11bc92724cfd891f5b86acd60dfd4f).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-185546066
  
    Merged build finished. Test PASSed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-174668709
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/50004/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184021972
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51284/
    Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53875856
  
    --- Diff: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala ---
    @@ -65,7 +65,11 @@ class NettyBlockRpcServer(
             val level: StorageLevel =
               serializer.newInstance().deserialize(ByteBuffer.wrap(uploadBlock.metadata))
             val data = new NioManagedBuffer(ByteBuffer.wrap(uploadBlock.blockData))
    -        blockManager.putBlockData(BlockId(uploadBlock.blockId), data, level)
    +        val blockId = BlockId(uploadBlock.blockId)
    +        val putSucceeded = blockManager.putBlockData(blockId, data, level)
    --- End diff --
    
    You seem to do this in a bunch of places, perhaps consider the switched semantics that put does not retain the lock by default. Not sure if this is better, something to think about for later.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53724680
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -597,6 +586,9 @@ private[spark] class BlockManager(
     
       /**
        * Get a block from the block manager (either local or remote).
    +   *
    +   * This acquires a read lock on the block if the block was stored locally and does not acquire
    +   * any locks if the block was fetched from a remote block manager.
    --- End diff --
    
    we should expand on this comment that the caller may check whether the `readMethod` of the `BlockResult` is network, and if so, do not release the lock (i.e. what we do in `CacheManager`)


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53744323
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    --- End diff --
    
    Done.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by nongli <gi...@git.apache.org>.
Github user nongli commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53688589
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockManager.scala ---
    @@ -805,13 +814,12 @@ private[spark] class BlockManager(
               }
             }
           } finally {
    -        // If we failed in putting the block to memory/disk, notify other possible readers
    -        // that it has failed, and then remove it from the block info map.
    -        if (!marked) {
    -          // Note that the remove must happen before markFailure otherwise another thread
    -          // could've inserted a new BlockInfo before we remove it.
    -          blockInfo.remove(blockId)
    -          putBlockInfo.markFailure()
    +        if (!blockWasSuccessfullyStored) {
    +          // Guard against the fact that MemoryStore might have already removed the block if the
    +          // put() failed and the block could not be dropped to disk.
    +          if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) {
    --- End diff --
    
    How can you fail to lock for writing here? The lock was acquired on line 722 right?


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184884781
  
    **[Test build #51384 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51384/consoleFull)** for PR 10705 at commit [`0c08731`](https://github.com/apache/spark/commit/0c08731890f1a34606b95ffd31d063fc931b4e21).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-188351914
  
    **[Test build #51883 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51883/consoleFull)** for PR 10705 at commit [`9becde3`](https://github.com/apache/spark/commit/9becde3d94ea41bba9e275b9108c61b91074f035).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-170711333
  
    Merged build finished. Test FAILed.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-178932711
  
    **[Test build #50601 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50601/consoleFull)** for PR 10705 at commit [`76fc9f5`](https://github.com/apache/spark/commit/76fc9f5580b384d6f9fc990078f7f080bf298ada).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757][WIP] Use reference counting to p...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10705#issuecomment-184076622
  
    **[Test build #51287 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51287/consoleFull)** for PR 10705 at commit [`f0b6d71`](https://github.com/apache/spark/commit/f0b6d711b29e7e7b71a954860a66df27c26380b1).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12757] Add block-level read/write locks...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10705#discussion_r53684951
  
    --- Diff: core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
    @@ -0,0 +1,356 @@
    +/*
    + * 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.spark.storage
    +
    +import java.lang
    +import javax.annotation.concurrent.GuardedBy
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
    +import com.google.common.collect.ConcurrentHashMultiset
    +
    +import org.apache.spark.{Logging, TaskContext}
    +
    +
    +/**
    + * Tracks metadata for an individual block.
    + *
    + * @param level the block's storage level. This is the requested persistence level, not the
    + *              effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this
    + *              does not imply that the block is actually resident in memory).
    + * @param tellMaster whether state changes for this block should be reported to the master. This
    + *                   is true for most blocks, but is false for broadcast blocks.
    + */
    +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
    +
    +  /**
    +   * The size of the block (in bytes)
    +   */
    +  var size: Long = 0
    +
    +  /**
    +   * The number of times that this block has been locked for reading.
    +   */
    +  var readerCount: Int = 0
    +
    +  /**
    +   * The task attempt id of the task which currently holds the write lock for this block, or -1
    +   * if this block is not locked for writing.
    +   */
    +  var writerTask: Long = -1
    +
    +  // Invariants:
    +  //     (writerTask != -1) implies (readerCount == 0)
    +  //     (readerCount != 0) implies (writerTask == -1)
    +  // TODO: add assertions around every method
    +
    +  /**
    +   * True if this block has been removed from the BlockManager and false otherwise.
    +   * This field is used to communicate block deletion to blocked readers / writers (see its usage
    +   * in [[BlockInfoManager]]).
    +   */
    +  var removed: Boolean = false
    +
    +  // TODO: Add timestamps on lock acquisitions
    +}
    +// In debugging mode, check that locks haven't been held for too long.
    +// Every few minutes, dump debug info.
    +
    +/**
    + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking.
    + *
    + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is
    + * automatically associated with a running task and locks are automatically released upon task
    + * completion or failure.
    + *
    + * This class is thread-safe.
    + */
    +private[storage] class BlockInfoManager extends Logging {
    +
    +  private type TaskAttemptId = Long
    +
    +  /**
    +   * Used to look up metadata for individual blocks. Entries are added to this map via an atomic
    +   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed
    +   * by [[removeBlock()]].
    +   */
    +  @GuardedBy("this")
    +  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for writing.
    +   */
    +  @GuardedBy("this")
    +  private[this] val writeLocksByTask =
    +    new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
    +      with mutable.MultiMap[TaskAttemptId, BlockId]
    +
    +  /**
    +   * Tracks the set of blocks that each task has locked for reading, along with the number of times
    +   * that a block has been locked (since our read locks are re-entrant). This is thread-safe.
    +   */
    +  private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = {
    +    // We need to explicitly box as java.lang.Long to avoid a type mismatch error:
    +    val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] {
    +      override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]()
    +    }
    +    CacheBuilder.newBuilder().build(loader)
    +  }
    +
    +  // ----------------------------------------------------------------------------------------------
    +
    +  /**
    +   * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024
    +   * if called outside of a task (-1024 was chosen because it's different than the -1 which is used
    +   * in [[BlockInfo.writerTask]] to denote the absence of a write lock).
    +   */
    +  private def currentTaskAttemptId: TaskAttemptId = {
    +    // TODO(josh): assert that this only happens on the driver?
    +    // What about block transfer / getRemote()?
    +    Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L)
    +  }
    +
    +  /**
    +   * Lock a block for reading and return its metadata.
    +   *
    +   * A single task can lock a block multiple times for reading, in which case each lock will need
    +   * to be released separately.
    +   *
    +   * @param blockId the block to lock.
    +   * @param blocking if true (default), this call will block until the lock is acquired. If false,
    +   *                 this call will return immediately if the lock acquisition fails.
    +   * @return None if the block did not exist or was removed (in which case no lock is held), or
    +   *         Some(BlockInfo) (in which case the block is locked for reading).
    +   */
    +  def lockForReading(
    +      blockId: BlockId,
    +      blocking: Boolean = true): Option[BlockInfo] = synchronized {
    +    logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId")
    +    infos.get(blockId).map { info =>
    +      while (info.writerTask != -1) {
    --- End diff --
    
    can you make these `-1` and `-1024` constants somewhere, like `val NO_WRITER = -1` or something


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org