aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/spark/storage/BlockManager.scala
blob: 3b9cd8b5fc1bc948de63031176b3f8c1d19b107a (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
package spark.storage

import akka.dispatch.{Await, Future}
import akka.util.Duration

import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream

import java.io.{InputStream, OutputStream, Externalizable, ObjectInput, ObjectOutput}
import java.nio.{MappedByteBuffer, ByteBuffer}
import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue}

import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue}
import scala.collection.JavaConversions._

import spark.{CacheTracker, Logging, Serializer, SizeEstimator, SparkException, Utils}
import spark.network._
import spark.util.ByteBufferInputStream
import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
import sun.nio.ch.DirectBuffer


class BlockManagerId(var ip: String, var port: Int) extends Externalizable {
  def this() = this(null, 0)

  override def writeExternal(out: ObjectOutput) {
    out.writeUTF(ip)
    out.writeInt(port)
  }

  override def readExternal(in: ObjectInput) {
    ip = in.readUTF()
    port = in.readInt()
  }

  override def toString = "BlockManagerId(" + ip + ", " + port + ")"

  override def hashCode = ip.hashCode * 41 + port

  override def equals(that: Any) = that match {
    case id: BlockManagerId => port == id.port && ip == id.ip
    case _ => false
  }
}


case class BlockException(blockId: String, message: String, ex: Exception = null)
extends Exception(message)


class BlockLocker(numLockers: Int) {
  private val hashLocker = Array.fill(numLockers)(new Object())

  def getLock(blockId: String): Object = {
    return hashLocker(math.abs(blockId.hashCode % numLockers))
  }
}


