Code formatting...

This commit is contained in:
Mosharaf Chowdhury 2010-12-19 18:03:20 -08:00
parent 81f78282e1
commit 3b21a5fb26
4 changed files with 122 additions and 104 deletions

View file

@ -21,7 +21,8 @@ import scala.collection.mutable.{ArrayBuffer, HashMap}
* TODO: Add support for compression when spark.compress is set to true.
*/
@serializable
class CustomBlockedLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logging {
class CustomBlockedLocalFileShuffle[K, V, C]
extends Shuffle[K, V, C] with Logging {
@transient var totalSplits = 0
@transient var hasSplits = 0
@ -76,8 +77,8 @@ class CustomBlockedLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Loggi
buckets(i).foreach(pair => {
// Open a new file if necessary
if (!isDirty) {
file = CustomBlockedLocalFileShuffle.getOutputFile(shuffleId, myIndex, i,
blockNum)
file = CustomBlockedLocalFileShuffle.getOutputFile(shuffleId,
myIndex, i, blockNum)
writeStartTime = System.currentTimeMillis
logInfo("BEGIN WRITE: " + file)
@ -110,8 +111,8 @@ class CustomBlockedLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Loggi
}
// Write the BLOCKNUM file
file =
CustomBlockedLocalFileShuffle.getBlockNumOutputFile(shuffleId, myIndex, i)
file = CustomBlockedLocalFileShuffle.getBlockNumOutputFile(shuffleId,
myIndex, i)
out = new ObjectOutputStream(new FileOutputStream(file))
out.writeObject(blockNum)
out.close()
@ -166,6 +167,8 @@ class CustomBlockedLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Loggi
// Sleep for a while before creating new threads
Thread.sleep(CustomBlockedLocalFileShuffle.MinKnockInterval)
}
threadPool.shutdown()
combiners
})
}

View file

