Skip to content

Commit

Permalink
fix JsonProtocolSuie test failure
Browse files Browse the repository at this point in the history
  • Loading branch information
RongGu committed Mar 24, 2014
1 parent 2825a13 commit d827250
Show file tree
Hide file tree
Showing 2 changed files with 11 additions and 9 deletions.
2 changes: 2 additions & 0 deletions core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
Original file line number Diff line number Diff line change
Expand Up @@ -599,11 +599,13 @@ private[spark] object JsonProtocol {
val numPartitions = (json \ "Number of Partitions").extract[Int]
val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int]
val memSize = (json \ "Memory Size").extract[Long]
val tachyonSize = (json \ "Tachyon Size").extract[Long]
val diskSize = (json \ "Disk Size").extract[Long]

val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel)
rddInfo.numCachedPartitions = numCachedPartitions
rddInfo.memSize = memSize
rddInfo.tachyonSize = tachyonSize
rddInfo.diskSize = diskSize
rddInfo
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -471,19 +471,19 @@ class JsonProtocolSuite extends FunSuite {
"""
{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name":
"greetings","Number of Tasks":200,"RDD Info":{"RDD ID":100,"Name":"mayor","Storage
Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},
"Number of Partitions":200,"Number of Cached Partitions":300,"Memory Size":400,
"Disk Size":500},"Emitted Task Size Warning":false},"Properties":{"France":"Paris",
"Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}}
Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,
"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":300,
"Memory Size":400,"Disk Size":500,"Tachyon Size":0},"Emitted Task Size Warning":false},
"Properties":{"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}}
"""

private val stageCompletedJsonString =
"""
{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name":
"greetings","Number of Tasks":201,"RDD Info":{"RDD ID":101,"Name":"mayor","Storage
Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},
"Number of Partitions":201,"Number of Cached Partitions":301,"Memory Size":401,
"Disk Size":501},"Emitted Task Size Warning":false}}
Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,
"Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301,
"Memory Size":401,"Disk Size":501,"Tachyon Size":0},"Emitted Task Size Warning":false}}
"""

private val taskStartJsonString =
Expand Down Expand Up @@ -516,8 +516,8 @@ class JsonProtocolSuite extends FunSuite {
700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics":
{"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks":
[{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status":
{"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":false,
"Replication":2},"Memory Size":0,"Disk Size":0}}]}}
{"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false,
"Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}}
"""

private val jobStartJsonString =
Expand Down

0 comments on commit d827250

Please sign in to comment.