Skip to content

Commit

Permalink
Add explicit return types
Browse files Browse the repository at this point in the history
  • Loading branch information
ankurdave committed May 7, 2014
1 parent 04d3ae5 commit 75af062
Show file tree
Hide file tree
Showing 2 changed files with 20 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -64,5 +64,5 @@ class EdgeTriplet[VD, ED] extends Edge[ED] {

override def toString = ((srcId, srcAttr), (dstId, dstAttr), attr).toString()

def toTuple = ((srcId, srcAttr), (dstId, dstAttr), attr)
def toTuple: ((VertexId, VD), (VertexId, VD), ED) = ((srcId, srcAttr), (dstId, dstAttr), attr)
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,26 +29,28 @@ private[graphx]
class RoutingTableMessageSerializer extends Serializer with Serializable {
override def newInstance(): SerializerInstance = new ShuffleSerializerInstance {

override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) {
def writeObject[T](t: T) = {
val msg = t.asInstanceOf[RoutingTableMessage]
writeVarLong(msg.vid, optimizePositive = false)
writeUnsignedVarInt(msg.pid)
// TODO: Write only the bottom two bits of msg.position
s.write(msg.position)
this
override def serializeStream(s: OutputStream): SerializationStream =
new ShuffleSerializationStream(s) {
def writeObject[T](t: T): SerializationStream = {
val msg = t.asInstanceOf[RoutingTableMessage]
writeVarLong(msg.vid, optimizePositive = false)
writeUnsignedVarInt(msg.pid)
// TODO: Write only the bottom two bits of msg.position
s.write(msg.position)
this
}
}
}

override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) {
override def readObject[T](): T = {
val a = readVarLong(optimizePositive = false)
val b = readUnsignedVarInt()
val c = s.read()
if (c == -1) throw new EOFException
new RoutingTableMessage(a, b, c.toByte).asInstanceOf[T]
override def deserializeStream(s: InputStream): DeserializationStream =
new ShuffleDeserializationStream(s) {
override def readObject[T](): T = {
val a = readVarLong(optimizePositive = false)
val b = readUnsignedVarInt()
val c = s.read()
if (c == -1) throw new EOFException
new RoutingTableMessage(a, b, c.toByte).asInstanceOf[T]
}
}
}
}
}

Expand Down

0 comments on commit 75af062

Please sign in to comment.