Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-4715

ShuffleMemoryManager.tryToAcquire may return a negative value

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 1.1.0
    • 1.1.2, 1.2.0
    • Spark Core
    • None

    Description

      Here is a unit test to demonstrate it:

        test("threads should not be granted a negative size") {
          val manager = new ShuffleMemoryManager(1000L)
          manager.tryToAcquire(700L)
      
          val latch = new CountDownLatch(1)
          startThread("t1") {
             manager.tryToAcquire(300L)
            latch.countDown()
          }
          latch.await() // Wait until `t1` calls `tryToAcquire`
      
          val granted = manager.tryToAcquire(300L)
          assert(0 === granted, "granted is negative")
        }
      

      It outputs "0 did not equal -200 granted is negative"

      Attachments

        Activity

          People

            zsxwing Shixiong Zhu
            zsxwing Shixiong Zhu
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: