aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala/spark/CacheTrackerSuite.scala
blob: 3d170a6e22ef0cec8544454e5622d4432cb0c78c (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
package spark

import org.scalatest.FunSuite

import scala.collection.mutable.HashMap

import akka.actor._
import akka.actor.Actor
import akka.actor.Actor._

class CacheTrackerSuite extends FunSuite {

  test("CacheTrackerActor slave initialization & cache status") {
    //System.setProperty("spark.master.port", "1345")
    val initialSize = 2L << 20

    val tracker = actorOf(new CacheTrackerActor)
    tracker.start()

    tracker !! SlaveCacheStarted("host001", initialSize)

    assert((tracker ? GetCacheStatus).get === Seq(("host001", 2097152L, 0L)))

    tracker !! StopCacheTracker
  }

  test("RegisterRDD") {
    //System.setProperty("spark.master.port", "1345")
    val initialSize = 2L << 20

    val tracker = actorOf(new CacheTrackerActor)
    tracker.start()

    tracker !! SlaveCacheStarted("host001", initialSize)

    tracker !! RegisterRDD(1, 3)
    tracker !! RegisterRDD(2, 1)

    assert(getCacheLocations(tracker) === Map(1 -> List(List(), List(), List()), 2 -> List(List())))

    tracker !! StopCacheTracker
  }

  test("AddedToCache") {
    //System.setProperty("spark.master.port", "1345")
    val initialSize = 2L << 20

    val tracker = actorOf(new CacheTrackerActor)
    tracker.start()

    tracker !! SlaveCacheStarted("host001", initialSize)

    tracker !! RegisterRDD(1, 2)
    tracker !! RegisterRDD(2, 1)

    tracker !! AddedToCache(1, 0, "host001", 2L << 15)
    tracker !! AddedToCache(1, 1, "host001", 2L << 11)
    tracker !! AddedToCache(2, 0, "host001", 3L << 10)

    assert((tracker ? GetCacheStatus).get === Seq(("host001", 2097152L, 72704L)))

    assert(getCacheLocations(tracker) === 
      Map(1 -> List(List("host001"), List("host001")), 2 -> List(List("host001"))))

    tracker !! StopCacheTracker
  }

  test("DroppedFromCache") {
    //System.setProperty("spark.master.port", "1345")
    val initialSize = 2L << 20

    val tracker = actorOf(new CacheTrackerActor)
    tracker.start()

    tracker !! SlaveCacheStarted("host001", initialSize)

    tracker !! RegisterRDD(1, 2)
    tracker !! RegisterRDD(2, 1)

    tracker !! AddedToCache(1, 0, "host001", 2L << 15)
    tracker !! AddedToCache(1, 1, "host001", 2L << 11)
    tracker !! AddedToCache(2, 0, "host001", 3L << 10)

    assert((tracker ? GetCacheStatus).get === Seq(("host001", 2097152L, 72704L)))
    assert(getCacheLocations(tracker) ===
      Map(1 -> List(List("host001"), List("host001")), 2 -> List(List("host001"))))

    tracker !! DroppedFromCache(1, 1, "host001", 2L << 11)

    assert((tracker ? GetCacheStatus).get === Seq(("host001", 2097152L, 68608L)))
    assert(getCacheLocations(tracker) ===
      Map(1 -> List(List("host001"),List()), 2 -> List(List("host001"))))

    tracker !! StopCacheTracker
  }

  /**
   * Helper function to get cacheLocations from CacheTracker
   */
  def getCacheLocations(tracker: ActorRef) = (tracker ? GetCacheLocations).get match {
    case h: HashMap[_, _] => h.asInstanceOf[HashMap[Int, Array[List[String]]]].map {
      case (i, arr) => (i -> arr.toList)
    }
  }
}