@ -16,7 +16,8 @@ import scala.collection.mutable.{ArrayBuffer, HashMap}
* TODO: Add support for compression when spark.compress is set to true.
*/
@serializable
class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logging {
class CustomParallelLocalFileShuffle[K, V, C]
extends Shuffle[K, V, C] with Logging {
@transient var totalSplits = 0
@transient var hasSplits = 0
@transient var hasSplitsBitVector: BitSet = null
@ -57,17 +58,19 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
}
for (i <- 0 until numOutputSplits) {
val file = CustomParallelLocalFileShuffle.getOutputFile(shuffleId, myIndex, i)
val file = CustomParallelLocalFileShuffle.getOutputFile(shuffleId,
myIndex, i)
val writeStartTime = System.currentTimeMillis
logInfo("BEGIN WRITE: " + file)
val out = new ObjectOutputStream(new FileOutputStream(file))
buckets(i).foreach(pair => out.writeObject(pair))
out.close()
logInfo("END WRITE: " + file)
val writeTime = (System.currentTimeMillis - writeStartTime)
val writeTime = System.currentTimeMillis - writeStartTime
logInfo("Writing " + file + " of size " + file.length + " bytes took " + writeTime + " millis.")
}
(myIndex, CustomParallelLocalFileShuffle.serverAddress, CustomParallelLocalFileShuffle.serverPort)
(myIndex, CustomParallelLocalFileShuffle.serverAddress,
CustomParallelLocalFileShuffle.serverPort)
}).collect()
val splitsByUri = new ArrayBuffer[(String, Int, Int)]
@ -86,12 +89,12 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
splitsInRequestBitVector = new BitSet(totalSplits)
combiners = new HashMap[K, C]
var threadPool =
CustomParallelLocalFileShuffle.newDaemonFixedThreadPool (CustomParallelLocalFileShuffle.MaxConnections)
var threadPool = CustomParallelLocalFileShuffle.newDaemonFixedThreadPool(
CustomParallelLocalFileShuffle.MaxConnections)
while (hasSplits < totalSplits) {
var numThreadsToCreate =
Math.min (totalSplits, CustomParallelLocalFileShuffle.MaxConnections) -
var numThreadsToCreate = Math.min(totalSplits,
CustomParallelLocalFileShuffle.MaxConnections) -
threadPool.getActiveCount
while (hasSplits < totalSplits && numThreadsToCreate > 0) {
@ -118,7 +121,7 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
Thread.sleep(CustomParallelLocalFileShuffle.MinKnockInterval)
}
threadPool.shutdown
threadPool.shutdown()
combiners
})
}
@ -135,7 +138,8 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
}
if (requiredSplits.size > 0) {
requiredSplits(CustomParallelLocalFileShuffle.ranGen.nextInt (requiredSplits.size))
requiredSplits(CustomParallelLocalFileShuffle.ranGen.nextInt(
requiredSplits.size))
} else {
-1
}
@ -157,12 +161,13 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
// Setup the timeout mechanism
var timeOutTask = new TimerTask {
override def run: Unit = {
cleanUpConnections
cleanUpConnections()
}
}
var timeOutTimer = new Timer
timeOutTimer.schedule (timeOutTask, CustomParallelLocalFileShuffle.MaxKnockInterval)
timeOutTimer.schedule(timeOutTask,
CustomParallelLocalFileShuffle.MaxKnockInterval)
logInfo("ShuffleClient started... => %s:%d#%s".format(hostAddress, listenPort, requestPath))
@ -171,7 +176,7 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
peerSocketToSource = new Socket(hostAddress, listenPort)
oosSource =
new ObjectOutputStream(peerSocketToSource.getOutputStream)
oosSource.flush
oosSource.flush()
var isSource = peerSocketToSource.getInputStream
oisSource = new ObjectInputStream(isSource)
@ -183,7 +188,7 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
logInfo("Received requestedFileLen = " + requestedFileLen)
// Turn the timer OFF, if the sender responds before timeout
timeOutTimer.cancel
timeOutTimer.cancel()
// Receive the file
if (requestedFileLen != -1) {
@ -203,7 +208,7 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
} catch {
case e: EOFException => { }
}
inputStream.close
inputStream.close()
// Reception completed. Update stats.
hasSplitsBitVector.synchronized {
@ -219,7 +224,7 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
receptionSucceeded = true
logInfo("END READ: http://%s:%d/shuffle/%s".format(hostAddress, listenPort, requestPath))
val readTime = (System.currentTimeMillis - readStartTime)
val readTime = System.currentTimeMillis - readStartTime
logInfo("Reading http://%s:%d/shuffle/%s".format(hostAddress, listenPort, requestPath) + " took " + readTime + " millis.")
} else {
throw new SparkException("ShuffleServer " + hostAddress + " does not have " + requestPath)
@ -238,19 +243,19 @@ class CustomParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logg
splitsInRequestBitVector.set(splitIndex, false)
}
}
cleanUpConnections
cleanUpConnections()
}
}
private def cleanUpConnections: Unit = {
private def cleanUpConnections(): Unit = {
if (oisSource != null) {
oisSource.close
oisSource.close()
}
if (oosSource != null) {
oosSource.close
oosSource.close()
}
if (peerSocketToSource != null) {
peerSocketToSource.close
peerSocketToSource.close()
}
}
}
@ -280,13 +285,13 @@ object CustomParallelLocalFileShuffle extends Logging {
private def initializeIfNeeded() = synchronized {
if (!initialized) {
// Load config parameters
MinKnockInterval_ =
System.getProperty ("spark.parallelLocalFileShuffle.MinKnockInterval", "1000").toInt
MaxKnockInterval_ =
System.getProperty ("spark.parallelLocalFileShuffle.MaxKnockInterval", "5000").toInt
MinKnockInterval_ = System.getProperty(
"spark.parallelLocalFileShuffle.MinKnockInterval", "1000").toInt
MaxKnockInterval_ = System.getProperty(
"spark.parallelLocalFileShuffle.MaxKnockInterval", "5000").toInt
MaxConnections_ =
System.getProperty ("spark.parallelLocalFileShuffle.MaxConnections", "4").toInt
MaxConnections_ = System.getProperty(
"spark.parallelLocalFileShuffle.MaxConnections", "4").toInt
// TODO: localDir should be created by some mechanism common to Spark
// so that it can be shared among shuffle, broadcast, etc
@ -299,9 +304,9 @@ object CustomParallelLocalFileShuffle extends Logging {
while (!foundLocalDir && tries < 10) {
tries += 1
try {
localDirUuid = UUID.randomUUID()
localDirUuid = UUID.randomUUID
localDir = new File(localDirRoot, "spark-local-" + localDirUuid)
if (!localDir.exists()) {
if (!localDir.exists) {
localDir.mkdirs()
foundLocalDir = true
}
@ -368,7 +373,8 @@ object CustomParallelLocalFileShuffle extends Logging {
class ShuffleServer
extends Thread with Logging {
var threadPool = newDaemonFixedThreadPool(CustomParallelLocalFileShuffle.MaxConnections)
var threadPool =
newDaemonFixedThreadPool(CustomParallelLocalFileShuffle.MaxConnections)
var serverSocket: ServerSocket = null
@ -383,7 +389,7 @@ object CustomParallelLocalFileShuffle extends Logging {
while (true) {
var clientSocket: Socket = null
try {
clientSocket = serverSocket.accept
clientSocket = serverSocket.accept()
} catch {
case e: Exception => { }
}
@ -394,7 +400,7 @@ object CustomParallelLocalFileShuffle extends Logging {
} catch {
// In failure, close socket here; else, the thread will close it
case ioe: IOException => {
clientSocket.close
clientSocket.close()
}
}
}
@ -402,21 +408,21 @@ object CustomParallelLocalFileShuffle extends Logging {
} finally {
if (serverSocket != null) {
logInfo("ShuffleServer now stopping...")
serverSocket.close
serverSocket.close()
}
}
// Shutdown the thread pool
threadPool.shutdown
threadPool.shutdown()
}
class ShuffleServerThread(val clientSocket: Socket)
extends Thread with Logging {
private val os = clientSocket.getOutputStream.asInstanceOf[OutputStream]
os.flush
os.flush()
private val bos = new BufferedOutputStream(os)
bos.flush
bos.flush()
private val oos = new ObjectOutputStream(os)
oos.flush
oos.flush()
private val ois = new ObjectInputStream(clientSocket.getInputStream)
logInfo("new ShuffleServerThread is running")
@ -442,7 +448,7 @@ object CustomParallelLocalFileShuffle extends Logging {
// In the case of receiver timeout and connection close, this will
// throw a java.net.SocketException: Broken pipe
oos.writeObject(requestedFileLen)
oos.flush
oos.flush()
logInfo("requestedFileLen = " + requestedFileLen)
@ -463,16 +469,16 @@ object CustomParallelLocalFileShuffle extends Logging {
alreadyRead = alreadyRead + bytesRead
}
}
bis.close
bis.close()
// Send
bos.write(byteArray, 0, byteArray.length)
bos.flush
bos.flush()
} else {
// Close the connection
}
} catch {
// If something went wrong, e.g., the worker at the other end died etc.
// If something went wrong, e.g., the worker at the other end died etc
// then close everything up
// Exception can happen if the receiver stops receiving
case e: Exception => {
@ -480,11 +486,11 @@ object CustomParallelLocalFileShuffle extends Logging {
}
} finally {
logInfo("ShuffleServerThread is closing streams and sockets")
ois.close
ois.close()
// TODO: Following can cause "java.net.SocketException: Socket closed"
oos.close
bos.close
clientSocket.close
oos.close()
bos.close()
clientSocket.close()
}
}
}

View file

@ -21,7 +21,8 @@ import scala.collection.mutable.{ArrayBuffer, HashMap}
* TODO: Add support for compression when spark.compress is set to true.
*/
@serializable
class HttpBlockedLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logging {
class HttpBlockedLocalFileShuffle[K, V, C]
extends Shuffle[K, V, C] with Logging {
@transient var totalSplits = 0
@transient var hasSplits = 0
@ -69,7 +70,8 @@ class HttpBlockedLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logging
for (i <- 0 until numOutputSplits) {
// Open the INDEX file
var indexFile: File =
HttpBlockedLocalFileShuffle.getBlockIndexOutputFile(shuffleId, myIndex, i)
HttpBlockedLocalFileShuffle.getBlockIndexOutputFile(shuffleId,
myIndex, i)
var indexOut = new ObjectOutputStream(new FileOutputStream(indexFile))
var indexDirty: Boolean = true
var alreadyWritten: Long = 0
@ -88,7 +90,8 @@ class HttpBlockedLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logging
indexDirty = true
// Update the INDEX file if more than blockSize limit has been written
if (file.length - alreadyWritten > HttpBlockedLocalFileShuffle.BlockSize) {
if (file.length - alreadyWritten >
HttpBlockedLocalFileShuffle.BlockSize) {
indexOut.writeObject(file.length)
indexDirty = false
alreadyWritten = file.length
@ -158,6 +161,8 @@ class HttpBlockedLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logging
// Sleep for a while before creating new threads
Thread.sleep(HttpBlockedLocalFileShuffle.MinKnockInterval)
}
threadPool.shutdown()
combiners
})
}

View file

@ -16,7 +16,8 @@ import scala.collection.mutable.{ArrayBuffer, HashMap}
* TODO: Add support for compression when spark.compress is set to true.
*/
@serializable
class HttpParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Logging {
class HttpParallelLocalFileShuffle[K, V, C]
extends Shuffle[K, V, C] with Logging {
@transient var totalSplits = 0
@transient var hasSplits = 0
@ -58,7 +59,8 @@ class HttpParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Loggin
}
for (i <- 0 until numOutputSplits) {
val file = HttpParallelLocalFileShuffle.getOutputFile(shuffleId, myIndex, i)
val file =
HttpParallelLocalFileShuffle.getOutputFile(shuffleId, myIndex, i)
val writeStartTime = System.currentTimeMillis
logInfo("BEGIN WRITE: " + file)
val out = new ObjectOutputStream(new FileOutputStream(file))
@ -115,6 +117,8 @@ class HttpParallelLocalFileShuffle[K, V, C] extends Shuffle[K, V, C] with Loggin
// Sleep for a while before creating new threads
Thread.sleep(HttpParallelLocalFileShuffle.MinKnockInterval)
}
threadPool.shutdown()
combiners
})
}