class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, maxMemory: Long)
  extends Logging {

  class BlockInfo(val level: StorageLevel, val tellMaster: Boolean, var pending: Boolean = true) {
    def waitForReady() {
      if (pending) {
        synchronized {
          while (pending) this.wait()
        }
      }
    }

    def markReady() {
      pending = false
      synchronized {
        this.notifyAll()
      }
    }
  }

  private val NUM_LOCKS = 337
  private val locker = new BlockLocker(NUM_LOCKS)

  private val blockInfo = new ConcurrentHashMap[String, BlockInfo]()

  private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory)
  private[storage] val diskStore: BlockStore =
    new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")))

  val connectionManager = new ConnectionManager(0)
  implicit val futureExecContext = connectionManager.futureExecContext

  val connectionManagerId = connectionManager.id
  val blockManagerId = new BlockManagerId(connectionManagerId.host, connectionManagerId.port)

  // TODO: This will be removed after cacheTracker is removed from the code base.
  var cacheTracker: CacheTracker = null

  val numParallelFetches = BlockManager.getNumParallelFetchesFromSystemProperties
  val compress = System.getProperty("spark.blockManager.compress", "false").toBoolean

  initialize()

  /**
   * Construct a BlockManager with a memory limit set based on system properties.
   */
  def this(master: BlockManagerMaster, serializer: Serializer) = {
    this(master, serializer, BlockManager.getMaxMemoryFromSystemProperties)
  }

  /**
   * Initialize the BlockManager. Register to the BlockManagerMaster, and start the
   * BlockManagerWorker actor.
   */
  private def initialize() {
    master.mustRegisterBlockManager(
      RegisterBlockManager(blockManagerId, maxMemory))
    BlockManagerWorker.startBlockManagerWorker(this)
  }

  /**
   * Get storage level of local block. If no info exists for the block, then returns null.
   */
  def getLevel(blockId: String): StorageLevel = {
    val info = blockInfo.get(blockId)
    if (info != null) info.level else null
  }

  /**
   * Tell the master about the current storage status of a block. This will send a heartbeat
   * message reflecting the current status, *not* the desired storage level in its block info.
   * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk.
   */
  def reportBlockStatus(blockId: String) {
    locker.getLock(blockId).synchronized {
      val curLevel = blockInfo.get(blockId) match {
        case null =>
          StorageLevel.NONE
        case info =>
          info.level match {
            case null =>
              StorageLevel.NONE
            case level =>
              val inMem = level.useMemory && memoryStore.contains(blockId)
              val onDisk = level.useDisk && diskStore.contains(blockId)
              new StorageLevel(onDisk, inMem, level.deserialized, level.replication)
          }
      }
      master.mustHeartBeat(HeartBeat(
        blockManagerId,
        blockId,
        curLevel,
        if (curLevel.useMemory) memoryStore.getSize(blockId) else 0L,
        if (curLevel.useDisk) diskStore.getSize(blockId) else 0L))
      logDebug("Told master about block " + blockId)
    }
  }

  /**
   * Get locations of the block.
   */
  def getLocations(blockId: String): Seq[String] = {
    val startTimeMs = System.currentTimeMillis
    var managers = master.mustGetLocations(GetLocations(blockId))
    val locations = managers.map(_.ip)
    logDebug("Get block locations in " + Utils.getUsedTimeMs(startTimeMs))
    return locations
  }

  /**
   * Get locations of an array of blocks.
   */
  def getLocations(blockIds: Array[String]): Array[Seq[String]] = {
    val startTimeMs = System.currentTimeMillis
    val locations = master.mustGetLocationsMultipleBlockIds(
      GetLocationsMultipleBlockIds(blockIds)).map(_.map(_.ip).toSeq).toArray
    logDebug("Get multiple block location in " + Utils.getUsedTimeMs(startTimeMs))
    return locations
  }

  /**
   * Get block from local block manager.
   */
  def getLocal(blockId: String): Option[Iterator[Any]] = {
    logDebug("Getting local block " + blockId)
    locker.getLock(blockId).synchronized {
      val info = blockInfo.get(blockId)
      if (info != null) {
        info.waitForReady() // In case the block is still being put() by another thread
        val level = info.level
        logDebug("Level for block " + blockId + " is " + level)

        // Look for the block in memory
        if (level.useMemory) {
          logDebug("Getting block " + blockId + " from memory")
          memoryStore.getValues(blockId) match {
            case Some(iterator) =>
              logDebug("Block " + blockId + " found in memory")
              return Some(iterator)
            case None =>
              logDebug("Block " + blockId + " not found in memory")
          }
        }

        // Look for block on disk
        if (level.useDisk) {
          logDebug("Getting block " + blockId + " from disk")
          diskStore.getValues(blockId) match {
            case Some(iterator) =>
              logDebug("Block " + blockId + " found in disk")
              return Some(iterator)
            case None =>
              throw new Exception("Block " + blockId + " not found on disk, though it should be")
              return None
          }
        }
      } else {
        logDebug("Block " + blockId + " not registered locally")
      }
    }
    return None
  }

  /**
   * Get block from the local block manager as serialized bytes.
   */
  def getLocalBytes(blockId: String): Option[ByteBuffer] = {
    logDebug("Getting local block " + blockId + " as bytes")
    locker.getLock(blockId).synchronized {
      val info = blockInfo.get(blockId)
      if (info != null) {
        info.waitForReady() // In case the block is still being put() by another thread
        val level = info.level
        logDebug("Level for block " + blockId + " is " + level + " on local machine")

        // Look for the block in memory
        if (level.useMemory) {
          logDebug("Getting block " + blockId + " from memory")
          memoryStore.getBytes(blockId) match {
            case Some(bytes) =>
              logDebug("Block " + blockId + " found in memory")
              return Some(bytes)
            case None =>
              logDebug("Block " + blockId + " not found in memory")
          }
        }

        // Look for block on disk
        if (level.useDisk) {
          logDebug("Getting block " + blockId + " from disk")
          diskStore.getBytes(blockId) match {
            case Some(bytes) =>
              logDebug("Block " + blockId + " found in disk")
              return Some(bytes)
            case None =>
              throw new Exception("Block " + blockId + " not found on disk, though it should be")
              return None
          }
        }
      } else {
        logDebug("Block " + blockId + " not registered locally")
      }
    }
    return None
  }

  /**
   * Get block from remote block managers.
   */
  def getRemote(blockId: String): Option[Iterator[Any]] = {
    if (blockId == null) {
      throw new IllegalArgumentException("Block Id is null")
    }
    logDebug("Getting remote block " + blockId)
    // Get locations of block
    val locations = master.mustGetLocations(GetLocations(blockId))

    // Get block from remote locations
    for (loc <- locations) {
      logDebug("Getting remote block " + blockId + " from " + loc)
      val data = BlockManagerWorker.syncGetBlock(
          GetBlock(blockId), ConnectionManagerId(loc.ip, loc.port))
      if (data != null) {
        logDebug("Data is not null: " + data)
        return Some(dataDeserialize(data))
      }
      logDebug("Data is null")
    }
    logDebug("Data not found")
    return None
  }

  /**
   * Get a block from the block manager (either local or remote).
   */
  def get(blockId: String): Option[Iterator[Any]] = {
    getLocal(blockId).orElse(getRemote(blockId))
  }

  /**
   * Get multiple blocks from local and remote block manager using their BlockManagerIds. Returns
   * an Iterator of (block ID, value) pairs so that clients may handle blocks in a pipelined
   * fashion as they're received.
   */
  def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[String])])
      : Iterator[(String, Option[Iterator[Any]])] = {

    if (blocksByAddress == null) {
      throw new IllegalArgumentException("BlocksByAddress is null")
    }
    val totalBlocks = blocksByAddress.map(_._2.size).sum
    logDebug("Getting " + totalBlocks + " blocks")
    var startTime = System.currentTimeMillis
    val localBlockIds = new ArrayBuffer[String]()
    val remoteBlockIds = new HashSet[String]()

    // A queue to hold our results. Because we want all the deserializing the happen in the
    // caller's thread, this will actually hold functions to produce the Iterator for each block.
    // For local blocks we'll have an iterator already, while for remote ones we'll deserialize.
    val results = new LinkedBlockingQueue[(String, Option[() => Iterator[Any]])]

    // Bound the number and memory usage of fetched remote blocks.
    val blocksToRequest = new Queue[(BlockManagerId, BlockMessage)]

    def sendRequest(bmId: BlockManagerId, blockMessages: Seq[BlockMessage]) {
      val cmId = new ConnectionManagerId(bmId.ip, bmId.port)
      val blockMessageArray = new BlockMessageArray(blockMessages)
      val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage)
      future.onSuccess {
        case Some(message) => {
          val bufferMessage = message.asInstanceOf[BufferMessage]
          val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage)
          for (blockMessage <- blockMessageArray) {
            if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) {
              throw new SparkException(
                "Unexpected message " + blockMessage.getType + " received from " + cmId)
            }
            val blockId = blockMessage.getId
            results.put((blockId, Some(() => dataDeserialize(blockMessage.getData))))
            logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
          }
        }
        case None => {
          logError("Could not get block(s) from " + cmId)
          for (blockMessage <- blockMessages) {
            results.put((blockMessage.getId, None))
          }
        }
      }
    }

    // Split local and remote blocks. Remote blocks are further split into ones that will
    // be requested initially and ones that will be added to a queue of blocks to request.
    val initialRequestBlocks = new HashMap[BlockManagerId, ArrayBuffer[BlockMessage]]()
    var initialRequests = 0
    val blocksToGetLater = new ArrayBuffer[(BlockManagerId, BlockMessage)]
    for ((address, blockIds) <- Utils.randomize(blocksByAddress)) {
      if (address == blockManagerId) {
        localBlockIds ++= blockIds
      } else {
        remoteBlockIds ++= blockIds
        for (blockId <- blockIds) {
          val blockMessage = BlockMessage.fromGetBlock(GetBlock(blockId))
          if (initialRequests < numParallelFetches) {
            initialRequestBlocks.getOrElseUpdate(address, new ArrayBuffer[BlockMessage])
              .append(blockMessage)
            initialRequests += 1
          } else {
            blocksToGetLater.append((address, blockMessage))
          }
        }
      }
    }
    // Add the remaining blocks into a queue to pull later in a random order
    blocksToRequest ++= Utils.randomize(blocksToGetLater)

    // Send out initial request(s) for 'numParallelFetches' blocks.
    for ((bmId, blockMessages) <- initialRequestBlocks) {
      sendRequest(bmId, blockMessages)
    }

    logDebug("Started remote gets for " + numParallelFetches + " blocks in " +
      Utils.getUsedTimeMs(startTime) + " ms")

    // Get the local blocks while remote blocks are being fetched.
    startTime = System.currentTimeMillis
    for (id <- localBlockIds) {
      getLocal(id) match {
        case Some(block) => {
          results.put((id, Some(() => block)))
          logDebug("Got local block " + id)
        }
        case None => {
          throw new BlockException(id, "Could not get block " + id + " from local machine")
        }
      }
    }
    logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms")

    // Return an iterator that will read fetched blocks off the queue as they arrive.
    return new Iterator[(String, Option[Iterator[Any]])] {
      var resultsGotten = 0

      def hasNext: Boolean = resultsGotten < totalBlocks

      def next(): (String, Option[Iterator[Any]]) = {
        resultsGotten += 1
        val (blockId, functionOption) = results.take()
        if (remoteBlockIds.contains(blockId) && !blocksToRequest.isEmpty) {
          val (bmId, blockMessage) = blocksToRequest.dequeue()
          sendRequest(bmId, Seq(blockMessage))
        }
        (blockId, functionOption.map(_.apply()))
      }
    }
  }

  /**
   * Put a new block of values to the block manager.
   */
  def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean = true) {
    if (blockId == null) {
      throw new IllegalArgumentException("Block Id is null")
    }
    if (values == null) {
      throw new IllegalArgumentException("Values is null")
    }
    if (level == null || !level.isValid) {
      throw new IllegalArgumentException("Storage level is null or invalid")
    }

    if (blockInfo.containsKey(blockId)) {
      logWarning("Block " + blockId + " already exists on this machine; not re-adding it")
      return
    }

    // Remember the block's storage level so that we can correctly drop it to disk if it needs
    // to be dropped right after it got put into memory. Note, however, that other threads will
    // not be able to get() this block until we call markReady on its BlockInfo.
    val myInfo = new BlockInfo(level, tellMaster)
    blockInfo.put(blockId, myInfo)

    val startTimeMs = System.currentTimeMillis
    var bytes: ByteBuffer = null

    // If we need to replicate the data, we'll want access to the values, but because our
    // put will read the whole iterator, there will be no values left. For the case where
    // the put serializes data, we'll remember the bytes, above; but for the case where
    // it doesn't, such as MEMORY_ONLY_DESER, let's rely on the put returning an Iterator.
    var valuesAfterPut: Iterator[Any] = null

    locker.getLock(blockId).synchronized {
      logDebug("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)
        + " to get into synchronized block")

      if (level.useMemory) {
        // Save it just to memory first, even if it also has useDisk set to true; we will later
        // drop it to disk if the memory store can't hold it.
        memoryStore.putValues(blockId, values, level, true) match {
          case Right(newBytes) => bytes = newBytes
          case Left(newIterator) => valuesAfterPut = newIterator
        }
      } else {
        // Save directly to disk.
        val askForBytes = level.replication > 1 // Don't get back the bytes unless we replicate them
        diskStore.putValues(blockId, values, level, askForBytes) match {
          case Right(newBytes) => bytes = newBytes
          case _ =>
        }
      }

      // Now that the block is in either the memory or disk store, let other threads read it,
      // and tell the master about it.
      myInfo.markReady()
      if (tellMaster) {
        reportBlockStatus(blockId)
      }
    }
    logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs))

    // Replicate block if required
    if (level.replication > 1) {
      // Serialize the block if not already done
      if (bytes == null) {
        if (valuesAfterPut == null) {
          throw new SparkException(
            "Underlying put returned neither an Iterator nor bytes! This shouldn't happen.")
        }
        bytes = dataSerialize(valuesAfterPut)
      }
      replicate(blockId, bytes, level)
    }

    BlockManager.dispose(bytes)

    // TODO: This code will be removed when CacheTracker is gone.
    if (blockId.startsWith("rdd")) {
      notifyTheCacheTracker(blockId)
    }
    logDebug("Put block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs))
  }


  /**
   * Put a new block of serialized bytes to the block manager.
   */
  def putBytes(
    blockId: String, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) {

    if (blockId == null) {
      throw new IllegalArgumentException("Block Id is null")
    }
    if (bytes == null) {
      throw new IllegalArgumentException("Bytes is null")
    }
    if (level == null || !level.isValid) {
      throw new IllegalArgumentException("Storage level is null or invalid")
    }

    if (blockInfo.containsKey(blockId)) {
      logWarning("Block " + blockId + " already exists on this machine; not re-adding it")
      return
    }

    // Remember the block's storage level so that we can correctly drop it to disk if it needs
    // to be dropped right after it got put into memory. Note, however, that other threads will
    // not be able to get() this block until we call markReady on its BlockInfo.
    val myInfo = new BlockInfo(level, tellMaster)
    blockInfo.put(blockId, myInfo)

    val startTimeMs = System.currentTimeMillis

    // Initiate the replication before storing it locally. This is faster as
    // data is already serialized and ready for sending
    val replicationFuture = if (level.replication > 1) {
      val bufferView = bytes.duplicate() // Doesn't copy the bytes, just creates a wrapper
      Future {
        replicate(blockId, bufferView, level)
      }
    } else {
      null
    }

    locker.getLock(blockId).synchronized {
      logDebug("PutBytes for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)
        + " to get into synchronized block")

      if (level.useMemory) {
        // Store it only in memory at first, even if useDisk is also set to true
        bytes.rewind()
        memoryStore.putBytes(blockId, bytes, level)
      } else {
        bytes.rewind()
        diskStore.putBytes(blockId, bytes, level)
      }

      // Now that the block is in either the memory or disk store, let other threads read it,
      // and tell the master about it.
      myInfo.markReady()
      if (tellMaster) {
        reportBlockStatus(blockId)
      }
    }

    // TODO: This code will be removed when CacheTracker is gone.
    if (blockId.startsWith("rdd")) {
      notifyTheCacheTracker(blockId)
    }

    // If replication had started, then wait for it to finish
    if (level.replication > 1) {
      if (replicationFuture == null) {
        throw new Exception("Unexpected")
      }
      Await.ready(replicationFuture, Duration.Inf)
    }

    if (level.replication > 1) {
      logDebug("PutBytes for block " + blockId + " with replication took " +
        Utils.getUsedTimeMs(startTimeMs))
    } else {
      logDebug("PutBytes for block " + blockId + " without replication took " +
        Utils.getUsedTimeMs(startTimeMs))
    }
  }

  /**
   * Replicate block to another node.
   */
  var cachedPeers: Seq[BlockManagerId] = null
  private def replicate(blockId: String, data: ByteBuffer, level: StorageLevel) {
    val tLevel: StorageLevel =
      new StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1)
    if (cachedPeers == null) {
      cachedPeers = master.mustGetPeers(GetPeers(blockManagerId, level.replication - 1))
    }
    for (peer: BlockManagerId <- cachedPeers) {
      val start = System.nanoTime
      data.rewind()
      logDebug("Try to replicate BlockId " + blockId + " once; The size of the data is "
        + data.limit() + " Bytes. To node: " + peer)
      if (!BlockManagerWorker.syncPutBlock(PutBlock(blockId, data, tLevel),
        new ConnectionManagerId(peer.ip, peer.port))) {
        logError("Failed to call syncPutBlock to " + peer)
      }
      logDebug("Replicated BlockId " + blockId + " once used " +
        (System.nanoTime - start) / 1e6 + " s; The size of the data is " +
        data.limit() + " bytes.")
    }
  }

  // TODO: This code will be removed when CacheTracker is gone.
  private def notifyTheCacheTracker(key: String) {
    val rddInfo = key.split("_")
    val rddId: Int = rddInfo(1).toInt
    val splitIndex: Int = rddInfo(2).toInt
    val host = System.getProperty("spark.hostname", Utils.localHostName())
    cacheTracker.notifyTheCacheTrackerFromBlockManager(spark.AddedToCache(rddId, splitIndex, host))
  }

  /**
   * Read a block consisting of a single object.
   */
  def getSingle(blockId: String): Option[Any] = {
    get(blockId).map(_.next())
  }

  /**
   * Write a block consisting of a single object.
   */
  def putSingle(blockId: String, value: Any, level: StorageLevel, tellMaster: Boolean = true) {
    put(blockId, Iterator(value), level, tellMaster)
  }

  /**
   * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory
   * store reaches its limit and needs to free up space.
   */
  def dropFromMemory(blockId: String, data: Either[Iterator[_], ByteBuffer]) {
    logInfo("Dropping block " + blockId + " from memory")
    locker.getLock(blockId).synchronized {
      val info = blockInfo.get(blockId)
      val level = info.level
      if (level.useDisk && !diskStore.contains(blockId)) {
        logInfo("Writing block " + blockId + " to disk")
        data match {
          case Left(iterator) =>
            diskStore.putValues(blockId, iterator, level, false)
          case Right(bytes) =>
            diskStore.putBytes(blockId, bytes, level)
        }
      }
      memoryStore.remove(blockId)
      if (info.tellMaster) {
        reportBlockStatus(blockId)
      }
      if (!level.useDisk) {
        // The block is completely gone from this node; forget it so we can put() it again later.
        blockInfo.remove(blockId)
      }
    }
  }

  /**
   * Wrap an output stream for compression if block compression is enabled
   */
  def wrapForCompression(s: OutputStream): OutputStream = {
    if (compress) new LZFOutputStream(s) else s
  }

  /**
   * Wrap an input stream for compression if block compression is enabled
   */
  def wrapForCompression(s: InputStream): InputStream = {
    if (compress) new LZFInputStream(s) else s
  }

  def dataSerialize(values: Iterator[Any]): ByteBuffer = {
    val byteStream = new FastByteArrayOutputStream(4096)
    val ser = serializer.newInstance()
    ser.serializeStream(wrapForCompression(byteStream)).writeAll(values).close()
    byteStream.trim()
    ByteBuffer.wrap(byteStream.array)
  }

  /**
   * Deserializes a ByteBuffer into an iterator of values and disposes of it when the end of
   * the iterator is reached.
   */
  def dataDeserialize(bytes: ByteBuffer): Iterator[Any] = {
    bytes.rewind()
    val ser = serializer.newInstance()
    ser.deserializeStream(wrapForCompression(new ByteBufferInputStream(bytes, true))).asIterator
  }

  def stop() {
    connectionManager.stop()
    blockInfo.clear()
    memoryStore.clear()
    diskStore.clear()
    logInfo("BlockManager stopped")
  }
}

private[spark]
object BlockManager extends Logging {
  def getNumParallelFetchesFromSystemProperties: Int = {
    System.getProperty("spark.blockManager.parallelFetches", "4").toInt
  }

  def getMaxMemoryFromSystemProperties: Long = {
    val memoryFraction = System.getProperty("spark.storage.memoryFraction", "0.66").toDouble
    (Runtime.getRuntime.maxMemory * memoryFraction).toLong
  }

  /**
   * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that
   * might cause errors if one attempts to read from the unmapped buffer, but it's better than
   * waiting for the GC to find it because that could lead to huge numbers of open files. There's
   * unfortunately no standard API to do this.
   */
  def dispose(buffer: ByteBuffer) {
    if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) {
      logDebug("Unmapping " + buffer)
      if (buffer.asInstanceOf[DirectBuffer].cleaner() != null) {
        buffer.asInstanceOf[DirectBuffer].cleaner().clean()
      }
    }
  }
}