From 10cf3828adce1c73c805904d33af16b7211dc9fb Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 3 Apr 2010 23:50:04 -0700 Subject: Imported Mosharaf's multi-tracker branch --- alltests | 2 +- run | 3 +- src/examples/SparkALS.scala | 5 + src/scala/spark/Broadcast.scala | 672 ++++++++++++++++++++--------------- src/scala/spark/HdfsFile.scala | 38 +- src/scala/spark/NexusScheduler.scala | 317 ++++++++--------- src/scala/spark/Task.scala | 4 +- src/test/spark/repl/ReplSuite.scala | 18 +- 8 files changed, 556 insertions(+), 503 deletions(-) diff --git a/alltests b/alltests index 3c9db301c4..8beab92952 100755 --- a/alltests +++ b/alltests @@ -1,3 +1,3 @@ #!/bin/bash FWDIR=`dirname $0` -$FWDIR/run org.scalatest.tools.Runner -p $FWDIR/build/classes -o $@ +$FWDIR/run org.scalatest.tools.Runner -p $FWDIR/classes -o $@ diff --git a/run b/run index c1156892ad..82a12b0113 100755 --- a/run +++ b/run @@ -4,7 +4,8 @@ FWDIR=`dirname $0` # Set JAVA_OPTS to be able to load libnexus.so and set various other misc options -export JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xmx2000m -Dspark.broadcast.masterHostAddress=127.0.0.1 -Dspark.broadcast.masterListenPort=11111 -Dspark.broadcast.blockSize=1024 -Dspark.broadcast.maxRetryCount=2 -Dspark.broadcast.serverSocketTimout=50000 -Dspark.broadcast.dualMode=false" +export JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xmx2000m -Dspark.broadcast.masterHostAddress=127.0.0.1 -Dspark.broadcast.masterTrackerPort=11111 -Dspark.broadcast.blockSize=1024 -Dspark.broadcast.maxRetryCount=2 -Dspark.broadcast.serverSocketTimout=50000 -Dspark.broadcast.dualMode=false" + if [ -e $FWDIR/conf/java-opts ] ; then JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" fi diff --git a/src/examples/SparkALS.scala b/src/examples/SparkALS.scala index 38dd0e665d..cbbbba3c79 100644 --- a/src/examples/SparkALS.scala +++ b/src/examples/SparkALS.scala @@ -122,6 +122,8 @@ object SparkALS { var msc = spark.broadcast(ms) var usc = spark.broadcast(us) for (iter <- 1 to ITERATIONS) { + val start = System.nanoTime + println("Iteration " + iter + ":") ms = spark.parallelize(0 until M, slices) .map(i => updateMovie(i, msc.value(i), usc.value, Rc.value)) @@ -133,6 +135,9 @@ object SparkALS { usc = spark.broadcast(us) // Re-broadcast us because it was updated println("RMSE = " + rmse(R, ms, us)) println() + + val time = (System.nanoTime - start) / 1e9 + println( "This iteration took " + time + " s") } } } diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 2da5e28a0a..73b9ea39e8 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -8,9 +8,6 @@ import com.google.common.collect.MapMaker import java.util.concurrent.{Executors, ExecutorService} -import scala.actors.Actor -import scala.actors.Actor._ - import scala.collection.mutable.Map import org.apache.hadoop.conf.Configuration @@ -29,8 +26,6 @@ trait BroadcastRecipe { override def toString = "spark.Broadcast(" + uuid + ")" } -// TODO: Should think about storing in HDFS in the future -// TODO: Right, now no parallelization between multiple broadcasts @serializable class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) extends BroadcastRecipe { @@ -39,31 +34,81 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) BroadcastCS.synchronized { BroadcastCS.values.put (uuid, value_) } + @transient var arrayOfBlocks: Array[BroadcastBlock] = null + @transient var totalBytes = -1 + @transient var totalBlocks = -1 + @transient var hasBlocks = 0 + + @transient var listenPortLock = new Object + @transient var guidePortLock = new Object + @transient var totalBlocksLock = new Object + @transient var hasBlocksLock = new Object + + @transient var pqOfSources = new PriorityQueue[SourceInfo] + + @transient var serveMR: ServeMultipleRequests = null + @transient var guideMR: GuideMultipleRequests = null + + @transient var hostAddress = InetAddress.getLocalHost.getHostAddress + @transient var listenPort = -1 + @transient var guidePort = -1 + if (!local) { sendBroadcast } def sendBroadcast () { // Create a variableInfo object and store it in valueInfos var variableInfo = blockifyObject (value_, BroadcastCS.blockSize) - // TODO: Even though this part is not in use now, there is problem in the - // following statement. Shouldn't use constant port and hostAddress anymore? - // val masterSource = - // new SourceInfo (BroadcastCS.masterHostAddress, BroadcastCS.masterListenPort, - // variableInfo.totalBlocks, variableInfo.totalBytes, 0) - // variableInfo.pqOfSources.add (masterSource) + guideMR = new GuideMultipleRequests + // guideMR.setDaemon (true) + guideMR.start + // println (System.currentTimeMillis + ": " + "GuideMultipleRequests started") + + serveMR = new ServeMultipleRequests + // serveMR.setDaemon (true) + serveMR.start + // println (System.currentTimeMillis + ": " + "ServeMultipleRequests started") + + // Prepare the value being broadcasted + // TODO: Refactoring and clean-up required here + arrayOfBlocks = variableInfo.arrayOfBlocks + totalBytes = variableInfo.totalBytes + totalBlocks = variableInfo.totalBlocks + hasBlocks = variableInfo.totalBlocks + + while (listenPort == -1) { + listenPortLock.synchronized { + listenPortLock.wait + } + } + + pqOfSources = new PriorityQueue[SourceInfo] + val masterSource_0 = + new SourceInfo (hostAddress, listenPort, totalBlocks, totalBytes, 0) + pqOfSources.add (masterSource_0) + // Add one more time to have two replicas of any seeds in the PQ + if (BroadcastCS.dualMode) { + val masterSource_1 = + new SourceInfo (hostAddress, listenPort, totalBlocks, totalBytes, 1) + pqOfSources.add (masterSource_1) + } + + // Register with the Tracker + while (guidePort == -1) { + guidePortLock.synchronized { + guidePortLock.wait + } + } + BroadcastCS.synchronized { - // BroadcastCS.valueInfos.put (uuid, variableInfo) - - // TODO: Not using variableInfo in current implementation. Manually - // setting all the variables inside BroadcastCS object - - BroadcastCS.initializeVariable (variableInfo) + BroadcastCS.registerValue (uuid, guidePort) } + // TODO: Make it a separate thread? // Now store a persistent copy in HDFS, just in case val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) out.writeObject (value_) - out.close + out.close } private def readObject (in: ObjectInputStream) { @@ -75,9 +120,18 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } else { // Only a single worker (the first one) in the same node can ever be // here. The rest will always get the value ready + + // Initializing everything because Master will only send null/0 values + initializeSlaveVariables + + serveMR = new ServeMultipleRequests + // serveMR.setDaemon (true) + serveMR.start + // println (System.currentTimeMillis + ": " + "ServeMultipleRequests started") + val start = System.nanoTime - val retByteArray = BroadcastCS.receiveBroadcast (uuid) + val retByteArray = receiveBroadcast (uuid) // If does not succeed, then get from HDFS copy if (retByteArray != null) { value_ = byteArrayToObject[T] (retByteArray) @@ -97,6 +151,19 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } } + private def initializeSlaveVariables = { + arrayOfBlocks = null + totalBytes = -1 + totalBlocks = -1 + hasBlocks = 0 + listenPortLock = new Object + totalBlocksLock = new Object + hasBlocksLock = new Object + serveMR = null + hostAddress = InetAddress.getLocalHost.getHostAddress + listenPort = -1 + } + private def blockifyObject (obj: T, blockSize: Int): VariableInfo = { val baos = new ByteArrayOutputStream val oos = new ObjectOutputStream (baos) @@ -108,7 +175,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) var blockNum = (byteArray.length / blockSize) if (byteArray.length % blockSize != 0) - blockNum += 1 + blockNum += 1 var retVal = new Array[BroadcastBlock] (blockNum) var blockID = 0 @@ -145,199 +212,50 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) bOut.close return bOut } -} - -@serializable -class CentralizedHDFSBroadcast[T](@transient var value_ : T, local: Boolean) - extends BroadcastRecipe { - - def value = value_ - - BroadcastCH.synchronized { BroadcastCH.values.put(uuid, value_) } - - if (!local) { sendBroadcast } - - def sendBroadcast () { - val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) - out.writeObject (value_) - out.close - } - - // Called by Java when deserializing an object - private def readObject(in: ObjectInputStream) { - in.defaultReadObject - BroadcastCH.synchronized { - val cachedVal = BroadcastCH.values.get(uuid) - if (cachedVal != null) { - value_ = cachedVal.asInstanceOf[T] - } else { - val start = System.nanoTime - - val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) - value_ = fileIn.readObject.asInstanceOf[T] - BroadcastCH.values.put(uuid, value_) - fileIn.close - - val time = (System.nanoTime - start) / 1e9 - println( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") - } - } - } -} - -@serializable -case class SourceInfo (val hostAddress: String, val listenPort: Int, - val totalBlocks: Int, val totalBytes: Int, val replicaID: Int) - extends Comparable [SourceInfo]{ - - var currentLeechers = 0 - var receptionFailed = false - - def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers) -} - -@serializable -case class BroadcastBlock (val blockID: Int, val byteArray: Array[Byte]) { } - -@serializable -case class VariableInfo (@transient val arrayOfBlocks : Array[BroadcastBlock], - val totalBlocks: Int, val totalBytes: Int) { - - @transient var hasBlocks = 0 - - val listenPortLock = new AnyRef - val totalBlocksLock = new AnyRef - val hasBlocksLock = new AnyRef - - @transient var pqOfSources = new PriorityQueue[SourceInfo] -} - -private object Broadcast { - private var initialized = false - - // Will be called by SparkContext or Executor before using Broadcast - // Calls all other initializers here - def initialize (isMaster: Boolean) { - synchronized { - if (!initialized) { - // Initialization for CentralizedHDFSBroadcast - BroadcastCH.initialize - // Initialization for ChainedStreamingBroadcast - BroadcastCS.initialize (isMaster) - - initialized = true - } - } - } -} - -private object BroadcastCS { - val values = new MapMaker ().softValues ().makeMap[UUID, Any] - // val valueInfos = new MapMaker ().softValues ().makeMap[UUID, Any] - - // private var valueToPort = Map[UUID, Int] () - - private var initialized = false - private var isMaster_ = false - - private var masterHostAddress_ = "127.0.0.1" - private var masterListenPort_ : Int = 11111 - private var blockSize_ : Int = 512 * 1024 - private var maxRetryCount_ : Int = 2 - private var serverSocketTimout_ : Int = 50000 - private var dualMode_ : Boolean = false - - private val hostAddress = InetAddress.getLocalHost.getHostAddress - private var listenPort = -1 - - var arrayOfBlocks: Array[BroadcastBlock] = null - var totalBytes = -1 - var totalBlocks = -1 - var hasBlocks = 0 - - val listenPortLock = new Object - val totalBlocksLock = new Object - val hasBlocksLock = new Object - - var pqOfSources = new PriorityQueue[SourceInfo] - private var serveMR: ServeMultipleRequests = null - private var guideMR: GuideMultipleRequests = null - - def initialize (isMaster__ : Boolean) { - synchronized { - if (!initialized) { - masterHostAddress_ = - System.getProperty ("spark.broadcast.masterHostAddress", "127.0.0.1") - masterListenPort_ = - System.getProperty ("spark.broadcast.masterListenPort", "11111").toInt - blockSize_ = - System.getProperty ("spark.broadcast.blockSize", "512").toInt * 1024 - maxRetryCount_ = - System.getProperty ("spark.broadcast.maxRetryCount", "2").toInt - serverSocketTimout_ = - System.getProperty ("spark.broadcast.serverSocketTimout", "50000").toInt - dualMode_ = - System.getProperty ("spark.broadcast.dualMode", "false").toBoolean - - isMaster_ = isMaster__ - - if (isMaster) { - guideMR = new GuideMultipleRequests - // guideMR.setDaemon (true) - guideMR.start - println (System.currentTimeMillis + ": " + "GuideMultipleRequests started") - } - serveMR = new ServeMultipleRequests - // serveMR.setDaemon (true) - serveMR.start - - println (System.currentTimeMillis + ": " + "ServeMultipleRequests started") - - println (System.currentTimeMillis + ": " + "BroadcastCS object has been initialized") - - initialized = true - } - } - } - - // TODO: This should change in future implementation. - // Called from the Master constructor to setup states for this particular that - // is being broadcasted - def initializeVariable (variableInfo: VariableInfo) { - arrayOfBlocks = variableInfo.arrayOfBlocks - totalBytes = variableInfo.totalBytes - totalBlocks = variableInfo.totalBlocks - hasBlocks = variableInfo.totalBlocks + def receiveBroadcast (variableUUID: UUID): Array[Byte] = { + var clientSocketToTracker: Socket = null + var oisTracker: ObjectInputStream = null + var oosTracker: ObjectOutputStream = null - // listenPort should already be valid - assert (listenPort != -1) + var masterListenPort: Int = -1 - pqOfSources = new PriorityQueue[SourceInfo] - val masterSource_0 = - new SourceInfo (hostAddress, listenPort, totalBlocks, totalBytes, 0) - BroadcastCS.pqOfSources.add (masterSource_0) - // Add one more time to have two replicas of any seeds in the PQ - if (BroadcastCS.dualMode) { - val masterSource_1 = - new SourceInfo (hostAddress, listenPort, totalBlocks, totalBytes, 1) - BroadcastCS.pqOfSources.add (masterSource_1) - } - } - - def masterHostAddress = masterHostAddress_ - def masterListenPort = masterListenPort_ - def blockSize = blockSize_ - def maxRetryCount = maxRetryCount_ - def serverSocketTimout = serverSocketTimout_ - def dualMode = dualMode_ + // masterListenPort aka guidePort value legend + // 0 = missed the broadcast, read from HDFS; + // <0 = hasn't started yet, wait & retry; (never happens) + // >0 = Read from this port + var retriesLeft = BroadcastCS.maxRetryCount + do { + try { + // Connect to the tracker to find out the guide + val clientSocketToTracker = + new Socket(BroadcastCS.masterHostAddress, BroadcastCS.masterTrackerPort) + val oisTracker = + new ObjectInputStream (clientSocketToTracker.getInputStream) + val oosTracker = + new ObjectOutputStream (clientSocketToTracker.getOutputStream) + + // Send UUID and receive masterListenPort + oosTracker.writeObject (uuid) + masterListenPort = oisTracker.readObject.asInstanceOf[Int] + } catch { + // In case of any failure, set masterListenPort = 0 to read from HDFS + case e: Exception => (masterListenPort = 0) + } finally { + if (oisTracker != null) { oisTracker.close } + if (oosTracker != null) { oosTracker.close } + if (clientSocketToTracker != null) { clientSocketToTracker.close } + } - def isMaster = isMaster_ + retriesLeft -= 1 + } while (retriesLeft > 0 && masterListenPort < 0) + // println (System.currentTimeMillis + ": " + "Got this guidePort from Tracker: " + masterListenPort) + + // If Tracker says that there is no guide for this object, read from HDFS + if (masterListenPort == 0) { return null } - def receiveBroadcast (variableUUID: UUID): Array[Byte] = { // Wait until hostAddress and listenPort are created by the // ServeMultipleRequests thread - // NO need to wait; ServeMultipleRequests is created much further ahead while (listenPort == -1) { listenPortLock.synchronized { listenPortLock.wait @@ -346,13 +264,13 @@ private object BroadcastCS { // Connect and receive broadcast from the specified source, retrying the // specified number of times in case of failures - var retriesLeft = BroadcastCS.maxRetryCount + retriesLeft = BroadcastCS.maxRetryCount var retByteArray: Array[Byte] = null - do { + do { // Connect to Master and send this worker's Information val clientSocketToMaster = - new Socket(BroadcastCS.masterHostAddress, BroadcastCS.masterListenPort) - println (System.currentTimeMillis + ": " + "Connected to Master's guiding object") + new Socket(BroadcastCS.masterHostAddress, masterListenPort) + // println (System.currentTimeMillis + ": " + "Connected to Master's guiding object") // TODO: Guiding object connection is reusable val oisMaster = new ObjectInputStream (clientSocketToMaster.getInputStream) @@ -371,11 +289,11 @@ private object BroadcastCS { } totalBytes = sourceInfo.totalBytes - println (System.currentTimeMillis + ": " + "Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort) + // println (System.currentTimeMillis + ": " + "Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort) retByteArray = receiveSingleTransmission (sourceInfo) - println (System.currentTimeMillis + ": " + "I got this from receiveSingleTransmission: " + retByteArray) + // println (System.currentTimeMillis + ": " + "I got this from receiveSingleTransmission: " + retByteArray) // TODO: Update sourceInfo to add error notifactions for Master if (retByteArray == null) { sourceInfo.receptionFailed = true } @@ -414,8 +332,8 @@ private object BroadcastCS { oisSource = new ObjectInputStream (clientSocketToSource.getInputStream) - println (System.currentTimeMillis + ": " + "Inside receiveSingleTransmission") - println (System.currentTimeMillis + ": " + "totalBlocks: "+ totalBlocks + " " + "hasBlocks: " + hasBlocks) + // println (System.currentTimeMillis + ": " + "Inside receiveSingleTransmission") + // println (System.currentTimeMillis + ": " + "totalBlocks: "+ totalBlocks + " " + "hasBlocks: " + hasBlocks) retByteArray = new Array[Byte] (totalBytes) for (i <- 0 until totalBlocks) { val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock] @@ -426,14 +344,14 @@ private object BroadcastCS { hasBlocksLock.synchronized { hasBlocksLock.notifyAll } - println (System.currentTimeMillis + ": " + "Received block: " + i + " " + bcBlock) + // println (System.currentTimeMillis + ": " + "Received block: " + i + " " + bcBlock) } assert (hasBlocks == totalBlocks) - println (System.currentTimeMillis + ": " + "After the receive loop") + // println (System.currentTimeMillis + ": " + "After the receive loop") } catch { case e: Exception => { retByteArray = null - println (System.currentTimeMillis + ": " + "receiveSingleTransmission had a " + e) + // println (System.currentTimeMillis + ": " + "receiveSingleTransmission had a " + e) } } finally { if (oisSource != null) { oisSource.close } @@ -445,91 +363,35 @@ private object BroadcastCS { return retByteArray } - - class TrackMultipleValues extends Thread { - override def run = { - var threadPool = Executors.newCachedThreadPool - var serverSocket: ServerSocket = null - - serverSocket = new ServerSocket (BroadcastCS.masterListenPort) - println (System.currentTimeMillis + ": " + "TrackMultipleVariables" + serverSocket + " " + listenPort) - - var keepAccepting = true - try { - while (true) { - var clientSocket: Socket = null - try { - serverSocket.setSoTimeout (serverSocketTimout) - clientSocket = serverSocket.accept - } catch { - case e: Exception => { - println ("TrackMultipleValues Timeout. Stopping listening...") - keepAccepting = false - } - } - println (System.currentTimeMillis + ": " + "TrackMultipleValues:Got new request:" + clientSocket) - if (clientSocket != null) { - try { - threadPool.execute (new Runnable { - def run = { - val oos = new ObjectOutputStream (clientSocket.getOutputStream) - val ois = new ObjectInputStream (clientSocket.getInputStream) - try { - val variableUUID = ois.readObject.asInstanceOf[UUID] - var contactPort = 0 - // TODO: Add logic and data structures to find out UUID->port - // mapping. 0 = missed the broadcast, read from HDFS; <0 = - // Haven't started yet, wait & retry; >0 = Read from this port - oos.writeObject (contactPort) - } catch { - case e: Exception => { } - } finally { - ois.close - oos.close - clientSocket.close - } - } - }) - } catch { - // In failure, close the socket here; else, the thread will close it - case ioe: IOException => clientSocket.close - } - } - } - } finally { - serverSocket.close - } - } - } - - class TrackSingleValue { - - } - + class GuideMultipleRequests extends Thread { override def run = { var threadPool = Executors.newCachedThreadPool var serverSocket: ServerSocket = null - serverSocket = new ServerSocket (BroadcastCS.masterListenPort) - // listenPort = BroadcastCS.masterListenPort - println (System.currentTimeMillis + ": " + "GuideMultipleRequests" + serverSocket + " " + listenPort) + serverSocket = new ServerSocket (0) + guidePort = serverSocket.getLocalPort + // println (System.currentTimeMillis + ": " + "GuideMultipleRequests" + serverSocket + " " + guidePort) + guidePortLock.synchronized { + guidePortLock.notifyAll + } + var keepAccepting = true try { while (keepAccepting) { var clientSocket: Socket = null try { - serverSocket.setSoTimeout (serverSocketTimout) + serverSocket.setSoTimeout (BroadcastCS.serverSocketTimout) clientSocket = serverSocket.accept } catch { case e: Exception => { - println ("GuideMultipleRequests Timeout. Stopping listening...") + // println ("GuideMultipleRequests Timeout. Stopping listening...") keepAccepting = false } } if (clientSocket != null) { - println (System.currentTimeMillis + ": " + "Guide:Accepted new client connection:" + clientSocket) + // println (System.currentTimeMillis + ": " + "Guide:Accepted new client connection:" + clientSocket) try { threadPool.execute (new GuideSingleRequest (clientSocket)) } catch { @@ -552,21 +414,21 @@ private object BroadcastCS { def run = { try { - println (System.currentTimeMillis + ": " + "new GuideSingleRequest is running") + // println (System.currentTimeMillis + ": " + "new GuideSingleRequest is running") // Connecting worker is sending in its hostAddress and listenPort it will // be listening to. ReplicaID is 0 and other fields are invalid (-1) var sourceInfo = ois.readObject.asInstanceOf[SourceInfo] // Select a suitable source and send it back to the worker selectedSourceInfo = selectSuitableSource (sourceInfo) - println (System.currentTimeMillis + ": " + "Sending selectedSourceInfo:" + selectedSourceInfo) + // println (System.currentTimeMillis + ": " + "Sending selectedSourceInfo:" + selectedSourceInfo) oos.writeObject (selectedSourceInfo) oos.flush // Add this new (if it can finish) source to the PQ of sources thisWorkerInfo = new SourceInfo(sourceInfo.hostAddress, sourceInfo.listenPort, totalBlocks, totalBytes, 0) - println (System.currentTimeMillis + ": " + "Adding possible new source to pqOfSources: " + thisWorkerInfo) + // println (System.currentTimeMillis + ": " + "Adding possible new source to pqOfSources: " + thisWorkerInfo) pqOfSources.synchronized { pqOfSources.add (thisWorkerInfo) } @@ -649,7 +511,7 @@ private object BroadcastCS { serverSocket = new ServerSocket (0) listenPort = serverSocket.getLocalPort - println (System.currentTimeMillis + ": " + "ServeMultipleRequests" + serverSocket + " " + listenPort) + // println (System.currentTimeMillis + ": " + "ServeMultipleRequests" + serverSocket + " " + listenPort) listenPortLock.synchronized { listenPortLock.notifyAll @@ -660,16 +522,16 @@ private object BroadcastCS { while (keepAccepting) { var clientSocket: Socket = null try { - serverSocket.setSoTimeout (serverSocketTimout) + serverSocket.setSoTimeout (BroadcastCS.serverSocketTimout) clientSocket = serverSocket.accept } catch { case e: Exception => { - println ("ServeMultipleRequests Timeout. Stopping listening...") + // println ("ServeMultipleRequests Timeout. Stopping listening...") keepAccepting = false } } if (clientSocket != null) { - println (System.currentTimeMillis + ": " + "Serve:Accepted new client connection:" + clientSocket) + // println (System.currentTimeMillis + ": " + "Serve:Accepted new client connection:" + clientSocket) try { threadPool.execute (new ServeSingleRequest (clientSocket)) } catch { @@ -689,17 +551,17 @@ private object BroadcastCS { def run = { try { - println (System.currentTimeMillis + ": " + "new ServeSingleRequest is running") + // println (System.currentTimeMillis + ": " + "new ServeSingleRequest is running") sendObject } catch { // TODO: Need to add better exception handling here // If something went wrong, e.g., the worker at the other end died etc. // then close everything up case e: Exception => { - println (System.currentTimeMillis + ": " + "ServeSingleRequest had a " + e) + // println (System.currentTimeMillis + ": " + "ServeSingleRequest had a " + e) } } finally { - println (System.currentTimeMillis + ": " + "ServeSingleRequest is closing streams and sockets") + // println (System.currentTimeMillis + ": " + "ServeSingleRequest is closing streams and sockets") ois.close oos.close clientSocket.close @@ -726,11 +588,229 @@ private object BroadcastCS { } catch { case e: Exception => { } } - println (System.currentTimeMillis + ": " + "Send block: " + i + " " + arrayOfBlocks(i)) + // println (System.currentTimeMillis + ": " + "Send block: " + i + " " + arrayOfBlocks(i)) } } } - + } +} + +@serializable +class CentralizedHDFSBroadcast[T](@transient var value_ : T, local: Boolean) + extends BroadcastRecipe { + + def value = value_ + + BroadcastCH.synchronized { BroadcastCH.values.put(uuid, value_) } + + if (!local) { sendBroadcast } + + def sendBroadcast () { + val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) + out.writeObject (value_) + out.close + } + + // Called by Java when deserializing an object + private def readObject(in: ObjectInputStream) { + in.defaultReadObject + BroadcastCH.synchronized { + val cachedVal = BroadcastCH.values.get(uuid) + if (cachedVal != null) { + value_ = cachedVal.asInstanceOf[T] + } else { + // println( System.currentTimeMillis + ": " + "Started reading Broadcasted variable " + uuid) + val start = System.nanoTime + + val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) + value_ = fileIn.readObject.asInstanceOf[T] + BroadcastCH.values.put(uuid, value_) + fileIn.close + + val time = (System.nanoTime - start) / 1e9 + println( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") + } + } + } +} + +@serializable +case class SourceInfo (val hostAddress: String, val listenPort: Int, + val totalBlocks: Int, val totalBytes: Int, val replicaID: Int) + extends Comparable [SourceInfo]{ + + var currentLeechers = 0 + var receptionFailed = false + + def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers) +} + +@serializable +case class BroadcastBlock (val blockID: Int, val byteArray: Array[Byte]) { } + +@serializable +case class VariableInfo (@transient val arrayOfBlocks : Array[BroadcastBlock], + val totalBlocks: Int, val totalBytes: Int) { + + @transient var hasBlocks = 0 + + val listenPortLock = new AnyRef + val totalBlocksLock = new AnyRef + val hasBlocksLock = new AnyRef + + @transient var pqOfSources = new PriorityQueue[SourceInfo] +} + +private object Broadcast { + private var initialized = false + + // Will be called by SparkContext or Executor before using Broadcast + // Calls all other initializers here + def initialize (isMaster: Boolean) { + synchronized { + if (!initialized) { + // Initialization for CentralizedHDFSBroadcast + BroadcastCH.initialize + // Initialization for ChainedStreamingBroadcast + BroadcastCS.initialize (isMaster) + + initialized = true + } + } + } +} + +private object BroadcastCS { + val values = new MapMaker ().softValues ().makeMap[UUID, Any] + // val valueInfos = new MapMaker ().softValues ().makeMap[UUID, Any] + + var valueToGuidePortMap = Map[UUID, Int] () + + private var initialized = false + private var isMaster_ = false + + private var masterHostAddress_ = "127.0.0.1" + private var masterTrackerPort_ : Int = 11111 + private var blockSize_ : Int = 512 * 1024 + private var maxRetryCount_ : Int = 2 + private var serverSocketTimout_ : Int = 50000 + private var dualMode_ : Boolean = false + + private var trackMV: TrackMultipleValues = null + + def initialize (isMaster__ : Boolean) { + synchronized { + if (!initialized) { + masterHostAddress_ = + System.getProperty ("spark.broadcast.masterHostAddress", "127.0.0.1") + masterTrackerPort_ = + System.getProperty ("spark.broadcast.masterTrackerPort", "11111").toInt + blockSize_ = + System.getProperty ("spark.broadcast.blockSize", "512").toInt * 1024 + maxRetryCount_ = + System.getProperty ("spark.broadcast.maxRetryCount", "2").toInt + serverSocketTimout_ = + System.getProperty ("spark.broadcast.serverSocketTimout", "50000").toInt + dualMode_ = + System.getProperty ("spark.broadcast.dualMode", "false").toBoolean + + isMaster_ = isMaster__ + + if (isMaster) { + trackMV = new TrackMultipleValues + // trackMV.setDaemon (true) + trackMV.start + // println (System.currentTimeMillis + ": " + "TrackMultipleValues started") + } + + initialized = true + } + } + } + + def masterHostAddress = masterHostAddress_ + def masterTrackerPort = masterTrackerPort_ + def blockSize = blockSize_ + def maxRetryCount = maxRetryCount_ + def serverSocketTimout = serverSocketTimout_ + def dualMode = dualMode_ + + def isMaster = isMaster_ + + def registerValue (uuid: UUID, guidePort: Int) = { + valueToGuidePortMap.synchronized { + valueToGuidePortMap += (uuid -> guidePort) + // println (System.currentTimeMillis + ": " + "New value registered with the Tracker " + valueToGuidePortMap) + } + } + + // TODO: Who call this and when? + def unregisterValue (uuid: UUID) { + valueToGuidePortMap.synchronized { + valueToGuidePortMap (uuid) = 0 + // println (System.currentTimeMillis + ": " + "Value unregistered from the Tracker " + valueToGuidePortMap) + } + } + + class TrackMultipleValues extends Thread { + override def run = { + var threadPool = Executors.newCachedThreadPool + var serverSocket: ServerSocket = null + + serverSocket = new ServerSocket (BroadcastCS.masterTrackerPort) + // println (System.currentTimeMillis + ": " + "TrackMultipleValues" + serverSocket) + + var keepAccepting = true + try { + while (true) { + var clientSocket: Socket = null + try { + // TODO: + // serverSocket.setSoTimeout (serverSocketTimout) + clientSocket = serverSocket.accept + } catch { + case e: Exception => { + // println ("TrackMultipleValues Timeout. Stopping listening...") + keepAccepting = false + } + } + + if (clientSocket != null) { + try { + threadPool.execute (new Runnable { + def run = { + val oos = new ObjectOutputStream (clientSocket.getOutputStream) + val ois = new ObjectInputStream (clientSocket.getInputStream) + try { + val uuid = ois.readObject.asInstanceOf[UUID] + // masterListenPort/guidePort value legend + // 0 = missed the broadcast, read from HDFS; + // <0 = hasn't started yet, wait & retry; (never happens) + // >0 = Read from this port + var guidePort = if (valueToGuidePortMap.contains (uuid)) { + valueToGuidePortMap (uuid) + } else -1 + // println (System.currentTimeMillis + ": " + "TrackMultipleValues:Got new request: " + clientSocket + " for " + uuid + " : " + guidePort) + oos.writeObject (guidePort) + } catch { + case e: Exception => { } + } finally { + ois.close + oos.close + clientSocket.close + } + } + }) + } catch { + // In failure, close the socket here; else, the thread will close it + case ioe: IOException => clientSocket.close + } + } + } + } finally { + serverSocket.close + } + } } } diff --git a/src/scala/spark/HdfsFile.scala b/src/scala/spark/HdfsFile.scala index 87d8e8cc81..8050683f99 100644 --- a/src/scala/spark/HdfsFile.scala +++ b/src/scala/spark/HdfsFile.scala @@ -27,9 +27,9 @@ import org.apache.hadoop.mapred.Reporter abstract class DistributedFile[T, Split](@transient sc: SparkContext) { def splits: Array[Split] def iterator(split: Split): Iterator[T] - def preferredLocations(split: Split): Seq[String] + def prefers(split: Split, slot: SlaveOffer): Boolean - def taskStarted(split: Split, offer: SlaveOffer) {} + def taskStarted(split: Split, slot: SlaveOffer) {} def sparkContext = sc @@ -87,8 +87,8 @@ abstract class DistributedFile[T, Split](@transient sc: SparkContext) { abstract class FileTask[U, T, Split](val file: DistributedFile[T, Split], val split: Split) extends Task[U] { - override def preferredLocations: Seq[String] = file.preferredLocations(split) - override def markStarted(offer: SlaveOffer) { file.taskStarted(split, offer) } + override def prefers(slot: SlaveOffer) = file.prefers(split, slot) + override def markStarted(slot: SlaveOffer) { file.taskStarted(split, slot) } } class ForeachTask[T, Split](file: DistributedFile[T, Split], @@ -124,31 +124,31 @@ extends FileTask[Option[T], T, Split](file, split) { class MappedFile[U, T, Split](prev: DistributedFile[T, Split], f: T => U) extends DistributedFile[U, Split](prev.sparkContext) { override def splits = prev.splits - override def preferredLocations(sp: Split) = prev.preferredLocations(sp) + override def prefers(split: Split, slot: SlaveOffer) = prev.prefers(split, slot) override def iterator(split: Split) = prev.iterator(split).map(f) - override def taskStarted(split: Split, offer: SlaveOffer) = prev.taskStarted(split, offer) + override def taskStarted(split: Split, slot: SlaveOffer) = prev.taskStarted(split, slot) } class FilteredFile[T, Split](prev: DistributedFile[T, Split], f: T => Boolean) extends DistributedFile[T, Split](prev.sparkContext) { override def splits = prev.splits - override def preferredLocations(sp: Split) = prev.preferredLocations(sp) + override def prefers(split: Split, slot: SlaveOffer) = prev.prefers(split, slot) override def iterator(split: Split) = prev.iterator(split).filter(f) - override def taskStarted(split: Split, offer: SlaveOffer) = prev.taskStarted(split, offer) + override def taskStarted(split: Split, slot: SlaveOffer) = prev.taskStarted(split, slot) } class CachedFile[T, Split](prev: DistributedFile[T, Split]) extends DistributedFile[T, Split](prev.sparkContext) { val id = CachedFile.newId() - @transient val cacheLocs = Map[Split, List[String]]() + @transient val cacheLocs = Map[Split, List[Int]]() override def splits = prev.splits - override def preferredLocations(split: Split): Seq[String] = { + override def prefers(split: Split, slot: SlaveOffer): Boolean = { if (cacheLocs.contains(split)) - cacheLocs(split) + cacheLocs(split).contains(slot.getSlaveId) else - prev.preferredLocations(split) + prev.prefers(split, slot) } override def iterator(split: Split): Iterator[T] = { @@ -183,11 +183,11 @@ extends DistributedFile[T, Split](prev.sparkContext) { } } - override def taskStarted(split: Split, offer: SlaveOffer) { + override def taskStarted(split: Split, slot: SlaveOffer) { val oldList = cacheLocs.getOrElse(split, Nil) - val host = offer.getHost - if (!oldList.contains(host)) - cacheLocs(split) = host :: oldList + val slaveId = slot.getSlaveId + if (!oldList.contains(slaveId)) + cacheLocs(split) = slaveId :: oldList } } @@ -251,10 +251,8 @@ extends DistributedFile[String, HdfsSplit](sc) { } } - override def preferredLocations(split: HdfsSplit) = { - // TODO: Filtering out "localhost" in case of file:// URLs - split.value.getLocations().filter(_ != "localhost").toArray - } + override def prefers(split: HdfsSplit, slot: SlaveOffer) = + split.value.getLocations().contains(slot.getHost) } object ConfigureLock {} diff --git a/src/scala/spark/NexusScheduler.scala b/src/scala/spark/NexusScheduler.scala index a8a5e2947a..a96fca9350 100644 --- a/src/scala/spark/NexusScheduler.scala +++ b/src/scala/spark/NexusScheduler.scala @@ -1,11 +1,11 @@ package spark import java.io.File +import java.util.concurrent.Semaphore -import scala.collection.mutable.Map - -import nexus.{Scheduler => NScheduler} -import nexus._ +import nexus.{ExecutorInfo, TaskDescription, TaskState, TaskStatus} +import nexus.{SlaveOffer, SchedulerDriver, NexusSchedulerDriver} +import nexus.{SlaveOfferVector, TaskDescriptionVector, StringMap} // The main Scheduler implementation, which talks to Nexus. Clients are expected // to first call start(), then submit tasks through the runTasks method. @@ -21,26 +21,30 @@ import nexus._ // can be made cleaner. private class NexusScheduler( master: String, frameworkName: String, execArg: Array[Byte]) -extends NScheduler with spark.Scheduler +extends nexus.Scheduler with spark.Scheduler { - // Lock used by runTasks to ensure only one thread can be in it - val runTasksMutex = new Object() + // Semaphore used by runTasks to ensure only one thread can be in it + val semaphore = new Semaphore(1) // Lock used to wait for scheduler to be registered var isRegistered = false val registeredLock = new Object() + // Trait representing a set of scheduler callbacks + trait Callbacks { + def slotOffer(s: SlaveOffer): Option[TaskDescription] + def taskFinished(t: TaskStatus): Unit + def error(code: Int, message: String): Unit + } + // Current callback object (may be null) - var activeOp: ParallelOperation = null + var callbacks: Callbacks = null // Incrementing task ID - private var nextTaskId = 0 + var nextTaskId = 0 - def newTaskId(): Int = { - val id = nextTaskId; - nextTaskId += 1; - return id - } + // Maximum time to wait to run a task in a preferred location (in ms) + val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "1000").toLong // Driver for talking to Nexus var driver: SchedulerDriver = null @@ -62,27 +66,125 @@ extends NScheduler with spark.Scheduler new ExecutorInfo(new File("spark-executor").getCanonicalPath(), execArg) override def runTasks[T](tasks: Array[Task[T]]): Array[T] = { - runTasksMutex.synchronized { - waitForRegister() - val myOp = new SimpleParallelOperation(this, tasks) + val results = new Array[T](tasks.length) + if (tasks.length == 0) + return results + + val launched = new Array[Boolean](tasks.length) + + val callingThread = currentThread + + var errorHappened = false + var errorCode = 0 + var errorMessage = "" + + // Wait for scheduler to be registered with Nexus + waitForRegister() + + try { + // Acquire the runTasks semaphore + semaphore.acquire() + + val myCallbacks = new Callbacks { + val firstTaskId = nextTaskId + var tasksLaunched = 0 + var tasksFinished = 0 + var lastPreferredLaunchTime = System.currentTimeMillis + + def slotOffer(slot: SlaveOffer): Option[TaskDescription] = { + try { + if (tasksLaunched < tasks.length) { + // TODO: Add a short wait if no task with location pref is found + // TODO: Figure out why a function is needed around this to + // avoid scala.runtime.NonLocalReturnException + def findTask: Option[TaskDescription] = { + var checkPrefVals: Array[Boolean] = Array(true) + val time = System.currentTimeMillis + if (time - lastPreferredLaunchTime > LOCALITY_WAIT) + checkPrefVals = Array(true, false) // Allow non-preferred tasks + // TODO: Make desiredCpus and desiredMem configurable + val desiredCpus = 1 + val desiredMem = 750L * 1024L * 1024L + if (slot.getParams.get("cpus").toInt < desiredCpus || + slot.getParams.get("mem").toLong < desiredMem) + return None + for (checkPref <- checkPrefVals; + i <- 0 until tasks.length; + if !launched(i) && (!checkPref || tasks(i).prefers(slot))) + { + val taskId = nextTaskId + nextTaskId += 1 + printf("Starting task %d as TID %d on slave %d: %s (%s)\n", + i, taskId, slot.getSlaveId, slot.getHost, + if(checkPref) "preferred" else "non-preferred") + tasks(i).markStarted(slot) + launched(i) = true + tasksLaunched += 1 + if (checkPref) + lastPreferredLaunchTime = time + val params = new StringMap + params.set("cpus", "" + desiredCpus) + params.set("mem", "" + desiredMem) + val serializedTask = Utils.serialize(tasks(i)) + return Some(new TaskDescription(taskId, slot.getSlaveId, + "task_" + taskId, params, serializedTask)) + } + return None + } + return findTask + } else { + return None + } + } catch { + case e: Exception => { + e.printStackTrace + System.exit(1) + return None + } + } + } - try { - this.synchronized { - this.activeOp = myOp + def taskFinished(status: TaskStatus) { + println("Finished TID " + status.getTaskId) + // Deserialize task result + val result = Utils.deserialize[TaskResult[T]](status.getData) + results(status.getTaskId - firstTaskId) = result.value + // Update accumulators + Accumulators.add(callingThread, result.accumUpdates) + // Stop if we've finished all the tasks + tasksFinished += 1 + if (tasksFinished == tasks.length) { + NexusScheduler.this.callbacks = null + NexusScheduler.this.notifyAll() + } } - driver.reviveOffers(); - myOp.join(); - } finally { - this.synchronized { - this.activeOp = null + + def error(code: Int, message: String) { + // Save the error message + errorHappened = true + errorCode = code + errorMessage = message + // Indicate to caller thread that we're done + NexusScheduler.this.callbacks = null + NexusScheduler.this.notifyAll() } } - if (myOp.errorHappened) - throw new SparkException(myOp.errorMessage, myOp.errorCode) - else - return myOp.results + this.synchronized { + this.callbacks = myCallbacks + } + driver.reviveOffers(); + this.synchronized { + while (this.callbacks != null) this.wait() + } + } finally { + semaphore.release() } + + if (errorHappened) + throw new SparkException(errorMessage, errorCode) + else + return results } override def registered(d: SchedulerDriver, frameworkId: Int) { @@ -95,19 +197,18 @@ extends NScheduler with spark.Scheduler override def waitForRegister() { registeredLock.synchronized { - while (!isRegistered) - registeredLock.wait() + while (!isRegistered) registeredLock.wait() } } override def resourceOffer( - d: SchedulerDriver, oid: Long, offers: SlaveOfferVector) { + d: SchedulerDriver, oid: Long, slots: SlaveOfferVector) { synchronized { val tasks = new TaskDescriptionVector - if (activeOp != null) { + if (callbacks != null) { try { - for (i <- 0 until offers.size.toInt) { - activeOp.slaveOffer(offers.get(i)) match { + for (i <- 0 until slots.size.toInt) { + callbacks.slotOffer(slots.get(i)) match { case Some(task) => tasks.add(task) case None => {} } @@ -124,21 +225,21 @@ extends NScheduler with spark.Scheduler override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { synchronized { - try { - if (activeOp != null) { - activeOp.statusUpdate(status) + if (callbacks != null && status.getState == TaskState.TASK_FINISHED) { + try { + callbacks.taskFinished(status) + } catch { + case e: Exception => e.printStackTrace } - } catch { - case e: Exception => e.printStackTrace } } } override def error(d: SchedulerDriver, code: Int, message: String) { synchronized { - if (activeOp != null) { + if (callbacks != null) { try { - activeOp.error(code, message) + callbacks.error(code, message) } catch { case e: Exception => e.printStackTrace } @@ -155,135 +256,3 @@ extends NScheduler with spark.Scheduler driver.stop() } } - - -// Trait representing a set of scheduler callbacks -trait ParallelOperation { - def slaveOffer(s: SlaveOffer): Option[TaskDescription] - def statusUpdate(t: TaskStatus): Unit - def error(code: Int, message: String): Unit -} - - -class SimpleParallelOperation[T](sched: NexusScheduler, tasks: Array[Task[T]]) -extends ParallelOperation -{ - // Maximum time to wait to run a task in a preferred location (in ms) - val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "1000").toLong - - val callingThread = currentThread - val numTasks = tasks.length - val results = new Array[T](numTasks) - val launched = new Array[Boolean](numTasks) - val finished = new Array[Boolean](numTasks) - val tidToIndex = Map[Int, Int]() - - var allFinished = false - val joinLock = new Object() - - var errorHappened = false - var errorCode = 0 - var errorMessage = "" - - var tasksLaunched = 0 - var tasksFinished = 0 - var lastPreferredLaunchTime = System.currentTimeMillis - - def setAllFinished() { - joinLock.synchronized { - allFinished = true - joinLock.notifyAll() - } - } - - def join() { - joinLock.synchronized { - while (!allFinished) - joinLock.wait() - } - } - - def slaveOffer(offer: SlaveOffer): Option[TaskDescription] = { - if (tasksLaunched < numTasks) { - var checkPrefVals: Array[Boolean] = Array(true) - val time = System.currentTimeMillis - if (time - lastPreferredLaunchTime > LOCALITY_WAIT) - checkPrefVals = Array(true, false) // Allow non-preferred tasks - // TODO: Make desiredCpus and desiredMem configurable - val desiredCpus = 1 - val desiredMem = 750L * 1024L * 1024L - if (offer.getParams.get("cpus").toInt < desiredCpus || - offer.getParams.get("mem").toLong < desiredMem) - return None - for (checkPref <- checkPrefVals; i <- 0 until numTasks) { - if (!launched(i) && (!checkPref || - tasks(i).preferredLocations.contains(offer.getHost) || - tasks(i).preferredLocations.isEmpty)) - { - val taskId = sched.newTaskId() - tidToIndex(taskId) = i - printf("Starting task %d as TID %d on slave %d: %s (%s)\n", - i, taskId, offer.getSlaveId, offer.getHost, - if(checkPref) "preferred" else "non-preferred") - tasks(i).markStarted(offer) - launched(i) = true - tasksLaunched += 1 - if (checkPref) - lastPreferredLaunchTime = time - val params = new StringMap - params.set("cpus", "" + desiredCpus) - params.set("mem", "" + desiredMem) - val serializedTask = Utils.serialize(tasks(i)) - return Some(new TaskDescription(taskId, offer.getSlaveId, - "task_" + taskId, params, serializedTask)) - } - } - } - return None - } - - def statusUpdate(status: TaskStatus) { - status.getState match { - case TaskState.TASK_FINISHED => - taskFinished(status) - case TaskState.TASK_LOST => - taskLost(status) - case TaskState.TASK_FAILED => - taskLost(status) - case TaskState.TASK_KILLED => - taskLost(status) - case _ => - } - } - - def taskFinished(status: TaskStatus) { - val tid = status.getTaskId - println("Finished TID " + tid) - // Deserialize task result - val result = Utils.deserialize[TaskResult[T]](status.getData) - results(tidToIndex(tid)) = result.value - // Update accumulators - Accumulators.add(callingThread, result.accumUpdates) - // Mark finished and stop if we've finished all the tasks - finished(tidToIndex(tid)) = true - tasksFinished += 1 - if (tasksFinished == numTasks) - setAllFinished() - } - - def taskLost(status: TaskStatus) { - val tid = status.getTaskId - println("Lost TID " + tid) - launched(tidToIndex(tid)) = false - tasksLaunched -= 1 - } - - def error(code: Int, message: String) { - // Save the error message - errorHappened = true - errorCode = code - errorMessage = message - // Indicate to caller thread that we're done - setAllFinished() - } -} diff --git a/src/scala/spark/Task.scala b/src/scala/spark/Task.scala index efb864472d..e559996a37 100644 --- a/src/scala/spark/Task.scala +++ b/src/scala/spark/Task.scala @@ -5,8 +5,8 @@ import nexus._ @serializable trait Task[T] { def run: T - def preferredLocations: Seq[String] = Nil - def markStarted(offer: SlaveOffer) {} + def prefers(slot: SlaveOffer): Boolean = true + def markStarted(slot: SlaveOffer) {} } @serializable diff --git a/src/test/spark/repl/ReplSuite.scala b/src/test/spark/repl/ReplSuite.scala index 43ef296efe..d71fe20a94 100644 --- a/src/test/spark/repl/ReplSuite.scala +++ b/src/test/spark/repl/ReplSuite.scala @@ -85,15 +85,15 @@ class ReplSuite extends FunSuite { assertContains("res2: Int = 100", output) } - test ("broadcast vars") { - // Test that the value that a broadcast var had when it was created is used, - // even if that broadcast var is then modified in the driver program + test ("cached vars") { + // Test that the value that a cached var had when it was created is used, + // even if that cached var is then modified in the driver program val output = runInterpreter("local", """ var array = new Array[Int](5) - val broadcastedArray = sc.broadcast(array) - sc.parallelize(0 to 4).map(x => broadcastedArray.value(x)).toArray + val cachedArray = sc.cache(array) + sc.parallelize(0 to 4).map(x => cachedArray.value(x)).toArray array(0) = 5 - sc.parallelize(0 to 4).map(x => broadcastedArray.value(x)).toArray + sc.parallelize(0 to 4).map(x => cachedArray.value(x)).toArray """) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -109,10 +109,10 @@ class ReplSuite extends FunSuite { v = 10 sc.parallelize(1 to 10).map(x => getV()).toArray.reduceLeft(_+_) var array = new Array[Int](5) - val broadcastedArray = sc.broadcast(array) - sc.parallelize(0 to 4).map(x => broadcastedArray.value(x)).toArray + val cachedArray = sc.cache(array) + sc.parallelize(0 to 4).map(x => cachedArray.value(x)).toArray array(0) = 5 - sc.parallelize(0 to 4).map(x => broadcastedArray.value(x)).toArray + sc.parallelize(0 to 4).map(x => cachedArray.value(x)).toArray """) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) -- cgit v1.2.3 From e0db4e04821822b50294da4ff040e36f16163627 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Sun, 4 Apr 2010 13:50:25 -0700 Subject: - HDFS storing is in separate thread. - Receivers now ask for a range instead of expecting the whole variable. But, they are still asking for the whole range from a single source. - Next step: make receivers ask for different parts from different sources. Also, make sure that Master sends back a list of sources instead of a single one. --- run | 2 +- src/examples/BroadcastTest.scala | 2 +- src/scala/spark/Broadcast.scala | 54 ++++++++++++++++++++++++++------------- third_party/nexus.jar | Bin 34246 -> 34246 bytes 4 files changed, 38 insertions(+), 20 deletions(-) diff --git a/run b/run index 82a12b0113..415d50cabf 100755 --- a/run +++ b/run @@ -4,7 +4,7 @@ FWDIR=`dirname $0` # Set JAVA_OPTS to be able to load libnexus.so and set various other misc options -export JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xmx2000m -Dspark.broadcast.masterHostAddress=127.0.0.1 -Dspark.broadcast.masterTrackerPort=11111 -Dspark.broadcast.blockSize=1024 -Dspark.broadcast.maxRetryCount=2 -Dspark.broadcast.serverSocketTimout=50000 -Dspark.broadcast.dualMode=false" +export JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xmx2000m -Dspark.broadcast.masterHostAddress=127.0.0.1 -Dspark.broadcast.masterTrackerPort=11111 -Dspark.broadcast.blockSize=4096 -Dspark.broadcast.maxRetryCount=2 -Dspark.broadcast.serverSocketTimout=50000 -Dspark.broadcast.dualMode=false" if [ -e $FWDIR/conf/java-opts ] ; then JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" diff --git a/src/examples/BroadcastTest.scala b/src/examples/BroadcastTest.scala index 7764013413..05778668bd 100644 --- a/src/examples/BroadcastTest.scala +++ b/src/examples/BroadcastTest.scala @@ -3,7 +3,7 @@ import spark.SparkContext object BroadcastTest { def main(args: Array[String]) { if (args.length == 0) { - System.err.println("Usage: BroadcastTest []") + System.err.println("Usage: BroadcastTest [] []") System.exit(1) } val spark = new SparkContext(args(0), "Broadcast Test") diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 73b9ea39e8..cc81bac01f 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -52,6 +52,8 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) @transient var hostAddress = InetAddress.getLocalHost.getHostAddress @transient var listenPort = -1 @transient var guidePort = -1 + + @transient var hasCopyInHDFS = false if (!local) { sendBroadcast } @@ -99,16 +101,17 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) guidePortLock.wait } } - - BroadcastCS.synchronized { - BroadcastCS.registerValue (uuid, guidePort) - } + BroadcastCS.registerValue (uuid, guidePort) - // TODO: Make it a separate thread? - // Now store a persistent copy in HDFS, just in case - val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) - out.writeObject (value_) - out.close + // Now store a persistent copy in HDFS, in a separate thread + new Runnable { + override def run = { + val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) + out.writeObject (value_) + out.close + hasCopyInHDFS = true + } + } } private def readObject (in: ObjectInputStream) { @@ -218,12 +221,12 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) var oisTracker: ObjectInputStream = null var oosTracker: ObjectOutputStream = null - var masterListenPort: Int = -1 - // masterListenPort aka guidePort value legend // 0 = missed the broadcast, read from HDFS; - // <0 = hasn't started yet, wait & retry; (never happens) + // <0 = hasn't started yet, wait & retry; // >0 = Read from this port + var masterListenPort: Int = -1 + var retriesLeft = BroadcastCS.maxRetryCount do { try { @@ -314,7 +317,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) return retByteArray } - // Tries to receive broadcast from the Master and returns Boolean status. + // Tries to receive broadcast from the source and returns Boolean status. // This might be called multiple times to retry a defined number of times. private def receiveSingleTransmission(sourceInfo: SourceInfo): Array[Byte] = { var clientSocketToSource: Socket = null @@ -334,6 +337,10 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // println (System.currentTimeMillis + ": " + "Inside receiveSingleTransmission") // println (System.currentTimeMillis + ": " + "totalBlocks: "+ totalBlocks + " " + "hasBlocks: " + hasBlocks) + + // Send the range + oosSource.writeObject((0, totalBlocks)) + retByteArray = new Array[Byte] (totalBytes) for (i <- 0 until totalBlocks) { val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock] @@ -379,7 +386,8 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) var keepAccepting = true try { - while (keepAccepting) { + // Don't stop until there is a copy in HDFS + while (keepAccepting || !hasCopyInHDFS) { var clientSocket: Socket = null try { serverSocket.setSoTimeout (BroadcastCS.serverSocketTimout) @@ -400,6 +408,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } } } + BroadcastCS.unregisterValue (uuid) } finally { serverSocket.close } @@ -549,9 +558,18 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) private val oos = new ObjectOutputStream (clientSocket.getOutputStream) private val ois = new ObjectInputStream (clientSocket.getInputStream) + private var sendFrom = 0 + private var sendUntil = totalBlocks + def run = { try { // println (System.currentTimeMillis + ": " + "new ServeSingleRequest is running") + + // Receive range to send + var sendRange = ois.readObject.asInstanceOf[(Int, Int)] + sendFrom = sendRange._1 + sendUntil = sendRange._2 + sendObject } catch { // TODO: Need to add better exception handling here @@ -576,7 +594,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } } - for (i <- 0 until totalBlocks) { + for (i <- sendFrom until sendUntil) { while (i == hasBlocks) { hasBlocksLock.synchronized { hasBlocksLock.wait @@ -744,9 +762,9 @@ private object BroadcastCS { } } - // TODO: Who call this and when? def unregisterValue (uuid: UUID) { valueToGuidePortMap.synchronized { + // Set to 0 to make sure that people read it from HDFS valueToGuidePortMap (uuid) = 0 // println (System.currentTimeMillis + ": " + "Value unregistered from the Tracker " + valueToGuidePortMap) } @@ -778,14 +796,14 @@ private object BroadcastCS { if (clientSocket != null) { try { threadPool.execute (new Runnable { - def run = { + override def run = { val oos = new ObjectOutputStream (clientSocket.getOutputStream) val ois = new ObjectInputStream (clientSocket.getInputStream) try { val uuid = ois.readObject.asInstanceOf[UUID] // masterListenPort/guidePort value legend // 0 = missed the broadcast, read from HDFS; - // <0 = hasn't started yet, wait & retry; (never happens) + // <0 = hasn't started yet, wait & retry; // >0 = Read from this port var guidePort = if (valueToGuidePortMap.contains (uuid)) { valueToGuidePortMap (uuid) diff --git a/third_party/nexus.jar b/third_party/nexus.jar index 3c0b10ed62..6d007e8ac4 100644 Binary files a/third_party/nexus.jar and b/third_party/nexus.jar differ -- cgit v1.2.3 From c6962f516ea00025232e68000d509096c33bc73f Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Wed, 14 Apr 2010 21:19:32 -0700 Subject: Several things, but the most important one is that now we are using node speed to select source instead of leecher count. --- run | 2 +- src/scala/spark/Broadcast.scala | 84 +++++++++++++++++++++++++++++++---------- 2 files changed, 66 insertions(+), 20 deletions(-) diff --git a/run b/run index 415d50cabf..2e78025d7c 100755 --- a/run +++ b/run @@ -4,7 +4,7 @@ FWDIR=`dirname $0` # Set JAVA_OPTS to be able to load libnexus.so and set various other misc options -export JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xmx2000m -Dspark.broadcast.masterHostAddress=127.0.0.1 -Dspark.broadcast.masterTrackerPort=11111 -Dspark.broadcast.blockSize=4096 -Dspark.broadcast.maxRetryCount=2 -Dspark.broadcast.serverSocketTimout=50000 -Dspark.broadcast.dualMode=false" +export JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xmx2000m -Dspark.broadcast.masterHostAddress=127.0.0.1 -Dspark.broadcast.masterTrackerPort=11111 -Dspark.broadcast.blockSize=4096 -Dspark.broadcast.maxRetryCount=2 -Dspark.broadcast.serverSocketTimout=5000 -Dspark.broadcast.dualMode=false" if [ -e $FWDIR/conf/java-opts ] ; then JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index cc81bac01f..e6cac5389f 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -104,14 +104,16 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) BroadcastCS.registerValue (uuid, guidePort) // Now store a persistent copy in HDFS, in a separate thread - new Runnable { - override def run = { + // new Runnable { + // override def run = { + // TODO: When threaded, its not written to file + // TODO: On second thought, its better to have it stored before anything starts val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) out.writeObject (value_) out.close hasCopyInHDFS = true - } - } + // } + // } } private def readObject (in: ObjectInputStream) { @@ -139,8 +141,6 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) if (retByteArray != null) { value_ = byteArrayToObject[T] (retByteArray) BroadcastCS.values.put (uuid, value_) - // val variableInfo = blockifyObject (value_, BroadcastCS.blockSize) - // BroadcastCS.valueInfos.put (uuid, variableInfo) } else { val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) value_ = fileIn.readObject.asInstanceOf[T] @@ -294,15 +294,17 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // println (System.currentTimeMillis + ": " + "Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort) + val start = System.nanoTime retByteArray = receiveSingleTransmission (sourceInfo) + val time = (System.nanoTime - start) / 1e9 // println (System.currentTimeMillis + ": " + "I got this from receiveSingleTransmission: " + retByteArray) // TODO: Update sourceInfo to add error notifactions for Master if (retByteArray == null) { sourceInfo.receptionFailed = true } - // TODO: Supposed to update values here, but we don't support advanced - // statistics right now. Master can handle leecherCount by itself. + // Updating some statistics here. Master will be using them later + sourceInfo.MBps = (sourceInfo.totalBytes.toDouble / 1048576) / time // Send back statistics to the Master oosMaster.writeObject (sourceInfo) @@ -362,9 +364,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } } finally { if (oisSource != null) { oisSource.close } - if (oosSource != null) { - oosSource.close - } + if (oosSource != null) { oosSource.close } if (clientSocketToSource != null) { clientSocketToSource.close } } @@ -373,6 +373,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) class GuideMultipleRequests extends Thread { override def run = { + // TODO: Cached threadpool has 60 s keep alive timer var threadPool = Executors.newCachedThreadPool var serverSocket: ServerSocket = null @@ -394,7 +395,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) clientSocket = serverSocket.accept } catch { case e: Exception => { - // println ("GuideMultipleRequests Timeout. Stopping listening...") + // println ("GuideMultipleRequests Timeout. Stopping listening..." + hasCopyInHDFS) keepAccepting = false } } @@ -453,11 +454,18 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // Remove first pqOfSources.remove (selectedSourceInfo) // TODO: Removing a source based on just one failure notification! - // Update leecher count and put it back in IF reception succeeded + // Update sourceInfo and put it back in, IF reception succeeded if (!sourceInfo.receptionFailed) { selectedSourceInfo.currentLeechers -= 1 + selectedSourceInfo.MBps = sourceInfo.MBps + + // Put it back pqOfSources.add (selectedSourceInfo) + // Update global source speed statistics + BroadcastCS.setSourceSpeed ( + sourceInfo.hostAddress, sourceInfo.MBps) + // No need to find and update thisWorkerInfo, but add its replica if (BroadcastCS.dualMode) { pqOfSources.add (new SourceInfo (thisWorkerInfo.hostAddress, @@ -469,7 +477,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // If something went wrong, e.g., the worker at the other end died etc. // then close everything up case e: Exception => { - // Assuming that exception caused due to receiver worker failure + // Assuming that exception caused due to receiver worker failure. // Remove failed worker from pqOfSources and update leecherCount of // corresponding source worker pqOfSources.synchronized { @@ -659,8 +667,27 @@ case class SourceInfo (val hostAddress: String, val listenPort: Int, var currentLeechers = 0 var receptionFailed = false + var MBps: Double = 0.0 - def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers) + // Ascending sort based on leecher count + // def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers) + + // Descending sort based on speed + // def compareTo (o: SourceInfo): Int = { + // if (MBps > o.MBps) -1 + // else if (MBps < o.MBps) 1 + // else 0 + // } + + // Descending sort based on globally stored speed + def compareTo (o: SourceInfo): Int = { + val mySpeed = BroadcastCS.getSourceSpeed (hostAddress) + val urSpeed = BroadcastCS.getSourceSpeed (o.hostAddress) + + if (mySpeed > urSpeed) -1 + else if (mySpeed < urSpeed) 1 + else 0 + } } @serializable @@ -700,9 +727,10 @@ private object Broadcast { private object BroadcastCS { val values = new MapMaker ().softValues ().makeMap[UUID, Any] - // val valueInfos = new MapMaker ().softValues ().makeMap[UUID, Any] var valueToGuidePortMap = Map[UUID, Int] () + + var sourceToSpeedMap = Map[String, Double] () private var initialized = false private var isMaster_ = false @@ -716,6 +744,9 @@ private object BroadcastCS { private var trackMV: TrackMultipleValues = null + // newSpeed = ALPHA * oldSpeed + (1 - ALPHA) * curSpeed + private val ALPHA = 0.7 + def initialize (isMaster__ : Boolean) { synchronized { if (!initialized) { @@ -736,7 +767,7 @@ private object BroadcastCS { if (isMaster) { trackMV = new TrackMultipleValues - // trackMV.setDaemon (true) + trackMV.setDaemon (true) trackMV.start // println (System.currentTimeMillis + ": " + "TrackMultipleValues started") } @@ -762,7 +793,7 @@ private object BroadcastCS { } } - def unregisterValue (uuid: UUID) { + def unregisterValue (uuid: UUID) = { valueToGuidePortMap.synchronized { // Set to 0 to make sure that people read it from HDFS valueToGuidePortMap (uuid) = 0 @@ -770,6 +801,20 @@ private object BroadcastCS { } } + def getSourceSpeed (hostAddress: String): Double = { + sourceToSpeedMap.synchronized { + sourceToSpeedMap.getOrElseUpdate(hostAddress, 0.0) + } + } + + def setSourceSpeed (hostAddress: String, MBps: Double) = { + sourceToSpeedMap.synchronized { + var oldSpeed = sourceToSpeedMap.getOrElseUpdate(hostAddress, 0.0) + var newSpeed = ALPHA * oldSpeed + (1 - ALPHA) * MBps + sourceToSpeedMap.update (hostAddress, newSpeed) + } + } + class TrackMultipleValues extends Thread { override def run = { var threadPool = Executors.newCachedThreadPool @@ -789,7 +834,8 @@ private object BroadcastCS { } catch { case e: Exception => { // println ("TrackMultipleValues Timeout. Stopping listening...") - keepAccepting = false + // TODO: Tracking should be explicitly stopped by the SparkContext + // keepAccepting = false } } -- cgit v1.2.3 From ee6c524fdf2046a52152d5327789ca492169ecf4 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Thu, 15 Apr 2010 21:39:57 -0700 Subject: Fixed some bugs in speed-based PQ-ing. --- src/scala/spark/Broadcast.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index e6cac5389f..fb1a877a71 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -183,7 +183,6 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) var retVal = new Array[BroadcastBlock] (blockNum) var blockID = 0 - // TODO: What happens in byteArray.length == 0 => blockNum == 0 for (i <- 0 until (byteArray.length, blockSize)) { val thisBlockSize = Math.min (blockSize, byteArray.length - i) var tempByteArray = new Array[Byte] (thisBlockSize) @@ -300,10 +299,8 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // println (System.currentTimeMillis + ": " + "I got this from receiveSingleTransmission: " + retByteArray) - // TODO: Update sourceInfo to add error notifactions for Master + // Updating some statistics in sourceInfo. Master will be using them later if (retByteArray == null) { sourceInfo.receptionFailed = true } - - // Updating some statistics here. Master will be using them later sourceInfo.MBps = (sourceInfo.totalBytes.toDouble / 1048576) / time // Send back statistics to the Master @@ -667,7 +664,7 @@ case class SourceInfo (val hostAddress: String, val listenPort: Int, var currentLeechers = 0 var receptionFailed = false - var MBps: Double = 0.0 + var MBps: Double = BroadcastCS.MaxMBps // Ascending sort based on leecher count // def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers) @@ -746,6 +743,8 @@ private object BroadcastCS { // newSpeed = ALPHA * oldSpeed + (1 - ALPHA) * curSpeed private val ALPHA = 0.7 + // 125.0 MBps = 1 Gbps link + private val MaxMBps_ = 125.0 def initialize (isMaster__ : Boolean) { synchronized { @@ -786,6 +785,8 @@ private object BroadcastCS { def isMaster = isMaster_ + def MaxMBps = MaxMBps_ + def registerValue (uuid: UUID, guidePort: Int) = { valueToGuidePortMap.synchronized { valueToGuidePortMap += (uuid -> guidePort) @@ -803,13 +804,13 @@ private object BroadcastCS { def getSourceSpeed (hostAddress: String): Double = { sourceToSpeedMap.synchronized { - sourceToSpeedMap.getOrElseUpdate(hostAddress, 0.0) + sourceToSpeedMap.getOrElseUpdate(hostAddress, MaxMBps) } } def setSourceSpeed (hostAddress: String, MBps: Double) = { sourceToSpeedMap.synchronized { - var oldSpeed = sourceToSpeedMap.getOrElseUpdate(hostAddress, 0.0) + var oldSpeed = sourceToSpeedMap.getOrElseUpdate(hostAddress, MaxMBps) var newSpeed = ALPHA * oldSpeed + (1 - ALPHA) * MBps sourceToSpeedMap.update (hostAddress, newSpeed) } -- cgit v1.2.3 From bb0178d1e49d1360e684240645ada421f36a3818 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Thu, 15 Apr 2010 22:41:44 -0700 Subject: - Receiving retry now starts from where the last try left off, not from the very beginning. - Some refactoring. --- src/scala/spark/Broadcast.scala | 68 +++++++++++++++++++++++------------------ 1 file changed, 39 insertions(+), 29 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index fb1a877a71..40353decde 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -136,10 +136,11 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) val start = System.nanoTime - val retByteArray = receiveBroadcast (uuid) + val receptionSucceeded = receiveBroadcast (uuid) // If does not succeed, then get from HDFS copy - if (retByteArray != null) { - value_ = byteArrayToObject[T] (retByteArray) + if (receptionSucceeded) { + // value_ = byteArrayToObject[T] (retByteArray) + value_ = unBlockifyObject[T] BroadcastCS.values.put (uuid, value_) } else { val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) @@ -199,6 +200,15 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) return variableInfo } + private def unBlockifyObject[A]: A = { + var retByteArray = new Array[Byte] (totalBytes) + for (i <- 0 until totalBlocks) { + System.arraycopy (arrayOfBlocks(i).byteArray, 0, retByteArray, + i * BroadcastCS.blockSize, arrayOfBlocks(i).byteArray.length) + } + byteArrayToObject (retByteArray) + } + private def byteArrayToObject[A] (bytes: Array[Byte]): A = { val in = new ObjectInputStream (new ByteArrayInputStream (bytes)) val retVal = in.readObject.asInstanceOf[A] @@ -215,15 +225,15 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) return bOut } - def receiveBroadcast (variableUUID: UUID): Array[Byte] = { + // masterListenPort aka guidePort value legend + // 0 = missed the broadcast, read from HDFS; + // <0 = hasn't started yet, wait & retry; + // >0 = Read from this port + def getMasterListenPort (variableUUID: UUID): Int = { var clientSocketToTracker: Socket = null var oisTracker: ObjectInputStream = null var oosTracker: ObjectOutputStream = null - // masterListenPort aka guidePort value legend - // 0 = missed the broadcast, read from HDFS; - // <0 = hasn't started yet, wait & retry; - // >0 = Read from this port var masterListenPort: Int = -1 var retriesLeft = BroadcastCS.maxRetryCount @@ -248,13 +258,18 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) if (oosTracker != null) { oosTracker.close } if (clientSocketToTracker != null) { clientSocketToTracker.close } } - retriesLeft -= 1 + // TODO: Should wait before retrying } while (retriesLeft > 0 && masterListenPort < 0) // println (System.currentTimeMillis + ": " + "Got this guidePort from Tracker: " + masterListenPort) - + return masterListenPort + } + + def receiveBroadcast (variableUUID: UUID): Boolean = { + // Get masterListenPort for this variable from the Tracker + val masterListenPort = getMasterListenPort (variableUUID) // If Tracker says that there is no guide for this object, read from HDFS - if (masterListenPort == 0) { return null } + if (masterListenPort == 0) { return false } // Wait until hostAddress and listenPort are created by the // ServeMultipleRequests thread @@ -266,8 +281,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // Connect and receive broadcast from the specified source, retrying the // specified number of times in case of failures - retriesLeft = BroadcastCS.maxRetryCount - var retByteArray: Array[Byte] = null + var retriesLeft = BroadcastCS.maxRetryCount do { // Connect to Master and send this worker's Information val clientSocketToMaster = @@ -294,13 +308,13 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // println (System.currentTimeMillis + ": " + "Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort) val start = System.nanoTime - retByteArray = receiveSingleTransmission (sourceInfo) + val receptionSucceeded = receiveSingleTransmission (sourceInfo) val time = (System.nanoTime - start) / 1e9 // println (System.currentTimeMillis + ": " + "I got this from receiveSingleTransmission: " + retByteArray) // Updating some statistics in sourceInfo. Master will be using them later - if (retByteArray == null) { sourceInfo.receptionFailed = true } + if (!receptionSucceeded) { sourceInfo.receptionFailed = true } sourceInfo.MBps = (sourceInfo.totalBytes.toDouble / 1048576) / time // Send back statistics to the Master @@ -308,23 +322,22 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) oisMaster.close oosMaster.close - clientSocketToMaster.close + clientSocketToMaster.close retriesLeft -= 1 - } while (retriesLeft > 0 && retByteArray == null) + } while (retriesLeft > 0 && hasBlocks != totalBlocks) - return retByteArray + return (hasBlocks == totalBlocks) } // Tries to receive broadcast from the source and returns Boolean status. // This might be called multiple times to retry a defined number of times. - private def receiveSingleTransmission(sourceInfo: SourceInfo): Array[Byte] = { + private def receiveSingleTransmission(sourceInfo: SourceInfo): Boolean = { var clientSocketToSource: Socket = null var oisSource: ObjectInputStream = null var oosSource: ObjectOutputStream = null - var retByteArray:Array[Byte] = null - + var receptionSucceeded = false try { // Connect to the source to get the object itself clientSocketToSource = @@ -338,25 +351,22 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // println (System.currentTimeMillis + ": " + "totalBlocks: "+ totalBlocks + " " + "hasBlocks: " + hasBlocks) // Send the range - oosSource.writeObject((0, totalBlocks)) + oosSource.writeObject((hasBlocks, totalBlocks)) - retByteArray = new Array[Byte] (totalBytes) - for (i <- 0 until totalBlocks) { + for (i <- hasBlocks until totalBlocks) { val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock] - System.arraycopy (bcBlock.byteArray, 0, retByteArray, - i * BroadcastCS.blockSize, bcBlock.byteArray.length) arrayOfBlocks(hasBlocks) = bcBlock hasBlocks += 1 + // Set to true if at least one block is received + receptionSucceeded = true hasBlocksLock.synchronized { hasBlocksLock.notifyAll } // println (System.currentTimeMillis + ": " + "Received block: " + i + " " + bcBlock) } - assert (hasBlocks == totalBlocks) // println (System.currentTimeMillis + ": " + "After the receive loop") } catch { case e: Exception => { - retByteArray = null // println (System.currentTimeMillis + ": " + "receiveSingleTransmission had a " + e) } } finally { @@ -365,7 +375,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) if (clientSocketToSource != null) { clientSocketToSource.close } } - return retByteArray + return receptionSucceeded } class GuideMultipleRequests extends Thread { -- cgit v1.2.3 From dc2c69e65943d4f4ba42ac9785b7b6da5b35ea4c Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Mon, 19 Apr 2010 00:14:53 -0700 Subject: SplitStream implementation in progress. --- Makefile | 1 + run | 5 +- src/scala/spark/Broadcast.scala | 141 ++++++++++++++++++++++++++++++++++++++++ third_party/FreePastry-2.1.jar | Bin 0 -> 2895564 bytes third_party/libnexus.so | Bin 0 -> 5823410 bytes 5 files changed, 145 insertions(+), 2 deletions(-) create mode 100644 third_party/FreePastry-2.1.jar create mode 100755 third_party/libnexus.so diff --git a/Makefile b/Makefile index cec6f755f4..d30f543826 100644 --- a/Makefile +++ b/Makefile @@ -10,6 +10,7 @@ JARS += third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar JARS += third_party/hadoop-0.20.0/lib/commons-logging-1.0.4.jar JARS += third_party/scalatest-1.0/scalatest-1.0.jar JARS += third_party/ScalaCheck-1.5.jar +JARS += third_party/FreePastry-2.1.jar CLASSPATH = $(subst $(SPACE),:,$(JARS)) SCALA_SOURCES = src/examples/*.scala src/scala/spark/*.scala src/scala/spark/repl/*.scala diff --git a/run b/run index 2e78025d7c..c11b2b8456 100755 --- a/run +++ b/run @@ -18,8 +18,9 @@ CLASSPATH+=:$FWDIR/third_party/asm-3.2/lib/all/asm-all-3.2.jar CLASSPATH+=:$FWDIR/third_party/colt.jar CLASSPATH+=:$FWDIR/third_party/google-collect-1.0-rc5/google-collect-1.0-rc5.jar CLASSPATH+=:$FWDIR/third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar -CLASSPATH+=:third_party/scalatest-1.0/scalatest-1.0.jar -CLASSPATH+=:third_party/ScalaCheck-1.5.jar +CLASSPATH+=:$FWDIR/third_party/scalatest-1.0/scalatest-1.0.jar +CLASSPATH+=:$FWDIR/third_party/ScalaCheck-1.5.jar +CLASSPATH+=:$FWDIR/third_party/FreePastry-2.1.jar for jar in $FWDIR/third_party/hadoop-0.20.0/lib/*.jar; do CLASSPATH+=:$jar done diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 40353decde..1a95860e32 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -15,6 +15,17 @@ import org.apache.hadoop.fs.{FileSystem, Path, RawLocalFileSystem} import spark.compress.lzf.{LZFInputStream, LZFOutputStream} +import rice.environment.Environment +import rice.p2p.commonapi._ +import rice.p2p.commonapi.rawserialization.RawMessage +import rice.pastry._ +import rice.pastry.commonapi.PastryIdFactory +import rice.pastry.direct._ +import rice.pastry.socket.SocketPastryNodeFactory +import rice.pastry.standard.RandomNodeIdFactory +import rice.p2p.scribe._ +import rice.p2p.splitstream._ + @serializable trait BroadcastRecipe { val uuid = UUID.randomUUID @@ -628,6 +639,33 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } } +@serializable +class SplitStreamBroadcast[T](@transient var value_ : T, local: Boolean) + extends BroadcastRecipe { + + def value = value_ + + BroadcastSS.synchronized { BroadcastSS.values.put (uuid, value_) } + + if (!local) { sendBroadcast } + + def sendBroadcast () { + + } + + private def readObject (in: ObjectInputStream) { + in.defaultReadObject + BroadcastSS.synchronized { + val cachedVal = BroadcastSS.values.get(uuid) + if (cachedVal != null) { + value_ = cachedVal.asInstanceOf[T] + } else { + + } + } + } +} + @serializable class CentralizedHDFSBroadcast[T](@transient var value_ : T, local: Boolean) extends BroadcastRecipe { @@ -725,6 +763,8 @@ private object Broadcast { BroadcastCH.initialize // Initialization for ChainedStreamingBroadcast BroadcastCS.initialize (isMaster) + // Initialization for SplitStreamBroadcast + BroadcastSS.initialize (isMaster) initialized = true } @@ -889,6 +929,107 @@ private object BroadcastCS { } } +private object BroadcastSS { + val values = new MapMaker ().softValues ().makeMap[UUID, Any] + + private var initialized = false + private var isMaster_ = false + + private var masterBootHost_ = "127.0.0.1" + private var masterBootPort_ : Int = 11111 + private var blockSize_ : Int = 512 * 1024 + private var maxRetryCount_ : Int = 2 + + private var masterBootAddress_ : InetSocketAddress = null + private var localBindPort_ : Int = -1 + + private var pEnvironment_ : Environment = null + private var pastryNode_ : PastryNode = null + + def initialize (isMaster__ : Boolean) { + synchronized { + if (!initialized) { + masterBootHost_ = + System.getProperty ("spark.broadcast.masterHostAddress", "127.0.0.1") + masterBootPort_ = + System.getProperty ("spark.broadcast.masterTrackerPort", "11111").toInt + + masterBootAddress_ = new InetSocketAddress(masterBootHost_, + masterBootPort_) + + blockSize_ = + System.getProperty ("spark.broadcast.blockSize", "512").toInt * 1024 + maxRetryCount_ = + System.getProperty ("spark.broadcast.maxRetryCount", "2").toInt + + isMaster_ = isMaster__ + + // Initialize the SplitStream tree + initializeSplitStream + + initialized = true + } + } + } + + def masterBootAddress = masterBootAddress_ + def blockSize = blockSize_ + def maxRetryCount = maxRetryCount_ + def pEnvironment = pEnvironment_ + def pastryNode = pastryNode_ + def localBindPort = { + if (localBindPort_ == -1) { + if (isMaster) { localBindPort_ = masterBootPort_ } + else { + // TODO: What's the best way of finding a free port? + val sSocket = new ServerSocket (0) + val sPort = sSocket.getLocalPort + sSocket.close + localBindPort_ = sPort + } + } + localBindPort_ + } + + def isMaster = isMaster_ + + private def initializeSplitStream = { + pEnvironment_ = new Environment + + // Generate the NodeIds Randomly + val nidFactory = new RandomNodeIdFactory (pEnvironment) + + // Construct the PastryNodeFactory + val pastryNodeFactory = new SocketPastryNodeFactory (nidFactory, + localBindPort, pEnvironment) + + // Construct a Pastry node + pastryNode_ = pastryNodeFactory.newNode + + // Boot the node. If its Master, start a new ring. + if (isMaster) { pastryNode.boot (null) } + else { pastryNode.boot (masterBootAddress)} + + // The node may require sending several messages to fully boot into the ring + pastryNode.synchronized { + while(!pastryNode.isReady && !pastryNode.joinFailed) { + // Delay so we don't busy-wait + pastryNode.wait(500); + + // Abort if can't join + if (pastryNode.joinFailed()) { + // TODO: throw new IOException("Join failed " + node.joinFailedReason) + } + } + } + + // construct a new splitstream application + // val app = new MySplitStreamClient(pastryNode) + // app.subscribe + // if (isMaster) { app.startPublishTask } + } +} + private object BroadcastCH { val values = new MapMaker ().softValues ().makeMap[UUID, Any] diff --git a/third_party/FreePastry-2.1.jar b/third_party/FreePastry-2.1.jar new file mode 100644 index 0000000000..51146e1541 Binary files /dev/null and b/third_party/FreePastry-2.1.jar differ diff --git a/third_party/libnexus.so b/third_party/libnexus.so new file mode 100755 index 0000000000..80c151751f Binary files /dev/null and b/third_party/libnexus.so differ -- cgit v1.2.3 From 1c1ac3161de50af7fbdb70f735f7ff7d8cc597d6 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Mon, 19 Apr 2010 20:32:17 -0700 Subject: More porting of SplitStream code. --- src/scala/spark/Broadcast.scala | 132 ++++++++++++++++++++++++++++++++++++++-- 1 file changed, 127 insertions(+), 5 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 1a95860e32..6aee3d5a26 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -640,7 +640,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } @serializable -class SplitStreamBroadcast[T](@transient var value_ : T, local: Boolean) +class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) extends BroadcastRecipe { def value = value_ @@ -664,6 +664,123 @@ class SplitStreamBroadcast[T](@transient var value_ : T, local: Boolean) } } } + + class SSClient (pastryNode: PastryNode) extends SplitStreamClient with Application { + // Length of a message in bytes. + val DATA_LENGTH = 10 + // Number of messages to publish. + val NUM_PUBLISHES = 10 + + /** + * The message sequence number. Will be incremented after each send. + * Out of laziness we are encoding this as a byte in the stream, so the range is limited + */ + var seqNum: Byte = 0 + + // Data source... + // protected RandomSource random; + + //This task kicks off publishing and anycasting. We hold it around in case + // we ever want to cancel the publishTask. + var publishTask: CancellableTask = null + + // The Endpoint represents the underlying node. By making calls on the + // Endpoint, it assures that the message will be delivered to a MyApp on + // whichever node the message is intended for. + protected val endpoint = pastryNode.buildEndpoint (this, "myInstance") + + // use this to generate data + // this.random = endpoint.getEnvironment().getRandomSource() + + // Handle to a SplitStream implementation + val mySplitStream = new SplitStreamImpl (pastryNode, "splitStreamImpl") + + // The ChannelId is constructed from a normal PastryId + val tmp = new PastryIdFactory (pastryNode.getEnvironment).buildId ("myChannel") + val myChannelId = new ChannelId (tmp) + + // The channel. + var myChannel: Channel= null + + // The stripes. Acquired from myChannel. + var myStripes: Array[Stripe] = null + + // Now we can receive messages + endpoint.register + + // Subscribes to all stripes in myChannelId. + def subscribe = { + // Attaching makes you part of the Channel, and volunteers to be an + // internal node of one of the trees + myChannel = mySplitStream.attachChannel (myChannelId) + + // Subscribing notifies your application when data comes through the tree + myStripes = myChannel.getStripes + for (curStripe <- myStripes) { curStripe.subscribe (this) } + } + + // Starts the publish task. + def startPublishTask = { + // TODO: The last two parameters are delays to wait before delivering the + // first and the subsequent messages. Needs tweaking. + publishTask = endpoint.scheduleMessage (new PublishContent, 5000, 5000) + } + + + // Part of the Application interface. Will receive PublishContent. + def deliver (id: rice.p2p.commonapi.Id, message: Message) = { + // TODO: Couldn't perform dynamic type checking. This can cause problems. + // if (message.isInstanceof[PublishContent]) + { publish } + } + + // Called whenever we receive a published message. + def deliver (s: Stripe, data: Array[Byte]) = { + println(endpoint.getId()+" deliver("+s+"):seq:"+data(0)+" stripe:"+data(1)+" "+data+")") + } + + /** + * Multicasts data. + */ + def publish = { + + for (curStripe <- myStripes) { + // format of the data: + // first byte: seqNum + // second byte: stripe + // rest: random + var data = new Array[Byte] (DATA_LENGTH) + + // yes, we waste some random bytes here + // random.nextBytes(data) + data(0) = seqNum + data(1) = 13 // curStripe + + // print what we are sending + println("Node "+endpoint.getLocalNodeHandle+" publishing "+seqNum+" "+data) + + // publish the data + curStripe.publish (data) + } + + // increment the sequence number + // seqNum = seqNum + 1 + + // cancel after sending all the messages + if (seqNum >= NUM_PUBLISHES) { publishTask.cancel } + } + + class PublishContent extends Message { + def getPriority: Int = { Message.MEDIUM_PRIORITY } + } + + // Error handling + def joinFailed(s: Stripe) = { println ("joinFailed(" + s + ")") } + + // Rest of the Application interface. NOT USED. + def forward (message: RouteMessage): Boolean = false + def update (handle: rice.p2p.commonapi.NodeHandle, joined: Boolean) = { } + } } @serializable @@ -965,7 +1082,7 @@ private object BroadcastSS { isMaster_ = isMaster__ // Initialize the SplitStream tree - initializeSplitStream + initializeSplitStream initialized = true } @@ -975,8 +1092,10 @@ private object BroadcastSS { def masterBootAddress = masterBootAddress_ def blockSize = blockSize_ def maxRetryCount = maxRetryCount_ + def pEnvironment = pEnvironment_ def pastryNode = pastryNode_ + def localBindPort = { if (localBindPort_ == -1) { if (isMaster) { localBindPort_ = masterBootPort_ } @@ -993,7 +1112,9 @@ private object BroadcastSS { def isMaster = isMaster_ - private def initializeSplitStream = { + private def initializeSplitStream: PastryNode = { + if (pastryNode != null) { return pastryNode } + pEnvironment_ = new Environment // Generate the NodeIds Randomly @@ -1006,9 +1127,9 @@ private object BroadcastSS { // Construct a Pastry node pastryNode_ = pastryNodeFactory.newNode - // Boot the node. If its Master, start a new ring. + // Boot the node. If its the Master, start a new ring. if (isMaster) { pastryNode.boot (null) } - else { pastryNode.boot (masterBootAddress)} + else { pastryNode.boot (masterBootAddress) } // The node may require sending several messages to fully boot into the ring pastryNode.synchronized { @@ -1023,6 +1144,7 @@ private object BroadcastSS { } } + return pastryNode // construct a new splitstream application // val app = new MySplitStreamClient(pastryNode) // app.subscribe -- cgit v1.2.3 From d2f1d0151a8ec8dc3da12569579251f2db154d8c Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Tue, 20 Apr 2010 02:08:48 -0700 Subject: SplitStream integration in progress. --- run | 2 +- src/scala/spark/Broadcast.scala | 195 +++++++++++++++---------------------- src/scala/spark/SparkContext.scala | 5 +- 3 files changed, 83 insertions(+), 119 deletions(-) diff --git a/run b/run index c11b2b8456..61d23adba4 100755 --- a/run +++ b/run @@ -4,7 +4,7 @@ FWDIR=`dirname $0` # Set JAVA_OPTS to be able to load libnexus.so and set various other misc options -export JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xmx2000m -Dspark.broadcast.masterHostAddress=127.0.0.1 -Dspark.broadcast.masterTrackerPort=11111 -Dspark.broadcast.blockSize=4096 -Dspark.broadcast.maxRetryCount=2 -Dspark.broadcast.serverSocketTimout=5000 -Dspark.broadcast.dualMode=false" +export JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xmx2000m -Dspark.broadcast.masterHostAddress=10.0.2.15 -Dspark.broadcast.masterTrackerPort=11111 -Dspark.broadcast.masterBootPort=22222 -Dspark.broadcast.blockSize=4096 -Dspark.broadcast.maxRetryCount=2 -Dspark.broadcast.serverSocketTimout=5000 -Dspark.broadcast.dualMode=false" if [ -e $FWDIR/conf/java-opts ] ; then JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 6aee3d5a26..bed2991713 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -69,6 +69,12 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) if (!local) { sendBroadcast } def sendBroadcast () { + // Store a persistent copy in HDFS + val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) + out.writeObject (value_) + out.close + hasCopyInHDFS = true + // Create a variableInfo object and store it in valueInfos var variableInfo = blockifyObject (value_, BroadcastCS.blockSize) @@ -112,19 +118,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) guidePortLock.wait } } - BroadcastCS.registerValue (uuid, guidePort) - - // Now store a persistent copy in HDFS, in a separate thread - // new Runnable { - // override def run = { - // TODO: When threaded, its not written to file - // TODO: On second thought, its better to have it stored before anything starts - val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) - out.writeObject (value_) - out.close - hasCopyInHDFS = true - // } - // } + BroadcastCS.registerValue (uuid, guidePort) } private def readObject (in: ObjectInputStream) { @@ -226,16 +220,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) in.close return retVal } - - private def getByteArrayOutputStream (obj: T): ByteArrayOutputStream = { - val bOut = new ByteArrayOutputStream - val out = new ObjectOutputStream (bOut) - out.writeObject (obj) - out.close - bOut.close - return bOut - } - + // masterListenPort aka guidePort value legend // 0 = missed the broadcast, read from HDFS; // <0 = hasn't started yet, wait & retry; @@ -649,8 +634,18 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) if (!local) { sendBroadcast } + @transient var hasCopyInHDFS = false + def sendBroadcast () { - + // Store a persistent copy in HDFS + val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) + out.writeObject (value_) + out.close + hasCopyInHDFS = true + + val ssClient = new SSClient (BroadcastSS.pastryNode) + ssClient.subscribe + ssClient.publish[T] (value, BroadcastSS.blockSize) } private def readObject (in: ObjectInputStream) { @@ -660,53 +655,43 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) if (cachedVal != null) { value_ = cachedVal.asInstanceOf[T] } else { - + val ssClient = new SSClient (BroadcastSS.pastryNode) + ssClient.subscribe } } } - class SSClient (pastryNode: PastryNode) extends SplitStreamClient with Application { - // Length of a message in bytes. - val DATA_LENGTH = 10 - // Number of messages to publish. - val NUM_PUBLISHES = 10 + class PublishThread (ssClient: SSClient) extends Runnable { + def run = { + + } + } - /** - * The message sequence number. Will be incremented after each send. - * Out of laziness we are encoding this as a byte in the stream, so the range is limited - */ - var seqNum: Byte = 0 - - // Data source... - // protected RandomSource random; + class SSClient (pastryNode: PastryNode) extends SplitStreamClient + with Application { + // Bytes reserved before each published block. 8 byte = 2 integer + val preAmbleSize = 8 - //This task kicks off publishing and anycasting. We hold it around in case - // we ever want to cancel the publishTask. - var publishTask: CancellableTask = null - // The Endpoint represents the underlying node. By making calls on the - // Endpoint, it assures that the message will be delivered to a MyApp on + // Endpoint, it assures that the message will be delivered to the App on // whichever node the message is intended for. - protected val endpoint = pastryNode.buildEndpoint (this, "myInstance") + protected val endPoint = pastryNode.buildEndpoint (this, "myInstance") - // use this to generate data - // this.random = endpoint.getEnvironment().getRandomSource() - // Handle to a SplitStream implementation - val mySplitStream = new SplitStreamImpl (pastryNode, "splitStreamImpl") + val mySplitStream = new SplitStreamImpl (pastryNode, "mySplitStream") - // The ChannelId is constructed from a normal PastryId - val tmp = new PastryIdFactory (pastryNode.getEnvironment).buildId ("myChannel") - val myChannelId = new ChannelId (tmp) + // The ChannelId is constructed from a normal PastryId based on the UUID + val myChannelId = new ChannelId (new PastryIdFactory + (pastryNode.getEnvironment).buildId ("myChannel")) - // The channel. - var myChannel: Channel= null + // The channel + var myChannel: Channel = null // The stripes. Acquired from myChannel. var myStripes: Array[Stripe] = null // Now we can receive messages - endpoint.register + endPoint.register // Subscribes to all stripes in myChannelId. def subscribe = { @@ -719,65 +704,52 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) for (curStripe <- myStripes) { curStripe.subscribe (this) } } - // Starts the publish task. - def startPublishTask = { - // TODO: The last two parameters are delays to wait before delivering the - // first and the subsequent messages. Needs tweaking. - publishTask = endpoint.scheduleMessage (new PublishContent, 5000, 5000) - } - - - // Part of the Application interface. Will receive PublishContent. - def deliver (id: rice.p2p.commonapi.Id, message: Message) = { - // TODO: Couldn't perform dynamic type checking. This can cause problems. - // if (message.isInstanceof[PublishContent]) - { publish } - } - - // Called whenever we receive a published message. + // Part of SplitStreamClient. Called when a published message is received. def deliver (s: Stripe, data: Array[Byte]) = { - println(endpoint.getId()+" deliver("+s+"):seq:"+data(0)+" stripe:"+data(1)+" "+data+")") + // TODO: Do real work here. + if (!BroadcastSS.isMaster) { + } + println(endPoint.getId + " deliver(" + s + "):seq:" + data(0) + " stripe:" + data(1) + " " + data + ")") } - /** - * Multicasts data. - */ - def publish = { - - for (curStripe <- myStripes) { - // format of the data: - // first byte: seqNum - // second byte: stripe - // rest: random - var data = new Array[Byte] (DATA_LENGTH) - - // yes, we waste some random bytes here - // random.nextBytes(data) - data(0) = seqNum - data(1) = 13 // curStripe - - // print what we are sending - println("Node "+endpoint.getLocalNodeHandle+" publishing "+seqNum+" "+data) + private def objectToByteArray[A] (obj: A): Array[Byte] = { + val baos = new ByteArrayOutputStream + val oos = new ObjectOutputStream (baos) + oos.writeObject (obj) + oos.close + baos.close + return baos.toByteArray + } - // publish the data - curStripe.publish (data) - } - - // increment the sequence number - // seqNum = seqNum + 1 + // Multicasts data. + def publish[A] (obj: A, blockSize: Int) = { + val byteArray = objectToByteArray[A] (obj) - // cancel after sending all the messages - if (seqNum >= NUM_PUBLISHES) { publishTask.cancel } + var blockNum = (byteArray.length / blockSize) + if (byteArray.length % blockSize != 0) + blockNum += 1 + + var blockID = 0 + for (i <- 0 until (byteArray.length, blockSize)) { + val thisBlockSize = Math.min (blockSize, byteArray.length - i) + var tempByteArray = new Array[Byte] (thisBlockSize + preAmbleSize) + System.arraycopy (byteArray, i * blockSize, + tempByteArray, preAmbleSize, thisBlockSize) + + myStripes(blockID % myStripes.length).publish (tempByteArray) + blockID += 1 + } } - class PublishContent extends Message { + /* class PublishContent extends Message { def getPriority: Int = { Message.MEDIUM_PRIORITY } - } + } */ // Error handling def joinFailed(s: Stripe) = { println ("joinFailed(" + s + ")") } // Rest of the Application interface. NOT USED. + def deliver (id: rice.p2p.commonapi.Id, message: Message) = { } def forward (message: RouteMessage): Boolean = false def update (handle: rice.p2p.commonapi.NodeHandle, joined: Boolean) = { } } @@ -1053,7 +1025,7 @@ private object BroadcastSS { private var isMaster_ = false private var masterBootHost_ = "127.0.0.1" - private var masterBootPort_ : Int = 11111 + private var masterBootPort_ : Int = 22222 private var blockSize_ : Int = 512 * 1024 private var maxRetryCount_ : Int = 2 @@ -1069,7 +1041,7 @@ private object BroadcastSS { masterBootHost_ = System.getProperty ("spark.broadcast.masterHostAddress", "127.0.0.1") masterBootPort_ = - System.getProperty ("spark.broadcast.masterTrackerPort", "11111").toInt + System.getProperty ("spark.broadcast.masterBootPort", "22222").toInt masterBootAddress_ = new InetSocketAddress(masterBootHost_, masterBootPort_) @@ -1112,9 +1084,7 @@ private object BroadcastSS { def isMaster = isMaster_ - private def initializeSplitStream: PastryNode = { - if (pastryNode != null) { return pastryNode } - + private def initializeSplitStream = { pEnvironment_ = new Environment // Generate the NodeIds Randomly @@ -1127,28 +1097,21 @@ private object BroadcastSS { // Construct a Pastry node pastryNode_ = pastryNodeFactory.newNode - // Boot the node. If its the Master, start a new ring. - if (isMaster) { pastryNode.boot (null) } - else { pastryNode.boot (masterBootAddress) } + // Boot the node. + pastryNode.boot (masterBootAddress) // The node may require sending several messages to fully boot into the ring pastryNode.synchronized { while(!pastryNode.isReady && !pastryNode.joinFailed) { // Delay so we don't busy-wait - pastryNode.wait(500); + pastryNode.wait (500) // Abort if can't join if (pastryNode.joinFailed()) { // TODO: throw new IOException("Join failed " + node.joinFailedReason) } } - } - - return pastryNode - // construct a new splitstream application - // val app = new MySplitStreamClient(pastryNode) - // app.subscribe - // if (isMaster) { app.startPublishTask } + } } } diff --git a/src/scala/spark/SparkContext.scala b/src/scala/spark/SparkContext.scala index 7972702205..a80183ed70 100644 --- a/src/scala/spark/SparkContext.scala +++ b/src/scala/spark/SparkContext.scala @@ -6,7 +6,7 @@ import java.util.UUID import scala.collection.mutable.ArrayBuffer class SparkContext(master: String, frameworkName: String) { - Broadcast.initialize(true) + Broadcast.initialize (true) def parallelize[T](seq: Seq[T], numSlices: Int): ParallelArray[T] = new SimpleParallelArray[T](this, seq, numSlices) @@ -17,7 +17,8 @@ class SparkContext(master: String, frameworkName: String) { new Accumulator(initialValue, param) // TODO: Keep around a weak hash map of values to Cached versions? - def broadcast[T](value: T) = new ChainedStreamingBroadcast (value, local) + def broadcast[T](value: T) = new SplitStreamBroadcast (value, local) + // def broadcast[T](value: T) = new ChainedStreamingBroadcast (value, local) // def broadcast[T](value: T) = new CentralizedHDFSBroadcast (value, local) def textFile(path: String) = new HdfsTextFile(this, path) -- cgit v1.2.3 From e2f21279bedd5c1879c48ac90f58e4be92f2ac85 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Tue, 20 Apr 2010 19:16:27 -0700 Subject: Moved SplitStreamClient inside the BroadcastSS object with the decision that there should be only a single SSClient for the whole Spark program instead of one for each broadcasted variable. It's still working well though. --- src/scala/spark/Broadcast.scala | 219 ++++++++++++++++++++++------------------ 1 file changed, 120 insertions(+), 99 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index bed2991713..3edf54edb7 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -45,6 +45,8 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) BroadcastCS.synchronized { BroadcastCS.values.put (uuid, value_) } + if (!local) { sendBroadcast } + @transient var arrayOfBlocks: Array[BroadcastBlock] = null @transient var totalBytes = -1 @transient var totalBlocks = -1 @@ -65,8 +67,6 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) @transient var guidePort = -1 @transient var hasCopyInHDFS = false - - if (!local) { sendBroadcast } def sendBroadcast () { // Store a persistent copy in HDFS @@ -634,6 +634,7 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) if (!local) { sendBroadcast } + @transient var publishThread: PublishThread = null @transient var hasCopyInHDFS = false def sendBroadcast () { @@ -642,10 +643,9 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) out.writeObject (value_) out.close hasCopyInHDFS = true - - val ssClient = new SSClient (BroadcastSS.pastryNode) - ssClient.subscribe - ssClient.publish[T] (value, BroadcastSS.blockSize) + + publishThread = new PublishThread + publishThread.start } private def readObject (in: ObjectInputStream) { @@ -655,103 +655,20 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) if (cachedVal != null) { value_ = cachedVal.asInstanceOf[T] } else { - val ssClient = new SSClient (BroadcastSS.pastryNode) - ssClient.subscribe + // TODO: Do something + Thread.sleep (10000) } } } - class PublishThread (ssClient: SSClient) extends Runnable { - def run = { - - } - } - - class SSClient (pastryNode: PastryNode) extends SplitStreamClient - with Application { - // Bytes reserved before each published block. 8 byte = 2 integer - val preAmbleSize = 8 - - // The Endpoint represents the underlying node. By making calls on the - // Endpoint, it assures that the message will be delivered to the App on - // whichever node the message is intended for. - protected val endPoint = pastryNode.buildEndpoint (this, "myInstance") - - // Handle to a SplitStream implementation - val mySplitStream = new SplitStreamImpl (pastryNode, "mySplitStream") - - // The ChannelId is constructed from a normal PastryId based on the UUID - val myChannelId = new ChannelId (new PastryIdFactory - (pastryNode.getEnvironment).buildId ("myChannel")) - - // The channel - var myChannel: Channel = null - - // The stripes. Acquired from myChannel. - var myStripes: Array[Stripe] = null - - // Now we can receive messages - endPoint.register - - // Subscribes to all stripes in myChannelId. - def subscribe = { - // Attaching makes you part of the Channel, and volunteers to be an - // internal node of one of the trees - myChannel = mySplitStream.attachChannel (myChannelId) - - // Subscribing notifies your application when data comes through the tree - myStripes = myChannel.getStripes - for (curStripe <- myStripes) { curStripe.subscribe (this) } - } - - // Part of SplitStreamClient. Called when a published message is received. - def deliver (s: Stripe, data: Array[Byte]) = { - // TODO: Do real work here. - if (!BroadcastSS.isMaster) { - } - println(endPoint.getId + " deliver(" + s + "):seq:" + data(0) + " stripe:" + data(1) + " " + data + ")") - } - - private def objectToByteArray[A] (obj: A): Array[Byte] = { - val baos = new ByteArrayOutputStream - val oos = new ObjectOutputStream (baos) - oos.writeObject (obj) - oos.close - baos.close - return baos.toByteArray - } - - // Multicasts data. - def publish[A] (obj: A, blockSize: Int) = { - val byteArray = objectToByteArray[A] (obj) - - var blockNum = (byteArray.length / blockSize) - if (byteArray.length % blockSize != 0) - blockNum += 1 - - var blockID = 0 - for (i <- 0 until (byteArray.length, blockSize)) { - val thisBlockSize = Math.min (blockSize, byteArray.length - i) - var tempByteArray = new Array[Byte] (thisBlockSize + preAmbleSize) - System.arraycopy (byteArray, i * blockSize, - tempByteArray, preAmbleSize, thisBlockSize) - - myStripes(blockID % myStripes.length).publish (tempByteArray) - blockID += 1 + class PublishThread extends Thread { + override def run = { + // TODO: Put some delay here to give time others to register + Thread.sleep (10000) + BroadcastSS.synchronized { + BroadcastSS.publish[T] (value) } } - - /* class PublishContent extends Message { - def getPriority: Int = { Message.MEDIUM_PRIORITY } - } */ - - // Error handling - def joinFailed(s: Stripe) = { println ("joinFailed(" + s + ")") } - - // Rest of the Application interface. NOT USED. - def deliver (id: rice.p2p.commonapi.Id, message: Message) = { } - def forward (message: RouteMessage): Boolean = false - def update (handle: rice.p2p.commonapi.NodeHandle, joined: Boolean) = { } } } @@ -1034,6 +951,7 @@ private object BroadcastSS { private var pEnvironment_ : Environment = null private var pastryNode_ : PastryNode = null + private var ssClient: SSClient = null def initialize (isMaster__ : Boolean) { synchronized { @@ -1065,8 +983,15 @@ private object BroadcastSS { def blockSize = blockSize_ def maxRetryCount = maxRetryCount_ - def pEnvironment = pEnvironment_ - def pastryNode = pastryNode_ + def pEnvironment: Environment = { + if (pEnvironment_ == null) { initializeSplitStream } + pEnvironment_ + } + + def pastryNode: PastryNode = { + if (pastryNode_ == null) { initializeSplitStream } + pastryNode_ + } def localBindPort = { if (localBindPort_ == -1) { @@ -1099,6 +1024,7 @@ private object BroadcastSS { // Boot the node. pastryNode.boot (masterBootAddress) + // TODO: Some unknown messages are dropped in slaves at this point // The node may require sending several messages to fully boot into the ring pastryNode.synchronized { @@ -1112,6 +1038,101 @@ private object BroadcastSS { } } } + + // Create the SplitStream client and subscribe + ssClient = new SSClient (BroadcastSS.pastryNode) + ssClient.subscribe + } + + def publish[A] (obj: A) = { + ssClient.publish[A] (obj) + } + + class SSClient (pastryNode: PastryNode) extends SplitStreamClient + with Application { + // Bytes reserved before each published block. 8 byte = 2 integer + val preAmbleSize = 8 + + // The Endpoint represents the underlying node. By making calls on the + // Endpoint, it assures that the message will be delivered to the App on + // whichever node the message is intended for. + protected val endPoint = pastryNode.buildEndpoint (this, "myInstance") + + // Handle to a SplitStream implementation + val mySplitStream = new SplitStreamImpl (pastryNode, "mySplitStream") + + // The ChannelId is constructed from a normal PastryId based on the UUID + val myChannelId = new ChannelId (new PastryIdFactory + (pastryNode.getEnvironment).buildId ("myChannel")) + + // The channel + var myChannel: Channel = null + + // The stripes. Acquired from myChannel. + var myStripes: Array[Stripe] = null + + // Now we can receive messages + endPoint.register + + // Subscribes to all stripes in myChannelId. + def subscribe = { + // Attaching makes you part of the Channel, and volunteers to be an + // internal node of one of the trees + myChannel = mySplitStream.attachChannel (myChannelId) + + // Subscribing notifies your application when data comes through the tree + myStripes = myChannel.getStripes + for (curStripe <- myStripes) { curStripe.subscribe (this) } + } + + // Part of SplitStreamClient. Called when a published message is received. + def deliver (s: Stripe, data: Array[Byte]) = { + // TODO: Do real work here. + if (!BroadcastSS.isMaster) { + } + println(endPoint.getId + " deliver(" + s + "):seq:" + data(0) + " stripe:" + data(1) + " " + data + ")") + } + + private def objectToByteArray[A] (obj: A): Array[Byte] = { + val baos = new ByteArrayOutputStream + val oos = new ObjectOutputStream (baos) + oos.writeObject (obj) + oos.close + baos.close + return baos.toByteArray + } + + // Multicasts data. + def publish[A] (obj: A) = { + val byteArray = objectToByteArray[A] (obj) + + var blockNum = (byteArray.length / blockSize) + if (byteArray.length % blockSize != 0) + blockNum += 1 + + var blockID = 0 + for (i <- 0 until (byteArray.length, blockSize)) { + val thisBlockSize = Math.min (blockSize, byteArray.length - i) + var tempByteArray = new Array[Byte] (thisBlockSize + preAmbleSize) + System.arraycopy (byteArray, i * blockSize, + tempByteArray, preAmbleSize, thisBlockSize) + + myStripes(blockID % myStripes.length).publish (tempByteArray) + blockID += 1 + } + } + + /* class PublishContent extends Message { + def getPriority: Int = { Message.MEDIUM_PRIORITY } + } */ + + // Error handling + def joinFailed(s: Stripe) = { println ("joinFailed(" + s + ")") } + + // Rest of the Application interface. NOT USED. + def deliver (id: rice.p2p.commonapi.Id, message: Message) = { } + def forward (message: RouteMessage): Boolean = false + def update (handle: rice.p2p.commonapi.NodeHandle, joined: Boolean) = { } } } -- cgit v1.2.3 From c0117f9473a5a14adf8812d0e58bfa1764efd2c9 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Wed, 21 Apr 2010 02:01:34 -0700 Subject: Added flesh to publish/deliver functions. --- src/scala/spark/Broadcast.scala | 141 ++++++++++++++++++++++++++++++++++------ 1 file changed, 122 insertions(+), 19 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 3edf54edb7..1360d94a84 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -666,7 +666,7 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) // TODO: Put some delay here to give time others to register Thread.sleep (10000) BroadcastSS.synchronized { - BroadcastSS.publish[T] (value) + BroadcastSS.publish[T] (uuid, value) } } } @@ -1044,15 +1044,31 @@ private object BroadcastSS { ssClient.subscribe } - def publish[A] (obj: A) = { - ssClient.publish[A] (obj) + def publish[A] (uuid: UUID, obj: A) = { + ssClient.synchronized { + ssClient.publish[A] (uuid, obj) + } } + // TODO: Receive function/callback! + class SSClient (pastryNode: PastryNode) extends SplitStreamClient with Application { - // Bytes reserved before each published block. 8 byte = 2 integer - val preAmbleSize = 8 - + // Magic bits: 11111100001100100100110000111111 + val magicBits = 0xFC324C3F + + // Message Types + val INFO_MSG = 1 + val DATA_MSG = 2 + + // Current transmission state variables + var curUUID: UUID = null + var curTotalBlocks = -1 + var curTotalBytes = -1 + var curHasBlocks = -1 + var curBlockBitmap: Array[Boolean] = null + var curArrayOfBytes: Array[Byte] = null + // The Endpoint represents the underlying node. By making calls on the // Endpoint, it assures that the message will be delivered to the App on // whichever node the message is intended for. @@ -1086,11 +1102,55 @@ private object BroadcastSS { } // Part of SplitStreamClient. Called when a published message is received. - def deliver (s: Stripe, data: Array[Byte]) = { - // TODO: Do real work here. - if (!BroadcastSS.isMaster) { + def deliver (s: Stripe, data: Array[Byte]) = { + // Unpack and verify magicBits + val topLevelInfo = byteArrayToObject[(Int, Int, Array[Byte])] (data) + + // Process only if magicBits are OK + if (topLevelInfo._1 == magicBits) { + // Process only for slaves + if (!BroadcastSS.isMaster) { + // Match on Message Type + topLevelInfo._2 match { + case INFO_MSG => { + val realInfo = byteArrayToObject[(UUID, Int, Int)] ( + topLevelInfo._3) + + // Setup states for impending transmission + curUUID = realInfo._1 // TODO: + curTotalBlocks = realInfo._2 + curTotalBytes = realInfo._3 + + curHasBlocks = 0 + curBlockBitmap = new Array[Boolean] (curTotalBlocks) + curArrayOfBytes = new Array[Byte] (curTotalBytes) + + println (curUUID + " " + curTotalBlocks + " " + curTotalBytes) + } + case DATA_MSG => { + val realInfo = byteArrayToObject[(Int, Array[Byte])] ( + topLevelInfo._3) + val blockIndex = realInfo._1 + val blockData = realInfo._2 + + // Update everything + curHasBlocks += 1 + curBlockBitmap(blockIndex) = true + System.arraycopy (blockData, 0, curArrayOfBytes, + blockIndex * blockSize, blockData.length) + + // Done receiving + if (curHasBlocks == curTotalBlocks) { // TODO: + + } + } + case _ => { + // Should never happen + } + } + } + println(endPoint.getId + " deliver(" + s + "):seq:" + data(0) + " stripe:" + data(1) + " " + data + ")") } - println(endPoint.getId + " deliver(" + s + "):seq:" + data(0) + " stripe:" + data(1) + " " + data + ")") } private def objectToByteArray[A] (obj: A): Array[Byte] = { @@ -1102,25 +1162,68 @@ private object BroadcastSS { return baos.toByteArray } + private def byteArrayToObject[A] (bytes: Array[Byte]): A = { + val in = new ObjectInputStream (new ByteArrayInputStream (bytes)) + val retVal = in.readObject.asInstanceOf[A] + in.close + return retVal + } + + private def intToByteArray (value: Int): Array[Byte] = { + var retVal = new Array[Byte] (4) + for (i <- 0 until 4) + retVal(i) = (value >> ((4 - 1 - i) * 8)).toByte + return retVal + } + + private def byteArrayToInt (arr: Array[Byte], offset: Int): Int = { + var retVal = 0 + for (i <- 0 until 4) + retVal += ((arr(i + offset).toInt & 0x000000FF) << ((4 - 1 - i) * 8)) + return retVal + } + // Multicasts data. - def publish[A] (obj: A) = { + def publish[A] (uuid: UUID, obj: A) = { val byteArray = objectToByteArray[A] (obj) var blockNum = (byteArray.length / blockSize) if (byteArray.length % blockSize != 0) - blockNum += 1 - + blockNum += 1 + + // ------------------------------------- + // INFO_MSG: | UUID | Total Blocks | Total Bytes | + // ------------------------------------- + var infoByteArray = objectToByteArray[(UUID, Int, Int)] ((uuid, blockNum, + byteArray.length)) + doPublish (0, INFO_MSG, infoByteArray) + + // ------------------------------ + // DATA_MSG: | Block Index | Single Block | + // ------------------------------ var blockID = 0 - for (i <- 0 until (byteArray.length, blockSize)) { - val thisBlockSize = Math.min (blockSize, byteArray.length - i) - var tempByteArray = new Array[Byte] (thisBlockSize + preAmbleSize) - System.arraycopy (byteArray, i * blockSize, - tempByteArray, preAmbleSize, thisBlockSize) + for (i <- 0 until (byteArray.length, blockSize)) { + val thisBlockSize = Math.min (blockSize, byteArray.length - i) + var thisBlockData = new Array[Byte] (thisBlockSize) + System.arraycopy (byteArray, i * blockSize, thisBlockData, 0, + thisBlockSize) + + var dataByteArray = objectToByteArray[(Int, Array[Byte])] ((blockID, + thisBlockData)) + doPublish (blockID % myStripes.length, DATA_MSG, dataByteArray) - myStripes(blockID % myStripes.length).publish (tempByteArray) blockID += 1 } } + + // -------------------------------- + // Message Format: | MagicBits | Type | Real Data | + // -------------------------------- + private def doPublish (stripeID: Int, msgType: Int, data: Array[Byte]) = { + val bytesToSend = objectToByteArray[(Int, Int, Array[Byte])] ((magicBits, + msgType, data)) + myStripes(stripeID).publish (bytesToSend) + } /* class PublishContent extends Message { def getPriority: Int = { Message.MEDIUM_PRIORITY } -- cgit v1.2.3 From d0a92571ddea2d4e946a000d09109595a7a2641b Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Wed, 21 Apr 2010 19:48:25 -0700 Subject: - Should work, but not tested yet. - Right now, each variable has to come one after another. Within a single variable, blocks can come out-of-order. --- src/scala/spark/Broadcast.scala | 173 ++++++++++++++++++++++++++-------------- 1 file changed, 113 insertions(+), 60 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 1360d94a84..65c0a9095c 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -144,13 +144,12 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) val receptionSucceeded = receiveBroadcast (uuid) // If does not succeed, then get from HDFS copy if (receptionSucceeded) { - // value_ = byteArrayToObject[T] (retByteArray) value_ = unBlockifyObject[T] BroadcastCS.values.put (uuid, value_) } else { val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) value_ = fileIn.readObject.asInstanceOf[T] - BroadcastCH.values.put(uuid, value_) + BroadcastCS.values.put(uuid, value_) fileIn.close } @@ -655,6 +654,23 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) if (cachedVal != null) { value_ = cachedVal.asInstanceOf[T] } else { + val start = System.nanoTime + + val receptionSucceeded = BroadcastSS.receiveVariable (uuid) + // If does not succeed, then get from HDFS copy + if (receptionSucceeded) { + value_ = BroadcastSS.values.get(uuid).asInstanceOf[T] + } else { + val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) + value_ = fileIn.readObject.asInstanceOf[T] + BroadcastSS.values.put(uuid, value_) + fileIn.close + } + + val time = (System.nanoTime - start) / 1e9 + println( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") + + // TODO: Do something Thread.sleep (10000) } @@ -666,7 +682,7 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) // TODO: Put some delay here to give time others to register Thread.sleep (10000) BroadcastSS.synchronized { - BroadcastSS.publish[T] (uuid, value) + BroadcastSS.publishVariable[T] (uuid, value) } } } @@ -749,12 +765,6 @@ case class VariableInfo (@transient val arrayOfBlocks : Array[BroadcastBlock], val totalBlocks: Int, val totalBytes: Int) { @transient var hasBlocks = 0 - - val listenPortLock = new AnyRef - val totalBlocksLock = new AnyRef - val hasBlocksLock = new AnyRef - - @transient var pqOfSources = new PriorityQueue[SourceInfo] } private object Broadcast { @@ -938,6 +948,8 @@ private object BroadcastCS { private object BroadcastSS { val values = new MapMaker ().softValues ().makeMap[UUID, Any] + private val valueBytes = new MapMaker().softValues().makeMap[UUID,Array[Byte]] + private var initialized = false private var isMaster_ = false @@ -953,6 +965,16 @@ private object BroadcastSS { private var pastryNode_ : PastryNode = null private var ssClient: SSClient = null + // Current transmission state variables + private var curUUID: UUID = null + private var curTotalBlocks = -1 + private var curTotalBytes = -1 + private var curHasBlocks = -1 + private var curBlockBitmap: Array[Boolean] = null + private var curArrayOfBytes: Array[Byte] = null + + // TODO: Add stuff so that we can handle out of order variable broadcast + def initialize (isMaster__ : Boolean) { synchronized { if (!initialized) { @@ -1044,14 +1066,71 @@ private object BroadcastSS { ssClient.subscribe } - def publish[A] (uuid: UUID, obj: A) = { + def publishVariable[A] (uuid: UUID, obj: A) = { ssClient.synchronized { ssClient.publish[A] (uuid, obj) } } - // TODO: Receive function/callback! + // Return status of the reception + def receiveVariable[A] (uuid: UUID): Boolean = { + // TODO: Things will change if out-of-order variable recepetion is supported + + // Check in valueBytes + if (xferValueBytesToValues[A] (uuid)) { return true } + + // Check if its in progress + for (i <- 0 until maxRetryCount) { + while (uuid == curUUID) { Thread.sleep (100) } // TODO: How long to sleep + if (xferValueBytesToValues[A] (uuid)) { return true } + + // Wait for a while to see if we've reached here before xmission started + Thread.sleep (100) + } + return false + } + + private def xferValueBytesToValues[A] (uuid: UUID): Boolean = { + var cachedValueBytes: Array[Byte] = null + valueBytes.synchronized { cachedValueBytes = valueBytes.get (uuid) } + if (cachedValueBytes != null) { + val cachedValue = byteArrayToObject[A] (cachedValueBytes) + values.synchronized { values.put (uuid, cachedValue) } + return true + } + return false + } + private def objectToByteArray[A] (obj: A): Array[Byte] = { + val baos = new ByteArrayOutputStream + val oos = new ObjectOutputStream (baos) + oos.writeObject (obj) + oos.close + baos.close + return baos.toByteArray + } + + private def byteArrayToObject[A] (bytes: Array[Byte]): A = { + val in = new ObjectInputStream (new ByteArrayInputStream (bytes)) + val retVal = in.readObject.asInstanceOf[A] + in.close + return retVal + } + + private def intToByteArray (value: Int): Array[Byte] = { + var retVal = new Array[Byte] (4) + for (i <- 0 until 4) + retVal(i) = (value >> ((4 - 1 - i) * 8)).toByte + return retVal + } + + private def byteArrayToInt (arr: Array[Byte], offset: Int): Int = { + var retVal = 0 + for (i <- 0 until 4) + retVal += ((arr(i + offset).toInt & 0x000000FF) << ((4 - 1 - i) * 8)) + return retVal + } + class SSClient (pastryNode: PastryNode) extends SplitStreamClient with Application { // Magic bits: 11111100001100100100110000111111 @@ -1060,15 +1139,7 @@ private object BroadcastSS { // Message Types val INFO_MSG = 1 val DATA_MSG = 2 - - // Current transmission state variables - var curUUID: UUID = null - var curTotalBlocks = -1 - var curTotalBytes = -1 - var curHasBlocks = -1 - var curBlockBitmap: Array[Boolean] = null - var curArrayOfBytes: Array[Byte] = null - + // The Endpoint represents the underlying node. By making calls on the // Endpoint, it assures that the message will be delivered to the App on // whichever node the message is intended for. @@ -1128,10 +1199,16 @@ private object BroadcastSS { println (curUUID + " " + curTotalBlocks + " " + curTotalBytes) } case DATA_MSG => { - val realInfo = byteArrayToObject[(Int, Array[Byte])] ( - topLevelInfo._3) - val blockIndex = realInfo._1 - val blockData = realInfo._2 + val realInfo = byteArrayToObject[(UUID, Int, Array[Byte])] ( + topLevelInfo._3) + val blockUUID = realInfo._1 + val blockIndex = realInfo._2 + val blockData = realInfo._3 + + // TODO: Will change in future implementation. Right now we + // require broadcast in order on the variable level. Blocks can + // come out of order though + assert (blockUUID == curUUID) // Update everything curHasBlocks += 1 @@ -1140,8 +1217,14 @@ private object BroadcastSS { blockIndex * blockSize, blockData.length) // Done receiving - if (curHasBlocks == curTotalBlocks) { // TODO: + if (curHasBlocks == curTotalBlocks) { + // Store as a Array[Byte] + valueBytes.synchronized { + valueBytes.put (curUUID, curArrayOfBytes) + } + // RESET + curUUID = null } } case _ => { @@ -1153,36 +1236,6 @@ private object BroadcastSS { } } - private def objectToByteArray[A] (obj: A): Array[Byte] = { - val baos = new ByteArrayOutputStream - val oos = new ObjectOutputStream (baos) - oos.writeObject (obj) - oos.close - baos.close - return baos.toByteArray - } - - private def byteArrayToObject[A] (bytes: Array[Byte]): A = { - val in = new ObjectInputStream (new ByteArrayInputStream (bytes)) - val retVal = in.readObject.asInstanceOf[A] - in.close - return retVal - } - - private def intToByteArray (value: Int): Array[Byte] = { - var retVal = new Array[Byte] (4) - for (i <- 0 until 4) - retVal(i) = (value >> ((4 - 1 - i) * 8)).toByte - return retVal - } - - private def byteArrayToInt (arr: Array[Byte], offset: Int): Int = { - var retVal = 0 - for (i <- 0 until 4) - retVal += ((arr(i + offset).toInt & 0x000000FF) << ((4 - 1 - i) * 8)) - return retVal - } - // Multicasts data. def publish[A] (uuid: UUID, obj: A) = { val byteArray = objectToByteArray[A] (obj) @@ -1198,9 +1251,9 @@ private object BroadcastSS { byteArray.length)) doPublish (0, INFO_MSG, infoByteArray) - // ------------------------------ - // DATA_MSG: | Block Index | Single Block | - // ------------------------------ + // ------------------------------------- + // DATA_MSG: | UUID | Block Index | Single Block | + // ------------------------------------- var blockID = 0 for (i <- 0 until (byteArray.length, blockSize)) { val thisBlockSize = Math.min (blockSize, byteArray.length - i) @@ -1208,8 +1261,8 @@ private object BroadcastSS { System.arraycopy (byteArray, i * blockSize, thisBlockData, 0, thisBlockSize) - var dataByteArray = objectToByteArray[(Int, Array[Byte])] ((blockID, - thisBlockData)) + var dataByteArray = objectToByteArray[(UUID, Int, Array[Byte])] ((uuid, + blockID, thisBlockData)) doPublish (blockID % myStripes.length, DATA_MSG, dataByteArray) blockID += 1 -- cgit v1.2.3 From 53a2367c9c477f5016cd9a3f68031ce5fe236db1 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Tue, 4 May 2010 15:32:30 -0700 Subject: - SplitStream working in local machine for single variable broadcast - Removed delays before publishing and receiving. - Commented out some prints. --- src/scala/spark/Broadcast.scala | 23 ++++++++++++++--------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 65c0a9095c..747b643f25 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -154,7 +154,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } val time = (System.nanoTime - start) / 1e9 - println( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") + println( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") } } } @@ -656,11 +656,13 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) } else { val start = System.nanoTime + // Thread.sleep (5000) // TODO: val receptionSucceeded = BroadcastSS.receiveVariable (uuid) // If does not succeed, then get from HDFS copy if (receptionSucceeded) { value_ = BroadcastSS.values.get(uuid).asInstanceOf[T] } else { + // println (System.currentTimeMillis + ": " + "Reading from HDFS") val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) value_ = fileIn.readObject.asInstanceOf[T] BroadcastSS.values.put(uuid, value_) @@ -669,10 +671,6 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) val time = (System.nanoTime - start) / 1e9 println( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") - - - // TODO: Do something - Thread.sleep (10000) } } } @@ -680,7 +678,8 @@ class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) class PublishThread extends Thread { override def run = { // TODO: Put some delay here to give time others to register - Thread.sleep (10000) + // Thread.sleep (5000) + // println (System.currentTimeMillis + ": " + "Waited. Now sending...") BroadcastSS.synchronized { BroadcastSS.publishVariable[T] (uuid, value) } @@ -1076,11 +1075,14 @@ private object BroadcastSS { def receiveVariable[A] (uuid: UUID): Boolean = { // TODO: Things will change if out-of-order variable recepetion is supported + // println (System.currentTimeMillis + ": " + "In receiveVariable") + // Check in valueBytes if (xferValueBytesToValues[A] (uuid)) { return true } // Check if its in progress for (i <- 0 until maxRetryCount) { + // println (System.currentTimeMillis + ": " + uuid + " " + curUUID) while (uuid == curUUID) { Thread.sleep (100) } // TODO: How long to sleep if (xferValueBytesToValues[A] (uuid)) { return true } @@ -1196,7 +1198,7 @@ private object BroadcastSS { curBlockBitmap = new Array[Boolean] (curTotalBlocks) curArrayOfBytes = new Array[Byte] (curTotalBytes) - println (curUUID + " " + curTotalBlocks + " " + curTotalBytes) + // println (System.currentTimeMillis + ": " + curUUID + " " + curTotalBlocks + " " + curTotalBytes) } case DATA_MSG => { val realInfo = byteArrayToObject[(UUID, Int, Array[Byte])] ( @@ -1215,7 +1217,9 @@ private object BroadcastSS { curBlockBitmap(blockIndex) = true System.arraycopy (blockData, 0, curArrayOfBytes, blockIndex * blockSize, blockData.length) - + + // println (System.currentTimeMillis + ": " + "Got stuff for: " + blockUUID) + // Done receiving if (curHasBlocks == curTotalBlocks) { // Store as a Array[Byte] @@ -1223,6 +1227,8 @@ private object BroadcastSS { valueBytes.put (curUUID, curArrayOfBytes) } + // println (System.currentTimeMillis + ": " + "Finished reading. Stored in valueBytes") + // RESET curUUID = null } @@ -1232,7 +1238,6 @@ private object BroadcastSS { } } } - println(endPoint.getId + " deliver(" + s + "):seq:" + data(0) + " stripe:" + data(1) + " " + data + ")") } } -- cgit v1.2.3 From 4f0b7eb02d077765336e31ad2bc80e8ab0395046 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Sun, 16 May 2010 13:00:47 -0700 Subject: SplitStream was not working in EC2. We have turned OFF SSB for now. --- src/scala/spark/Broadcast.scala | 3 ++- src/scala/spark/SparkContext.scala | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 747b643f25..833fd439a2 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -779,7 +779,8 @@ private object Broadcast { // Initialization for ChainedStreamingBroadcast BroadcastCS.initialize (isMaster) // Initialization for SplitStreamBroadcast - BroadcastSS.initialize (isMaster) + // TODO: SplitStream turned OFF + // BroadcastSS.initialize (isMaster) initialized = true } diff --git a/src/scala/spark/SparkContext.scala b/src/scala/spark/SparkContext.scala index a80183ed70..cfb6c2f458 100644 --- a/src/scala/spark/SparkContext.scala +++ b/src/scala/spark/SparkContext.scala @@ -17,8 +17,8 @@ class SparkContext(master: String, frameworkName: String) { new Accumulator(initialValue, param) // TODO: Keep around a weak hash map of values to Cached versions? - def broadcast[T](value: T) = new SplitStreamBroadcast (value, local) - // def broadcast[T](value: T) = new ChainedStreamingBroadcast (value, local) + // def broadcast[T](value: T) = new SplitStreamBroadcast (value, local) + def broadcast[T](value: T) = new ChainedStreamingBroadcast (value, local) // def broadcast[T](value: T) = new CentralizedHDFSBroadcast (value, local) def textFile(path: String) = new HdfsTextFile(this, path) -- cgit v1.2.3 From 520b594bdf355b2b26f660f10bed7dfed4a0a22a Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Sun, 16 May 2010 16:20:00 -0700 Subject: "if (!local) { sendBroadcast }" must be called after all the variables have been created /initialized --- src/scala/spark/Broadcast.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 833fd439a2..1da0773845 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -45,8 +45,6 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) BroadcastCS.synchronized { BroadcastCS.values.put (uuid, value_) } - if (!local) { sendBroadcast } - @transient var arrayOfBlocks: Array[BroadcastBlock] = null @transient var totalBytes = -1 @transient var totalBlocks = -1 @@ -68,6 +66,9 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) @transient var hasCopyInHDFS = false + // Must call this after all the variables have been created/initialized + if (!local) { sendBroadcast } + def sendBroadcast () { // Store a persistent copy in HDFS val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) -- cgit v1.2.3 From e85bb3f04d51cc7c6cc7b6c198c4bc0cd4eede31 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Sun, 16 May 2010 16:24:48 -0700 Subject: All ObjectOutputStream objects created before OInputS objects. --- src/scala/spark/Broadcast.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 1da0773845..ebc1876961 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -227,8 +227,8 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // >0 = Read from this port def getMasterListenPort (variableUUID: UUID): Int = { var clientSocketToTracker: Socket = null - var oisTracker: ObjectInputStream = null var oosTracker: ObjectOutputStream = null + var oisTracker: ObjectInputStream = null var masterListenPort: Int = -1 @@ -238,10 +238,10 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // Connect to the tracker to find out the guide val clientSocketToTracker = new Socket(BroadcastCS.masterHostAddress, BroadcastCS.masterTrackerPort) - val oisTracker = - new ObjectInputStream (clientSocketToTracker.getInputStream) val oosTracker = new ObjectOutputStream (clientSocketToTracker.getOutputStream) + val oisTracker = + new ObjectInputStream (clientSocketToTracker.getInputStream) // Send UUID and receive masterListenPort oosTracker.writeObject (uuid) @@ -284,10 +284,10 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) new Socket(BroadcastCS.masterHostAddress, masterListenPort) // println (System.currentTimeMillis + ": " + "Connected to Master's guiding object") // TODO: Guiding object connection is reusable - val oisMaster = - new ObjectInputStream (clientSocketToMaster.getInputStream) val oosMaster = new ObjectOutputStream (clientSocketToMaster.getOutputStream) + val oisMaster = + new ObjectInputStream (clientSocketToMaster.getInputStream) oosMaster.writeObject(new SourceInfo (hostAddress, listenPort, -1, -1, 0)) oosMaster.flush @@ -330,8 +330,8 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // This might be called multiple times to retry a defined number of times. private def receiveSingleTransmission(sourceInfo: SourceInfo): Boolean = { var clientSocketToSource: Socket = null - var oisSource: ObjectInputStream = null var oosSource: ObjectOutputStream = null + var oisSource: ObjectInputStream = null var receptionSucceeded = false try { -- cgit v1.2.3 From 2d381c974e2c27ea9cd3c37f899625200bbcd0fb Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Sun, 16 May 2010 17:06:05 -0700 Subject: Added flush calls after all writeObject calls as well as after creating every ObjectOutputStream object. --- src/scala/spark/Broadcast.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index ebc1876961..0c1aa43285 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -240,11 +240,13 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) new Socket(BroadcastCS.masterHostAddress, BroadcastCS.masterTrackerPort) val oosTracker = new ObjectOutputStream (clientSocketToTracker.getOutputStream) + oosTracker.flush val oisTracker = new ObjectInputStream (clientSocketToTracker.getInputStream) // Send UUID and receive masterListenPort oosTracker.writeObject (uuid) + oosTracker.flush masterListenPort = oisTracker.readObject.asInstanceOf[Int] } catch { // In case of any failure, set masterListenPort = 0 to read from HDFS @@ -286,6 +288,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // TODO: Guiding object connection is reusable val oosMaster = new ObjectOutputStream (clientSocketToMaster.getOutputStream) + oosMaster.flush val oisMaster = new ObjectInputStream (clientSocketToMaster.getInputStream) @@ -340,6 +343,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) new Socket (sourceInfo.hostAddress, sourceInfo.listenPort) oosSource = new ObjectOutputStream (clientSocketToSource.getOutputStream) + oosSource.flush oisSource = new ObjectInputStream (clientSocketToSource.getInputStream) @@ -348,6 +352,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // Send the range oosSource.writeObject((hasBlocks, totalBlocks)) + oosSource.flush for (i <- hasBlocks until totalBlocks) { val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock] @@ -420,6 +425,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) class GuideSingleRequest (val clientSocket: Socket) extends Runnable { private val oos = new ObjectOutputStream (clientSocket.getOutputStream) + oos.flush private val ois = new ObjectInputStream (clientSocket.getInputStream) private var selectedSourceInfo: SourceInfo = null @@ -567,6 +573,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) class ServeSingleRequest (val clientSocket: Socket) extends Runnable { private val oos = new ObjectOutputStream (clientSocket.getOutputStream) + oos.flush private val ois = new ObjectInputStream (clientSocket.getInputStream) private var sendFrom = 0 @@ -912,6 +919,7 @@ private object BroadcastCS { threadPool.execute (new Runnable { override def run = { val oos = new ObjectOutputStream (clientSocket.getOutputStream) + oos.flush val ois = new ObjectInputStream (clientSocket.getInputStream) try { val uuid = ois.readObject.asInstanceOf[UUID] -- cgit v1.2.3 From 7ab703117aa6947de0d05c8f58c6f0ae2ecb37ab Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Tue, 18 May 2010 20:49:59 -0700 Subject: Added timers around BroadcastTest and sendBroadcast. Turned OFF saving to HDFS for now for stress tests. pqOfSources is ordered by least leechers again. --- src/examples/BroadcastTest.scala | 3 ++ src/scala/spark/Broadcast.scala | 73 +++++++++++++++++++++------------------- 2 files changed, 42 insertions(+), 34 deletions(-) diff --git a/src/examples/BroadcastTest.scala b/src/examples/BroadcastTest.scala index 05778668bd..6d31ade95b 100644 --- a/src/examples/BroadcastTest.scala +++ b/src/examples/BroadcastTest.scala @@ -14,11 +14,14 @@ object BroadcastTest { for (i <- 0 until arr.length) arr(i) = i + val start = System.nanoTime val barr = spark.broadcast(arr) spark.parallelize(1 to 10, slices).foreach { println("in task: barr = " + barr) i => println(barr.value.size) } + val time = (System.nanoTime - start) / 1e9 + println("BroadcastTest took " + time + " s") } } diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index 0c1aa43285..b2114eb151 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -67,13 +67,19 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) @transient var hasCopyInHDFS = false // Must call this after all the variables have been created/initialized - if (!local) { sendBroadcast } + if (!local) { + val start = System.nanoTime + sendBroadcast + val time = (System.nanoTime - start) / 1e9 + println("sendBroadcast took " + time + " s") + } - def sendBroadcast () { + def sendBroadcast () { // Store a persistent copy in HDFS - val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) - out.writeObject (value_) - out.close + // TODO: Turned OFF for now + // val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) + // out.writeObject (value_) + // out.close hasCopyInHDFS = true // Create a variableInfo object and store it in valueInfos @@ -438,17 +444,17 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // be listening to. ReplicaID is 0 and other fields are invalid (-1) var sourceInfo = ois.readObject.asInstanceOf[SourceInfo] - // Select a suitable source and send it back to the worker - selectedSourceInfo = selectSuitableSource (sourceInfo) - // println (System.currentTimeMillis + ": " + "Sending selectedSourceInfo:" + selectedSourceInfo) - oos.writeObject (selectedSourceInfo) - oos.flush - - // Add this new (if it can finish) source to the PQ of sources - thisWorkerInfo = new SourceInfo(sourceInfo.hostAddress, - sourceInfo.listenPort, totalBlocks, totalBytes, 0) - // println (System.currentTimeMillis + ": " + "Adding possible new source to pqOfSources: " + thisWorkerInfo) pqOfSources.synchronized { + // Select a suitable source and send it back to the worker + selectedSourceInfo = selectSuitableSource (sourceInfo) + // println (System.currentTimeMillis + ": " + "Sending selectedSourceInfo:" + selectedSourceInfo) + oos.writeObject (selectedSourceInfo) + oos.flush + + // Add this new (if it can finish) source to the PQ of sources + thisWorkerInfo = new SourceInfo(sourceInfo.hostAddress, + sourceInfo.listenPort, totalBlocks, totalBytes, 0) + // println (System.currentTimeMillis + ": " + "Adding possible new source to pqOfSources: " + thisWorkerInfo) pqOfSources.add (thisWorkerInfo) } @@ -509,23 +515,22 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } } + // TODO: Caller must have a synchronized block on pqOfSources // TODO: If a worker fails to get the broadcasted variable from a source and // comes back to Master, this function might choose the worker itself as a // source tp create a dependency cycle (this worker was put into pqOfSources // as a streming source when it first arrived). The length of this cycle can // be arbitrarily long. private def selectSuitableSource(skipSourceInfo: SourceInfo): SourceInfo = { - // Select one with the lowest number of leechers - pqOfSources.synchronized { - // take is a blocking call removing the element from PQ - var selectedSource = pqOfSources.poll - assert (selectedSource != null) - // Update leecher count - selectedSource.currentLeechers += 1 - // Add it back and then return - pqOfSources.add (selectedSource) - return selectedSource - } + // Select one based on the ordering strategy (e.g., least leechers etc.) + // take is a blocking call removing the element from PQ + var selectedSource = pqOfSources.poll + assert (selectedSource != null) + // Update leecher count + selectedSource.currentLeechers += 1 + // Add it back and then return + pqOfSources.add (selectedSource) + return selectedSource } } } @@ -744,7 +749,7 @@ case class SourceInfo (val hostAddress: String, val listenPort: Int, var MBps: Double = BroadcastCS.MaxMBps // Ascending sort based on leecher count - // def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers) + def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers) // Descending sort based on speed // def compareTo (o: SourceInfo): Int = { @@ -754,14 +759,14 @@ case class SourceInfo (val hostAddress: String, val listenPort: Int, // } // Descending sort based on globally stored speed - def compareTo (o: SourceInfo): Int = { - val mySpeed = BroadcastCS.getSourceSpeed (hostAddress) - val urSpeed = BroadcastCS.getSourceSpeed (o.hostAddress) + // def compareTo (o: SourceInfo): Int = { + // val mySpeed = BroadcastCS.getSourceSpeed (hostAddress) + // val urSpeed = BroadcastCS.getSourceSpeed (o.hostAddress) - if (mySpeed > urSpeed) -1 - else if (mySpeed < urSpeed) 1 - else 0 - } + // if (mySpeed > urSpeed) -1 + // else if (mySpeed < urSpeed) 1 + // else 0 + // } } @serializable -- cgit v1.2.3 From e73a5f3491ff502cb6218de8464ab56e71ea0958 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Tue, 12 Oct 2010 13:05:32 -0700 Subject: Now compiles with Scala 2.8.0, but doesn't run with nexus.jar Must update it to use mesos.jar --- src/examples/BroadcastTest.scala | 23 ++++++++++++----------- src/examples/LocalFileLR.scala | 2 +- src/scala/spark/SparkContext.scala | 4 ++-- 3 files changed, 15 insertions(+), 14 deletions(-) diff --git a/src/examples/BroadcastTest.scala b/src/examples/BroadcastTest.scala index 6d31ade95b..40c2be8f6d 100644 --- a/src/examples/BroadcastTest.scala +++ b/src/examples/BroadcastTest.scala @@ -3,25 +3,26 @@ import spark.SparkContext object BroadcastTest { def main(args: Array[String]) { if (args.length == 0) { - System.err.println("Usage: BroadcastTest [] []") + System.err.println("Usage: BroadcastTest []") System.exit(1) } val spark = new SparkContext(args(0), "Broadcast Test") val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 - var arr = new Array[Int](num) - for (i <- 0 until arr.length) - arr(i) = i + var arr1 = new Array[Int](num) + for (i <- 0 until arr1.length) + arr1(i) = i - val start = System.nanoTime - val barr = spark.broadcast(arr) +// var arr2 = new Array[Int](num * 2) +// for (i <- 0 until arr2.length) +// arr2(i) = i + + val barr1 = spark.broadcast(arr1) +// val barr2 = spark.broadcast(arr2) spark.parallelize(1 to 10, slices).foreach { - println("in task: barr = " + barr) - i => println(barr.value.size) +// i => println(barr1.value.size + barr2.value.size) + i => println(barr1.value.size) } - val time = (System.nanoTime - start) / 1e9 - println("BroadcastTest took " + time + " s") } } - diff --git a/src/examples/LocalFileLR.scala b/src/examples/LocalFileLR.scala index 988442755a..3d3bb60677 100644 --- a/src/examples/LocalFileLR.scala +++ b/src/examples/LocalFileLR.scala @@ -13,7 +13,7 @@ object LocalFileLR { } def main(args: Array[String]) { - val lines = scala.io.Source.fromPath(args(0)).getLines() + val lines = scala.io.Source.fromFile(args(0)).getLines() val points = lines.map(parsePoint _) val ITERATIONS = args(1).toInt diff --git a/src/scala/spark/SparkContext.scala b/src/scala/spark/SparkContext.scala index 9b8c34421f..50d8e4908a 100644 --- a/src/scala/spark/SparkContext.scala +++ b/src/scala/spark/SparkContext.scala @@ -18,8 +18,8 @@ class SparkContext(master: String, frameworkName: String) { new Accumulator(initialValue, param) // TODO: Keep around a weak hash map of values to Cached versions? - def broadcast[T](value: T) = new CentralizedHDFSBroadcast(value, local) - //def broadcast[T](value: T) = new ChainedStreamingBroadcast(value, local) + // def broadcast[T](value: T) = new CentralizedHDFSBroadcast(value, local) + def broadcast[T](value: T) = new ChainedStreamingBroadcast(value, local) def textFile(path: String) = new HdfsTextFile(this, path) -- cgit v1.2.3 From 4fdd48295b966ff9aae950e58f9cd6001e7a0087 Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Tue, 12 Oct 2010 13:10:31 -0700 Subject: Added mesos.jar. Still not working. Major changes required. --- third_party/libnexus.so | Bin 5823410 -> 0 bytes third_party/mesos.jar | Bin 0 -> 33618 bytes third_party/nexus.jar | Bin 39840 -> 0 bytes 3 files changed, 0 insertions(+), 0 deletions(-) delete mode 100755 third_party/libnexus.so create mode 100644 third_party/mesos.jar delete mode 100644 third_party/nexus.jar diff --git a/third_party/libnexus.so b/third_party/libnexus.so deleted file mode 100755 index 80c151751f..0000000000 Binary files a/third_party/libnexus.so and /dev/null differ diff --git a/third_party/mesos.jar b/third_party/mesos.jar new file mode 100644 index 0000000000..60d299c8af Binary files /dev/null and b/third_party/mesos.jar differ diff --git a/third_party/nexus.jar b/third_party/nexus.jar deleted file mode 100644 index 1a53acf591..0000000000 Binary files a/third_party/nexus.jar and /dev/null differ -- cgit v1.2.3 From 0d67bc1cee62594f2bc9c24734b1cf90c88b8bce Mon Sep 17 00:00:00 2001 From: Mosharaf Chowdhury Date: Tue, 12 Oct 2010 15:39:53 -0700 Subject: multi-tracker branch now compiles and runs; but it crashes right before the end. The same problem is seen also in the master branch (in the ChainedStreaming implementation) --- Makefile | 14 +- README | 29 +- conf/java-opts | 1 + conf/log4j.properties | 8 + conf/spark-env.sh | 13 + run | 40 +- spark-executor | 3 +- src/examples/BroadcastTest.scala | 18 +- src/examples/SparkALS.scala | 5 - src/examples/Vector.scala | 2 +- src/scala/spark/Accumulators.scala | 12 +- src/scala/spark/Broadcast.scala | 952 +++++++-------- src/scala/spark/ClosureCleaner.scala | 8 +- src/scala/spark/Executor.scala | 32 +- src/scala/spark/HdfsFile.scala | 33 +- src/scala/spark/LocalScheduler.scala | 22 +- src/scala/spark/Logging.scala | 49 + src/scala/spark/MesosScheduler.scala | 346 ++++++ src/scala/spark/NexusScheduler.scala | 290 ----- src/scala/spark/ParallelArray.scala | 15 +- src/scala/spark/RDD.scala | 196 ++- src/scala/spark/Scheduler.scala | 1 + src/scala/spark/SparkContext.scala | 26 +- src/scala/spark/Split.scala | 13 + src/scala/spark/Task.scala | 2 +- src/scala/spark/Utils.scala | 27 +- src/scala/spark/repl/ClassServer.scala | 77 ++ src/scala/spark/repl/ExecutorClassLoader.scala | 42 +- src/scala/spark/repl/SparkInterpreter.scala | 44 +- src/scala/spark/repl/SparkInterpreterLoop.scala | 2 +- src/scala/ubiquifs/Header.scala | 21 - src/scala/ubiquifs/Master.scala | 49 - src/scala/ubiquifs/Message.scala | 14 - src/scala/ubiquifs/Slave.scala | 141 --- src/scala/ubiquifs/UbiquiFS.scala | 11 - src/scala/ubiquifs/Utils.scala | 12 - third_party/apache-log4j-1.2.16/log4j-1.2.16.jar | Bin 0 -> 481534 bytes third_party/google-collect-1.0-rc5/COPYING | 202 ---- .../google-collect-1.0-rc5.jar | Bin 577311 -> 0 bytes third_party/guava-r06/COPYING | 202 ++++ third_party/guava-r06/README | 28 + third_party/guava-r06/guava-r06.jar | Bin 0 -> 934385 bytes third_party/hadoop-0.20.0/contrib/hod/CHANGES.txt | 248 ---- third_party/hadoop-0.20.0/contrib/hod/README | 104 -- third_party/hadoop-0.20.0/contrib/hod/bin/VERSION | 1 - .../hadoop-0.20.0/contrib/hod/bin/checknodes | 31 - third_party/hadoop-0.20.0/contrib/hod/bin/hod | 577 --------- .../hadoop-0.20.0/contrib/hod/bin/hodcleanup | 183 --- third_party/hadoop-0.20.0/contrib/hod/bin/hodring | 287 ----- .../hadoop-0.20.0/contrib/hod/bin/ringmaster | 349 ------ .../hadoop-0.20.0/contrib/hod/bin/verify-account | 11 - third_party/hadoop-0.20.0/contrib/hod/build.xml | 81 -- third_party/hadoop-0.20.0/contrib/hod/conf/hodrc | 46 - third_party/hadoop-0.20.0/contrib/hod/config.txt | 172 --- .../hadoop-0.20.0/contrib/hod/getting_started.txt | 233 ---- .../hod/hodlib/AllocationManagers/__init__.py | 16 - .../AllocationManagers/goldAllocationManager.py | 104 -- .../contrib/hod/hodlib/Common/__init__.py | 15 - .../hod/hodlib/Common/allocationManagerUtil.py | 27 - .../contrib/hod/hodlib/Common/desc.py | 298 ----- .../contrib/hod/hodlib/Common/descGenerator.py | 72 -- .../contrib/hod/hodlib/Common/hodsvc.py | 228 ---- .../contrib/hod/hodlib/Common/logger.py | 788 ------------ .../contrib/hod/hodlib/Common/miniHTMLParser.py | 45 - .../contrib/hod/hodlib/Common/nodepoolutil.py | 26 - .../contrib/hod/hodlib/Common/setup.py | 1058 ---------------- .../contrib/hod/hodlib/Common/socketServers.py | 621 ---------- .../hadoop-0.20.0/contrib/hod/hodlib/Common/tcp.py | 176 --- .../contrib/hod/hodlib/Common/threads.py | 389 ------ .../contrib/hod/hodlib/Common/types.py | 1266 -------------------- .../contrib/hod/hodlib/Common/util.py | 309 ----- .../contrib/hod/hodlib/Common/xmlrpc.py | 57 - .../contrib/hod/hodlib/GridServices/__init__.py | 18 - .../contrib/hod/hodlib/GridServices/hdfs.py | 310 ----- .../contrib/hod/hodlib/GridServices/mapred.py | 272 ----- .../contrib/hod/hodlib/GridServices/service.py | 266 ---- .../contrib/hod/hodlib/Hod/__init__.py | 15 - .../hadoop-0.20.0/contrib/hod/hodlib/Hod/hadoop.py | 747 ------------ .../hadoop-0.20.0/contrib/hod/hodlib/Hod/hod.py | 754 ------------ .../contrib/hod/hodlib/Hod/nodePool.py | 128 -- .../contrib/hod/hodlib/HodRing/__init__.py | 15 - .../contrib/hod/hodlib/HodRing/hodRing.py | 928 -------------- .../contrib/hod/hodlib/NodePools/__init__.py | 15 - .../contrib/hod/hodlib/NodePools/torque.py | 334 ------ .../contrib/hod/hodlib/RingMaster/__init__.py | 15 - .../hod/hodlib/RingMaster/idleJobTracker.py | 218 ---- .../contrib/hod/hodlib/RingMaster/ringMaster.py | 1019 ---------------- .../contrib/hod/hodlib/Schedulers/__init__.py | 15 - .../contrib/hod/hodlib/Schedulers/torque.py | 175 --- .../contrib/hod/hodlib/ServiceProxy/__init__.py | 15 - .../hod/hodlib/ServiceProxy/serviceProxy.py | 49 - .../contrib/hod/hodlib/ServiceRegistry/__init__.py | 15 - .../hod/hodlib/ServiceRegistry/serviceRegistry.py | 127 -- .../hadoop-0.20.0/contrib/hod/hodlib/__init__.py | 16 - third_party/hadoop-0.20.0/contrib/hod/ivy.xml | 22 - .../contrib/hod/ivy/libraries.properties | 5 - .../contrib/hod/support/checklimits.sh | 57 - .../contrib/hod/support/logcondense.py | 212 ---- .../hadoop-0.20.0/contrib/hod/testing/__init__.py | 15 - .../hadoop-0.20.0/contrib/hod/testing/helper.py | 33 - .../hadoop-0.20.0/contrib/hod/testing/lib.py | 113 -- .../hadoop-0.20.0/contrib/hod/testing/main.py | 83 -- .../contrib/hod/testing/testHadoop.py | 123 -- .../hadoop-0.20.0/contrib/hod/testing/testHod.py | 310 ----- .../contrib/hod/testing/testHodCleanup.py | 113 -- .../contrib/hod/testing/testHodRing.py | 117 -- .../contrib/hod/testing/testModule.py | 88 -- .../contrib/hod/testing/testRingmasterRPCs.py | 171 --- .../contrib/hod/testing/testThreads.py | 99 -- .../hadoop-0.20.0/contrib/hod/testing/testTypes.py | 180 --- .../hadoop-0.20.0/contrib/hod/testing/testUtil.py | 62 - .../contrib/hod/testing/testXmlrpc.py | 109 -- third_party/hadoop-0.20.0/lib/slf4j-api-1.4.3.jar | Bin 15345 -> 0 bytes .../hadoop-0.20.0/lib/slf4j-log4j12-1.4.3.jar | Bin 8601 -> 0 bytes .../jetty-server-7.1.6.v20100715.jar | Bin 0 -> 647178 bytes .../jetty-7.1.6.v20100715/servlet-api-2.5.jar | Bin 0 -> 105112 bytes third_party/mesos.jar | Bin 33618 -> 34562 bytes third_party/scalacheck_2.8.0-1.7.jar | Bin 0 -> 745883 bytes third_party/scalacheck_2.8.0.RC3-1.7.jar | Bin 745630 -> 0 bytes .../scalatest-1.2-for-scala-2.8.0.RC3-SNAPSHOT.jar | Bin 1784234 -> 0 bytes third_party/scalatest-1.2/LICENSE | 202 ++++ third_party/scalatest-1.2/NOTICE | 7 + third_party/scalatest-1.2/README.txt | 58 + third_party/scalatest-1.2/scalatest-1.2.jar | Bin 0 -> 1784096 bytes third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar | Bin 0 -> 25496 bytes third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar | Bin 0 -> 9753 bytes 126 files changed, 1892 insertions(+), 16573 deletions(-) create mode 100644 conf/java-opts create mode 100644 conf/log4j.properties create mode 100755 conf/spark-env.sh create mode 100644 src/scala/spark/Logging.scala create mode 100644 src/scala/spark/MesosScheduler.scala delete mode 100644 src/scala/spark/NexusScheduler.scala create mode 100644 src/scala/spark/Split.scala create mode 100644 src/scala/spark/repl/ClassServer.scala delete mode 100644 src/scala/ubiquifs/Header.scala delete mode 100644 src/scala/ubiquifs/Master.scala delete mode 100644 src/scala/ubiquifs/Message.scala delete mode 100644 src/scala/ubiquifs/Slave.scala delete mode 100644 src/scala/ubiquifs/UbiquiFS.scala delete mode 100644 src/scala/ubiquifs/Utils.scala create mode 100644 third_party/apache-log4j-1.2.16/log4j-1.2.16.jar delete mode 100644 third_party/google-collect-1.0-rc5/COPYING delete mode 100644 third_party/google-collect-1.0-rc5/google-collect-1.0-rc5.jar create mode 100644 third_party/guava-r06/COPYING create mode 100644 third_party/guava-r06/README create mode 100644 third_party/guava-r06/guava-r06.jar delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/CHANGES.txt delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/README delete mode 100755 third_party/hadoop-0.20.0/contrib/hod/bin/VERSION delete mode 100755 third_party/hadoop-0.20.0/contrib/hod/bin/checknodes delete mode 100755 third_party/hadoop-0.20.0/contrib/hod/bin/hod delete mode 100755 third_party/hadoop-0.20.0/contrib/hod/bin/hodcleanup delete mode 100755 third_party/hadoop-0.20.0/contrib/hod/bin/hodring delete mode 100755 third_party/hadoop-0.20.0/contrib/hod/bin/ringmaster delete mode 100755 third_party/hadoop-0.20.0/contrib/hod/bin/verify-account delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/build.xml delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/conf/hodrc delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/config.txt delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/getting_started.txt delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/AllocationManagers/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/AllocationManagers/goldAllocationManager.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/allocationManagerUtil.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/desc.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/descGenerator.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/hodsvc.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/logger.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/miniHTMLParser.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/nodepoolutil.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/setup.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/socketServers.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/tcp.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/threads.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/types.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/util.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/xmlrpc.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/hdfs.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/mapred.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/service.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/hadoop.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/hod.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/nodePool.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/HodRing/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/HodRing/hodRing.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/NodePools/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/NodePools/torque.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/idleJobTracker.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/ringMaster.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Schedulers/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/Schedulers/torque.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceProxy/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceProxy/serviceProxy.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceRegistry/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceRegistry/serviceRegistry.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/hodlib/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/ivy.xml delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/ivy/libraries.properties delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/support/checklimits.sh delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/support/logcondense.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/__init__.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/helper.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/lib.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/main.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testHadoop.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testHod.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testHodCleanup.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testHodRing.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testModule.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testRingmasterRPCs.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testThreads.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testTypes.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testUtil.py delete mode 100644 third_party/hadoop-0.20.0/contrib/hod/testing/testXmlrpc.py delete mode 100644 third_party/hadoop-0.20.0/lib/slf4j-api-1.4.3.jar delete mode 100644 third_party/hadoop-0.20.0/lib/slf4j-log4j12-1.4.3.jar create mode 100644 third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar create mode 100644 third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar create mode 100644 third_party/scalacheck_2.8.0-1.7.jar delete mode 100644 third_party/scalacheck_2.8.0.RC3-1.7.jar delete mode 100644 third_party/scalatest-1.2-for-scala-2.8.0.RC3-SNAPSHOT.jar create mode 100644 third_party/scalatest-1.2/LICENSE create mode 100644 third_party/scalatest-1.2/NOTICE create mode 100644 third_party/scalatest-1.2/README.txt create mode 100644 third_party/scalatest-1.2/scalatest-1.2.jar create mode 100644 third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar create mode 100644 third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar diff --git a/Makefile b/Makefile index 1e49402a1b..c5d004fb10 100644 --- a/Makefile +++ b/Makefile @@ -2,15 +2,19 @@ EMPTY = SPACE = $(EMPTY) $(EMPTY) # Build up classpath by concatenating some strings -JARS = third_party/nexus.jar +JARS = third_party/mesos.jar JARS += third_party/asm-3.2/lib/all/asm-all-3.2.jar JARS += third_party/colt.jar -JARS += third_party/google-collect-1.0-rc5/google-collect-1.0-rc5.jar +JARS += third_party/guava-r06/guava-r06.jar JARS += third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar JARS += third_party/hadoop-0.20.0/lib/commons-logging-1.0.4.jar -JARS += third_party/scalatest-1.2-for-scala-2.8.0.RC3-SNAPSHOT.jar -JARS += third_party/scalacheck_2.8.0.RC3-1.7.jar -JARS += third_party/FreePastry-2.1.jar +JARS += third_party/scalatest-1.2/scalatest-1.2.jar +JARS += third_party/scalacheck_2.8.0-1.7.jar +JARS += third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar +JARS += third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar +JARS += third_party/apache-log4j-1.2.16/log4j-1.2.16.jar +JARS += third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar +JARS += third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar CLASSPATH = $(subst $(SPACE),:,$(JARS)) SCALA_SOURCES = src/examples/*.scala src/scala/spark/*.scala src/scala/spark/repl/*.scala diff --git a/README b/README index e0c7603632..f084f22a1f 100644 --- a/README +++ b/README @@ -1,9 +1,11 @@ -Spark requires Scala 2.8. This version has been tested with 2.8.0RC3. +BUILDING + +Spark requires Scala 2.8. This version has been tested with 2.8.0.final. To build and run Spark, you will need to have Scala's bin in your $PATH, or you will need to set the SCALA_HOME environment variable to point to where you've installed Scala. Scala must be accessible through one -of these methods on Nexus slave nodes as well as on the master. +of these methods on Mesos slave nodes as well as on the master. To build Spark and the example programs, run make. @@ -11,5 +13,28 @@ To run one of the examples, use ./run . For example, ./run SparkLR will run the Logistic Regression example. Each of the example programs prints usage help if no params are given. +All of the Spark samples take a parameter that is the Mesos master +to connect to. This can be a Mesos URL, or "local" to run locally with one +thread, or "local[N]" to run locally with N threads. + Tip: If you are building Spark and examples repeatedly, export USE_FSC=1 to have the Makefile use the fsc compiler daemon instead of scalac. + +CONFIGURATION + +Spark can be configured through two files: conf/java-opts and conf/spark-env.sh. + +In java-opts, you can add flags to be passed to the JVM when running Spark. + +In spark-env.sh, you can set any environment variables you wish to be available +when running Spark programs, such as PATH, SCALA_HOME, etc. There are also +several Spark-specific variables you can set: +- SPARK_CLASSPATH: Extra entries to be added to the classpath, separated by ":". +- SPARK_MEM: Memory for Spark to use, in the format used by java's -Xmx option + (for example, 200m meams 200 MB, 1g means 1 GB, etc). +- SPARK_LIBRARY_PATH: Extra entries to add to java.library.path for locating + shared libraries. +- SPARK_JAVA_OPTS: Extra options to pass to JVM. + +Note that spark-env.sh must be a shell script (it must be executable and start +with a #! header to specify the shell to use). diff --git a/conf/java-opts b/conf/java-opts new file mode 100644 index 0000000000..b61e8163b5 --- /dev/null +++ b/conf/java-opts @@ -0,0 +1 @@ +-Dspark.broadcast.masterHostAddress=127.0.0.1 -Dspark.broadcast.masterTrackerPort=11111 -Dspark.broadcast.blockSize=1024 -Dspark.broadcast.maxRetryCount=2 -Dspark.broadcast.serverSocketTimout=50000 -Dspark.broadcast.dualMode=false diff --git a/conf/log4j.properties b/conf/log4j.properties new file mode 100644 index 0000000000..d72dbadc39 --- /dev/null +++ b/conf/log4j.properties @@ -0,0 +1,8 @@ +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN diff --git a/conf/spark-env.sh b/conf/spark-env.sh new file mode 100755 index 0000000000..77f9cb69b9 --- /dev/null +++ b/conf/spark-env.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +# Set Spark environment variables for your site in this file. Some useful +# variables to set are: +# - MESOS_HOME, to point to your Mesos installation +# - SCALA_HOME, to point to your Scala installation +# - SPARK_CLASSPATH, to add elements to Spark's classpath +# - SPARK_JAVA_OPTS, to add JVM options +# - SPARK_MEM, to change the amount of memory used per node (this should +# be in the same format as the JVM's -Xmx option, e.g. 300m or 1g). +# - SPARK_LIBRARY_PATH, to add extra search paths for native libraries. + +MESOS_HOME=/home/mosharaf/Work/mesos diff --git a/run b/run index 00696b2c1a..f28b39af9b 100755 --- a/run +++ b/run @@ -3,27 +3,49 @@ # Figure out where the Scala framework is installed FWDIR=`dirname $0` -# Set JAVA_OPTS to be able to load libnexus.so and set various other misc options -export JAVA_OPTS="-Djava.library.path=$FWDIR/third_party:$FWDIR/src/native -Xms100m -Xmx750m" +# Load environment variables from conf/spark-env.sh, if it exists +if [ -e $FWDIR/conf/spark-env.sh ] ; then + . $FWDIR/conf/spark-env.sh +fi + +if [ "x$MESOS_HOME" != "x" ] ; then + SPARK_CLASSPATH="$MESOS_HOME/lib/java/mesos.jar:$SPARK_CLASSPATH" + SPARK_LIBRARY_PATH="$MESOS_HOME/lib/java:$SPARK_LIBARY_PATH" +fi + +if [ "x$SPARK_MEM" == "x" ] ; then + SPARK_MEM="300m" +fi + +# Set JAVA_OPTS to be able to load native libraries and to set heap size +JAVA_OPTS="$SPARK_JAVA_OPTS" +JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH:$FWDIR/third_party:$FWDIR/src/native" +JAVA_OPTS+=" -Xms$SPARK_MEM -Xmx$SPARK_MEM" +# Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e $FWDIR/conf/java-opts ] ; then JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" fi export JAVA_OPTS # Build up classpath -CLASSPATH=$FWDIR/build/classes -CLASSPATH+=:$FWDIR/third_party/nexus.jar +CLASSPATH="$SPARK_CLASSPATH:$FWDIR/build/classes" +CLASSPATH+=:$FWDIR/conf +CLASSPATH+=:$FWDIR/third_party/mesos.jar CLASSPATH+=:$FWDIR/third_party/asm-3.2/lib/all/asm-all-3.2.jar CLASSPATH+=:$FWDIR/third_party/colt.jar -CLASSPATH+=:$FWDIR/third_party/google-collect-1.0-rc5/google-collect-1.0-rc5.jar +CLASSPATH+=:$FWDIR/third_party/guava-r06/guava-r06.jar CLASSPATH+=:$FWDIR/third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar -CLASSPATH+=:third_party/scalatest-1.2-for-scala-2.8.0.RC3-SNAPSHOT.jar -CLASSPATH+=:third_party/scalacheck_2.8.0.RC3-1.7.jar -CLASSPATH+=:$FWDIR/third_party/FreePastry-2.1.jar +CLASSPATH+=:$FWDIR/third_party/scalatest-1.2/scalatest-1.2.jar +CLASSPATH+=:$FWDIR/third_party/scalacheck_2.8.0-1.7.jar +CLASSPATH+=:$FWDIR/third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar +CLASSPATH+=:$FWDIR/third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar +CLASSPATH+=:$FWDIR/third_party/apache-log4j-1.2.16/log4j-1.2.16.jar +CLASSPATH+=:$FWDIR/third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar +CLASSPATH+=:$FWDIR/third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar for jar in $FWDIR/third_party/hadoop-0.20.0/lib/*.jar; do CLASSPATH+=:$jar done -export CLASSPATH +export CLASSPATH # Needed for spark-shell if [ -n "$SCALA_HOME" ]; then SCALA=${SCALA_HOME}/bin/scala diff --git a/spark-executor b/spark-executor index ee847cfff0..0f9b9b1ece 100755 --- a/spark-executor +++ b/spark-executor @@ -1,5 +1,4 @@ #!/bin/sh -echo "In spark-executor" FWDIR="`dirname $0`" -echo Framework dir: $FWDIR +echo "Running spark-executor with framework dir = $FWDIR" exec $FWDIR/run spark.Executor diff --git a/src/examples/BroadcastTest.scala b/src/examples/BroadcastTest.scala index 40c2be8f6d..7764013413 100644 --- a/src/examples/BroadcastTest.scala +++ b/src/examples/BroadcastTest.scala @@ -10,19 +10,15 @@ object BroadcastTest { val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 - var arr1 = new Array[Int](num) - for (i <- 0 until arr1.length) - arr1(i) = i + var arr = new Array[Int](num) + for (i <- 0 until arr.length) + arr(i) = i -// var arr2 = new Array[Int](num * 2) -// for (i <- 0 until arr2.length) -// arr2(i) = i - - val barr1 = spark.broadcast(arr1) -// val barr2 = spark.broadcast(arr2) + val barr = spark.broadcast(arr) spark.parallelize(1 to 10, slices).foreach { -// i => println(barr1.value.size + barr2.value.size) - i => println(barr1.value.size) + println("in task: barr = " + barr) + i => println(barr.value.size) } } } + diff --git a/src/examples/SparkALS.scala b/src/examples/SparkALS.scala index 7c4dce5869..6fae3c0940 100644 --- a/src/examples/SparkALS.scala +++ b/src/examples/SparkALS.scala @@ -123,8 +123,6 @@ object SparkALS { var msc = spark.broadcast(ms) var usc = spark.broadcast(us) for (iter <- 1 to ITERATIONS) { - val start = System.nanoTime - println("Iteration " + iter + ":") ms = spark.parallelize(0 until M, slices) .map(i => updateMovie(i, msc.value(i), usc.value, Rc.value)) @@ -136,9 +134,6 @@ object SparkALS { usc = spark.broadcast(us) // Re-broadcast us because it was updated println("RMSE = " + rmse(R, ms, us)) println() - - val time = (System.nanoTime - start) / 1e9 - println( "This iteration took " + time + " s") } } } diff --git a/src/examples/Vector.scala b/src/examples/Vector.scala index 0ae2cbc6e8..e9fbdca752 100644 --- a/src/examples/Vector.scala +++ b/src/examples/Vector.scala @@ -57,7 +57,7 @@ object Vector { implicit def doubleToMultiplier(num: Double) = new Multiplier(num) implicit object VectorAccumParam extends spark.AccumulatorParam[Vector] { - def add(t1: Vector, t2: Vector) = t1 + t2 + def addInPlace(t1: Vector, t2: Vector) = t1 + t2 def zero(initialValue: Vector) = Vector.zeros(initialValue.length) } } diff --git a/src/scala/spark/Accumulators.scala b/src/scala/spark/Accumulators.scala index 3e4cd4935a..ee93d3c85c 100644 --- a/src/scala/spark/Accumulators.scala +++ b/src/scala/spark/Accumulators.scala @@ -4,15 +4,17 @@ import java.io._ import scala.collection.mutable.Map -@serializable class Accumulator[T](initialValue: T, param: AccumulatorParam[T]) +@serializable class Accumulator[T]( + @transient initialValue: T, param: AccumulatorParam[T]) { val id = Accumulators.newId - @transient var value_ = initialValue + @transient var value_ = initialValue // Current value on master + val zero = param.zero(initialValue) // Zero value to be passed to workers var deserialized = false Accumulators.register(this) - def += (term: T) { value_ = param.add(value_, term) } + def += (term: T) { value_ = param.addInPlace(value_, term) } def value = this.value_ def value_= (t: T) { if (!deserialized) value_ = t @@ -22,7 +24,7 @@ import scala.collection.mutable.Map // Called by Java when deserializing an object private def readObject(in: ObjectInputStream) { in.defaultReadObject - value_ = param.zero(initialValue) + value_ = zero deserialized = true Accumulators.register(this) } @@ -31,7 +33,7 @@ import scala.collection.mutable.Map } @serializable trait AccumulatorParam[T] { - def add(t1: T, t2: T): T + def addInPlace(t1: T, t2: T): T def zero(initialValue: T): T } diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala index b2114eb151..23ab9b345c 100644 --- a/src/scala/spark/Broadcast.scala +++ b/src/scala/spark/Broadcast.scala @@ -15,16 +15,16 @@ import org.apache.hadoop.fs.{FileSystem, Path, RawLocalFileSystem} import spark.compress.lzf.{LZFInputStream, LZFOutputStream} -import rice.environment.Environment -import rice.p2p.commonapi._ -import rice.p2p.commonapi.rawserialization.RawMessage -import rice.pastry._ -import rice.pastry.commonapi.PastryIdFactory -import rice.pastry.direct._ -import rice.pastry.socket.SocketPastryNodeFactory -import rice.pastry.standard.RandomNodeIdFactory -import rice.p2p.scribe._ -import rice.p2p.splitstream._ +//import rice.environment.Environment +//import rice.p2p.commonapi._ +//import rice.p2p.commonapi.rawserialization.RawMessage +//import rice.pastry._ +//import rice.pastry.commonapi.PastryIdFactory +//import rice.pastry.direct._ +//import rice.pastry.socket.SocketPastryNodeFactory +//import rice.pastry.standard.RandomNodeIdFactory +//import rice.p2p.scribe._ +//import rice.p2p.splitstream._ @serializable trait BroadcastRecipe { @@ -39,7 +39,7 @@ trait BroadcastRecipe { @serializable class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) - extends BroadcastRecipe { + extends BroadcastRecipe with Logging { def value = value_ @@ -71,7 +71,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) val start = System.nanoTime sendBroadcast val time = (System.nanoTime - start) / 1e9 - println("sendBroadcast took " + time + " s") + logInfo("sendBroadcast took " + time + " s") } def sendBroadcast () { @@ -86,14 +86,14 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) var variableInfo = blockifyObject (value_, BroadcastCS.blockSize) guideMR = new GuideMultipleRequests - // guideMR.setDaemon (true) + guideMR.setDaemon (true) guideMR.start - // println (System.currentTimeMillis + ": " + "GuideMultipleRequests started") + logInfo (System.currentTimeMillis + ": " + "GuideMultipleRequests started") serveMR = new ServeMultipleRequests - // serveMR.setDaemon (true) + serveMR.setDaemon (true) serveMR.start - // println (System.currentTimeMillis + ": " + "ServeMultipleRequests started") + logInfo (System.currentTimeMillis + ": " + "ServeMultipleRequests started") // Prepare the value being broadcasted // TODO: Refactoring and clean-up required here @@ -142,9 +142,9 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) initializeSlaveVariables serveMR = new ServeMultipleRequests - // serveMR.setDaemon (true) + serveMR.setDaemon (true) serveMR.start - // println (System.currentTimeMillis + ": " + "ServeMultipleRequests started") + logInfo (System.currentTimeMillis + ": " + "ServeMultipleRequests started") val start = System.nanoTime @@ -161,7 +161,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } val time = (System.nanoTime - start) / 1e9 - println( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") + logInfo( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") } } } @@ -265,7 +265,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) retriesLeft -= 1 // TODO: Should wait before retrying } while (retriesLeft > 0 && masterListenPort < 0) - // println (System.currentTimeMillis + ": " + "Got this guidePort from Tracker: " + masterListenPort) + logInfo (System.currentTimeMillis + ": " + "Got this guidePort from Tracker: " + masterListenPort) return masterListenPort } @@ -290,7 +290,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // Connect to Master and send this worker's Information val clientSocketToMaster = new Socket(BroadcastCS.masterHostAddress, masterListenPort) - // println (System.currentTimeMillis + ": " + "Connected to Master's guiding object") + logInfo (System.currentTimeMillis + ": " + "Connected to Master's guiding object") // TODO: Guiding object connection is reusable val oosMaster = new ObjectOutputStream (clientSocketToMaster.getOutputStream) @@ -310,14 +310,12 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } totalBytes = sourceInfo.totalBytes - // println (System.currentTimeMillis + ": " + "Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort) + logInfo (System.currentTimeMillis + ": " + "Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort) val start = System.nanoTime val receptionSucceeded = receiveSingleTransmission (sourceInfo) val time = (System.nanoTime - start) / 1e9 - // println (System.currentTimeMillis + ": " + "I got this from receiveSingleTransmission: " + retByteArray) - // Updating some statistics in sourceInfo. Master will be using them later if (!receptionSucceeded) { sourceInfo.receptionFailed = true } sourceInfo.MBps = (sourceInfo.totalBytes.toDouble / 1048576) / time @@ -353,8 +351,8 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) oisSource = new ObjectInputStream (clientSocketToSource.getInputStream) - // println (System.currentTimeMillis + ": " + "Inside receiveSingleTransmission") - // println (System.currentTimeMillis + ": " + "totalBlocks: "+ totalBlocks + " " + "hasBlocks: " + hasBlocks) + logInfo (System.currentTimeMillis + ": " + "Inside receiveSingleTransmission") + logInfo (System.currentTimeMillis + ": " + "totalBlocks: "+ totalBlocks + " " + "hasBlocks: " + hasBlocks) // Send the range oosSource.writeObject((hasBlocks, totalBlocks)) @@ -369,12 +367,12 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) hasBlocksLock.synchronized { hasBlocksLock.notifyAll } - // println (System.currentTimeMillis + ": " + "Received block: " + i + " " + bcBlock) + logInfo (System.currentTimeMillis + ": " + "Received block: " + i + " " + bcBlock) } - // println (System.currentTimeMillis + ": " + "After the receive loop") + logInfo (System.currentTimeMillis + ": " + "After the receive loop") } catch { case e: Exception => { - // println (System.currentTimeMillis + ": " + "receiveSingleTransmission had a " + e) + logInfo (System.currentTimeMillis + ": " + "receiveSingleTransmission had a " + e) } } finally { if (oisSource != null) { oisSource.close } @@ -385,7 +383,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) return receptionSucceeded } - class GuideMultipleRequests extends Thread { + class GuideMultipleRequests extends Thread with Logging { override def run = { // TODO: Cached threadpool has 60 s keep alive timer var threadPool = Executors.newCachedThreadPool @@ -393,7 +391,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) serverSocket = new ServerSocket (0) guidePort = serverSocket.getLocalPort - // println (System.currentTimeMillis + ": " + "GuideMultipleRequests" + serverSocket + " " + guidePort) + logInfo (System.currentTimeMillis + ": " + "GuideMultipleRequests" + serverSocket + " " + guidePort) guidePortLock.synchronized { guidePortLock.notifyAll @@ -409,12 +407,12 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) clientSocket = serverSocket.accept } catch { case e: Exception => { - // println ("GuideMultipleRequests Timeout. Stopping listening..." + hasCopyInHDFS) + logInfo ("GuideMultipleRequests Timeout. Stopping listening..." + hasCopyInHDFS) keepAccepting = false } } if (clientSocket != null) { - // println (System.currentTimeMillis + ": " + "Guide:Accepted new client connection:" + clientSocket) + logInfo (System.currentTimeMillis + ": " + "Guide:Accepted new client connection:" + clientSocket) try { threadPool.execute (new GuideSingleRequest (clientSocket)) } catch { @@ -429,7 +427,8 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } } - class GuideSingleRequest (val clientSocket: Socket) extends Runnable { + class GuideSingleRequest (val clientSocket: Socket) + extends Runnable with Logging { private val oos = new ObjectOutputStream (clientSocket.getOutputStream) oos.flush private val ois = new ObjectInputStream (clientSocket.getInputStream) @@ -439,7 +438,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) def run = { try { - // println (System.currentTimeMillis + ": " + "new GuideSingleRequest is running") + logInfo (System.currentTimeMillis + ": " + "new GuideSingleRequest is running") // Connecting worker is sending in its hostAddress and listenPort it will // be listening to. ReplicaID is 0 and other fields are invalid (-1) var sourceInfo = ois.readObject.asInstanceOf[SourceInfo] @@ -447,14 +446,14 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) pqOfSources.synchronized { // Select a suitable source and send it back to the worker selectedSourceInfo = selectSuitableSource (sourceInfo) - // println (System.currentTimeMillis + ": " + "Sending selectedSourceInfo:" + selectedSourceInfo) + logInfo (System.currentTimeMillis + ": " + "Sending selectedSourceInfo:" + selectedSourceInfo) oos.writeObject (selectedSourceInfo) oos.flush // Add this new (if it can finish) source to the PQ of sources thisWorkerInfo = new SourceInfo(sourceInfo.hostAddress, sourceInfo.listenPort, totalBlocks, totalBytes, 0) - // println (System.currentTimeMillis + ": " + "Adding possible new source to pqOfSources: " + thisWorkerInfo) + logInfo (System.currentTimeMillis + ": " + "Adding possible new source to pqOfSources: " + thisWorkerInfo) pqOfSources.add (thisWorkerInfo) } @@ -535,14 +534,14 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } } - class ServeMultipleRequests extends Thread { + class ServeMultipleRequests extends Thread with Logging { override def run = { var threadPool = Executors.newCachedThreadPool var serverSocket: ServerSocket = null serverSocket = new ServerSocket (0) listenPort = serverSocket.getLocalPort - // println (System.currentTimeMillis + ": " + "ServeMultipleRequests" + serverSocket + " " + listenPort) + logInfo (System.currentTimeMillis + ": " + "ServeMultipleRequests" + serverSocket + " " + listenPort) listenPortLock.synchronized { listenPortLock.notifyAll @@ -557,12 +556,12 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) clientSocket = serverSocket.accept } catch { case e: Exception => { - // println ("ServeMultipleRequests Timeout. Stopping listening...") + logInfo ("ServeMultipleRequests Timeout. Stopping listening...") keepAccepting = false } } if (clientSocket != null) { - // println (System.currentTimeMillis + ": " + "Serve:Accepted new client connection:" + clientSocket) + logInfo (System.currentTimeMillis + ": " + "Serve:Accepted new client connection:" + clientSocket) try { threadPool.execute (new ServeSingleRequest (clientSocket)) } catch { @@ -576,7 +575,8 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } } - class ServeSingleRequest (val clientSocket: Socket) extends Runnable { + class ServeSingleRequest (val clientSocket: Socket) + extends Runnable with Logging { private val oos = new ObjectOutputStream (clientSocket.getOutputStream) oos.flush private val ois = new ObjectInputStream (clientSocket.getInputStream) @@ -586,7 +586,7 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) def run = { try { - // println (System.currentTimeMillis + ": " + "new ServeSingleRequest is running") + logInfo (System.currentTimeMillis + ": " + "new ServeSingleRequest is running") // Receive range to send var sendRange = ois.readObject.asInstanceOf[(Int, Int)] @@ -599,10 +599,10 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) // If something went wrong, e.g., the worker at the other end died etc. // then close everything up case e: Exception => { - // println (System.currentTimeMillis + ": " + "ServeSingleRequest had a " + e) + logInfo (System.currentTimeMillis + ": " + "ServeSingleRequest had a " + e) } } finally { - // println (System.currentTimeMillis + ": " + "ServeSingleRequest is closing streams and sockets") + logInfo (System.currentTimeMillis + ": " + "ServeSingleRequest is closing streams and sockets") ois.close oos.close clientSocket.close @@ -629,80 +629,80 @@ class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) } catch { case e: Exception => { } } - // println (System.currentTimeMillis + ": " + "Send block: " + i + " " + arrayOfBlocks(i)) + logInfo (System.currentTimeMillis + ": " + "Send block: " + i + " " + arrayOfBlocks(i)) } } } } } -@serializable -class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) - extends BroadcastRecipe { - - def value = value_ - - BroadcastSS.synchronized { BroadcastSS.values.put (uuid, value_) } - - if (!local) { sendBroadcast } - - @transient var publishThread: PublishThread = null - @transient var hasCopyInHDFS = false - - def sendBroadcast () { - // Store a persistent copy in HDFS - val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) - out.writeObject (value_) - out.close - hasCopyInHDFS = true - - publishThread = new PublishThread - publishThread.start - } - - private def readObject (in: ObjectInputStream) { - in.defaultReadObject - BroadcastSS.synchronized { - val cachedVal = BroadcastSS.values.get(uuid) - if (cachedVal != null) { - value_ = cachedVal.asInstanceOf[T] - } else { - val start = System.nanoTime - - // Thread.sleep (5000) // TODO: - val receptionSucceeded = BroadcastSS.receiveVariable (uuid) - // If does not succeed, then get from HDFS copy - if (receptionSucceeded) { - value_ = BroadcastSS.values.get(uuid).asInstanceOf[T] - } else { - // println (System.currentTimeMillis + ": " + "Reading from HDFS") - val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) - value_ = fileIn.readObject.asInstanceOf[T] - BroadcastSS.values.put(uuid, value_) - fileIn.close - } - - val time = (System.nanoTime - start) / 1e9 - println( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") - } - } - } - - class PublishThread extends Thread { - override def run = { - // TODO: Put some delay here to give time others to register - // Thread.sleep (5000) - // println (System.currentTimeMillis + ": " + "Waited. Now sending...") - BroadcastSS.synchronized { - BroadcastSS.publishVariable[T] (uuid, value) - } - } - } -} +//@serializable +//class SplitStreamBroadcast[T] (@transient var value_ : T, local: Boolean) +// extends BroadcastRecipe with Logging { + +// def value = value_ + +// BroadcastSS.synchronized { BroadcastSS.values.put (uuid, value_) } +// +// if (!local) { sendBroadcast } +// +// @transient var publishThread: PublishThread = null +// @transient var hasCopyInHDFS = false +// +// def sendBroadcast () { +// // Store a persistent copy in HDFS +// val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) +// out.writeObject (value_) +// out.close +// hasCopyInHDFS = true +// +// publishThread = new PublishThread +// publishThread.start +// } +// +// private def readObject (in: ObjectInputStream) { +// in.defaultReadObject +// BroadcastSS.synchronized { +// val cachedVal = BroadcastSS.values.get(uuid) +// if (cachedVal != null) { +// value_ = cachedVal.asInstanceOf[T] +// } else { +// val start = System.nanoTime + +// // Thread.sleep (5000) // TODO: +// val receptionSucceeded = BroadcastSS.receiveVariable (uuid) +// // If does not succeed, then get from HDFS copy +// if (receptionSucceeded) { +// value_ = BroadcastSS.values.get(uuid).asInstanceOf[T] +// } else { +// logInfo (System.currentTimeMillis + ": " + "Reading from HDFS") +// val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) +// value_ = fileIn.readObject.asInstanceOf[T] +// BroadcastSS.values.put(uuid, value_) +// fileIn.close +// } +// +// val time = (System.nanoTime - start) / 1e9 +// logInfo( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") +// } +// } +// } +// +// class PublishThread extends Thread with Logging { +// override def run = { +// // TODO: Put some delay here to give time others to register +// // Thread.sleep (5000) +// logInfo (System.currentTimeMillis + ": " + "Waited. Now sending...") +// BroadcastSS.synchronized { +// BroadcastSS.publishVariable[T] (uuid, value) +// } +// } +// } +//} @serializable class CentralizedHDFSBroadcast[T](@transient var value_ : T, local: Boolean) - extends BroadcastRecipe { + extends BroadcastRecipe with Logging { def value = value_ @@ -724,7 +724,7 @@ class CentralizedHDFSBroadcast[T](@transient var value_ : T, local: Boolean) if (cachedVal != null) { value_ = cachedVal.asInstanceOf[T] } else { - // println( System.currentTimeMillis + ": " + "Started reading Broadcasted variable " + uuid) + logInfo( System.currentTimeMillis + ": " + "Started reading Broadcasted variable " + uuid) val start = System.nanoTime val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) @@ -733,7 +733,7 @@ class CentralizedHDFSBroadcast[T](@transient var value_ : T, local: Boolean) fileIn.close val time = (System.nanoTime - start) / 1e9 - println( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") + logInfo( System.currentTimeMillis + ": " + "Reading Broadcasted variable " + uuid + " took " + time + " s") } } } @@ -742,7 +742,7 @@ class CentralizedHDFSBroadcast[T](@transient var value_ : T, local: Boolean) @serializable case class SourceInfo (val hostAddress: String, val listenPort: Int, val totalBlocks: Int, val totalBytes: Int, val replicaID: Int) - extends Comparable [SourceInfo]{ + extends Comparable [SourceInfo] with Logging { var currentLeechers = 0 var receptionFailed = false @@ -779,7 +779,7 @@ case class VariableInfo (@transient val arrayOfBlocks : Array[BroadcastBlock], @transient var hasBlocks = 0 } -private object Broadcast { +private object Broadcast extends Logging { private var initialized = false // Will be called by SparkContext or Executor before using Broadcast @@ -801,7 +801,7 @@ private object Broadcast { } } -private object BroadcastCS { +private object BroadcastCS extends Logging { val values = new MapMaker ().softValues ().makeMap[UUID, Any] var valueToGuidePortMap = Map[UUID, Int] () @@ -847,7 +847,7 @@ private object BroadcastCS { trackMV = new TrackMultipleValues trackMV.setDaemon (true) trackMV.start - // println (System.currentTimeMillis + ": " + "TrackMultipleValues started") + logInfo (System.currentTimeMillis + ": " + "TrackMultipleValues started") } initialized = true @@ -869,7 +869,7 @@ private object BroadcastCS { def registerValue (uuid: UUID, guidePort: Int) = { valueToGuidePortMap.synchronized { valueToGuidePortMap += (uuid -> guidePort) - // println (System.currentTimeMillis + ": " + "New value registered with the Tracker " + valueToGuidePortMap) + logInfo (System.currentTimeMillis + ": " + "New value registered with the Tracker " + valueToGuidePortMap) } } @@ -877,7 +877,7 @@ private object BroadcastCS { valueToGuidePortMap.synchronized { // Set to 0 to make sure that people read it from HDFS valueToGuidePortMap (uuid) = 0 - // println (System.currentTimeMillis + ": " + "Value unregistered from the Tracker " + valueToGuidePortMap) + logInfo (System.currentTimeMillis + ": " + "Value unregistered from the Tracker " + valueToGuidePortMap) } } @@ -895,13 +895,13 @@ private object BroadcastCS { } } - class TrackMultipleValues extends Thread { + class TrackMultipleValues extends Thread with Logging { override def run = { var threadPool = Executors.newCachedThreadPool var serverSocket: ServerSocket = null serverSocket = new ServerSocket (BroadcastCS.masterTrackerPort) - // println (System.currentTimeMillis + ": " + "TrackMultipleValues" + serverSocket) + logInfo (System.currentTimeMillis + ": " + "TrackMultipleValues" + serverSocket) var keepAccepting = true try { @@ -913,7 +913,7 @@ private object BroadcastCS { clientSocket = serverSocket.accept } catch { case e: Exception => { - // println ("TrackMultipleValues Timeout. Stopping listening...") + logInfo ("TrackMultipleValues Timeout. Stopping listening...") // TODO: Tracking should be explicitly stopped by the SparkContext // keepAccepting = false } @@ -935,7 +935,7 @@ private object BroadcastCS { var guidePort = if (valueToGuidePortMap.contains (uuid)) { valueToGuidePortMap (uuid) } else -1 - // println (System.currentTimeMillis + ": " + "TrackMultipleValues:Got new request: " + clientSocket + " for " + uuid + " : " + guidePort) + logInfo (System.currentTimeMillis + ": " + "TrackMultipleValues:Got new request: " + clientSocket + " for " + uuid + " : " + guidePort) oos.writeObject (guidePort) } catch { case e: Exception => { } @@ -959,360 +959,360 @@ private object BroadcastCS { } } -private object BroadcastSS { - val values = new MapMaker ().softValues ().makeMap[UUID, Any] - - private val valueBytes = new MapMaker().softValues().makeMap[UUID,Array[Byte]] - - private var initialized = false - private var isMaster_ = false - - private var masterBootHost_ = "127.0.0.1" - private var masterBootPort_ : Int = 22222 - private var blockSize_ : Int = 512 * 1024 - private var maxRetryCount_ : Int = 2 - - private var masterBootAddress_ : InetSocketAddress = null - private var localBindPort_ : Int = -1 - - private var pEnvironment_ : Environment = null - private var pastryNode_ : PastryNode = null - private var ssClient: SSClient = null - - // Current transmission state variables - private var curUUID: UUID = null - private var curTotalBlocks = -1 - private var curTotalBytes = -1 - private var curHasBlocks = -1 - private var curBlockBitmap: Array[Boolean] = null - private var curArrayOfBytes: Array[Byte] = null - - // TODO: Add stuff so that we can handle out of order variable broadcast - - def initialize (isMaster__ : Boolean) { - synchronized { - if (!initialized) { - masterBootHost_ = - System.getProperty ("spark.broadcast.masterHostAddress", "127.0.0.1") - masterBootPort_ = - System.getProperty ("spark.broadcast.masterBootPort", "22222").toInt - - masterBootAddress_ = new InetSocketAddress(masterBootHost_, - masterBootPort_) - - blockSize_ = - System.getProperty ("spark.broadcast.blockSize", "512").toInt * 1024 - maxRetryCount_ = - System.getProperty ("spark.broadcast.maxRetryCount", "2").toInt - - isMaster_ = isMaster__ - - // Initialize the SplitStream tree - initializeSplitStream - - initialized = true - } - } - } - - def masterBootAddress = masterBootAddress_ - def blockSize = blockSize_ - def maxRetryCount = maxRetryCount_ - - def pEnvironment: Environment = { - if (pEnvironment_ == null) { initializeSplitStream } - pEnvironment_ - } - - def pastryNode: PastryNode = { - if (pastryNode_ == null) { initializeSplitStream } - pastryNode_ - } - - def localBindPort = { - if (localBindPort_ == -1) { - if (isMaster) { localBindPort_ = masterBootPort_ } - else { - // TODO: What's the best way of finding a free port? - val sSocket = new ServerSocket (0) - val sPort = sSocket.getLocalPort - sSocket.close - localBindPort_ = sPort - } - } - localBindPort_ - } - - def isMaster = isMaster_ - - private def initializeSplitStream = { - pEnvironment_ = new Environment - - // Generate the NodeIds Randomly - val nidFactory = new RandomNodeIdFactory (pEnvironment) - - // Construct the PastryNodeFactory - val pastryNodeFactory = new SocketPastryNodeFactory (nidFactory, - localBindPort, pEnvironment) - - // Construct a Pastry node - pastryNode_ = pastryNodeFactory.newNode - - // Boot the node. - pastryNode.boot (masterBootAddress) - // TODO: Some unknown messages are dropped in slaves at this point - - // The node may require sending several messages to fully boot into the ring - pastryNode.synchronized { - while(!pastryNode.isReady && !pastryNode.joinFailed) { - // Delay so we don't busy-wait - pastryNode.wait (500) - - // Abort if can't join - if (pastryNode.joinFailed()) { - // TODO: throw new IOException("Join failed " + node.joinFailedReason) - } - } - } - - // Create the SplitStream client and subscribe - ssClient = new SSClient (BroadcastSS.pastryNode) - ssClient.subscribe - } - - def publishVariable[A] (uuid: UUID, obj: A) = { - ssClient.synchronized { - ssClient.publish[A] (uuid, obj) - } - } - - // Return status of the reception - def receiveVariable[A] (uuid: UUID): Boolean = { - // TODO: Things will change if out-of-order variable recepetion is supported - - // println (System.currentTimeMillis + ": " + "In receiveVariable") - - // Check in valueBytes - if (xferValueBytesToValues[A] (uuid)) { return true } - - // Check if its in progress - for (i <- 0 until maxRetryCount) { - // println (System.currentTimeMillis + ": " + uuid + " " + curUUID) - while (uuid == curUUID) { Thread.sleep (100) } // TODO: How long to sleep - if (xferValueBytesToValues[A] (uuid)) { return true } - - // Wait for a while to see if we've reached here before xmission started - Thread.sleep (100) - } - return false - } - - private def xferValueBytesToValues[A] (uuid: UUID): Boolean = { - var cachedValueBytes: Array[Byte] = null - valueBytes.synchronized { cachedValueBytes = valueBytes.get (uuid) } - if (cachedValueBytes != null) { - val cachedValue = byteArrayToObject[A] (cachedValueBytes) - values.synchronized { values.put (uuid, cachedValue) } - return true - } - return false - } - - private def objectToByteArray[A] (obj: A): Array[Byte] = { - val baos = new ByteArrayOutputStream - val oos = new ObjectOutputStream (baos) - oos.writeObject (obj) - oos.close - baos.close - return baos.toByteArray - } - - private def byteArrayToObject[A] (bytes: Array[Byte]): A = { - val in = new ObjectInputStream (new ByteArrayInputStream (bytes)) - val retVal = in.readObject.asInstanceOf[A] - in.close - return retVal - } - - private def intToByteArray (value: Int): Array[Byte] = { - var retVal = new Array[Byte] (4) - for (i <- 0 until 4) - retVal(i) = (value >> ((4 - 1 - i) * 8)).toByte - return retVal - } - - private def byteArrayToInt (arr: Array[Byte], offset: Int): Int = { - var retVal = 0 - for (i <- 0 until 4) - retVal += ((arr(i + offset).toInt & 0x000000FF) << ((4 - 1 - i) * 8)) - return retVal - } - - class SSClient (pastryNode: PastryNode) extends SplitStreamClient - with Application { - // Magic bits: 11111100001100100100110000111111 - val magicBits = 0xFC324C3F - - // Message Types - val INFO_MSG = 1 - val DATA_MSG = 2 - - // The Endpoint represents the underlying node. By making calls on the - // Endpoint, it assures that the message will be delivered to the App on - // whichever node the message is intended for. - protected val endPoint = pastryNode.buildEndpoint (this, "myInstance") - - // Handle to a SplitStream implementation - val mySplitStream = new SplitStreamImpl (pastryNode, "mySplitStream") - - // The ChannelId is constructed from a normal PastryId based on the UUID - val myChannelId = new ChannelId (new PastryIdFactory - (pastryNode.getEnvironment).buildId ("myChannel")) - - // The channel - var myChannel: Channel = null - - // The stripes. Acquired from myChannel. - var myStripes: Array[Stripe] = null - - // Now we can receive messages - endPoint.register - - // Subscribes to all stripes in myChannelId. - def subscribe = { - // Attaching makes you part of the Channel, and volunteers to be an - // internal node of one of the trees - myChannel = mySplitStream.attachChannel (myChannelId) - - // Subscribing notifies your application when data comes through the tree - myStripes = myChannel.getStripes - for (curStripe <- myStripes) { curStripe.subscribe (this) } - } - - // Part of SplitStreamClient. Called when a published message is received. - def deliver (s: Stripe, data: Array[Byte]) = { - // Unpack and verify magicBits - val topLevelInfo = byteArrayToObject[(Int, Int, Array[Byte])] (data) - - // Process only if magicBits are OK - if (topLevelInfo._1 == magicBits) { - // Process only for slaves - if (!BroadcastSS.isMaster) { - // Match on Message Type - topLevelInfo._2 match { - case INFO_MSG => { - val realInfo = byteArrayToObject[(UUID, Int, Int)] ( - topLevelInfo._3) - - // Setup states for impending transmission - curUUID = realInfo._1 // TODO: - curTotalBlocks = realInfo._2 - curTotalBytes = realInfo._3 - - curHasBlocks = 0 - curBlockBitmap = new Array[Boolean] (curTotalBlocks) - curArrayOfBytes = new Array[Byte] (curTotalBytes) - - // println (System.currentTimeMillis + ": " + curUUID + " " + curTotalBlocks + " " + curTotalBytes) - } - case DATA_MSG => { - val realInfo = byteArrayToObject[(UUID, Int, Array[Byte])] ( - topLevelInfo._3) - val blockUUID = realInfo._1 - val blockIndex = realInfo._2 - val blockData = realInfo._3 - - // TODO: Will change in future implementation. Right now we - // require broadcast in order on the variable level. Blocks can - // come out of order though - assert (blockUUID == curUUID) - - // Update everything - curHasBlocks += 1 - curBlockBitmap(blockIndex) = true - System.arraycopy (blockData, 0, curArrayOfBytes, - blockIndex * blockSize, blockData.length) - - // println (System.currentTimeMillis + ": " + "Got stuff for: " + blockUUID) - - // Done receiving - if (curHasBlocks == curTotalBlocks) { - // Store as a Array[Byte] - valueBytes.synchronized { - valueBytes.put (curUUID, curArrayOfBytes) - } - - // println (System.currentTimeMillis + ": " + "Finished reading. Stored in valueBytes") - - // RESET - curUUID = null - } - } - case _ => { - // Should never happen - } - } - } - } - } - - // Multicasts data. - def publish[A] (uuid: UUID, obj: A) = { - val byteArray = objectToByteArray[A] (obj) - - var blockNum = (byteArray.length / blockSize) - if (byteArray.length % blockSize != 0) - blockNum += 1 - - // ------------------------------------- - // INFO_MSG: | UUID | Total Blocks | Total Bytes | - // ------------------------------------- - var infoByteArray = objectToByteArray[(UUID, Int, Int)] ((uuid, blockNum, - byteArray.length)) - doPublish (0, INFO_MSG, infoByteArray) - - // ------------------------------------- - // DATA_MSG: | UUID | Block Index | Single Block | - // ------------------------------------- - var blockID = 0 - for (i <- 0 until (byteArray.length, blockSize)) { - val thisBlockSize = Math.min (blockSize, byteArray.length - i) - var thisBlockData = new Array[Byte] (thisBlockSize) - System.arraycopy (byteArray, i * blockSize, thisBlockData, 0, - thisBlockSize) - - var dataByteArray = objectToByteArray[(UUID, Int, Array[Byte])] ((uuid, - blockID, thisBlockData)) - doPublish (blockID % myStripes.length, DATA_MSG, dataByteArray) - - blockID += 1 - } - } - - // -------------------------------- - // Message Format: | MagicBits | Type | Real Data | - // -------------------------------- - private def doPublish (stripeID: Int, msgType: Int, data: Array[Byte]) = { - val bytesToSend = objectToByteArray[(Int, Int, Array[Byte])] ((magicBits, - msgType, data)) - myStripes(stripeID).publish (bytesToSend) - } - - /* class PublishContent extends Message { - def getPriority: Int = { Message.MEDIUM_PRIORITY } - } */ - - // Error handling - def joinFailed(s: Stripe) = { println ("joinFailed(" + s + ")") } - - // Rest of the Application interface. NOT USED. - def deliver (id: rice.p2p.commonapi.Id, message: Message) = { } - def forward (message: RouteMessage): Boolean = false - def update (handle: rice.p2p.commonapi.NodeHandle, joined: Boolean) = { } - } -} - -private object BroadcastCH { +//private object BroadcastSS { +// val values = new MapMaker ().softValues ().makeMap[UUID, Any] + +// private val valueBytes = new MapMaker().softValues().makeMap[UUID,Array[Byte]] + +// private var initialized = false +// private var isMaster_ = false +// +// private var masterBootHost_ = "127.0.0.1" +// private var masterBootPort_ : Int = 22222 +// private var blockSize_ : Int = 512 * 1024 +// private var maxRetryCount_ : Int = 2 +// +// private var masterBootAddress_ : InetSocketAddress = null +// private var localBindPort_ : Int = -1 +// +// private var pEnvironment_ : Environment = null +// private var pastryNode_ : PastryNode = null +// private var ssClient: SSClient = null +// +// // Current transmission state variables +// private var curUUID: UUID = null +// private var curTotalBlocks = -1 +// private var curTotalBytes = -1 +// private var curHasBlocks = -1 +// private var curBlockBitmap: Array[Boolean] = null +// private var curArrayOfBytes: Array[Byte] = null +// +// // TODO: Add stuff so that we can handle out of order variable broadcast + +// def initialize (isMaster__ : Boolean) { +// synchronized { +// if (!initialized) { +// masterBootHost_ = +// System.getProperty ("spark.broadcast.masterHostAddress", "127.0.0.1") +// masterBootPort_ = +// System.getProperty ("spark.broadcast.masterBootPort", "22222").toInt +// +// masterBootAddress_ = new InetSocketAddress(masterBootHost_, +// masterBootPort_) +// +// blockSize_ = +// System.getProperty ("spark.broadcast.blockSize", "512").toInt * 1024 +// maxRetryCount_ = +// System.getProperty ("spark.broadcast.maxRetryCount", "2").toInt +// +// isMaster_ = isMaster__ +// +// // Initialize the SplitStream tree +// initializeSplitStream +// +// initialized = true +// } +// } +// } +// +// def masterBootAddress = masterBootAddress_ +// def blockSize = blockSize_ +// def maxRetryCount = maxRetryCount_ +// +// def pEnvironment: Environment = { +// if (pEnvironment_ == null) { initializeSplitStream } +// pEnvironment_ +// } +// +// def pastryNode: PastryNode = { +// if (pastryNode_ == null) { initializeSplitStream } +// pastryNode_ +// } +// +// def localBindPort = { +// if (localBindPort_ == -1) { +// if (isMaster) { localBindPort_ = masterBootPort_ } +// else { +// // TODO: What's the best way of finding a free port? +// val sSocket = new ServerSocket (0) +// val sPort = sSocket.getLocalPort +// sSocket.close +// localBindPort_ = sPort +// } +// } +// localBindPort_ +// } + +// def isMaster = isMaster_ +// +// private def initializeSplitStream = { +// pEnvironment_ = new Environment +// +// // Generate the NodeIds Randomly +// val nidFactory = new RandomNodeIdFactory (pEnvironment) +// +// // Construct the PastryNodeFactory +// val pastryNodeFactory = new SocketPastryNodeFactory (nidFactory, +// localBindPort, pEnvironment) +// +// // Construct a Pastry node +// pastryNode_ = pastryNodeFactory.newNode +// +// // Boot the node. +// pastryNode.boot (masterBootAddress) +// // TODO: Some unknown messages are dropped in slaves at this point +// +// // The node may require sending several messages to fully boot into the ring +// pastryNode.synchronized { +// while(!pastryNode.isReady && !pastryNode.joinFailed) { +// // Delay so we don't busy-wait +// pastryNode.wait (500) +// +// // Abort if can't join +// if (pastryNode.joinFailed()) { +// // TODO: throw new IOException("Join failed " + node.joinFailedReason) +// } +// } +// } +// +// // Create the SplitStream client and subscribe +// ssClient = new SSClient (BroadcastSS.pastryNode) +// ssClient.subscribe +// } +// +// def publishVariable[A] (uuid: UUID, obj: A) = { +// ssClient.synchronized { +// ssClient.publish[A] (uuid, obj) +// } +// } +// +// // Return status of the reception +// def receiveVariable[A] (uuid: UUID): Boolean = { +// // TODO: Things will change if out-of-order variable recepetion is supported +// +// logInfo (System.currentTimeMillis + ": " + "In receiveVariable") +// +// // Check in valueBytes +// if (xferValueBytesToValues[A] (uuid)) { return true } +// +// // Check if its in progress +// for (i <- 0 until maxRetryCount) { +// logInfo (System.currentTimeMillis + ": " + uuid + " " + curUUID) +// while (uuid == curUUID) { Thread.sleep (100) } // TODO: How long to sleep +// if (xferValueBytesToValues[A] (uuid)) { return true } +// +// // Wait for a while to see if we've reached here before xmission started +// Thread.sleep (100) +// } +// return false +// } +// +// private def xferValueBytesToValues[A] (uuid: UUID): Boolean = { +// var cachedValueBytes: Array[Byte] = null +// valueBytes.synchronized { cachedValueBytes = valueBytes.get (uuid) } +// if (cachedValueBytes != null) { +// val cachedValue = byteArrayToObject[A] (cachedValueBytes) +// values.synchronized { values.put (uuid, cachedValue) } +// return true +// } +// return false +// } +// +// private def objectToByteArray[A] (obj: A): Array[Byte] = { +// val baos = new ByteArrayOutputStream +// val oos = new ObjectOutputStream (baos) +// oos.writeObject (obj) +// oos.close +// baos.close +// return baos.toByteArray +// } + +// private def byteArrayToObject[A] (bytes: Array[Byte]): A = { +// val in = new ObjectInputStream (new ByteArrayInputStream (bytes)) +// val retVal = in.readObject.asInstanceOf[A] +// in.close +// return retVal +// } + +// private def intToByteArray (value: Int): Array[Byte] = { +// var retVal = new Array[Byte] (4) +// for (i <- 0 until 4) +// retVal(i) = (value >> ((4 - 1 - i) * 8)).toByte +// return retVal +// } + +// private def byteArrayToInt (arr: Array[Byte], offset: Int): Int = { +// var retVal = 0 +// for (i <- 0 until 4) +// retVal += ((arr(i + offset).toInt & 0x000000FF) << ((4 - 1 - i) * 8)) +// return retVal +// } + +// class SSClient (pastryNode: PastryNode) extends SplitStreamClient +// with Application { +// // Magic bits: 11111100001100100100110000111111 +// val magicBits = 0xFC324C3F +// +// // Message Types +// val INFO_MSG = 1 +// val DATA_MSG = 2 +// +// // The Endpoint represents the underlying node. By making calls on the +// // Endpoint, it assures that the message will be delivered to the App on +// // whichever node the message is intended for. +// protected val endPoint = pastryNode.buildEndpoint (this, "myInstance") + +// // Handle to a SplitStream implementation +// val mySplitStream = new SplitStreamImpl (pastryNode, "mySplitStream") + +// // The ChannelId is constructed from a normal PastryId based on the UUID +// val myChannelId = new ChannelId (new PastryIdFactory +// (pastryNode.getEnvironment).buildId ("myChannel")) +// +// // The channel +// var myChannel: Channel = null +// +// // The stripes. Acquired from myChannel. +// var myStripes: Array[Stripe] = null + +// // Now we can receive messages +// endPoint.register +// +// // Subscribes to all stripes in myChannelId. +// def subscribe = { +// // Attaching makes you part of the Channel, and volunteers to be an +// // internal node of one of the trees +// myChannel = mySplitStream.attachChannel (myChannelId) +// +// // Subscribing notifies your application when data comes through the tree +// myStripes = myChannel.getStripes +// for (curStripe <- myStripes) { curStripe.subscribe (this) } +// } +// +// // Part of SplitStreamClient. Called when a published message is received. +// def deliver (s: Stripe, data: Array[Byte]) = { +// // Unpack and verify magicBits +// val topLevelInfo = byteArrayToObject[(Int, Int, Array[Byte])] (data) +// +// // Process only if magicBits are OK +// if (topLevelInfo._1 == magicBits) { +// // Process only for slaves +// if (!BroadcastSS.isMaster) { +// // Match on Message Type +// topLevelInfo._2 match { +// case INFO_MSG => { +// val realInfo = byteArrayToObject[(UUID, Int, Int)] ( +// topLevelInfo._3) +// +// // Setup states for impending transmission +// curUUID = realInfo._1 // TODO: +// curTotalBlocks = realInfo._2 +// curTotalBytes = realInfo._3 +// +// curHasBlocks = 0 +// curBlockBitmap = new Array[Boolean] (curTotalBlocks) +// curArrayOfBytes = new Array[Byte] (curTotalBytes) +// +// logInfo (System.currentTimeMillis + ": " + curUUID + " " + curTotalBlocks + " " + curTotalBytes) +// } +// case DATA_MSG => { +// val realInfo = byteArrayToObject[(UUID, Int, Array[Byte])] ( +// topLevelInfo._3) +// val blockUUID = realInfo._1 +// val blockIndex = realInfo._2 +// val blockData = realInfo._3 +// +// // TODO: Will change in future implementation. Right now we +// // require broadcast in order on the variable level. Blocks can +// // come out of order though +// assert (blockUUID == curUUID) +// +// // Update everything +// curHasBlocks += 1 +// curBlockBitmap(blockIndex) = true +// System.arraycopy (blockData, 0, curArrayOfBytes, +// blockIndex * blockSize, blockData.length) +// +// logInfo (System.currentTimeMillis + ": " + "Got stuff for: " + blockUUID) +// +// // Done receiving +// if (curHasBlocks == curTotalBlocks) { +// // Store as a Array[Byte] +// valueBytes.synchronized { +// valueBytes.put (curUUID, curArrayOfBytes) +// } +// +// logInfo (System.currentTimeMillis + ": " + "Finished reading. Stored in valueBytes") +// +// // RESET +// curUUID = null +// } +// } +// case _ => { +// // Should never happen +// } +// } +// } +// } +// } + +// // Multicasts data. +// def publish[A] (uuid: UUID, obj: A) = { +// val byteArray = objectToByteArray[A] (obj) +// +// var blockNum = (byteArray.length / blockSize) +// if (byteArray.length % blockSize != 0) +// blockNum += 1 +// +// // ------------------------------------- +// // INFO_MSG: | UUID | Total Blocks | Total Bytes | +// // ------------------------------------- +// var infoByteArray = objectToByteArray[(UUID, Int, Int)] ((uuid, blockNum, +// byteArray.length)) +// doPublish (0, INFO_MSG, infoByteArray) +// +// // ------------------------------------- +// // DATA_MSG: | UUID | Block Index | Single Block | +// // ------------------------------------- +// var blockID = 0 +// for (i <- 0 until (byteArray.length, blockSize)) { +// val thisBlockSize = Math.min (blockSize, byteArray.length - i) +// var thisBlockData = new Array[Byte] (thisBlockSize) +// System.arraycopy (byteArray, i * blockSize, thisBlockData, 0, +// thisBlockSize) + +// var dataByteArray = objectToByteArray[(UUID, Int, Array[Byte])] ((uuid, +// blockID, thisBlockData)) +// doPublish (blockID % myStripes.length, DATA_MSG, dataByteArray) + +// blockID += 1 +// } +// } +// +// // -------------------------------- +// // Message Format: | MagicBits | Type | Real Data | +// // -------------------------------- +// private def doPublish (stripeID: Int, msgType: Int, data: Array[Byte]) = { +// val bytesToSend = objectToByteArray[(Int, Int, Array[Byte])] ((magicBits, +// msgType, data)) +// myStripes(stripeID).publish (bytesToSend) +// } + +// /* class PublishContent extends Message { +// def getPriority: Int = { Message.MEDIUM_PRIORITY } +// } */ +// +// // Error handling +// def joinFailed(s: Stripe) = { logInfo ("joinFailed(" + s + ")") } + +// // Rest of the Application interface. NOT USED. +// def deliver (id: rice.p2p.commonapi.Id, message: Message) = { } +// def forward (message: RouteMessage): Boolean = false +// def update (handle: rice.p2p.commonapi.NodeHandle, joined: Boolean) = { } +// } +//} + +private object BroadcastCH extends Logging { val values = new MapMaker ().softValues ().makeMap[UUID, Any] private var initialized = false diff --git a/src/scala/spark/ClosureCleaner.scala b/src/scala/spark/ClosureCleaner.scala index 8037434c38..0e0b3954d4 100644 --- a/src/scala/spark/ClosureCleaner.scala +++ b/src/scala/spark/ClosureCleaner.scala @@ -8,7 +8,7 @@ import org.objectweb.asm.commons.EmptyVisitor import org.objectweb.asm.Opcodes._ -object ClosureCleaner { +object ClosureCleaner extends Logging { private def getClassReader(cls: Class[_]): ClassReader = { new ClassReader(cls.getResourceAsStream( cls.getName.replaceFirst("^.*\\.", "") + ".class")) @@ -72,13 +72,13 @@ object ClosureCleaner { val field = cls.getDeclaredField(fieldName) field.setAccessible(true) val value = field.get(obj) - //println("1: Setting " + fieldName + " on " + cls + " to " + value); + //logInfo("1: Setting " + fieldName + " on " + cls + " to " + value); field.set(outer, value) } } if (outer != null) { - //println("2: Setting $outer on " + func.getClass + " to " + outer); + //logInfo("2: Setting $outer on " + func.getClass + " to " + outer); val field = func.getClass.getDeclaredField("$outer") field.setAccessible(true) field.set(func, outer) @@ -101,7 +101,7 @@ object ClosureCleaner { val newCtor = rf.newConstructorForSerialization(cls, parentCtor) val obj = newCtor.newInstance().asInstanceOf[AnyRef]; if (outer != null) { - //println("3: Setting $outer on " + cls + " to " + outer); + //logInfo("3: Setting $outer on " + cls + " to " + outer); val field = cls.getDeclaredField("$outer") field.setAccessible(true) field.set(obj, outer) diff --git a/src/scala/spark/Executor.scala b/src/scala/spark/Executor.scala index 679a61f3c0..be73aae541 100644 --- a/src/scala/spark/Executor.scala +++ b/src/scala/spark/Executor.scala @@ -2,14 +2,18 @@ package spark import java.util.concurrent.{Executors, ExecutorService} -import nexus.{ExecutorArgs, ExecutorDriver, NexusExecutorDriver} -import nexus.{TaskDescription, TaskState, TaskStatus} +import mesos.{ExecutorArgs, ExecutorDriver, MesosExecutorDriver} +import mesos.{TaskDescription, TaskState, TaskStatus} -object Executor { +/** + * The Mesos executor for Spark. + */ +object Executor extends Logging { def main(args: Array[String]) { - System.loadLibrary("nexus") + System.loadLibrary("mesos") - val exec = new nexus.Executor() { + // Create a new Executor implementation that will run our tasks + val exec = new mesos.Executor() { var classLoader: ClassLoader = null var threadPool: ExecutorService = null @@ -25,10 +29,10 @@ object Executor { // If the REPL is in use, create a ClassLoader that will be able to // read new classes defined by the REPL as the user types code classLoader = this.getClass.getClassLoader - val classDir = System.getProperty("spark.repl.current.classdir") - if (classDir != null) { - println("Using REPL classdir: " + classDir) - classLoader = new repl.ExecutorClassLoader(classDir, classLoader) + val classUri = System.getProperty("spark.repl.class.uri") + if (classUri != null) { + logInfo("Using REPL class URI: " + classUri) + classLoader = new repl.ExecutorClassLoader(classUri, classLoader) } Thread.currentThread.setContextClassLoader(classLoader) @@ -43,7 +47,7 @@ object Executor { val arg = desc.getArg threadPool.execute(new Runnable() { def run() = { - println("Running task ID " + taskId) + logInfo("Running task ID " + taskId) try { Accumulators.clear val task = Utils.deserialize[Task[Any]](arg, classLoader) @@ -52,12 +56,11 @@ object Executor { val result = new TaskResult(value, accumUpdates) d.sendStatusUpdate(new TaskStatus( taskId, TaskState.TASK_FINISHED, Utils.serialize(result))) - println("Finished task ID " + taskId) + logInfo("Finished task ID " + taskId) } catch { case e: Exception => { // TODO: Handle errors in tasks less dramatically - System.err.println("Exception in task ID " + taskId + ":") - e.printStackTrace + logError("Exception in task ID " + taskId, e) System.exit(1) } } @@ -66,6 +69,7 @@ object Executor { } } - new NexusExecutorDriver(exec).run() + // Start it running and connect it to the slave + new MesosExecutorDriver(exec).run() } } diff --git a/src/scala/spark/HdfsFile.scala b/src/scala/spark/HdfsFile.scala index 6aa0e22338..8637c6e30a 100644 --- a/src/scala/spark/HdfsFile.scala +++ b/src/scala/spark/HdfsFile.scala @@ -1,6 +1,6 @@ package spark -import nexus.SlaveOffer +import mesos.SlaveOffer import org.apache.hadoop.io.LongWritable import org.apache.hadoop.io.Text @@ -11,11 +11,16 @@ import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter -class HdfsSplit(@transient s: InputSplit) -extends SerializableWritable[InputSplit](s) {} +@serializable class HdfsSplit(@transient s: InputSplit) +extends Split { + val inputSplit = new SerializableWritable[InputSplit](s) + + override def getId() = inputSplit.toString // Hadoop makes this unique + // for each split of each file +} class HdfsTextFile(sc: SparkContext, path: String) -extends RDD[String, HdfsSplit](sc) { +extends RDD[String](sc) { @transient val conf = new JobConf() @transient val inputFormat = new TextInputFormat() @@ -23,11 +28,12 @@ extends RDD[String, HdfsSplit](sc) { ConfigureLock.synchronized { inputFormat.configure(conf) } @transient val splits_ = - inputFormat.getSplits(conf, 2).map(new HdfsSplit(_)).toArray + inputFormat.getSplits(conf, sc.scheduler.numCores).map(new HdfsSplit(_)).toArray - override def splits = splits_ + override def splits = splits_.asInstanceOf[Array[Split]] - override def iterator(split: HdfsSplit) = new Iterator[String] { + override def iterator(split_in: Split) = new Iterator[String] { + val split = split_in.asInstanceOf[HdfsSplit] var reader: RecordReader[LongWritable, Text] = null ConfigureLock.synchronized { val conf = new JobConf() @@ -35,7 +41,7 @@ extends RDD[String, HdfsSplit](sc) { System.getProperty("spark.buffer.size", "65536")) val tif = new TextInputFormat() tif.configure(conf) - reader = tif.getRecordReader(split.value, conf, Reporter.NULL) + reader = tif.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) } val lineNum = new LongWritable() val text = new Text() @@ -44,7 +50,12 @@ extends RDD[String, HdfsSplit](sc) { override def hasNext: Boolean = { if (!gotNext) { - finished = !reader.next(lineNum, text) + try { + finished = !reader.next(lineNum, text) + } catch { + case eofe: java.io.EOFException => + finished = true + } gotNext = true } !finished @@ -60,9 +71,9 @@ extends RDD[String, HdfsSplit](sc) { } } - override def preferredLocations(split: HdfsSplit) = { + override def preferredLocations(split: Split) = { // TODO: Filtering out "localhost" in case of file:// URLs - split.value.getLocations().filter(_ != "localhost") + split.asInstanceOf[HdfsSplit].inputSplit.value.getLocations().filter(_ != "localhost") } } diff --git a/src/scala/spark/LocalScheduler.scala b/src/scala/spark/LocalScheduler.scala index b33f3c863e..20954a1224 100644 --- a/src/scala/spark/LocalScheduler.scala +++ b/src/scala/spark/LocalScheduler.scala @@ -4,8 +4,10 @@ import java.util.concurrent._ import scala.collection.mutable.Map -// A simple Scheduler implementation that runs tasks locally in a thread pool. -private class LocalScheduler(threads: Int) extends Scheduler { +/** + * A simple Scheduler implementation that runs tasks locally in a thread pool. + */ +private class LocalScheduler(threads: Int) extends Scheduler with Logging { var threadPool: ExecutorService = Executors.newFixedThreadPool(threads, DaemonThreadFactory) @@ -20,25 +22,24 @@ private class LocalScheduler(threads: Int) extends Scheduler { for (i <- 0 until tasks.length) { futures(i) = threadPool.submit(new Callable[TaskResult[T]]() { def call(): TaskResult[T] = { - println("Running task " + i) + logInfo("Running task " + i) try { // Serialize and deserialize the task so that accumulators are // changed to thread-local ones; this adds a bit of unnecessary // overhead but matches how the Nexus Executor works Accumulators.clear val bytes = Utils.serialize(tasks(i)) - println("Size of task " + i + " is " + bytes.size + " bytes") + logInfo("Size of task " + i + " is " + bytes.size + " bytes") val task = Utils.deserialize[Task[T]]( bytes, currentThread.getContextClassLoader) val value = task.run val accumUpdates = Accumulators.values - println("Finished task " + i) + logInfo("Finished task " + i) new TaskResult[T](value, accumUpdates) } catch { case e: Exception => { // TODO: Do something nicer here - System.err.println("Exception in task " + i + ":") - e.printStackTrace() + logError("Exception in task " + i, e) System.exit(1) null } @@ -54,9 +55,14 @@ private class LocalScheduler(threads: Int) extends Scheduler { } override def stop() {} + + override def numCores() = threads } -// A ThreadFactory that creates daemon threads + +/** + * A ThreadFactory that creates daemon threads + */ private object DaemonThreadFactory extends ThreadFactory { override def newThread(r: Runnable): Thread = { val t = new Thread(r); diff --git a/src/scala/spark/Logging.scala b/src/scala/spark/Logging.scala new file mode 100644 index 0000000000..2d1feebbb1 --- /dev/null +++ b/src/scala/spark/Logging.scala @@ -0,0 +1,49 @@ +package spark + +import org.slf4j.Logger +import org.slf4j.LoggerFactory + +/** + * Utility trait for classes that want to log data. Creates a SLF4J logger + * for the class and allows logging messages at different levels using + * methods that only evaluate parameters lazily if the log level is enabled. + */ +trait Logging { + // Make the log field transient so that objects with Logging can + // be serialized and used on another machine + @transient private var log_ : Logger = null + + // Method to get or create the logger for this object + def log: Logger = { + if (log_ == null) { + var className = this.getClass().getName() + // Ignore trailing $'s in the class names for Scala objects + if (className.endsWith("$")) + className = className.substring(0, className.length - 1) + log_ = LoggerFactory.getLogger(className) + } + return log_ + } + + // Log methods that take only a String + def logInfo(msg: => String) = if (log.isInfoEnabled) log.info(msg) + + def logDebug(msg: => String) = if (log.isDebugEnabled) log.debug(msg) + + def logWarning(msg: => String) = if (log.isWarnEnabled) log.warn(msg) + + def logError(msg: => String) = if (log.isErrorEnabled) log.error(msg) + + // Log methods that take Throwables (Exceptions/Errors) too + def logInfo(msg: => String, throwable: Throwable) = + if (log.isInfoEnabled) log.info(msg) + + def logDebug(msg: => String, throwable: Throwable) = + if (log.isDebugEnabled) log.debug(msg) + + def logWarning(msg: => String, throwable: Throwable) = + if (log.isWarnEnabled) log.warn(msg, throwable) + + def logError(msg: => String, throwable: Throwable) = + if (log.isErrorEnabled) log.error(msg, throwable) +} diff --git a/src/scala/spark/MesosScheduler.scala b/src/scala/spark/MesosScheduler.scala new file mode 100644 index 0000000000..873a97c59c --- /dev/null +++ b/src/scala/spark/MesosScheduler.scala @@ -0,0 +1,346 @@ +package spark + +import java.io.File + +import scala.collection.mutable.Map +import scala.collection.mutable.Queue +import scala.collection.mutable.HashMap +import scala.collection.JavaConversions._ + +import mesos.{Scheduler => NScheduler} +import mesos._ + +// The main Scheduler implementation, which talks to Mesos. Clients are expected +// to first call start(), then submit tasks through the runTasks method. +// +// This implementation is currently a little quick and dirty. The following +// improvements need to be made to it: +// 1) Right now, the scheduler uses a linear scan through the tasks to find a +// local one for a given node. It would be faster to have a separate list of +// pending tasks for each node. +// 2) Presenting a single slave in ParallelOperation.slaveOffer makes it +// difficult to balance tasks across nodes. It would be better to pass +// all the offers to the ParallelOperation and have it load-balance. +private class MesosScheduler( + master: String, frameworkName: String, execArg: Array[Byte]) +extends NScheduler with spark.Scheduler with Logging +{ + // Lock used by runTasks to ensure only one thread can be in it + val runTasksMutex = new Object() + + // Lock used to wait for scheduler to be registered + var isRegistered = false + val registeredLock = new Object() + + // Current callback object (may be null) + var activeOpsQueue = new Queue[Int] + var activeOps = new HashMap[Int, ParallelOperation] + private var nextOpId = 0 + private[spark] var taskIdToOpId = new HashMap[Int, Int] + + def newOpId(): Int = { + val id = nextOpId + nextOpId += 1 + return id + } + + // Incrementing task ID + private var nextTaskId = 0 + + def newTaskId(): Int = { + val id = nextTaskId; + nextTaskId += 1; + return id + } + + // Driver for talking to Mesos + var driver: SchedulerDriver = null + + override def start() { + new Thread("Spark scheduler") { + setDaemon(true) + override def run { + val ns = MesosScheduler.this + ns.driver = new MesosSchedulerDriver(ns, master) + ns.driver.run() + } + }.start + } + + override def getFrameworkName(d: SchedulerDriver): String = frameworkName + + override def getExecutorInfo(d: SchedulerDriver): ExecutorInfo = + new ExecutorInfo(new File("spark-executor").getCanonicalPath(), execArg) + + override def runTasks[T: ClassManifest](tasks: Array[Task[T]]): Array[T] = { + var opId = 0 + waitForRegister() + this.synchronized { + opId = newOpId() + } + val myOp = new SimpleParallelOperation(this, tasks, opId) + + try { + this.synchronized { + this.activeOps(myOp.opId) = myOp + this.activeOpsQueue += myOp.opId + } + driver.reviveOffers(); + myOp.join(); + } finally { + this.synchronized { + this.activeOps.remove(myOp.opId) + this.activeOpsQueue.dequeueAll(x => (x == myOp.opId)) + } + } + + if (myOp.errorHappened) + throw new SparkException(myOp.errorMessage, myOp.errorCode) + else + return myOp.results + } + + override def registered(d: SchedulerDriver, frameworkId: String) { + logInfo("Registered as framework ID " + frameworkId) + registeredLock.synchronized { + isRegistered = true + registeredLock.notifyAll() + } + } + + override def waitForRegister() { + registeredLock.synchronized { + while (!isRegistered) + registeredLock.wait() + } + } + + override def resourceOffer( + d: SchedulerDriver, oid: String, offers: java.util.List[SlaveOffer]) { + synchronized { + val tasks = new java.util.ArrayList[TaskDescription] + val availableCpus = offers.map(_.getParams.get("cpus").toInt) + val availableMem = offers.map(_.getParams.get("mem").toInt) + var launchedTask = true + for (opId <- activeOpsQueue) { + launchedTask = true + while (launchedTask) { + launchedTask = false + for (i <- 0 until offers.size.toInt) { + try { + activeOps(opId).slaveOffer(offers.get(i), availableCpus(i), availableMem(i)) match { + case Some(task) => + tasks.add(task) + availableCpus(i) -= task.getParams.get("cpus").toInt + availableMem(i) -= task.getParams.get("mem").toInt + launchedTask = launchedTask || true + case None => {} + } + } catch { + case e: Exception => logError("Exception in resourceOffer", e) + } + } + } + } + val params = new java.util.HashMap[String, String] + params.put("timeout", "1") + d.replyToOffer(oid, tasks, params) // TODO: use smaller timeout + } + } + + override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + synchronized { + try { + taskIdToOpId.get(status.getTaskId) match { + case Some(opId) => + if (activeOps.contains(opId)) { + activeOps(opId).statusUpdate(status) + } + case None => + logInfo("TID " + status.getTaskId + " already finished") + } + + } catch { + case e: Exception => logError("Exception in statusUpdate", e) + } + } + } + + override def error(d: SchedulerDriver, code: Int, message: String) { + synchronized { + if (activeOps.size > 0) { + for ((opId, activeOp) <- activeOps) { + try { + activeOp.error(code, message) + } catch { + case e: Exception => logError("Exception in error callback", e) + } + } + } else { + logError("Mesos error: %s (error code: %d)".format(message, code)) + System.exit(1) + } + } + } + + override def stop() { + if (driver != null) + driver.stop() + } + + // TODO: query Mesos for number of cores + override def numCores() = System.getProperty("spark.default.parallelism", "2").toInt +} + + +// Trait representing an object that manages a parallel operation by +// implementing various scheduler callbacks. +trait ParallelOperation { + def slaveOffer(s: SlaveOffer, availableCpus: Int, availableMem: Int): Option[TaskDescription] + def statusUpdate(t: TaskStatus): Unit + def error(code: Int, message: String): Unit +} + + +class SimpleParallelOperation[T: ClassManifest]( + sched: MesosScheduler, tasks: Array[Task[T]], val opId: Int) +extends ParallelOperation with Logging +{ + // Maximum time to wait to run a task in a preferred location (in ms) + val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong + + val callingThread = currentThread + val numTasks = tasks.length + val results = new Array[T](numTasks) + val launched = new Array[Boolean](numTasks) + val finished = new Array[Boolean](numTasks) + val tidToIndex = Map[Int, Int]() + + var allFinished = false + val joinLock = new Object() + + var errorHappened = false + var errorCode = 0 + var errorMessage = "" + + var tasksLaunched = 0 + var tasksFinished = 0 + var lastPreferredLaunchTime = System.currentTimeMillis + + def setAllFinished() { + joinLock.synchronized { + allFinished = true + joinLock.notifyAll() + } + } + + def join() { + joinLock.synchronized { + while (!allFinished) + joinLock.wait() + } + } + + def slaveOffer(offer: SlaveOffer, availableCpus: Int, availableMem: Int): Option[TaskDescription] = { + if (tasksLaunched < numTasks) { + var checkPrefVals: Array[Boolean] = Array(true) + val time = System.currentTimeMillis + if (time - lastPreferredLaunchTime > LOCALITY_WAIT) + checkPrefVals = Array(true, false) // Allow non-preferred tasks + // TODO: Make desiredCpus and desiredMem configurable + val desiredCpus = 1 + val desiredMem = 500 + if ((availableCpus < desiredCpus) || (availableMem < desiredMem)) + return None + for (checkPref <- checkPrefVals; i <- 0 until numTasks) { + if (!launched(i) && (!checkPref || + tasks(i).preferredLocations.contains(offer.getHost) || + tasks(i).preferredLocations.isEmpty)) + { + val taskId = sched.newTaskId() + sched.taskIdToOpId(taskId) = opId + tidToIndex(taskId) = i + val preferred = if(checkPref) "preferred" else "non-preferred" + val message = + "Starting task %d as opId %d, TID %s on slave %s: %s (%s)".format( + i, opId, taskId, offer.getSlaveId, offer.getHost, preferred) + logInfo(message) + tasks(i).markStarted(offer) + launched(i) = true + tasksLaunched += 1 + if (checkPref) + lastPreferredLaunchTime = time + val params = new java.util.HashMap[String, String] + params.put("cpus", "" + desiredCpus) + params.put("mem", "" + desiredMem) + val serializedTask = Utils.serialize(tasks(i)) + //logInfo("Serialized size: " + serializedTask.size) + return Some(new TaskDescription(taskId, offer.getSlaveId, + "task_" + taskId, params, serializedTask)) + } + } + } + return None + } + + def statusUpdate(status: TaskStatus) { + status.getState match { + case TaskState.TASK_FINISHED => + taskFinished(status) + case TaskState.TASK_LOST => + taskLost(status) + case TaskState.TASK_FAILED => + taskLost(status) + case TaskState.TASK_KILLED => + taskLost(status) + case _ => + } + } + + def taskFinished(status: TaskStatus) { + val tid = status.getTaskId + val index = tidToIndex(tid) + if (!finished(index)) { + tasksFinished += 1 + logInfo("Finished opId %d TID %d (progress: %d/%d)".format( + opId, tid, tasksFinished, numTasks)) + // Deserialize task result + val result = Utils.deserialize[TaskResult[T]](status.getData) + results(index) = result.value + // Update accumulators + Accumulators.add(callingThread, result.accumUpdates) + // Mark finished and stop if we've finished all the tasks + finished(index) = true + // Remove TID -> opId mapping from sched + sched.taskIdToOpId.remove(tid) + if (tasksFinished == numTasks) + setAllFinished() + } else { + logInfo("Ignoring task-finished event for TID " + tid + + " because task " + index + " is already finished") + } + } + + def taskLost(status: TaskStatus) { + val tid = status.getTaskId + val index = tidToIndex(tid) + if (!finished(index)) { + logInfo("Lost opId " + opId + " TID " + tid) + launched(index) = false + sched.taskIdToOpId.remove(tid) + tasksLaunched -= 1 + } else { + logInfo("Ignoring task-lost event for TID " + tid + + " because task " + index + " is already finished") + } + } + + def error(code: Int, message: String) { + // Save the error message + errorHappened = true + errorCode = code + errorMessage = message + // Indicate to caller thread that we're done + setAllFinished() + } +} diff --git a/src/scala/spark/NexusScheduler.scala b/src/scala/spark/NexusScheduler.scala deleted file mode 100644 index a5343039ef..0000000000 --- a/src/scala/spark/NexusScheduler.scala +++ /dev/null @@ -1,290 +0,0 @@ -package spark - -import java.io.File - -import scala.collection.mutable.Map - -import nexus.{Scheduler => NScheduler} -import nexus._ - -// The main Scheduler implementation, which talks to Nexus. Clients are expected -// to first call start(), then submit tasks through the runTasks method. -// -// This implementation is currently a little quick and dirty. The following -// improvements need to be made to it: -// 1) Right now, the scheduler uses a linear scan through the tasks to find a -// local one for a given node. It would be faster to have a separate list of -// pending tasks for each node. -// 2) Presenting a single slave in ParallelOperation.slaveOffer makes it -// difficult to balance tasks across nodes. It would be better to pass -// all the offers to the ParallelOperation and have it load-balance. -private class NexusScheduler( - master: String, frameworkName: String, execArg: Array[Byte]) -extends NScheduler with spark.Scheduler -{ - // Lock used by runTasks to ensure only one thread can be in it - val runTasksMutex = new Object() - - // Lock used to wait for scheduler to be registered - var isRegistered = false - val registeredLock = new Object() - - // Current callback object (may be null) - var activeOp: ParallelOperation = null - - // Incrementing task ID - private var nextTaskId = 0 - - def newTaskId(): Int = { - val id = nextTaskId; - nextTaskId += 1; - return id - } - - // Driver for talking to Nexus - var driver: SchedulerDriver = null - - override def start() { - new Thread("Spark scheduler") { - setDaemon(true) - override def run { - val ns = NexusScheduler.this - ns.driver = new NexusSchedulerDriver(ns, master) - ns.driver.run() - } - }.start - } - - override def getFrameworkName(d: SchedulerDriver): String = frameworkName - - override def getExecutorInfo(d: SchedulerDriver): ExecutorInfo = - new ExecutorInfo(new File("spark-executor").getCanonicalPath(), execArg) - - override def runTasks[T: ClassManifest](tasks: Array[Task[T]]): Array[T] = { - runTasksMutex.synchronized { - waitForRegister() - val myOp = new SimpleParallelOperation(this, tasks) - - try { - this.synchronized { - this.activeOp = myOp - } - driver.reviveOffers(); - myOp.join(); - } finally { - this.synchronized { - this.activeOp = null - } - } - - if (myOp.errorHappened) - throw new SparkException(myOp.errorMessage, myOp.errorCode) - else - return myOp.results - } - } - - override def registered(d: SchedulerDriver, frameworkId: Int) { - println("Registered as framework ID " + frameworkId) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() - } - } - - override def waitForRegister() { - registeredLock.synchronized { - while (!isRegistered) - registeredLock.wait() - } - } - - override def resourceOffer( - d: SchedulerDriver, oid: Long, offers: SlaveOfferVector) { - synchronized { - val tasks = new TaskDescriptionVector - if (activeOp != null) { - try { - for (i <- 0 until offers.size.toInt) { - activeOp.slaveOffer(offers.get(i)) match { - case Some(task) => tasks.add(task) - case None => {} - } - } - } catch { - case e: Exception => e.printStackTrace - } - } - val params = new StringMap - params.set("timeout", "1") - d.replyToOffer(oid, tasks, params) // TODO: use smaller timeout - } - } - - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { - synchronized { - try { - if (activeOp != null) { - activeOp.statusUpdate(status) - } - } catch { - case e: Exception => e.printStackTrace - } - } - } - - override def error(d: SchedulerDriver, code: Int, message: String) { - synchronized { - if (activeOp != null) { - try { - activeOp.error(code, message) - } catch { - case e: Exception => e.printStackTrace - } - } else { - val msg = "Nexus error: %s (error code: %d)".format(message, code) - System.err.println(msg) - System.exit(1) - } - } - } - - override def stop() { - if (driver != null) - driver.stop() - } -} - - -// Trait representing an object that manages a parallel operation by -// implementing various scheduler callbacks. -trait ParallelOperation { - def slaveOffer(s: SlaveOffer): Option[TaskDescription] - def statusUpdate(t: TaskStatus): Unit - def error(code: Int, message: String): Unit -} - - -class SimpleParallelOperation[T: ClassManifest]( - sched: NexusScheduler, tasks: Array[Task[T]]) -extends ParallelOperation -{ - // Maximum time to wait to run a task in a preferred location (in ms) - val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "1000").toLong - - val callingThread = currentThread - val numTasks = tasks.length - val results = new Array[T](numTasks) - val launched = new Array[Boolean](numTasks) - val finished = new Array[Boolean](numTasks) - val tidToIndex = Map[Int, Int]() - - var allFinished = false - val joinLock = new Object() - - var errorHappened = false - var errorCode = 0 - var errorMessage = "" - - var tasksLaunched = 0 - var tasksFinished = 0 - var lastPreferredLaunchTime = System.currentTimeMillis - - def setAllFinished() { - joinLock.synchronized { - allFinished = true - joinLock.notifyAll() - } - } - - def join() { - joinLock.synchronized { - while (!allFinished) - joinLock.wait() - } - } - - def slaveOffer(offer: SlaveOffer): Option[TaskDescription] = { - if (tasksLaunched < numTasks) { - var checkPrefVals: Array[Boolean] = Array(true) - val time = System.currentTimeMillis - if (time - lastPreferredLaunchTime > LOCALITY_WAIT) - checkPrefVals = Array(true, false) // Allow non-preferred tasks - // TODO: Make desiredCpus and desiredMem configurable - val desiredCpus = 1 - val desiredMem = 750L * 1024L * 1024L - if (offer.getParams.get("cpus").toInt < desiredCpus || - offer.getParams.get("mem").toLong < desiredMem) - return None - for (checkPref <- checkPrefVals; i <- 0 until numTasks) { - if (!launched(i) && (!checkPref || - tasks(i).preferredLocations.contains(offer.getHost) || - tasks(i).preferredLocations.isEmpty)) - { - val taskId = sched.newTaskId() - tidToIndex(taskId) = i - printf("Starting task %d as TID %d on slave %d: %s (%s)\n", - i, taskId, offer.getSlaveId, offer.getHost, - if(checkPref) "preferred" else "non-preferred") - tasks(i).markStarted(offer) - launched(i) = true - tasksLaunched += 1 - if (checkPref) - lastPreferredLaunchTime = time - val params = new StringMap - params.set("cpus", "" + desiredCpus) - params.set("mem", "" + desiredMem) - val serializedTask = Utils.serialize(tasks(i)) - return Some(new TaskDescription(taskId, offer.getSlaveId, - "task_" + taskId, params, serializedTask)) - } - } - } - return None - } - - def statusUpdate(status: TaskStatus) { - status.getState match { - case TaskState.TASK_FINISHED => - taskFinished(status) - case TaskState.TASK_LOST => - taskLost(status) - case TaskState.TASK_FAILED => - taskLost(status) - case TaskState.TASK_KILLED => - taskLost(status) - case _ => - } - } - - def taskFinished(status: TaskStatus) { - val tid = status.getTaskId - println("Finished TID " + tid) - // Deserialize task result - val result = Utils.deserialize[TaskResult[T]](status.getData) - results(tidToIndex(tid)) = result.value - // Update accumulators - Accumulators.add(callingThread, result.accumUpdates) - // Mark finished and stop if we've finished all the tasks - finished(tidToIndex(tid)) = true - tasksFinished += 1 - if (tasksFinished == numTasks) - setAllFinished() - } - - def taskLost(status: TaskStatus) { - val tid = status.getTaskId - println("Lost TID " + tid) - launched(tidToIndex(tid)) = false - tasksLaunched -= 1 - } - - def error(code: Int, message: String) { - // Save the error message - errorHappened = true - errorCode = code - errorMessage = message - // Indicate to caller thread that we're done - setAllFinished() - } -} diff --git a/src/scala/spark/ParallelArray.scala b/src/scala/spark/ParallelArray.scala index 39ca867cb9..a01904d61c 100644 --- a/src/scala/spark/ParallelArray.scala +++ b/src/scala/spark/ParallelArray.scala @@ -1,11 +1,12 @@ package spark -import nexus.SlaveOffer +import mesos.SlaveOffer import java.util.concurrent.atomic.AtomicLong @serializable class ParallelArraySplit[T: ClassManifest]( - val arrayId: Long, val slice: Int, values: Seq[T]) { + val arrayId: Long, val slice: Int, values: Seq[T]) +extends Split { def iterator(): Iterator[T] = values.iterator override def hashCode(): Int = (41 * (41 + arrayId) + slice).toInt @@ -16,13 +17,13 @@ import java.util.concurrent.atomic.AtomicLong case _ => false } - override def toString() = + override def getId() = "ParallelArraySplit(arrayId %d, slice %d)".format(arrayId, slice) } class ParallelArray[T: ClassManifest]( sc: SparkContext, @transient data: Seq[T], numSlices: Int) -extends RDD[T, ParallelArraySplit[T]](sc) { +extends RDD[T](sc) { // TODO: Right now, each split sends along its full data, even if later down // the RDD chain it gets cached. It might be worthwhile to write the data to // a file in the DFS and read it in the split instead. @@ -34,11 +35,11 @@ extends RDD[T, ParallelArraySplit[T]](sc) { slices.indices.map(i => new ParallelArraySplit(id, i, slices(i))).toArray } - override def splits = splits_ + override def splits = splits_.asInstanceOf[Array[Split]] - override def iterator(s: ParallelArraySplit[T]) = s.iterator + override def iterator(s: Split) = s.asInstanceOf[ParallelArraySplit[T]].iterator - override def preferredLocations(s: ParallelArraySplit[T]): Seq[String] = Nil + override def preferredLocations(s: Split): Seq[String] = Nil } private object ParallelArray { diff --git a/src/scala/spark/RDD.scala b/src/scala/spark/RDD.scala index f9a16ed782..803c063865 100644 --- a/src/scala/spark/RDD.scala +++ b/src/scala/spark/RDD.scala @@ -3,16 +3,18 @@ package spark import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.ConcurrentHashMap import java.util.HashSet +import java.util.Random import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.Map +import scala.collection.mutable.HashMap -import nexus._ +import mesos._ import com.google.common.collect.MapMaker @serializable -abstract class RDD[T: ClassManifest, Split]( +abstract class RDD[T: ClassManifest]( @transient sc: SparkContext) { def splits: Array[Split] def iterator(split: Split): Iterator[T] @@ -24,8 +26,15 @@ abstract class RDD[T: ClassManifest, Split]( def map[U: ClassManifest](f: T => U) = new MappedRDD(this, sc.clean(f)) def filter(f: T => Boolean) = new FilteredRDD(this, sc.clean(f)) + def aggregateSplit() = new SplitRDD(this) def cache() = new CachedRDD(this) + def sample(withReplacement: Boolean, frac: Double, seed: Int) = + new SampledRDD(this, withReplacement, frac, seed) + + def flatMap[U: ClassManifest](f: T => Traversable[U]) = + new FlatMappedRDD(this, sc.clean(f)) + def foreach(f: T => Unit) { val cleanF = sc.clean(f) val tasks = splits.map(s => new ForeachTask(this, s, cleanF)).toArray @@ -73,43 +82,44 @@ abstract class RDD[T: ClassManifest, Split]( try { map(x => 1L).reduce(_+_) } catch { case e: UnsupportedOperationException => 0L } - def union[OtherSplit](other: RDD[T, OtherSplit]) = - new UnionRDD(sc, this, other) + def union(other: RDD[T]) = new UnionRDD(sc, this, other) + def cartesian[U: ClassManifest](other: RDD[U]) = new CartesianRDD(sc, this, other) + + def ++(other: RDD[T]) = this.union(other) - def ++[OtherSplit](other: RDD[T, OtherSplit]) = this.union(other) } @serializable -abstract class RDDTask[U: ClassManifest, T: ClassManifest, Split]( - val rdd: RDD[T, Split], val split: Split) +abstract class RDDTask[U: ClassManifest, T: ClassManifest]( + val rdd: RDD[T], val split: Split) extends Task[U] { override def preferredLocations() = rdd.preferredLocations(split) override def markStarted(slot: SlaveOffer) { rdd.taskStarted(split, slot) } } -class ForeachTask[T: ClassManifest, Split]( - rdd: RDD[T, Split], split: Split, func: T => Unit) -extends RDDTask[Unit, T, Split](rdd, split) { +class ForeachTask[T: ClassManifest]( + rdd: RDD[T], split: Split, func: T => Unit) +extends RDDTask[Unit, T](rdd, split) with Logging { override def run() { - println("Processing " + split) + logInfo("Processing " + split) rdd.iterator(split).foreach(func) } } -class CollectTask[T, Split]( - rdd: RDD[T, Split], split: Split)(implicit m: ClassManifest[T]) -extends RDDTask[Array[T], T, Split](rdd, split) { +class CollectTask[T]( + rdd: RDD[T], split: Split)(implicit m: ClassManifest[T]) +extends RDDTask[Array[T], T](rdd, split) with Logging { override def run(): Array[T] = { - println("Processing " + split) + logInfo("Processing " + split) rdd.iterator(split).toArray(m) } } -class ReduceTask[T: ClassManifest, Split]( - rdd: RDD[T, Split], split: Split, f: (T, T) => T) -extends RDDTask[Option[T], T, Split](rdd, split) { +class ReduceTask[T: ClassManifest]( + rdd: RDD[T], split: Split, f: (T, T) => T) +extends RDDTask[Option[T], T](rdd, split) with Logging { override def run(): Option[T] = { - println("Processing " + split) + logInfo("Processing " + split) val iter = rdd.iterator(split) if (iter.hasNext) Some(iter.reduceLeft(f)) @@ -118,27 +128,81 @@ extends RDDTask[Option[T], T, Split](rdd, split) { } } -class MappedRDD[U: ClassManifest, T: ClassManifest, Split]( - prev: RDD[T, Split], f: T => U) -extends RDD[U, Split](prev.sparkContext) { +class MappedRDD[U: ClassManifest, T: ClassManifest]( + prev: RDD[T], f: T => U) +extends RDD[U](prev.sparkContext) { override def splits = prev.splits override def preferredLocations(split: Split) = prev.preferredLocations(split) override def iterator(split: Split) = prev.iterator(split).map(f) override def taskStarted(split: Split, slot: SlaveOffer) = prev.taskStarted(split, slot) } -class FilteredRDD[T: ClassManifest, Split]( - prev: RDD[T, Split], f: T => Boolean) -extends RDD[T, Split](prev.sparkContext) { +class FilteredRDD[T: ClassManifest]( + prev: RDD[T], f: T => Boolean) +extends RDD[T](prev.sparkContext) { override def splits = prev.splits override def preferredLocations(split: Split) = prev.preferredLocations(split) override def iterator(split: Split) = prev.iterator(split).filter(f) override def taskStarted(split: Split, slot: SlaveOffer) = prev.taskStarted(split, slot) } -class CachedRDD[T, Split]( - prev: RDD[T, Split])(implicit m: ClassManifest[T]) -extends RDD[T, Split](prev.sparkContext) { +class FlatMappedRDD[U: ClassManifest, T: ClassManifest]( + prev: RDD[T], f: T => Traversable[U]) +extends RDD[U](prev.sparkContext) { + override def splits = prev.splits + override def preferredLocations(split: Split) = prev.preferredLocations(split) + override def iterator(split: Split) = + prev.iterator(split).toStream.flatMap(f).iterator + override def taskStarted(split: Split, slot: SlaveOffer) = prev.taskStarted(split, slot) +} + +class SplitRDD[T: ClassManifest](prev: RDD[T]) +extends RDD[Array[T]](prev.sparkContext) { + override def splits = prev.splits + override def preferredLocations(split: Split) = prev.preferredLocations(split) + override def iterator(split: Split) = Iterator.fromArray(Array(prev.iterator(split).toArray)) + override def taskStarted(split: Split, slot: SlaveOffer) = prev.taskStarted(split, slot) +} + + +@serializable class SeededSplit(val prev: Split, val seed: Int) extends Split { + override def getId() = + "SeededSplit(" + prev.getId() + ", seed " + seed + ")" +} + +class SampledRDD[T: ClassManifest]( + prev: RDD[T], withReplacement: Boolean, frac: Double, seed: Int) +extends RDD[T](prev.sparkContext) { + + @transient val splits_ = { val rg = new Random(seed); prev.splits.map(x => new SeededSplit(x, rg.nextInt)) } + + override def splits = splits_.asInstanceOf[Array[Split]] + + override def preferredLocations(split: Split) = prev.preferredLocations(split.asInstanceOf[SeededSplit].prev) + + override def iterator(splitIn: Split) = { + val split = splitIn.asInstanceOf[SeededSplit] + val rg = new Random(split.seed); + // Sampling with replacement (TODO: use reservoir sampling to make this more efficient?) + if (withReplacement) { + val oldData = prev.iterator(split.prev).toArray + val sampleSize = (oldData.size * frac).ceil.toInt + val sampledData = for (i <- 1 to sampleSize) yield oldData(rg.nextInt(oldData.size)) // all of oldData's indices are candidates, even if sampleSize < oldData.size + sampledData.iterator + } + // Sampling without replacement + else { + prev.iterator(split.prev).filter(x => (rg.nextDouble <= frac)) + } + } + + override def taskStarted(split: Split, slot: SlaveOffer) = prev.taskStarted(split.asInstanceOf[SeededSplit].prev, slot) +} + + +class CachedRDD[T]( + prev: RDD[T])(implicit m: ClassManifest[T]) +extends RDD[T](prev.sparkContext) with Logging { val id = CachedRDD.newId() @transient val cacheLocs = Map[Split, List[String]]() @@ -152,7 +216,8 @@ extends RDD[T, Split](prev.sparkContext) { } override def iterator(split: Split): Iterator[T] = { - val key = id + "::" + split.toString + val key = id + "::" + split.getId() + logInfo("CachedRDD split key is " + key) val cache = CachedRDD.cache val loading = CachedRDD.loading val cachedVal = cache.get(key) @@ -172,7 +237,7 @@ extends RDD[T, Split](prev.sparkContext) { } } // If we got here, we have to load the split - println("Loading and caching " + split) + logInfo("Loading and caching " + split) val array = prev.iterator(split).toArray(m) cache.put(key, array) loading.synchronized { @@ -203,23 +268,26 @@ private object CachedRDD { } @serializable -abstract class UnionSplit[T: ClassManifest] { +abstract class UnionSplit[T: ClassManifest] extends Split { def iterator(): Iterator[T] def preferredLocations(): Seq[String] + def getId(): String } @serializable -class UnionSplitImpl[T: ClassManifest, Split]( - rdd: RDD[T, Split], split: Split) +class UnionSplitImpl[T: ClassManifest]( + rdd: RDD[T], split: Split) extends UnionSplit[T] { override def iterator() = rdd.iterator(split) override def preferredLocations() = rdd.preferredLocations(split) + override def getId() = + "UnionSplitImpl(" + split.getId() + ")" } @serializable -class UnionRDD[T: ClassManifest, Split1, Split2]( - sc: SparkContext, rdd1: RDD[T, Split1], rdd2: RDD[T, Split2]) -extends RDD[T, UnionSplit[T]](sc) { +class UnionRDD[T: ClassManifest]( + sc: SparkContext, rdd1: RDD[T], rdd2: RDD[T]) +extends RDD[T](sc) { @transient val splits_ : Array[UnionSplit[T]] = { val a1 = rdd1.splits.map(s => new UnionSplitImpl(rdd1, s)) @@ -227,10 +295,58 @@ extends RDD[T, UnionSplit[T]](sc) { (a1 ++ a2).toArray } - override def splits = splits_ + override def splits = splits_.asInstanceOf[Array[Split]] + + override def iterator(s: Split): Iterator[T] = s.asInstanceOf[UnionSplit[T]].iterator() + + override def preferredLocations(s: Split): Seq[String] = + s.asInstanceOf[UnionSplit[T]].preferredLocations() +} + +@serializable class CartesianSplit(val s1: Split, val s2: Split) extends Split { + override def getId() = + "CartesianSplit(" + s1.getId() + ", " + s2.getId() + ")" +} + +@serializable +class CartesianRDD[T: ClassManifest, U:ClassManifest]( + sc: SparkContext, rdd1: RDD[T], rdd2: RDD[U]) +extends RDD[Pair[T, U]](sc) { + @transient val splits_ = { + // create the cross product split + rdd2.splits.map(y => rdd1.splits.map(x => new CartesianSplit(x, y))).flatten + } - override def iterator(s: UnionSplit[T]): Iterator[T] = s.iterator() + override def splits = splits_.asInstanceOf[Array[Split]] - override def preferredLocations(s: UnionSplit[T]): Seq[String] = - s.preferredLocations() + override def preferredLocations(split: Split) = { + val currSplit = split.asInstanceOf[CartesianSplit] + rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2) + } + + override def iterator(split: Split) = { + val currSplit = split.asInstanceOf[CartesianSplit] + for (x <- rdd1.iterator(currSplit.s1); y <- rdd2.iterator(currSplit.s2)) yield (x, y) + } + + override def taskStarted(split: Split, slot: SlaveOffer) = { + val currSplit = split.asInstanceOf[CartesianSplit] + rdd1.taskStarted(currSplit.s1, slot) + rdd2.taskStarted(currSplit.s2, slot) + } +} + +@serializable class PairRDDExtras[K, V](rdd: RDD[(K, V)]) { + def reduceByKey(func: (V, V) => V): Map[K, V] = { + def mergeMaps(m1: HashMap[K, V], m2: HashMap[K, V]): HashMap[K, V] = { + for ((k, v) <- m2) { + m1.get(k) match { + case None => m1(k) = v + case Some(w) => m1(k) = func(w, v) + } + } + return m1 + } + rdd.map(pair => HashMap(pair)).reduce(mergeMaps) + } } diff --git a/src/scala/spark/Scheduler.scala b/src/scala/spark/Scheduler.scala index 27cf48e9d2..b9f3128c82 100644 --- a/src/scala/spark/Scheduler.scala +++ b/src/scala/spark/Scheduler.scala @@ -6,4 +6,5 @@ private trait Scheduler { def waitForRegister() def runTasks[T](tasks: Array[Task[T]])(implicit m: ClassManifest[T]): Array[T] def stop() + def numCores(): Int } diff --git a/src/scala/spark/SparkContext.scala b/src/scala/spark/SparkContext.scala index 50d8e4908a..216ea4c0a9 100644 --- a/src/scala/spark/SparkContext.scala +++ b/src/scala/spark/SparkContext.scala @@ -4,15 +4,16 @@ import java.io._ import java.util.UUID import scala.collection.mutable.ArrayBuffer +import scala.actors.Actor._ -class SparkContext(master: String, frameworkName: String) { - Broadcast.initialize (true) +class SparkContext(master: String, frameworkName: String) extends Logging { + Broadcast.initialize(true) def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int) = new ParallelArray[T](this, seq, numSlices) def parallelize[T: ClassManifest](seq: Seq[T]): ParallelArray[T] = - parallelize(seq, 2) + parallelize(seq, scheduler.numCores) def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = new Accumulator(initialValue, param) @@ -25,11 +26,11 @@ class SparkContext(master: String, frameworkName: String) { val LOCAL_REGEX = """local\[([0-9]+)\]""".r - private var scheduler: Scheduler = master match { + private[spark] var scheduler: Scheduler = master match { case "local" => new LocalScheduler(1) case LOCAL_REGEX(threads) => new LocalScheduler(threads.toInt) - case _ => { System.loadLibrary("nexus"); - new NexusScheduler(master, frameworkName, createExecArg()) } + case _ => { System.loadLibrary("mesos"); + new MesosScheduler(master, frameworkName, createExecArg()) } } private val local = scheduler.isInstanceOf[LocalScheduler] @@ -55,10 +56,10 @@ class SparkContext(master: String, frameworkName: String) { private[spark] def runTaskObjects[T: ClassManifest](tasks: Seq[Task[T]]) : Array[T] = { - println("Running " + tasks.length + " tasks in parallel") + logInfo("Running " + tasks.length + " tasks in parallel") val start = System.nanoTime val result = scheduler.runTasks(tasks.toArray) - println("Tasks finished in " + (System.nanoTime - start) / 1e9 + " s") + logInfo("Tasks finished in " + (System.nanoTime - start) / 1e9 + " s") return result } @@ -81,12 +82,17 @@ class SparkContext(master: String, frameworkName: String) { object SparkContext { implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { - def add(t1: Double, t2: Double): Double = t1 + t2 + def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 } + implicit object IntAccumulatorParam extends AccumulatorParam[Int] { - def add(t1: Int, t2: Int): Int = t1 + t2 + def addInPlace(t1: Int, t2: Int): Int = t1 + t2 def zero(initialValue: Int) = 0 } + // TODO: Add AccumulatorParams for other types, e.g. lists and strings + + implicit def rddToPairRDDExtras[K, V](rdd: RDD[(K, V)]) = + new PairRDDExtras(rdd) } diff --git a/src/scala/spark/Split.scala b/src/scala/spark/Split.scala new file mode 100644 index 0000000000..0f7a21354d --- /dev/null +++ b/src/scala/spark/Split.scala @@ -0,0 +1,13 @@ +package spark + +/** + * A partition of an RDD. + */ +trait Split { + /** + * Get a unique ID for this split which can be used, for example, to + * set up caches based on it. The ID should stay the same if we serialize + * and then deserialize the split. + */ + def getId(): String +} diff --git a/src/scala/spark/Task.scala b/src/scala/spark/Task.scala index efb864472d..6e94009f6e 100644 --- a/src/scala/spark/Task.scala +++ b/src/scala/spark/Task.scala @@ -1,6 +1,6 @@ package spark -import nexus._ +import mesos._ @serializable trait Task[T] { diff --git a/src/scala/spark/Utils.scala b/src/scala/spark/Utils.scala index 52bcb89f00..27d73aefbd 100644 --- a/src/scala/spark/Utils.scala +++ b/src/scala/spark/Utils.scala @@ -2,7 +2,9 @@ package spark import java.io._ -private object Utils { +import scala.collection.mutable.ArrayBuffer + +object Utils { def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream val oos = new ObjectOutputStream(bos) @@ -25,4 +27,27 @@ private object Utils { } return ois.readObject.asInstanceOf[T] } + + def isAlpha(c: Char) = { + (c >= 'A' && c <= 'Z') || (c >= 'a' && c <= 'z') + } + + def splitWords(s: String): Seq[String] = { + val buf = new ArrayBuffer[String] + var i = 0 + while (i < s.length) { + var j = i + while (j < s.length && isAlpha(s.charAt(j))) { + j += 1 + } + if (j > i) { + buf += s.substring(i, j); + } + i = j + while (i < s.length && !isAlpha(s.charAt(i))) { + i += 1 + } + } + return buf + } } diff --git a/src/scala/spark/repl/ClassServer.scala b/src/scala/spark/repl/ClassServer.scala new file mode 100644 index 0000000000..6a40d92765 --- /dev/null +++ b/src/scala/spark/repl/ClassServer.scala @@ -0,0 +1,77 @@ +package spark.repl + +import java.io.File +import java.net.InetAddress + +import org.eclipse.jetty.server.Server +import org.eclipse.jetty.server.handler.DefaultHandler +import org.eclipse.jetty.server.handler.HandlerList +import org.eclipse.jetty.server.handler.ResourceHandler + +import spark.Logging + + +/** + * Exception type thrown by ClassServer when it is in the wrong state + * for an operation. + */ +class ServerStateException(message: String) extends Exception(message) + + +/** + * An HTTP server used by the interpreter to allow worker nodes to access + * class files created as the user types in lines of code. This is just a + * wrapper around a Jetty embedded HTTP server. + */ +class ClassServer(classDir: File) extends Logging { + private var server: Server = null + private var port: Int = -1 + + def start() { + if (server != null) { + throw new ServerStateException("Server is already started") + } else { + server = new Server(0) + val resHandler = new ResourceHandler + resHandler.setResourceBase(classDir.getAbsolutePath) + val handlerList = new HandlerList + handlerList.setHandlers(Array(resHandler, new DefaultHandler)) + server.setHandler(handlerList) + server.start() + port = server.getConnectors()(0).getLocalPort() + logDebug("ClassServer started at " + uri) + } + } + + def stop() { + if (server == null) { + throw new ServerStateException("Server is already stopped") + } else { + server.stop() + port = -1 + server = null + } + } + + /** + * Get the URI of this HTTP server (http://host:port) + */ + def uri: String = { + if (server == null) { + throw new ServerStateException("Server is not started") + } else { + return "http://" + getLocalIpAddress + ":" + port + } + } + + /** + * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4) + */ + private def getLocalIpAddress: String = { + // Get local IP as an array of four bytes + val bytes = InetAddress.getLocalHost().getAddress() + // Convert the bytes to ints (keeping in mind that they may be negative) + // and join them into a string + return bytes.map(b => (b.toInt + 256) % 256).mkString(".") + } +} diff --git a/src/scala/spark/repl/ExecutorClassLoader.scala b/src/scala/spark/repl/ExecutorClassLoader.scala index 7d91b20e79..13d81ec1cf 100644 --- a/src/scala/spark/repl/ExecutorClassLoader.scala +++ b/src/scala/spark/repl/ExecutorClassLoader.scala @@ -1,7 +1,7 @@ package spark.repl import java.io.{ByteArrayOutputStream, InputStream} -import java.net.{URI, URL, URLClassLoader} +import java.net.{URI, URL, URLClassLoader, URLEncoder} import java.util.concurrent.{Executors, ExecutorService} import org.apache.hadoop.conf.Configuration @@ -12,18 +12,34 @@ import org.objectweb.asm.commons.EmptyVisitor import org.objectweb.asm.Opcodes._ -// A ClassLoader that reads classes from a Hadoop FileSystem URL, used to load -// classes defined by the interpreter when the REPL is in use -class ExecutorClassLoader(classDir: String, parent: ClassLoader) +/** + * A ClassLoader that reads classes from a Hadoop FileSystem or HTTP URI, + * used to load classes defined by the interpreter when the REPL is used + */ +class ExecutorClassLoader(classUri: String, parent: ClassLoader) extends ClassLoader(parent) { - val fileSystem = FileSystem.get(new URI(classDir), new Configuration()) - val directory = new URI(classDir).getPath + val uri = new URI(classUri) + val directory = uri.getPath + + // Hadoop FileSystem object for our URI, if it isn't using HTTP + var fileSystem: FileSystem = { + if (uri.getScheme() == "http") + null + else + FileSystem.get(uri, new Configuration()) + } override def findClass(name: String): Class[_] = { try { - //println("repl.ExecutorClassLoader resolving " + name) - val path = new Path(directory, name.replace('.', '/') + ".class") - val bytes = readAndTransformClass(name, fileSystem.open(path)) + val pathInDirectory = name.replace('.', '/') + ".class" + val inputStream = { + if (fileSystem != null) + fileSystem.open(new Path(directory, pathInDirectory)) + else + new URL(classUri + "/" + urlEncode(pathInDirectory)).openStream() + } + val bytes = readAndTransformClass(name, inputStream) + inputStream.close() return defineClass(name, bytes, 0, bytes.length) } catch { case e: Exception => throw new ClassNotFoundException(name, e) @@ -57,6 +73,13 @@ extends ClassLoader(parent) { return bos.toByteArray } } + + /** + * URL-encode a string, preserving only slashes + */ + def urlEncode(str: String): String = { + str.split('/').map(part => URLEncoder.encode(part, "UTF-8")).mkString("/") + } } class ConstructorCleaner(className: String, cv: ClassVisitor) @@ -68,7 +91,6 @@ extends ClassAdapter(cv) { // This is the constructor, time to clean it; just output some new // instructions to mv that create the object and set the static MODULE$ // field in the class to point to it, but do nothing otherwise. - //println("Cleaning constructor of " + className) mv.visitCode() mv.visitVarInsn(ALOAD, 0) // load this mv.visitMethodInsn(INVOKESPECIAL, "java/lang/Object", "", "()V") diff --git a/src/scala/spark/repl/SparkInterpreter.scala b/src/scala/spark/repl/SparkInterpreter.scala index 6237c83625..ae2e7e8a68 100644 --- a/src/scala/spark/repl/SparkInterpreter.scala +++ b/src/scala/spark/repl/SparkInterpreter.scala @@ -90,31 +90,44 @@ class SparkInterpreter(val settings: Settings, out: PrintWriter) { val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") - /** directory to save .class files to */ - //val virtualDirectory = new VirtualDirectory("(memory)", None) - val virtualDirectory = { + /** Local directory to save .class files too */ + val outputDir = { val rootDir = new File(System.getProperty("spark.repl.classdir", System.getProperty("java.io.tmpdir"))) var attempts = 0 val maxAttempts = 10 - var outputDir: File = null - while (outputDir == null) { + var dir: File = null + while (dir == null) { attempts += 1 if (attempts > maxAttempts) { throw new IOException("Failed to create a temp directory " + "after " + maxAttempts + " attempts!") } try { - outputDir = new File(rootDir, "spark-" + UUID.randomUUID.toString) - if (outputDir.exists() || !outputDir.mkdirs()) - outputDir = null + dir = new File(rootDir, "spark-" + UUID.randomUUID.toString) + if (dir.exists() || !dir.mkdirs()) + dir = null } catch { case e: IOException => ; } } - System.setProperty("spark.repl.current.classdir", - "file://" + outputDir.getAbsolutePath + "/") - if (SPARK_DEBUG_REPL) - println("Output directory: " + outputDir) - new PlainFile(outputDir) + if (SPARK_DEBUG_REPL) { + println("Output directory: " + dir) + } + dir + } + + /** Scala compiler virtual directory for outputDir */ + //val virtualDirectory = new VirtualDirectory("(memory)", None) + val virtualDirectory = new PlainFile(outputDir) + + /** Jetty server that will serve our classes to worker nodes */ + val classServer = new ClassServer(outputDir) + + // Start the classServer and store its URI in a spark system property + // (which will be passed to executors so that they can connect to it) + classServer.start() + System.setProperty("spark.repl.class.uri", classServer.uri) + if (SPARK_DEBUG_REPL) { + println("ClassServer started, URI = " + classServer.uri) } /** reporter */ @@ -714,6 +727,7 @@ class SparkInterpreter(val settings: Settings, out: PrintWriter) { */ def close() { reporter.flush + classServer.stop() } /** A traverser that finds all mentioned identifiers, i.e. things @@ -956,7 +970,9 @@ class SparkInterpreter(val settings: Settings, out: PrintWriter) { """.stripMargin code println preamble - handlers foreach { _.resultExtractionCode(this, code) } + if (printResults) { + handlers foreach { _.resultExtractionCode(this, code) } + } code println postamble } diff --git a/src/scala/spark/repl/SparkInterpreterLoop.scala b/src/scala/spark/repl/SparkInterpreterLoop.scala index 26361fdc25..5bad0a37da 100644 --- a/src/scala/spark/repl/SparkInterpreterLoop.scala +++ b/src/scala/spark/repl/SparkInterpreterLoop.scala @@ -267,7 +267,7 @@ extends InterpreterControl { if (prop != null) prop else "local" } } - new SparkContext(master, "Spark REPL") + new SparkContext(master, "Spark shell") } /** The main read-eval-print loop for the interpreter. It calls diff --git a/src/scala/ubiquifs/Header.scala b/src/scala/ubiquifs/Header.scala deleted file mode 100644 index bdca83a2d5..0000000000 --- a/src/scala/ubiquifs/Header.scala +++ /dev/null @@ -1,21 +0,0 @@ -package ubiquifs - -import java.io.{DataInputStream, DataOutputStream} - -object RequestType { - val READ = 0 - val WRITE = 1 -} - -class Header(val requestType: Int, val path: String) { - def write(out: DataOutputStream) { - out.write(requestType) - out.writeUTF(path) - } -} - -object Header { - def read(in: DataInputStream): Header = { - new Header(in.read(), in.readUTF()) - } -} diff --git a/src/scala/ubiquifs/Master.scala b/src/scala/ubiquifs/Master.scala deleted file mode 100644 index 6854acd6a5..0000000000 --- a/src/scala/ubiquifs/Master.scala +++ /dev/null @@ -1,49 +0,0 @@ -package ubiquifs - -import scala.actors.Actor -import scala.actors.Actor._ -import scala.actors.remote.RemoteActor -import scala.actors.remote.RemoteActor._ -import scala.actors.remote.Node -import scala.collection.mutable.{ArrayBuffer, Map, Set} - -class Master(port: Int) extends Actor { - case class SlaveInfo(host: String, port: Int) - - val files = Set[String]() - val slaves = new ArrayBuffer[SlaveInfo]() - - def act() { - alive(port) - register('UbiquiFS, self) - println("Created UbiquiFS Master on port " + port) - - loop { - react { - case RegisterSlave(host, port) => - slaves += SlaveInfo(host, port) - sender ! RegisterSucceeded() - - case Create(path) => - if (files.contains(path)) { - sender ! CreateFailed("File already exists") - } else if (slaves.isEmpty) { - sender ! CreateFailed("No slaves registered") - } else { - files += path - sender ! CreateSucceeded(slaves(0).host, slaves(0).port) - } - - case m: Any => - println("Unknown message: " + m) - } - } - } -} - -object MasterMain { - def main(args: Array[String]) { - val port = args(0).toInt - new Master(port).start() - } -} diff --git a/src/scala/ubiquifs/Message.scala b/src/scala/ubiquifs/Message.scala deleted file mode 100644 index 153542f8de..0000000000 --- a/src/scala/ubiquifs/Message.scala +++ /dev/null @@ -1,14 +0,0 @@ -package ubiquifs - -sealed case class Message() - -case class RegisterSlave(host: String, port: Int) extends Message -case class RegisterSucceeded() extends Message - -case class Create(path: String) extends Message -case class CreateSucceeded(host: String, port: Int) extends Message -case class CreateFailed(message: String) extends Message - -case class Read(path: String) extends Message -case class ReadSucceeded(host: String, port: Int) extends Message -case class ReadFailed(message: String) extends Message diff --git a/src/scala/ubiquifs/Slave.scala b/src/scala/ubiquifs/Slave.scala deleted file mode 100644 index 328b73c828..0000000000 --- a/src/scala/ubiquifs/Slave.scala +++ /dev/null @@ -1,141 +0,0 @@ -package ubiquifs - -import java.io.{DataInputStream, DataOutputStream, IOException} -import java.net.{InetAddress, Socket, ServerSocket} -import java.util.concurrent.locks.ReentrantLock - -import scala.actors.Actor -import scala.actors.Actor._ -import scala.actors.remote.RemoteActor -import scala.actors.remote.RemoteActor._ -import scala.actors.remote.Node -import scala.collection.mutable.{ArrayBuffer, Map, Set} - -class Slave(myPort: Int, master: String) extends Thread("UbiquiFS slave") { - val CHUNK_SIZE = 1024 * 1024 - - val buffers = Map[String, Buffer]() - - override def run() { - // Create server socket - val socket = new ServerSocket(myPort) - - // Register with master - val (masterHost, masterPort) = Utils.parseHostPort(master) - val masterActor = select(Node(masterHost, masterPort), 'UbiquiFS) - val myHost = InetAddress.getLocalHost.getHostName - val reply = masterActor !? RegisterSlave(myHost, myPort) - println("Registered with master, reply = " + reply) - - while (true) { - val conn = socket.accept() - new ConnectionHandler(conn).start() - } - } - - class ConnectionHandler(conn: Socket) extends Thread("ConnectionHandler") { - try { - val in = new DataInputStream(conn.getInputStream) - val out = new DataOutputStream(conn.getOutputStream) - val header = Header.read(in) - header.requestType match { - case RequestType.READ => - performRead(header.path, out) - case RequestType.WRITE => - performWrite(header.path, in) - case other => - throw new IOException("Invalid header type " + other) - } - println("hi") - } catch { - case e: Exception => e.printStackTrace() - } finally { - conn.close() - } - } - - def performWrite(path: String, in: DataInputStream) { - var buffer = new Buffer() - synchronized { - if (buffers.contains(path)) - throw new IllegalArgumentException("Path " + path + " already exists") - buffers(path) = buffer - } - var chunk = new Array[Byte](CHUNK_SIZE) - var pos = 0 - while (true) { - var numRead = in.read(chunk, pos, chunk.size - pos) - if (numRead == -1) { - buffer.addChunk(chunk.subArray(0, pos), true) - return - } else { - pos += numRead - if (pos == chunk.size) { - buffer.addChunk(chunk, false) - chunk = new Array[Byte](CHUNK_SIZE) - pos = 0 - } - } - } - // TODO: launch a thread to write the data to disk, and when this finishes, - // remove the hard reference to buffer - } - - def performRead(path: String, out: DataOutputStream) { - var buffer: Buffer = null - synchronized { - if (!buffers.contains(path)) - throw new IllegalArgumentException("Path " + path + " doesn't exist") - buffer = buffers(path) - } - for (chunk <- buffer.iterator) { - out.write(chunk, 0, chunk.size) - } - } - - class Buffer { - val chunks = new ArrayBuffer[Array[Byte]] - var finished = false - val mutex = new ReentrantLock - val chunksAvailable = mutex.newCondition() - - def addChunk(chunk: Array[Byte], finish: Boolean) { - mutex.lock() - chunks += chunk - finished = finish - chunksAvailable.signalAll() - mutex.unlock() - } - - def iterator = new Iterator[Array[Byte]] { - var index = 0 - - def hasNext: Boolean = { - mutex.lock() - while (index >= chunks.size && !finished) - chunksAvailable.await() - val ret = (index < chunks.size) - mutex.unlock() - return ret - } - - def next: Array[Byte] = { - mutex.lock() - if (!hasNext) - throw new NoSuchElementException("End of file") - val ret = chunks(index) // hasNext ensures we advance past index - index += 1 - mutex.unlock() - return ret - } - } - } -} - -object SlaveMain { - def main(args: Array[String]) { - val port = args(0).toInt - val master = args(1) - new Slave(port, master).start() - } -} diff --git a/src/scala/ubiquifs/UbiquiFS.scala b/src/scala/ubiquifs/UbiquiFS.scala deleted file mode 100644 index 9ce0fd4f44..0000000000 --- a/src/scala/ubiquifs/UbiquiFS.scala +++ /dev/null @@ -1,11 +0,0 @@ -package ubiquifs - -import java.io.{InputStream, OutputStream} - -class UbiquiFS(master: String) { - private val (masterHost, masterPort) = Utils.parseHostPort(master) - - def create(path: String): OutputStream = null - - def open(path: String): InputStream = null -} diff --git a/src/scala/ubiquifs/Utils.scala b/src/scala/ubiquifs/Utils.scala deleted file mode 100644 index d6fd3f0181..0000000000 --- a/src/scala/ubiquifs/Utils.scala +++ /dev/null @@ -1,12 +0,0 @@ -package ubiquifs - -private[ubiquifs] object Utils { - private val HOST_PORT_RE = "([a-zA-Z0-9.-]+):([0-9]+)".r - - def parseHostPort(string: String): (String, Int) = { - string match { - case HOST_PORT_RE(host, port) => (host, port.toInt) - case _ => throw new IllegalArgumentException(string) - } - } -} diff --git a/third_party/apache-log4j-1.2.16/log4j-1.2.16.jar b/third_party/apache-log4j-1.2.16/log4j-1.2.16.jar new file mode 100644 index 0000000000..3f9d847618 Binary files /dev/null and b/third_party/apache-log4j-1.2.16/log4j-1.2.16.jar differ diff --git a/third_party/google-collect-1.0-rc5/COPYING b/third_party/google-collect-1.0-rc5/COPYING deleted file mode 100644 index d645695673..0000000000 --- a/third_party/google-collect-1.0-rc5/COPYING +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. diff --git a/third_party/google-collect-1.0-rc5/google-collect-1.0-rc5.jar b/third_party/google-collect-1.0-rc5/google-collect-1.0-rc5.jar deleted file mode 100644 index 36aa65a069..0000000000 Binary files a/third_party/google-collect-1.0-rc5/google-collect-1.0-rc5.jar and /dev/null differ diff --git a/third_party/guava-r06/COPYING b/third_party/guava-r06/COPYING new file mode 100644 index 0000000000..d645695673 --- /dev/null +++ b/third_party/guava-r06/COPYING @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/third_party/guava-r06/README b/third_party/guava-r06/README new file mode 100644 index 0000000000..a0e832dd54 --- /dev/null +++ b/third_party/guava-r06/README @@ -0,0 +1,28 @@ +Guava: Google Core Libraries for Java + +Requires JDK 5 or higher. + +Project page: + http://guava-libraries.googlecode.com + +Ask "how-to" and "why-didn't-it-work" questions at: + http://www.stackoverflow.com/questions/ask + (use the "guava" tag so we'll see it) + +Ask discussion questions at: + http://groups.google.com/group/guava-discuss + +Subscribe to project updates in your feed reader: + http://code.google.com/feeds/p/guava-libraries/updates/basic + +Warnings: + +All APIs marked @Beta at the class or method level are subject to +change. If your code is a library or framework that users outside +your control will include on their classpath, do not use @Beta +APIs (at least without repackaging them somehow). + +Serialized forms of ALL objects are subject to change. Do not +persist these and assume they can be read by a future version of +the library. + diff --git a/third_party/guava-r06/guava-r06.jar b/third_party/guava-r06/guava-r06.jar new file mode 100644 index 0000000000..8ff3a81748 Binary files /dev/null and b/third_party/guava-r06/guava-r06.jar differ diff --git a/third_party/hadoop-0.20.0/contrib/hod/CHANGES.txt b/third_party/hadoop-0.20.0/contrib/hod/CHANGES.txt deleted file mode 100644 index 95cf0710fe..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/CHANGES.txt +++ /dev/null @@ -1,248 +0,0 @@ -HOD Change Log - -Release 0.20.0 - (unreleased changes) - - INCOMPATIBLE CHANGES - - NEW FEATURES - - IMPROVEMENTS - - HADOOP-4705. Grant read permissions for files/directories - created by HOD. (Peeyush Bishnoi via yhemanth) - - HADOOP-4937. Include ringmaster RPC port in the notes - attribute. (Peeyush Bishnoi via yhemanth) - - OPTIMIZATIONS - - BUG FIXES - - HADOOP-4782. Revert umask changes in HADOOP-4705 so that - files are still securely created. (Peeyush Bishnoi via - yhemanth) - -Release 0.19.0 - 2008-11-18 - - INCOMPATIBLE CHANGES - - NEW FEATURES - - HADOOP-3695. Provide an ability to start multiple workers per node. - (Vinod Kumar Vavilapalli via yhemanth) - - IMPROVEMENTS - - OPTIMIZATIONS - - BUG FIXES - - HADOOP-3959. Pass options specified in resource_manager.options to - job submission. - (Craig Macdonald and Vinod Kumar Vavilapalli via yhemanth) - - HADOOP-3814. Remove generation of dfs.client.buffer.dir for the generated - hadoop-site.xml. (Vinod Kumar Vavilapalli via acmurthy) - -Release 0.18.2 - Unreleased - - BUG FIXES - - HADOOP-3786. Use HDFS instead of DFS in all docs and hyperlink to Torque. - (Vinod Kumar Vavilapalli via acmurthy) - -Release 0.18.1 - 2008-09-17 - - INCOMPATIBLE CHANGES - - HADOOP-4060. Modified HOD to rotate log files on the client side. - (Vinod Kumar Vavilapalli via yhemanth) - - IMPROVEMENTS - - HADOOP-4145. Add an accounting plugin (script) for HOD. - (Hemanth Yamijala via nigel) - - BUG FIXES - - HADOOP-4161. Fixed bug in HOD cleanup that had the potential to - hang clients. (Vinod Kumar Vavilapalli via nigel) - -Release 0.18.0 - 2008-08-19 - - INCOMPATIBLE CHANGES - - HADOOP-3483. Modified HOD to create a cluster directory if one does not - exist and to auto-deallocate a cluster while reallocating it, if it is - already dead. (Hemanth Yamijala via mukund) - - HADOOP-3184. Modified HOD to handle master failures on bad nodes by trying - to bring them up on another node in the ring. (Hemanth Yamijala via ddas) - - HADOOP-3610. Modified HOD to create cluster directory if one does not - exist when using the script option. (Vinod Kumar Vavilapalli via yhemanth) - - HADOOP-3808. Modified HOD to include RPC port of the JobTracker - into the notes attribute of the resource manager. (yhemanth) - - NEW FEATURES - - IMPROVEMENTS - - HADOOP-3376: Provide a mechanism to detect and handle violations to - resource manager limits. (Vinod Kumar Vavilapalli via ddas) - - HADOOP-3151. Improves error messages when reporting failures due to - incorrect parameters passed to HOD. (Vinod Kumar Vavilapalli via ddas) - - HADOOP-3464. Implemented a mechanism to transfer HOD errors that occur on - compute nodes to the submit node running the HOD client, so users have good - feedback on why an allocation failed. (Vinod Kumar Vavilapalli via mukund) - - HADOOP-3505. Updated HOD documentation with changes made for Hadoop - 0.18. (Vinod Kumar Vavilapalli via yhemanth) - - BUG FIXES - - HADOOP-2961. Avoids unnecessary checks for some configuration parameters - related to service configuration. (Vinod Kumar Vavilapalli via ddas) - - HADOOP-3523. Fixes auto-deallocation of cluster if job id is not found in - Torque's job list (Hemanth Yamijala via ddas) - - HADOOP-3531. Fixes a bug related to handling JobTracker failures because of - timing issues on slow nodes. (Hemanth Yamijala via ddas) - - HADOOP-3564. HOD generates values for the parameter dfs.datanode.ipc.address - in the hadoop-site.xml created on datanodes. - (Vinod Kumar Vavilapalli via ddas) - - HADOOP-3076. Fixes a bug related to a spurious message about the - script.exitcode file when a cluster directory is specified as a relative - path. (Vinod Kumar Vavilapalli via yhemanth) - - HADOOP-3668. Makes editorial changes to HOD documentation. - (Vinod Kumar Vavilapalli via yhemanth) - - HADOOP-3703. Fixes logcondense.py to use the new format of hadoop dfs -lsr - command line output format. (Vinod Kumar Vavilapalli via yhemanth) - -Release 0.17.3 - Unreleased - - BUG FIXES - - HADOOP-3217. Decrease the rate at which the hod queries the resource - manager for job status. (Hemanth Yamijala via acmurthy) - -Release 0.17.0 - 2008-05-18 - - INCOMPATIBLE CHANGES - - HADOOP-3137. Modified build script to pick up version automatically - from Hadoop build. (yhemanth) - - IMPROVEMENTS - - HADOOP-2775. Adds unit test framework for HOD. - (Vinod Kumar Vavilapalli via ddas). - - HADOOP-2848. [HOD]hod -o list and deallocate works even after deleting - the cluster directory. (Hemanth Yamijala via ddas) - - HADOOP-2899. [HOD] Cleans up hdfs:///mapredsystem directory after - deallocation. (Hemanth Yamijala via ddas) - - HADOOP-2796. Enables distinguishing exit codes from user code vis-a-vis - HOD's exit code. (Hemanth Yamijala via ddas) - - HADOOP-2947. HOD redirects stdout and stderr of daemons to assist - getting stack traces. (Vinod Kumar Vavilapalli via yhemanth) - - BUG FIXES - - HADOOP-2924. Fixes an address problem to do with TaskTracker binding - to an address. (Vinod Kumar Vavilapalli via ddas) - - HADOOP-2970. Fixes a problem to do with Wrong class definition for - hodlib/Hod/hod.py for Python < 2.5.1. - (Vinod Kumar Vavilapalli via ddas) - - HADOOP-2783. Fixes a problem to do with import in - hod/hodlib/Common/xmlrpc.py. (Vinod Kumar Vavilapalli via ddas) - - HADOOP-2936. Fixes HOD in a way that it generates hdfs://host:port on the - client side configs. (Vinod Kumar Vavilapalli via ddas) - - HADOOP-2983. [HOD] Fixes the problem - local_fqdn() returns None when - gethostbyname_ex doesnt return any FQDNs. (Craig Macdonald via ddas) - - HADOOP-2982. Fixes a problem in the way HOD looks for free nodes. - (Hemanth Yamijala via ddas) - - HADOOP-2855. Fixes the way HOD handles relative paths for cluster - directory, script file and other options. - (Vinod Kumar Vavilapalli via yhemanth) - - HADOOP-3153. Fixes the way HOD handles allocation if the user has no - permissions to update the clusters state file. - (Vinod Kumar Vavilapalli via yhemanth) - -Release 0.16.4 - 2008-05-05 - - BUG FIXES - - HADOOP-3304. [HOD] Fixes the way the logcondense.py utility searches - for log files that need to be deleted. (yhemanth via mukund) - -Release 0.16.2 - 2008-04-02 - - BUG FIXES - - HADOOP-3103. [HOD] Hadoop.tmp.dir should not be set to cluster - directory. (Vinod Kumar Vavilapalli via ddas). - -Release 0.16.1 - 2008-03-13 - - INCOMPATIBLE CHANGES - - HADOOP-2861. Improve the user interface for the HOD commands. - Command line structure has changed. (Hemanth Yamijala via nigel) - - IMPROVEMENTS - - HADOOP-2730. HOD documentation update. - (Vinod Kumar Vavilapalli via ddas) - - HADOOP-2911. Make the information printed by the HOD allocate and - info commands less verbose and clearer. (Vinod Kumar via nigel) - - BUG FIXES - - HADOOP-2766. Enables setting of HADOOP_OPTS env variable for the hadoop - daemons through HOD. (Vinod Kumar Vavilapalli via ddas) - - HADOOP-2809. Fix HOD syslog config syslog-address so that it works. - (Hemanth Yamijala via nigel) - - HADOOP-2847. Ensure idle cluster cleanup works even if the JobTracker - becomes unresponsive to RPC calls. (Hemanth Yamijala via nigel) - - HADOOP-2925. Fix HOD to create the mapred system directory using a - naming convention that will avoid clashes in multi-user shared - cluster scenario. (Hemanth Yamijala via nigel) - -Release 0.16.0 - 2008-02-07 - - NEW FEATURES - - HADOOP-1301. Hadoop-On-Demand (HOD): resource management - provisioning for Hadoop. (Hemanth Yamijala via nigel) - - BUG FIXES - - HADOOP-2720. Jumbo bug fix patch to HOD. Final sync of Apache SVN with - internal Yahoo SVN. (Hemanth Yamijala via nigel) - - HADOOP-2740. Fix HOD to work with the configuration variables changed in - HADOOP-2404. (Hemanth Yamijala via omalley) - diff --git a/third_party/hadoop-0.20.0/contrib/hod/README b/third_party/hadoop-0.20.0/contrib/hod/README deleted file mode 100644 index aaa7d35c3e..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/README +++ /dev/null @@ -1,104 +0,0 @@ - Hadoop On Demand - ================ - -1. Introduction: -================ - -The Hadoop On Demand (HOD) project is a system for provisioning and -managing independent Hadoop MapReduce instances on a shared cluster -of nodes. HOD uses a resource manager for allocation. At present it -supports Torque (http://www.clusterresources.com/pages/products/torque-resource-manager.php) -out of the box. - -2. Feature List: -================ - -The following are the features provided by HOD: - -2.1 Simplified interface for managing MapReduce clusters: - -The MapReduce user interacts with the cluster through a simple -command line interface, the HOD client. HOD brings up a virtual -MapReduce cluster with the required number of nodes, which the -user can use for running Hadoop jobs. When done, HOD will -automatically clean up the resources and make the nodes available -again. - -2.2 Automatic installation of Hadoop: - -With HOD, Hadoop does not need to be even installed on the cluster. -The user can provide a Hadoop tarball that HOD will automatically -distribute to all the nodes in the cluster. - -2.3 Configuring Hadoop: - -Dynamic parameters of Hadoop configuration, such as the NameNode and -JobTracker addresses and ports, and file system temporary directories -are generated and distributed by HOD automatically to all nodes in -the cluster. - -In addition, HOD allows the user to configure Hadoop parameters -at both the server (for e.g. JobTracker) and client (for e.g. JobClient) -level, including 'final' parameters, that were introduced with -Hadoop 0.15. - -2.4 Auto-cleanup of unused clusters: - -HOD has an automatic timeout so that users cannot misuse resources they -aren't using. The timeout applies only when there is no MapReduce job -running. - -2.5 Log services: - -HOD can be used to collect all MapReduce logs to a central location -for archiving and inspection after the job is completed. - -3. HOD Components -================= - -This is a brief overview of the various components of HOD and how they -interact to provision Hadoop. - -HOD Client: The HOD client is a Unix command that users use to allocate -Hadoop MapReduce clusters. The command provides other options to list -allocated clusters and deallocate them. The HOD client generates the -hadoop-site.xml in a user specified directory. The user can point to -this configuration file while running Map/Reduce jobs on the allocated -cluster. - -RingMaster: The RingMaster is a HOD process that is started on one node -per every allocated cluster. It is submitted as a 'job' to the resource -manager by the HOD client. It controls which Hadoop daemons start on -which nodes. It provides this information to other HOD processes, -such as the HOD client, so users can also determine this information. -The RingMaster is responsible for hosting and distributing the -Hadoop tarball to all nodes in the cluster. It also automatically -cleans up unused clusters. - -HodRing: The HodRing is a HOD process that runs on every allocated node -in the cluster. These processes are run by the RingMaster through the -resource manager, using a facility of parallel execution. The HodRings -are responsible for launching Hadoop commands on the nodes to bring up -the Hadoop daemons. They get the command to launch from the RingMaster. - -Hodrc / HOD configuration file: An INI style configuration file where -the users configure various options for the HOD system, including -install locations of different software, resource manager parameters, -log and temp file directories, parameters for their MapReduce jobs, -etc. - -Submit Nodes: Nodes where the HOD Client is run, from where jobs are -submitted to the resource manager system for allocating and running -clusters. - -Compute Nodes: Nodes which get allocated by a resource manager, -and on which the Hadoop daemons are provisioned and started. - -4. Next Steps: -============== - -- Read getting_started.txt to get an idea of how to get started with -installing, configuring and running HOD. - -- Read config.txt to get more details on configuration options for HOD. - diff --git a/third_party/hadoop-0.20.0/contrib/hod/bin/VERSION b/third_party/hadoop-0.20.0/contrib/hod/bin/VERSION deleted file mode 100755 index 5a03fb737b..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/bin/VERSION +++ /dev/null @@ -1 +0,0 @@ -0.20.0 diff --git a/third_party/hadoop-0.20.0/contrib/hod/bin/checknodes b/third_party/hadoop-0.20.0/contrib/hod/bin/checknodes deleted file mode 100755 index 5f9f92f166..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/bin/checknodes +++ /dev/null @@ -1,31 +0,0 @@ -#!/usr/bin/env bash - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -PBS_NODES_PATH=`which pbsnodes 2>/dev/null` -if [ -z $PBS_NODES_PATH ] -then - echo Could not find pbsnodes in path. Cannot check available number of nodes. >&2 - exit 1 -fi -if [ -z $1 ] -then - echo Usage: checknodes queue-name >&2 - exit 2 -fi -# the number of nodes marked 'free', and which do not contain a jobs attribute from the server or from the moms. -$PBS_NODES_PATH :$1 | awk 'BEGIN {c=0} /state = free/ {getline;getline;getline;getline; if ($0 !~ /jobs =/ && $0 !~ /jobs=[0-9].*/) c++ ; } END {print c}' diff --git a/third_party/hadoop-0.20.0/contrib/hod/bin/hod b/third_party/hadoop-0.20.0/contrib/hod/bin/hod deleted file mode 100755 index e87b2764db..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/bin/hod +++ /dev/null @@ -1,577 +0,0 @@ -#!/bin/sh - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -""":" -work_dir=$(dirname $0) -base_name=$(basename $0) -original_dir=$PWD -cd $work_dir - -if [ $HOD_PYTHON_HOME ]; then - exec $HOD_PYTHON_HOME -u -OO $base_name ${1+"$@"} --hod.original-dir $original_dir -elif [ -e /usr/bin/python ]; then - exec /usr/bin/python -u -OO $base_name ${1+"$@"} --hod.original-dir $original_dir -elif [ -e /usr/local/bin/python ]; then - exec /usr/local/bin/python -u -OO $base_name ${1+"$@"} --hod.original-dir $original_dir -else - exec python -u -OO $base_name ${1+"$@"} --hod.original-dir $work_dir -fi -":""" - -"""The executable to be used by the user""" -import sys, os, re, pwd, threading, sys - -myName = os.path.basename(sys.argv[0]) -myName = re.sub(".*/", "", myName) -binDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/bin/.*", "", binDirectory) -libDirectory = rootDirectory - -sys.path.append(libDirectory) - -from hodlib.Hod.hod import hodRunner -from hodlib.Common.setup import * -from hodlib.Common.descGenerator import * -from hodlib.Common.util import local_fqdn, need_to_allocate, filter_warnings,\ - get_exception_error_string, hodInterrupt, \ - HOD_INTERRUPTED_MESG, HOD_INTERRUPTED_CODE,\ - TORQUE_USER_LIMITS_COMMENT_FIELD -from hodlib.Common.tcp import tcpError, tcpSocket -from hodlib.Hod.hod import hodHelp - -filter_warnings() - -reVersion = re.compile(".*(\d+_\d+).*") - -VERSION = None -if os.path.exists("./VERSION"): - vFile = open("./VERSION", 'r') - VERSION = vFile.readline() - vFile.close() - -# Always look for hodrc file here unless otherwise specified with -c: -DEFAULT_LOC = os.path.join(rootDirectory, 'conf') -DEFAULT_HOD_DIR = os.path.join(os.environ['HOME'], ".hod") - -if not os.path.isdir(DEFAULT_HOD_DIR): - os.mkdir(DEFAULT_HOD_DIR, 0777) - -DEFAULT_CONFIG = os.path.join(DEFAULT_HOD_DIR, 'hodrc') -if not os.path.exists(DEFAULT_CONFIG): - if os.environ.has_key('HOD_CONF_DIR') and os.environ['HOD_CONF_DIR'] is not None: - DEFAULT_CONFIG = os.path.join(os.environ['HOD_CONF_DIR'], 'hodrc') - -# Definition tuple is of the form: -# (name, type, description, help?, default value, required?, validate?, -# short option) -# -defList = { 'hod' : ( - ('original-dir', 'directory', 'hod original start directory', - False, None, True, True, 'r'), - - ('clusterdir', 'directory', - 'Directory where cluster state information and hadoop-site.xml' + - ' will be stored.', - True, None, False, False, 'd'), - - ('syslog-address', 'address', 'Syslog address.', - False, None, False, True, 'y'), - - ('java-home', 'directory', 'Java home directory.', - True, None, True, True, 'j'), - - ('debug', 'pos_int', 'Debugging level, 0-4.', - True, 3, True, True, 'b'), - - ('stream', 'bool', 'Output to stderr.', - False, True, False, True), - - ('nodecount', 'pos_int', - 'Number of nodes to allocate at startup. ', - True, None, False, True, 'n'), - - ('script', 'file', 'Hadoop script to execute.', - True, None, False, False, 's'), - - ('userid', 'user_account', - 'User ID the hod shell is running under.', - False, pwd.getpwuid(os.getuid())[0], False, True, 'u'), - - ('allocate-wait-time', 'pos_int', - 'Time to wait for cluster allocation.', - False, 300, True, True, 'e'), - - ('operation', 'string', - 'Initiate a hod operation. (help, allocate, deallocate ...)', - False, None, False, True, 'o'), - - ('cluster-factor', 'pos_float', - 'The number of grid slots per machines', False, 1.9, False, True, - 'x'), - - ('cluster', 'string', 'Name of cluster being used.', - False, None, True, True, 'w'), - - ('proxy-xrs-address', 'address', - 'Address to Allocation Manager XML RPC proxy.', - False, None, False, True, 'p'), - - ('xrs-port-range', 'range', 'XML-RPC port range n-m.', - False, None, True, True), - - ('client-params', 'keyval', 'Hadoop client xml key/value list', - True, None, False, True, 'C'), - - ('hadoop-ui-log-dir', 'directory', 'Directory to store Web UI Logs of Hadoop', - True, None, False, True), - - ('temp-dir', 'directory', 'HOD temporary directories.', - False, None, True, False), - - ('update-worker-info', 'bool', 'Specifies whether to update Worker Info after allocation', - False, False, False, True), - - ('job-feasibility-attr', 'string', 'Specifies whether to check job feasibility - resource manager and/or scheduler limits, also gives the attribute value', - False, None, False, True), - - ('title', 'string', 'Title for the current HOD allocation.', - True, "HOD", False, True, 'N'), - - ('walltime', 'pos_int', 'Walltime in seconds for the current HOD allocation', - True, None, False, True, 'l'), - - ('script-wait-time', 'pos_int', 'Specifies the time to wait before running the script. Used with the hod.script option.', - True, 10, False, True, 'W'), - - ('log-rollover-count', 'pos_int', 'Specifies the number of rolled-over log files of HOD client. A zero value disables rollover.', - True, 5, False, True, 'L'), - - ('job-status-query-interval', 'pos_int', 'Specifies the time between checking for job status', - False, 30, False, True), - - ('job-command-failure-interval', 'pos_int', 'Specifies the time between checking for failed job status or submission commands', - False, 10, False, True), - - ('job-status-query-failure-retries', 'pos_int', 'Specifies the number of times job status failure queries are retried', - False, 3, False, True), - - ('job-submission-failure-retries', 'pos_int', 'Specifies the number of times job submission failure queries are retried', - False, 3, False, True)), - - 'resource_manager' : ( - ('id', 'string', 'Batch scheduler ID: torque|condor.', - False, None, True, True), - - ('pbs-user', 'user_account', 'User ID jobs are submitted under.', - False, None, False, True), - - ('pbs-account', 'string', 'User Account jobs are submitted under.', - True, None, False, False, 'A'), - - ('queue', 'string', 'Queue of the batch scheduler to query.', - True, 'batch', False, True, 'Q'), - - ('batch-home', 'directory', 'Scheduler installation directory.', - False, None, True, True), - - ('options', 'keyval', 'Options to pass to the scheduler.', - False, None, False, True), - - ('env-vars', 'keyval', 'Environment variables to pass to the submitted jobs.', - False, None, False, True)), - - 'ringmaster' : ( - ('work-dirs', 'list', 'hod work directories', - False, None, True, False), - - ('temp-dir', 'directory', 'Ringmaster temporary directory.', - False, None, True, False), - - ('log-dir', 'directory', 'hod logging directory.', - False, os.path.join(rootDirectory, 'logs'), False, False), - - ('syslog-address', 'address', 'Syslog address.', - False, None, False, True), - - ('xrs-port-range', 'range', 'XML-RPC port range n-m.', - False, None, True, True), - - ('http-port-range', 'range', 'HTTP port range n-m.', - False, None, True, True), - - ('debug', 'pos_int', 'Debugging level, 0-4.', - False, 4, True, True), - - ('register', 'bool', 'Register with service registry?', - False, True, True, True), - - ('stream', 'bool', 'Output to stderr.', - False, False, False, True), - - ('userid', 'user_account', - 'User ID the hod shell is running under.', - False, pwd.getpwuid(os.getuid())[0], False, True), - - ('svcrgy-addr', 'address', 'Download HTTP address.', - False, None, False, False), - - ('hadoop-tar-ball', 'uri', 'hadoop program tar ball.', - True, None, False, False, 't'), - - ('max-connect','pos_int','max connections allowed for a single tarball server', - False, 30, False, True), - - ('jt-poll-interval', 'pos_int', 'How often to poll the Job tracker for idleness', - False, 120, False, True), - - ('idleness-limit', 'pos_int', 'Limit after which to deallocate the cluster', - False, 3600, False, True), - - ('max-master-failures', 'pos_int', - 'Defines how many times a master can fail before' \ - ' failing cluster allocation', False, 5, True, True), - - ('workers_per_ring', 'pos_int', 'Defines number of workers per service per hodring', - False, 1, False, True)), - - 'gridservice-mapred' : ( - ('external', 'bool', "Connect to an already running MapRed?", - False, False, True, True), - - ('host', 'hostname', 'Mapred hostname.', - False, 'localhost', False, False), - - ('info_port', 'pos_int', 'Mapred info port.', - False, None, False, False), - - ('tracker_port', 'pos_int', 'Mapred job tracker port.', - False, None, False, False), - - ('cmdline-params', 'keyval', 'Hadoop cmdline key/value list.', - False, None, False, False), - - ('server-params', 'keyval', 'Hadoop xml key/value list', - True, None, False, True, 'M'), - - ('envs', 'keyval', 'environment to run this package in', - False, None, False, True), - - ('final-server-params', 'keyval', 'Hadoop final xml key/val list', - False, None, False, True, 'F'), - - ('pkgs', 'directory', "directory where the package is installed", - False, None, False, False)), - - - 'gridservice-hdfs' : ( - ('external', 'bool', "Connect to an already running HDFS?", - False, False, True, True), - - ('host', 'hostname', 'HDFS hostname.', - False, 'localhost', False, False), - - ('fs_port', 'pos_int', 'HDFS port.', - False, None, False, False), - - ('info_port', 'pos_int', 'HDFS info port.', - False, None, False, False), - - ('cmdline-params', 'keyval', 'Hadoop cmdline key/value list.', - False, None, False, False), - - ('server-params', 'keyval', 'Hadoop xml key/value list', - False, None, False, True, 'H'), - - ('final-server-params', 'keyval', 'Hadoop final xml key/value list', - False, None, False, True, 'S'), - - ('envs', 'keyval', 'Environment in which to run this package.', - False, None, False, True), - - ('pkgs', 'directory', "directory where the package is installed", - False, None, False, False)), - - - 'hodring' : ( - ('temp-dir', 'list', 'hodring temporary directory.', - False, None, True, False), - - ('log-dir', 'directory', 'hod logging directory.', - False, os.path.join(rootDirectory, 'logs'), False, False), - - ('log-destination-uri', 'string', - 'URI to store logs to, local://some_path or ' - + 'hdfs://host:port/some_path', - False, None, False, True), - - ('pkgs', 'directory', 'Path to Hadoop to use in case of uploading to HDFS', - False, None, False, False), - - ('syslog-address', 'address', 'Syslog address.', - False, None, False, True), - - ('java-home', 'directory', 'Java home directory.', - False, None, True, False), - - ('debug', 'pos_int', 'Debugging level, 0-4.', - False, 3, True, True), - - ('register', 'bool', 'Register with service registry?', - False, True, True, True), - - ('stream', 'bool', 'Output to stderr.', - False, False, False, True), - - ('userid', 'user_account', - 'User ID the hod shell is running under.', - False, pwd.getpwuid(os.getuid())[0], False, True), - - ('command', 'string', 'Command for hodring to run.', - False, None, False, True), - - ('xrs-port-range', 'range', 'XML-RPC port range n-m.', - False, None, True, True), - - ('http-port-range', 'range', 'HTTP port range n-m.', - False, None, True, True), - - ('service-id', 'string', 'Service ID.', - False, None, False, True), - - ('download-addr', 'string', 'Download HTTP address.', - False, None, False, True), - - ('svcrgy-addr', 'address', 'Download HTTP address.', - False, None, False, True), - - ('ringmaster-xrs-addr', 'address', 'Ringmaster XML-RPC address.', - False, None, False, True), - - ('tarball-retry-initial-time', 'pos_float','Initial Retry time for tarball download', - False, 1, False, True), - - ('tarball-retry-interval', 'pos_float','interval to spread retries for tarball download', - False, 3, False, True), - - ('cmd-retry-initial-time', 'pos_float','Initial retry time for getting commands', - False, 2, False, True), - - ('cmd-retry-interval', 'pos_float','interval to spread retries for getting commands', - False, 2, False, True), - - ('mapred-system-dir-root', 'string', 'Root under which mapreduce system directory names are generated by HOD.', - False, '/mapredsystem', False, False)) - } - -defOrder = [ 'hod', 'ringmaster', 'hodring', 'resource_manager', - 'gridservice-mapred', 'gridservice-hdfs' ] - -def printErrors(msgs): - for msg in msgs: - print msg - -def op_requires_pkgs(config): - if config['hod'].has_key('operation'): - return config['hod']['operation'].startswith('allocate') - else: - return config['hod'].has_key('script') - -if __name__ == '__main__': - try: - confDef = definition() - confDef.add_defs(defList, defOrder) - hodhelp = hodHelp() - usage = hodhelp.help() - - hodOptions = options(confDef, usage, - VERSION, withConfig=True, defaultConfig=DEFAULT_CONFIG, - name=myName ) - # hodConfig is a dict like object, hodConfig[section][name] - try: - hodConfig = config(hodOptions['config'], configDef=confDef, - originalDir=hodOptions['hod']['original-dir'], - options=hodOptions) - except IOError, e: - print >>sys.stderr,"error: %s not found. Specify the path to the HOD configuration file, or define the environment variable %s under which a file named hodrc can be found." % (hodOptions['config'], 'HOD_CONF_DIR') - sys.exit(1) - - # Conditional validation - statusMsgs = [] - - if hodConfig.normalizeValue('gridservice-hdfs', 'external'): - # For external HDFS - statusMsgs.extend(hodConfig.validateValue('gridservice-hdfs', - 'fs_port')) - statusMsgs.extend(hodConfig.validateValue('gridservice-hdfs', - 'info_port')) - statusMsgs.extend(hodConfig.validateValue('gridservice-hdfs', - 'host')) - else: - hodConfig['gridservice-hdfs']['fs_port'] = 0 # Dummy - hodConfig['gridservice-hdfs']['info_port'] = 0 # Not used at all - - if hodConfig.normalizeValue('gridservice-mapred', 'external'): - statusMsgs.extend(hodConfig.validateValue('gridservice-mapred', - 'tracker_port')) - statusMsgs.extend(hodConfig.validateValue('gridservice-mapred', - 'info_port')) - statusMsgs.extend(hodConfig.validateValue('gridservice-mapred', - 'host')) - else: - hodConfig['gridservice-mapred']['tracker_port'] = 0 # Dummy - hodConfig['gridservice-mapred']['info_port'] = 0 # Not used at all - - if len(statusMsgs) != 0: - for msg in statusMsgs: - print >>sys.stderr, msg - sys.exit(1) - # End of conditional validation - - status = True - statusMsgs = [] - - (status,statusMsgs) = hodConfig.verify() - if not status: - print >>sys.stderr,"error: bin/hod failed to start." - for msg in statusMsgs: - print >>sys.stderr,"%s" % (msg) - sys.exit(1) - - ## TODO : should move the dependency verification to hodConfig.verify - if hodConfig['hod'].has_key('operation') and \ - hodConfig['hod'].has_key('script'): - print "Script operation is mutually exclusive with other HOD operations" - hodOptions.print_help(sys.stderr) - sys.exit(1) - - if 'operation' not in hodConfig['hod'] and 'script' not in hodConfig['hod']: - print "HOD requires at least a script or operation be specified." - hodOptions.print_help(sys.stderr) - sys.exit(1) - - if hodConfig['gridservice-hdfs']['external']: - hdfsAddress = "%s:%s" % (hodConfig['gridservice-hdfs']['host'], - hodConfig['gridservice-hdfs']['fs_port']) - - hdfsSocket = tcpSocket(hdfsAddress) - - try: - hdfsSocket.open() - hdfsSocket.close() - except tcpError: - printErrors(hodConfig.var_error('hod', 'gridservice-hdfs', - "Failed to open a connection to external hdfs address: %s." % - hdfsAddress)) - sys.exit(1) - else: - hodConfig['gridservice-hdfs']['host'] = 'localhost' - - if hodConfig['gridservice-mapred']['external']: - mapredAddress = "%s:%s" % (hodConfig['gridservice-mapred']['host'], - hodConfig['gridservice-mapred']['tracker_port']) - - mapredSocket = tcpSocket(mapredAddress) - - try: - mapredSocket.open() - mapredSocket.close() - except tcpError: - printErrors(hodConfig.var_error('hod', 'gridservice-mapred', - "Failed to open a connection to external mapred address: %s." % - mapredAddress)) - sys.exit(1) - else: - hodConfig['gridservice-mapred']['host'] = 'localhost' - - if not hodConfig['ringmaster'].has_key('hadoop-tar-ball') and \ - not hodConfig['gridservice-hdfs'].has_key('pkgs') and \ - op_requires_pkgs(hodConfig): - printErrors(hodConfig.var_error('gridservice-hdfs', 'pkgs', - "gridservice-hdfs.pkgs must be defined if ringmaster.hadoop-tar-ball " - + "is not defined.")) - sys.exit(1) - - if not hodConfig['ringmaster'].has_key('hadoop-tar-ball') and \ - not hodConfig['gridservice-mapred'].has_key('pkgs') and \ - op_requires_pkgs(hodConfig): - printErrors(hodConfig.var_error('gridservice-mapred', 'pkgs', - "gridservice-mapred.pkgs must be defined if ringmaster.hadoop-tar-ball " - + "is not defined.")) - sys.exit(1) - - if hodConfig['hodring'].has_key('log-destination-uri'): - if hodConfig['hodring']['log-destination-uri'].startswith('file://'): - pass - elif hodConfig['hodring']['log-destination-uri'].startswith('hdfs://'): - hostPort = hodConfig['hodring']['log-destination-uri'][7:].split("/") - hostPort = hostPort[0] - socket = tcpSocket(hostPort) - try: - socket.open() - socket.close() - except: - printErrors(hodConfig.var_error('hodring', 'log-destination-uri', - "Unable to contact host/port specified in log destination uri: %s" % - hodConfig['hodring']['log-destination-uri'])) - sys.exit(1) - else: - printErrors(hodConfig.var_error('hodring', 'log-destination-uri', - "The log destiniation uri must be of type local:// or hdfs://.")) - sys.exit(1) - - if hodConfig['ringmaster']['workers_per_ring'] < 1: - printErrors(hodConfig.var_error('ringmaster', 'workers_per_ring', - "ringmaster.workers_per_ring must be a positive integer " + - "greater than or equal to 1")) - sys.exit(1) - - ## TODO : end of should move the dependency verification to hodConfig.verif - - hodConfig['hod']['base-dir'] = rootDirectory - hodConfig['hod']['user_state'] = DEFAULT_HOD_DIR - - dGen = DescGenerator(hodConfig) - hodConfig = dGen.initializeDesc() - - os.environ['JAVA_HOME'] = hodConfig['hod']['java-home'] - - if hodConfig['hod']['debug'] == 4: - print "" - print "Using Python: %s" % sys.version - print "" - - hod = hodRunner(hodConfig) - - # Initiate signal handling - hodInterrupt.set_log(hod.get_logger()) - hodInterrupt.init_signals() - # Interrupts set up. Now on we handle signals only when we wish to. - except KeyboardInterrupt: - print HOD_INTERRUPTED_MESG - sys.exit(HOD_INTERRUPTED_CODE) - - opCode = 0 - try: - if hodConfig['hod'].has_key('script'): - opCode = hod.script() - else: - opCode = hod.operation() - except Exception, e: - print "Uncaught Exception : %s" % e - finally: - sys.exit(opCode) diff --git a/third_party/hadoop-0.20.0/contrib/hod/bin/hodcleanup b/third_party/hadoop-0.20.0/contrib/hod/bin/hodcleanup deleted file mode 100755 index 51613eae0a..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/bin/hodcleanup +++ /dev/null @@ -1,183 +0,0 @@ -#!/bin/sh - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -""":" -work_dir=$(dirname $0) -base_name=$(basename $0) -original_dir=$PWD -cd $work_dir - -if [ $HOD_PYTHON_HOME ]; then - exec $HOD_PYTHON_HOME -u -OO $base_name ${1+"$@"} -elif [ -e /usr/bin/python ]; then - exec /usr/bin/python -u -OO $base_name ${1+"$@"} -elif [ -e /usr/local/bin/python ]; then - exec /usr/local/bin/python -u -OO $base_name ${1+"$@"} -else - exec python -u -OO $base_name ${1+"$@"} -fi -":""" - -"""The executable to be used by the user""" -import sys, os, re, pwd, threading, sys, random, time, pprint, shutil, time, re -from pprint import pformat -from optparse import OptionParser - -myName = os.path.basename(sys.argv[0]) -myName = re.sub(".*/", "", myName) -binDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/bin/.*", "", binDirectory) -libDirectory = rootDirectory - -sys.path.append(libDirectory) - -from hodlib.Common.threads import simpleCommand -from hodlib.Common.util import local_fqdn, tar, filter_warnings,\ - get_exception_string, get_exception_error_string -from hodlib.Common.logger import hodLog -from hodlib.Common.logger import getLogger -from hodlib.HodRing.hodRing import createMRSystemDirectoryManager - -filter_warnings() - -reVersion = re.compile(".*(\d+_\d+).*") -reHdfsURI = re.compile("(hdfs://.*?:\d+)(.*)") - -VERSION = None -if os.path.exists("./VERSION"): - vFile = open("./VERSION", 'r') - VERSION = vFile.readline() - vFile.close() - -def __archive_logs(conf, log): - # need log-destination-uri, __hadoopLogDirs, temp-dir - status = True - logUri = conf['log-destination-uri'] - hadoopLogDirs = conf['hadoop-log-dirs'] - if logUri: - try: - if hadoopLogDirs: - date = time.localtime() - for logDir in hadoopLogDirs: - (head, tail) = os.path.split(logDir) - (head, logType) = os.path.split(head) - tarBallFile = "%s-%s-%04d%02d%02d%02d%02d%02d-%s.tar.gz" % ( - logType, local_fqdn(), date[0], date[1], date[2], date[3], - date[4], date[5], random.randint(0,1000)) - - if logUri.startswith('file://'): - tarBallFile = os.path.join(logUri[7:], - tarBallFile) - else: - tarBallFile = os.path.join(conf['temp-dir'], tarBallFile) - - log.debug('archiving log files to: %s' % tarBallFile) - status = tar(tarBallFile, logDir, ['*',]) - log.info('archive %s status: %s' % (tarBallFile, status)) - if status and \ - logUri.startswith('hdfs://'): - __copy_archive_to_dfs(conf, tarBallFile) - log.info("copying archive to dfs finished") - dict = {} - except: - log.error(get_exception_string()) - status = False - return status - - -def __copy_archive_to_dfs(conf, archiveFile): - # need log-destination-uri, hadoopCommandstring and/or pkgs - hdfsURIMatch = reHdfsURI.match(conf['log-destination-uri']) - - (head, tail) = os.path.split(archiveFile) - destFile = os.path.join(hdfsURIMatch.group(2), conf['user-id'], 'hod-logs', conf['service-id'], tail) - - log.info("copying archive %s to DFS %s ..." % (archiveFile, destFile)) - - hadoopCmd = conf['hadoop-command-string'] - if conf['pkgs']: - hadoopCmd = os.path.join(conf['pkgs'], 'bin', 'hadoop') - - copyCommand = "%s dfs -fs %s -copyFromLocal %s %s" % (hadoopCmd, - hdfsURIMatch.group(1), archiveFile, destFile) - - log.debug(copyCommand) - - copyThread = simpleCommand('hadoop', copyCommand) - copyThread.start() - copyThread.wait() - copyThread.join() - log.debug(pprint.pformat(copyThread.output())) - - os.unlink(archiveFile) - -def unpack(): - parser = OptionParser() - option_list=["--log-destination-uri", "--hadoop-log-dirs", \ - "--temp-dir", "--hadoop-command-string", "--pkgs", "--user-id", \ - "--service-id", "--hodring-debug", "--hodring-log-dir", \ - "--hodring-syslog-address", "--hodring-cleanup-list", \ - "--jt-pid", "--mr-sys-dir", "--fs-name", "--hadoop-path"] - regexp = re.compile("^--") - for opt in option_list: - parser.add_option(opt,dest=regexp.sub("",opt),action="store") - option_list.append("--hodring-stream") - parser.add_option("--hodring-stream",dest="hodring-stream",metavar="bool",\ - action="store_true") - (options, args) = parser.parse_args() - _options= {} - _options['hodring'] = {} - for opt in dir(options): - if "--"+opt in option_list: - _options[opt] = getattr(options,opt) - if _options.has_key('hadoop-log-dirs') and _options['hadoop-log-dirs']: - _options['hadoop-log-dirs'] = _options['hadoop-log-dirs'].split(",") - if _options.has_key('hodring-syslog-address') and _options['hodring-syslog-address']: - _options['hodring']['syslog-address'] = \ - _options['hodring-syslog-address'].split(':') - _options['hodring']['debug'] = int(_options['hodring-debug']) - _options['hodring']['log-dir'] = _options['hodring-log-dir'] - _options['hodring']['stream'] = _options['hodring-stream'] - _options['hodring']['userid'] = _options['user-id'] - os.putenv('PBS_JOBID', _options['service-id'] ) - return _options - -if __name__ == '__main__': - log = None - try: - conf = unpack() - # Use the same log as hodring - log = getLogger(conf['hodring'],'hodring') - log.debug("Logger initialised successfully") - mrSysDirManager = createMRSystemDirectoryManager(conf, log) - if mrSysDirManager is not None: - mrSysDirManager.removeMRSystemDirectory() - - status = __archive_logs(conf,log) - log.info("Archive status : %s" % status) - list = conf['hodring-cleanup-list'].split(',') - log.info("now removing %s" % list) - for dir in list: - if os.path.exists(dir): - log.debug('removing %s' % (dir)) - shutil.rmtree(dir, True) - log.debug("done") - log.info("Cleanup successfully completed") - except Exception, e: - if log: - log.info("Stack trace:\n%s\n%s" %(get_exception_error_string(),get_exception_string())) diff --git a/third_party/hadoop-0.20.0/contrib/hod/bin/hodring b/third_party/hadoop-0.20.0/contrib/hod/bin/hodring deleted file mode 100755 index 1bb891c540..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/bin/hodring +++ /dev/null @@ -1,287 +0,0 @@ -#!/bin/sh - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -""":" -work_dir=$(dirname $0) -base_name=$(basename $0) -cd $work_dir - -if [ $HOD_PYTHON_HOME ]; then - exec $HOD_PYTHON_HOME -OO $base_name ${1+"$@"} -elif [ -e /usr/bin/python ]; then - exec /usr/bin/python -OO $base_name ${1+"$@"} -elif [ -e /usr/local/bin/python ]; then - exec /usr/local/bin/python -OO $base_name ${1+"$@"} -else - exec python -OO $base_name ${1+"$@"} -fi -":""" - -"""The executable to be used by the user""" -import sys, os, re - - -myName = os.path.basename(sys.argv[0]) -myName = re.sub(".*/", "", myName) -binDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/bin/.*", "", binDirectory) -libDirectory = rootDirectory - -sys.path.append(libDirectory) - -from hodlib.HodRing.hodRing import HodRing -from hodlib.Common.setup import * -from hodlib.Common.util import filter_warnings, get_exception_string, \ - get_exception_error_string, getMapredSystemDirectory, \ - to_http_url, local_fqdn -from hodlib.Common.logger import getLogger, ensureLogDir -from hodlib.Common.xmlrpc import hodXRClient - -filter_warnings() - -reVersion = re.compile(".*(\d+_\d+).*") - -VERSION = '$HeadURL$' - -reMatch = reVersion.match(VERSION) -if reMatch: - VERSION = reMatch.group(1) - VERSION = re.sub("_", ".", VERSION) -else: - VERSION = 'DEV' - -# Definition tuple is of the form: -# (name, type, description, default value, required?, validate?) -# -defList = { 'hodring' : ( - ('temp-dir', 'directory', 'hod work directories', - False, None, True, False), - - ('log-dir', 'directory', 'hod logging directory.', - False, os.path.join(rootDirectory, 'logs'), False, True), - - ('log-destination-uri', 'string', - 'URI to store logs to, local://some_path or ' - + 'hdfs://host:port/some_path', - False, None, False, True), - - ('pkgs', 'directory', 'Path to Hadoop to use in case of uploading to HDFS', - False, None, False, True), - - ('syslog-address', 'address', 'Syslog address.', - False, None, False, True), - - ('java-home', 'directory', 'Java home directory.', - False, None, True, True), - - ('debug', 'pos_int', 'Debugging level, 0-4.', - False, 3, True, True), - - ('register', 'bool', 'Register with service registry?', - False, True, True, True), - - ('stream', 'bool', 'Output to stderr.', - False, False, False, True), - - ('userid', 'user_account', - 'User ID the hod shell is running under.', - False, None, True, False), - - ('xrs-port-range', 'range', 'XML-RPC port range n-m.', - False, None, True, True), - - ('http-port-range', 'range', 'HTTP port range n-m.', - False, None, True, True), - - ('command', 'string', 'Command for hodring to run.', - False, None, False, True), - - ('service-id', 'string', 'Service ID.', - False, None, False, True), - - ('download-addr', 'string', 'Download HTTP address.', - False, None, False, True), - - ('svcrgy-addr', 'address', 'Service registry XMLRPC address.', - False, None, True, True), - - ('ringmaster-xrs-addr', 'address', 'Ringmaster XML-RPC address.', - False, None, False, True), - - ('tarball-retry-initial-time', 'pos_float','initial retry time for tarball download', - False, 1, False, True), - - ('tarball-retry-interval', 'pos_float','interval to spread retries for tarball download', - False, 3, False, True), - - ('cmd-retry-initial-time', 'pos_float','initial retry time for getting commands', - False, 2, False, True), - - ('cmd-retry-interval', 'pos_float','interval to spread retries for getting commands', - False, 2, False, True), - - ('mapred-system-dir-root', 'string', 'Root under which mapreduce system directory names are generated by HOD.', - False, '/mapredsystem', False, False)) - } - -if __name__ == '__main__': - - confDef = definition() - confDef.add_defs(defList) - hodRingOptions = options(confDef, "./%s [OPTIONS]" % myName, VERSION) - ensureLogDir(hodRingOptions['hodring']['log-dir']) - service = None - try: - (status, statusMsgs) = hodRingOptions.verify() - if not status: - raise Exception("%s" % statusMsgs) - hodRingOptions['hodring']['base-dir'] = rootDirectory - service = HodRing(hodRingOptions) - service.start() - service.wait() - - if service.log: - log = service.log - else: - log = getLogger(hodRingOptions['hodring'],'hodring') - - list = [] - - runningHadoops = service.getRunningValues() - - mrSysDirManager = None - for cmd in runningHadoops: - if cmd.name == 'jobtracker': - mrSysDirManager = cmd.getMRSystemDirectoryManager() - log.debug("addding %s to cleanup list..." % cmd) - cmd.addCleanup(list) - - list.append(service.getTempDir()) - log.debug(list) - - # archive_logs now - cmdString = os.path.join(rootDirectory, "bin", "hodcleanup") # same python - - if (len(runningHadoops) == 0): - log.info("len(runningHadoops) == 0, No running cluster?") - log.info("Skipping __copy_archive_to_dfs") - hadoopString = "" - else: hadoopString=runningHadoops[0].path - - #construct the arguments - if hodRingOptions['hodring'].has_key('log-destination-uri'): - cmdString = cmdString + " --log-destination-uri " \ - + hodRingOptions['hodring']['log-destination-uri'] - - hadoopLogDirs = service.getHadoopLogDirs() - if hadoopLogDirs: - cmdString = cmdString \ - + " --hadoop-log-dirs " \ - + ",".join(hadoopLogDirs) - - cmdString = cmdString \ - + " --temp-dir " \ - + service._cfg['temp-dir'] \ - + " --hadoop-command-string " \ - + hadoopString \ - + " --user-id " \ - + service._cfg['userid'] \ - + " --service-id " \ - + service._cfg['service-id'] \ - + " --hodring-debug " \ - + str(hodRingOptions['hodring']['debug']) \ - + " --hodring-log-dir " \ - + hodRingOptions['hodring']['log-dir'] \ - + " --hodring-cleanup-list " \ - + ",".join(list) - - if hodRingOptions['hodring'].has_key('syslog-address'): - syslogAddr = hodRingOptions['hodring']['syslog-address'][0] + \ - ':' + str(hodRingOptions['hodring']['syslog-address'][1]) - cmdString = cmdString + " --hodring-syslog-address " + syslogAddr - if service._cfg.has_key('pkgs'): - cmdString = cmdString + " --pkgs " + service._cfg['pkgs'] - - if mrSysDirManager is not None: - cmdString = "%s %s" % (cmdString, mrSysDirManager.toCleanupArgs()) - - log.info("cleanup commandstring : ") - log.info(cmdString) - - # clean up - cmd = ['/bin/sh', '-c', cmdString] - - mswindows = (sys.platform == "win32") - originalcwd = os.getcwd() - - if not mswindows: - try: - pid = os.fork() - if pid > 0: - # exit first parent - log.info("child(pid: %s) is now doing cleanup" % pid) - sys.exit(0) - except OSError, e: - log.error("fork failed: %d (%s)" % (e.errno, e.strerror)) - sys.exit(1) - - # decouple from parent environment - os.chdir("/") - os.setsid() - os.umask(0) - - MAXFD = 128 # more than enough file descriptors to close. Just in case. - for i in xrange(0, MAXFD): - try: - os.close(i) - except OSError: - pass - - try: - os.execvp(cmd[0], cmd) - finally: - log.critical("exec failed") - os._exit(1) - - except Exception, e: - if service: - if service.log: - log = service.log - else: - log = getLogger(hodRingOptions['hodring'], 'hodring') - log.error("Error in bin/hodring %s. \nStack trace:\n%s" %(get_exception_error_string(),get_exception_string())) - - log.info("now trying informing to ringmaster") - log.info(hodRingOptions['hodring']['ringmaster-xrs-addr']) - log.info(hodRingOptions.normalizeValue('hodring', 'ringmaster-xrs-addr')) - log.info(to_http_url(hodRingOptions.normalizeValue( \ - 'hodring', 'ringmaster-xrs-addr'))) - # Report errors to the Ringmaster if possible - try: - ringXRAddress = to_http_url(hodRingOptions.normalizeValue( \ - 'hodring', 'ringmaster-xrs-addr')) - log.debug("Creating ringmaster XML-RPC client.") - ringClient = hodXRClient(ringXRAddress) - if ringClient is not None: - addr = local_fqdn() + "_" + str(os.getpid()) - ringClient.setHodRingErrors(addr, str(e)) - log.info("Reported errors to ringmaster at %s" % ringXRAddress) - except Exception, e: - log.error("Failed to report errors to ringmaster at %s" % ringXRAddress) - log.error("Reason : %s" % get_exception_string()) - # End of reporting errors to the client diff --git a/third_party/hadoop-0.20.0/contrib/hod/bin/ringmaster b/third_party/hadoop-0.20.0/contrib/hod/bin/ringmaster deleted file mode 100755 index fc194f6d27..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/bin/ringmaster +++ /dev/null @@ -1,349 +0,0 @@ -#!/bin/sh - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -""":" -work_dir=$(dirname $0) -base_name=$(basename $0) -cd $work_dir - -if [ $HOD_PYTHON_HOME ]; then - exec $HOD_PYTHON_HOME -OO $base_name ${1+"$@"} -elif [ -e /usr/bin/python ]; then - exec /usr/bin/python -OO $base_name ${1+"$@"} -elif [ -e /usr/local/bin/python ]; then - exec /usr/local/bin/python -OO $base_name ${1+"$@"} -else - exec python -OO $base_name ${1+"$@"} -fi -":""" - -"""The executable to be used by the user""" -import sys, os, re, getpass - -myName = os.path.basename(sys.argv[0]) -myName = re.sub(".*/", "", myName) -binDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/bin/.*", "", binDirectory) -libDirectory = rootDirectory - -sys.path.append(libDirectory) - -from hodlib.RingMaster.ringMaster import main -from hodlib.Common.setup import * -from hodlib.Common.descGenerator import * -from hodlib.Common.util import local_fqdn, filter_warnings, to_http_url, \ - get_exception_string, get_exception_error_string -from hodlib.Common.logger import getLogger, ensureLogDir -from hodlib.Common.xmlrpc import hodXRClient -import logging - -filter_warnings() - -reVersion = re.compile(".*(\d+_\d+).*") - -VERSION = '$HeadURL$' - -reMatch = reVersion.match(VERSION) -if reMatch: - VERSION = reMatch.group(1) - VERSION = re.sub("_", ".", VERSION) -else: - VERSION = 'DEV' - -# Definition tuple is of the form: -# (name, type, description, default value, required?, validate?) -# -defList = { 'ringmaster' : ( - ('work-dirs', 'list', 'hod work directories', - False, None, True, False), - - ('temp-dir', 'directory', 'Ringmaster temporary directory.', - False, None, True, False), - - ('log-dir', 'directory', 'hod logging directory.', - False, os.path.join(rootDirectory, 'logs'), False, True), - - ('syslog-address', 'address', 'Syslog address.', - False, None, False, True), - - ('xrs-port-range', 'range', 'XML-RPC port range n-m.', - False, None, True, True), - - ('http-port-range', 'range', 'HTTP port range n-m.', - False, None, True, True), - - ('debug', 'pos_int', 'Debugging level, 0-4.', - False, 3, True, True), - - ('register', 'bool', 'Register with service registry?', - False, True, True, True), - - ('stream', 'bool', 'Output to stderr.', - False, False, False, True), - - ('userid', 'user_account', - 'User ID the hod shell is running under.', - False, None, True, False), - - ('svcrgy-addr', 'address', 'Download HTTP address.', - False, None, False, True), - - ('hadoop-tar-ball', 'uri', 'hadoop program tar ball.', - False, None, False, False), - - ('max-connect','pos_int','max connections allowed for a single tarball server', - False, 30, False, True), - - ('jt-poll-interval', 'pos_int', 'How often to poll the Job tracker for idleness', - False, 120, False, True), - - ('idleness-limit', 'pos_int', 'Limit after which to deallocate the cluster', - False, 3600, False, True), - - ('max-master-failures', 'pos_int', - 'Defines how many times a master can fail before' \ - ' failing cluster allocation', False, 5, True, True), - - ('workers_per_ring', 'pos_int', 'Defines number of workers per service per hodring', - False, 1, False, True)), - - 'resource_manager' : ( - ('id', 'string', 'Batch scheduler ID: torque|condor.', - False, None, True, True), - - ('pbs-user', 'user_account', 'User ID jobs are submitted under.', - False, None, False, True), - - ('pbs-server', 'hostname', 'Hostname of PBS server.', - False, None, False, True), - - ('pbs-account', 'string', 'User Account jobs are submitted under.', - False, None, False, False), - - ('queue', 'string', 'Queue of the batch scheduler to query.', - False, None, False, False), - - ('batch-home', 'directory', 'Scheduler installation directory.', - False, None, True, True), - - ('options', 'keyval', 'Options to pass to the scheduler.', - False, None, False, True), - - ('env-vars', 'keyval', 'Environment variables to pass to the submitted jobs.', - False, None, False, True)), - - 'gridservice-mapred' : ( - ('external', 'bool', "Connect to an already running MapRed?", - False, False, True, True), - - ('host', 'hostname', 'Mapred hostname.', - False, 'localhost', False, True), - - ('info_port', 'pos_int', 'Mapred info port.', - False, None, True, True), - - ('tracker_port', 'pos_int', 'Mapred job tracker port.', - False, None, True, True), - - ('cmdline-params', 'keyval', 'Hadoop cmdline key/value list.', - False, None, False, False), - - ('server-params', 'keyval', 'Hadoop xml key/value list', - False, None, False, False), - - ('final-server-params', 'keyval', 'Hadoop final xml params', - False, None, False, False), - - ('envs', 'keyval', 'environment to run this package in', - False, None, False, False), - - ('pkgs', 'directory', "directory where the package is installed", - False, None, False, False)), - - - 'gridservice-hdfs' : ( - ('external', 'bool', "Connect to an already running HDFS?", - False, False, True, True), - - ('host', 'hostname', 'HDFS hostname.', - False, 'localhost', True, True), - - ('fs_port', 'pos_int', 'HDFS port range.', - False, None, True, True), - - ('info_port', 'pos_int', 'HDFS info port.', - False, None, True, True), - - ('cmdline-params', 'keyval', 'Hadoop cmdline key/value list.', - False, None, False, False), - - ('server-params', 'keyval', 'Hadoop xml key/value list', - False, None, False, False), - - ('final-server-params', 'keyval', 'Hadoop final xml params', - False, None, False, False), - - ('envs', 'keyval', 'Environment in which to run this package.', - False, None, False, False), - - ('pkgs', 'directory', "directory where the package is installed", - False, None, False, False)), - - - 'hodring' : ( - ('temp-dir', 'directory', 'hod work directories', - False, None, True, False), - - ('log-dir', 'directory', 'hod logging directory.', - False, os.path.join(rootDirectory, 'logs'), False, False), - - ('log-destination-uri', 'string', - 'URI to store logs to, local://some_path or ' - + 'hdfs://host:port/some_path', - False, None, False, True), - - ('pkgs', 'directory', 'Path to Hadoop to use in case of uploading to HDFS', - False, None, False, True), - - ('syslog-address', 'address', 'Syslog address.', - False, None, False, True), - - ('java-home', 'directory', 'Java home directory.', - False, None, True, False), - - ('debug', 'pos_int', 'Debugging level, 0-4.', - False, 3, True, True), - - ('register', 'bool', 'Register with service registry?', - False, True, True, True), - - ('stream', 'bool', 'Output to stderr.', - False, False, False, True), - - ('userid', 'user_account', - 'User ID the hod shell is running under.', - False, None, True, False), - - ('xrs-port-range', 'range', 'XML-RPC port range n-m.', - False, None, True, True), - - ('http-port-range', 'range', 'HTTP port range n-m.', - False, None, True, True), - - ('command', 'string', 'Command for hodring to run.', - False, None, False, True), - - ('service-id', 'string', 'Service ID.', - False, None, False, True), - - ('download-addr', 'address', 'Download HTTP address.', - False, None, False, True), - - ('svcrgy-addr', 'address', 'Download HTTP address.', - False, None, False, True), - - ('ringmaster-xrs-addr', 'address', 'Ringmaster XML-RPC address.', - False, None, False, True), - - ('tarball-retry-initial-time', 'pos_float','initial retry time for tarball download', - False, 1, False, True), - - ('tarball-retry-interval', 'pos_float','interval to spread retries for tarball download', - False, 3, False, True), - - ('cmd-retry-initial-time', 'pos_float','initial retry time for getting commands', - False, 2, False, True), - - ('cmd-retry-interval', 'pos_float','interval to spread retries for getting commands', - False, 2, False, True), - - ('mapred-system-dir-root', 'string', 'Root under which mapreduce system directory names are generated by HOD.', - False, '/mapredsystem', False, False)) - } - - -defOrder = [ 'ringmaster', 'hodring', 'resource_manager', - 'gridservice-mapred', 'gridservice-hdfs' ] - -if __name__ == '__main__': - confDef = definition() - confDef.add_defs(defList, defOrder) - ringMasterOptions = options(confDef, "./%s [OPTIONS]" % myName, VERSION) - log = logging.getLogger() - - try: - - # Set up logging before anything else. - ensureLogDir(ringMasterOptions.normalizeValue('ringmaster', 'log-dir')) - log = getLogger(ringMasterOptions['ringmaster'],'ringmaster') - # End of setting up logging - - # Verify and process options - statusMsgs = [] - # Conditional validation - if not ringMasterOptions['ringmaster'].has_key('hadoop-tar-ball') or \ - not ringMasterOptions['ringmaster']['hadoop-tar-ball']: - # If tarball is not used - if not ringMasterOptions.normalizeValue('gridservice-hdfs', 'external'): - # And if hdfs is not external, validate gridservice-hdfs.pkgs - statusMsgs.extend(ringMasterOptions.validateValue( - 'gridservice-hdfs', 'pkgs')) - statusMsgs.extend(ringMasterOptions.validateValue( - 'gridservice-mapred', 'pkgs')) - - if len(statusMsgs) != 0: - # format status messages into a single string - errStr = '' - for msg in statusMsgs: - errStr = "%s%s\n" % (errStr, msg) - raise Exception("%s" % errStr) - # End of conditional validation - - (status, statusMsgs) = ringMasterOptions.verify() - if not status: - # format status messages into a single string - errStr = '' - for msg in statusMsgs: - errStr = "%s%s\n" % (errStr, msg) - raise Exception("%s" % errStr) - - ringMasterOptions.replace_escape_seqs() - ringMasterOptions['ringmaster']['base-dir'] = rootDirectory - # End of option processing - - ret = main(ringMasterOptions,log) - sys.exit(ret) - except Exception, e: - log.error("bin/ringmaster failed to start.%s. Stack trace follows:\n%s" % (get_exception_error_string(),get_exception_string())) - - # Report errors to the client if possible - try: - serviceAddr = to_http_url(ringMasterOptions.normalizeValue( \ - 'ringmaster', 'svcrgy-addr')) - serviceClient = hodXRClient(serviceAddr) - if serviceClient is not None: - serviceClient.setRMError([local_fqdn(), str(e), \ - get_exception_string()]) - log.info("Reported errors to service registry at %s" % serviceAddr) - except Exception, e: - log.error("Failed to report errors to service registry.") - log.error("Reason : %s" % get_exception_string()) - # End of reporting errors to the client - - # Ringmaster failing to start is a ringmaster error. Exit with the appropriate exit code. - sys.exit(6) diff --git a/third_party/hadoop-0.20.0/contrib/hod/bin/verify-account b/third_party/hadoop-0.20.0/contrib/hod/bin/verify-account deleted file mode 100755 index 65aa79ab44..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/bin/verify-account +++ /dev/null @@ -1,11 +0,0 @@ -#!/bin/sh -# This script file is a stub for systems that might want to include -# checks for the account name that is passed to HOD. It will be -# launched by HOD with the account name as an argument. The script -# should return a zero exit code if the account is valid, and a -# non zero exit code otherwise. Any output that the script generates -# would be reported to the user by HOD, in case of a non-zero exit -# code. -# -# By default, the script does nothing and returns a zero exit code. -exit 0 diff --git a/third_party/hadoop-0.20.0/contrib/hod/build.xml b/third_party/hadoop-0.20.0/contrib/hod/build.xml deleted file mode 100644 index e16b36dacf..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/build.xml +++ /dev/null @@ -1,81 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/third_party/hadoop-0.20.0/contrib/hod/conf/hodrc b/third_party/hadoop-0.20.0/contrib/hod/conf/hodrc deleted file mode 100644 index bc2866d4c8..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/conf/hodrc +++ /dev/null @@ -1,46 +0,0 @@ -[hod] -stream = True -java-home = ${JAVA_HOME} -cluster = ${CLUSTER_NAME} -cluster-factor = 1.8 -xrs-port-range = 32768-65536 -debug = 3 -allocate-wait-time = 3600 -temp-dir = /tmp/hod - -[ringmaster] -register = True -stream = False -temp-dir = /tmp/hod -http-port-range = 8000-9000 -work-dirs = /tmp/hod/1,/tmp/hod/2 -xrs-port-range = 32768-65536 -debug = 3 - -[hodring] -stream = False -temp-dir = /tmp/hod -register = True -java-home = ${JAVA_HOME} -http-port-range = 8000-9000 -xrs-port-range = 32768-65536 -debug = 3 - -[resource_manager] -queue = ${RM_QUEUE} -batch-home = ${RM_HOME} -id = torque -#env-vars = HOD_PYTHON_HOME=/foo/bar/python-2.5.1/bin/python - -[gridservice-mapred] -external = False -pkgs = ${HADOOP_HOME} -tracker_port = 8030 -info_port = 50080 - -[gridservice-hdfs] -external = False -pkgs = ${HADOOP_HOME} -fs_port = 8020 -info_port = 50070 - diff --git a/third_party/hadoop-0.20.0/contrib/hod/config.txt b/third_party/hadoop-0.20.0/contrib/hod/config.txt deleted file mode 100644 index ca894a702a..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/config.txt +++ /dev/null @@ -1,172 +0,0 @@ - HOD Configuration - ================= - -1. Introduction: -================ - -Configuration options for HOD are organized as sections and options -within them. They can be specified in two ways: a configuration file -in the INI format, and as command line options to the HOD shell, -specified in the format --section.option[=value]. If the same option is -specified in both places, the value specified on the command line -overrides the value in the configuration file. - -To get a simple description of all configuration options, you can type - hod --verbose-help - -This document explains some of the most important or commonly used -configuration options in some more detail. - -2. Sections: -============ - -The following are the various sections in the HOD configuration: - - * hod: Options for the HOD client - * resource_manager: Options for specifying which resource - manager to use, and other parameters for - using that resource manager - * ringmaster: Options for the RingMaster process, - * hodring: Options for the HodRing processes - * gridservice-mapred: Options for the MapReduce daemons - * gridservice-hdfs: Options for the HDFS daemons. - -The following are some of the important options in the HOD -configuration: - -3. Important / Commonly Used Configuration Options: -=================================================== - -3.1. Common configuration options: ----------------------------------- - -Certain configuration options are defined in most of the sections of -the HOD configuration. Options defined in a section, are used by the -process for which that section applies. These options have the same -meaning, but can have different values in each section. - -* temp-dir: Temporary directory for usage by the HOD processes. Make - sure that the users who will run hod have rights to create - directories under the directory specified here. - -* debug: A numeric value from 1-4. 4 produces the most log information, - and 1 the least. - -* log-dir: Directory where log files are stored. By default, this is - /logs/. The restrictions and notes for the - temp-dir variable apply here too. - -* xrs-port-range: A range of ports, among which an available port shall - be picked for use to run an XML-RPC server. - -* http-port-range: A range of ports, among which an available port shall - be picked for use to run an HTTP server. - -* java-home: Location of Java to be used by Hadoop. - -3.2 hod options: ----------------- - -* cluster: A descriptive name given to the cluster. For Torque, this is - specified as a 'Node property' for every node in the cluster. - HOD uses this value to compute the number of available nodes. - -* client-params: A comma-separated list of hadoop config parameters - specified as key-value pairs. These will be used to - generate a hadoop-site.xml on the submit node that - should be used for running MapReduce jobs. - -3.3 resource_manager options: ------------------------------ - -* queue: Name of the queue configured in the resource manager to which - jobs are to be submitted. - -* batch-home: Install directory to which 'bin' is appended and under - which the executables of the resource manager can be - found. - -* env-vars: This is a comma separated list of key-value pairs, - expressed as key=value, which would be passed to the jobs - launched on the compute nodes. - For example, if the python installation is - in a non-standard location, one can set the environment - variable 'HOD_PYTHON_HOME' to the path to the python - executable. The HOD processes launched on the compute nodes - can then use this variable. - -3.4 ringmaster options: ------------------------ - -* work-dirs: These are a list of comma separated paths that will serve - as the root for directories that HOD generates and passes - to Hadoop for use to store DFS / MapReduce data. For e.g. - this is where DFS data blocks will be stored. Typically, - as many paths are specified as there are disks available - to ensure all disks are being utilized. The restrictions - and notes for the temp-dir variable apply here too. - -3.5 gridservice-hdfs options: ------------------------------ - -* external: If false, this indicates that a HDFS cluster must be - bought up by the HOD system, on the nodes which it - allocates via the allocate command. Note that in that case, - when the cluster is de-allocated, it will bring down the - HDFS cluster, and all the data will be lost. - If true, it will try and connect to an externally configured - HDFS system. - Typically, because input for jobs are placed into HDFS - before jobs are run, and also the output from jobs in HDFS - is required to be persistent, an internal HDFS cluster is - of little value in a production system. However, it allows - for quick testing. - -* host: Hostname of the externally configured NameNode, if any - -* fs_port: Port to which NameNode RPC server is bound. - -* info_port: Port to which the NameNode web UI server is bound. - -* pkgs: Installation directory, under which bin/hadoop executable is - located. This can be used to use a pre-installed version of - Hadoop on the cluster. - -* server-params: A comma-separated list of hadoop config parameters - specified key-value pairs. These will be used to - generate a hadoop-site.xml that will be used by the - NameNode and DataNodes. - -* final-server-params: Same as above, except they will be marked final. - - -3.6 gridservice-mapred options: -------------------------------- - -* external: If false, this indicates that a MapReduce cluster must be - bought up by the HOD system on the nodes which it allocates - via the allocate command. - If true, if will try and connect to an externally - configured MapReduce system. - -* host: Hostname of the externally configured JobTracker, if any - -* tracker_port: Port to which the JobTracker RPC server is bound - -* info_port: Port to which the JobTracker web UI server is bound. - -* pkgs: Installation directory, under which bin/hadoop executable is - located - -* server-params: A comma-separated list of hadoop config parameters - specified key-value pairs. These will be used to - generate a hadoop-site.xml that will be used by the - JobTracker and TaskTrackers - -* final-server-params: Same as above, except they will be marked final. - -4. Known Issues: -================ - -HOD does not currently handle special characters such as space, comma -and equals in configuration values. diff --git a/third_party/hadoop-0.20.0/contrib/hod/getting_started.txt b/third_party/hadoop-0.20.0/contrib/hod/getting_started.txt deleted file mode 100644 index ae2b0738f9..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/getting_started.txt +++ /dev/null @@ -1,233 +0,0 @@ - Getting Started With Hadoop On Demand (HOD) - =========================================== - -1. Pre-requisites: -================== - -Hardware: -HOD requires a minimum of 3 nodes configured through a resource manager. - -Software: -The following components are assumed to be installed before using HOD: -* Torque: - (http://www.clusterresources.com/pages/products/torque-resource-manager.php) - Currently HOD supports Torque out of the box. We assume that you are - familiar with configuring Torque. You can get information about this - from the following link: - http://www.clusterresources.com/wiki/doku.php?id=torque:torque_wiki -* Python (http://www.python.org/) - We require version 2.5.1 of Python. - -The following components can be optionally installed for getting better -functionality from HOD: -* Twisted Python: This can be used for improving the scalability of HOD - (http://twistedmatrix.com/trac/) -* Hadoop: HOD can automatically distribute Hadoop to all nodes in the - cluster. However, it can also use a pre-installed version of Hadoop, - if it is available on all nodes in the cluster. - (http://hadoop.apache.org/core) - HOD currently supports Hadoop 0.15 and above. - -NOTE: HOD configuration requires the location of installs of these -components to be the same on all nodes in the cluster. It will also -make the configuration simpler to have the same location on the submit -nodes. - -2. Resource Manager Configuration Pre-requisites: -================================================= - -For using HOD with Torque: -* Install Torque components: pbs_server on a head node, pbs_moms on all - compute nodes, and PBS client tools on all compute nodes and submit - nodes. -* Create a queue for submitting jobs on the pbs_server. -* Specify a name for all nodes in the cluster, by setting a 'node - property' to all the nodes. - This can be done by using the 'qmgr' command. For example: - qmgr -c "set node node properties=cluster-name" -* Ensure that jobs can be submitted to the nodes. This can be done by - using the 'qsub' command. For example: - echo "sleep 30" | qsub -l nodes=3 -* More information about setting up Torque can be found by referring - to the documentation under: -http://www.clusterresources.com/pages/products/torque-resource-manager.php - -3. Setting up HOD: -================== - -* HOD is available under the 'contrib' section of Hadoop under the root - directory 'hod'. -* Distribute the files under this directory to all the nodes in the - cluster. Note that the location where the files are copied should be - the same on all the nodes. -* On the node from where you want to run hod, edit the file hodrc - which can be found in the /conf directory. This file - contains the minimal set of values required for running hod. -* Specify values suitable to your environment for the following - variables defined in the configuration file. Note that some of these - variables are defined at more than one place in the file. - - * ${JAVA_HOME}: Location of Java for Hadoop. Hadoop supports Sun JDK - 1.5.x - * ${CLUSTER_NAME}: Name of the cluster which is specified in the - 'node property' as mentioned in resource manager configuration. - * ${HADOOP_HOME}: Location of Hadoop installation on the compute and - submit nodes. - * ${RM_QUEUE}: Queue configured for submiting jobs in the resource - manager configuration. - * ${RM_HOME}: Location of the resource manager installation on the - compute and submit nodes. - -* The following environment variables *may* need to be set depending on - your environment. These variables must be defined where you run the - HOD client, and also be specified in the HOD configuration file as the - value of the key resource_manager.env-vars. Multiple variables can be - specified as a comma separated list of key=value pairs. - - * HOD_PYTHON_HOME: If you install python to a non-default location - of the compute nodes, or submit nodes, then, this variable must be - defined to point to the python executable in the non-standard - location. - - -NOTE: - -You can also review other configuration options in the file and -modify them to suit your needs. Refer to the file config.txt for -information about the HOD configuration. - - -4. Running HOD: -=============== - -4.1 Overview: -------------- - -A typical session of HOD will involve atleast three steps: allocate, -run hadoop jobs, deallocate. - -4.1.1 Operation allocate ------------------------- - -The allocate operation is used to allocate a set of nodes and install and -provision Hadoop on them. It has the following syntax: - - hod -c config_file -t hadoop_tarball_location -o "allocate \ - cluster_dir number_of_nodes" - -The hadoop_tarball_location must be a location on a shared file system -accesible from all nodes in the cluster. Note, the cluster_dir must exist -before running the command. If the command completes successfully then -cluster_dir/hadoop-site.xml will be generated and will contain information -about the allocated cluster's JobTracker and NameNode. - -For example, the following command uses a hodrc file in ~/hod-config/hodrc and -allocates Hadoop (provided by the tarball ~/share/hadoop.tar.gz) on 10 nodes, -storing the generated Hadoop configuration in a directory named -~/hadoop-cluster: - - $ hod -c ~/hod-config/hodrc -t ~/share/hadoop.tar.gz -o "allocate \ - ~/hadoop-cluster 10" - -HOD also supports an environment variable called HOD_CONF_DIR. If this is -defined, HOD will look for a default hodrc file at $HOD_CONF_DIR/hodrc. -Defining this allows the above command to also be run as follows: - - $ export HOD_CONF_DIR=~/hod-config - $ hod -t ~/share/hadoop.tar.gz -o "allocate ~/hadoop-cluster 10" - -4.1.2 Running Hadoop jobs using the allocated cluster ------------------------------------------------------ - -Now, one can run Hadoop jobs using the allocated cluster in the usual manner: - - hadoop --config cluster_dir hadoop_command hadoop_command_args - -Continuing our example, the following command will run a wordcount example on -the allocated cluster: - - $ hadoop --config ~/hadoop-cluster jar \ - /path/to/hadoop/hadoop-examples.jar wordcount /path/to/input /path/to/output - -4.1.3 Operation deallocate --------------------------- - -The deallocate operation is used to release an allocated cluster. When -finished with a cluster, deallocate must be run so that the nodes become free -for others to use. The deallocate operation has the following syntax: - - hod -o "deallocate cluster_dir" - -Continuing our example, the following command will deallocate the cluster: - - $ hod -o "deallocate ~/hadoop-cluster" - -4.2 Command Line Options ------------------------- - -This section covers the major command line options available via the hod -command: - ---help -Prints out the help message to see the basic options. - ---verbose-help -All configuration options provided in the hodrc file can be passed on the -command line, using the syntax --section_name.option_name[=value]. When -provided this way, the value provided on command line overrides the option -provided in hodrc. The verbose-help command lists all the available options in -the hodrc file. This is also a nice way to see the meaning of the -configuration options. - --c config_file -Provides the configuration file to use. Can be used with all other options of -HOD. Alternatively, the HOD_CONF_DIR environment variable can be defined to -specify a directory that contains a file named hodrc, alleviating the need to -specify the configuration file in each HOD command. - --b 1|2|3|4 -Enables the given debug level. Can be used with all other options of HOD. 4 is -most verbose. - --o "help" -Lists the operations available in the operation mode. - --o "allocate cluster_dir number_of_nodes" -Allocates a cluster on the given number of cluster nodes, and store the -allocation information in cluster_dir for use with subsequent hadoop commands. -Note that the cluster_dir must exist before running the command. - --o "list" -Lists the clusters allocated by this user. Information provided includes the -Torque job id corresponding to the cluster, the cluster directory where the -allocation information is stored, and whether the Map/Reduce daemon is still -active or not. - --o "info cluster_dir" -Lists information about the cluster whose allocation information is stored in -the specified cluster directory. - --o "deallocate cluster_dir" -Deallocates the cluster whose allocation information is stored in the -specified cluster directory. - --t hadoop_tarball -Provisions Hadoop from the given tar.gz file. This option is only applicable -to the allocate operation. For better distribution performance it is -recommended that the Hadoop tarball contain only the libraries and binaries, -and not the source or documentation. - --Mkey1=value1 -Mkey2=value2 -Provides configuration parameters for the provisioned Map/Reduce daemons -(JobTracker and TaskTrackers). A hadoop-site.xml is generated with these -values on the cluster nodes - --Hkey1=value1 -Hkey2=value2 -Provides configuration parameters for the provisioned HDFS daemons (NameNode -and DataNodes). A hadoop-site.xml is generated with these values on the -cluster nodes - --Ckey1=value1 -Ckey2=value2 -Provides configuration parameters for the client from where jobs can be -submitted. A hadoop-site.xml is generated with these values on the submit -node. diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/AllocationManagers/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/AllocationManagers/__init__.py deleted file mode 100644 index 56759d7963..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/AllocationManagers/__init__.py +++ /dev/null @@ -1,16 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/AllocationManagers/goldAllocationManager.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/AllocationManagers/goldAllocationManager.py deleted file mode 100644 index 2794c50354..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/AllocationManagers/goldAllocationManager.py +++ /dev/null @@ -1,104 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -"""Gold Allocation Manager Implementation""" -# -*- python -*- - -import sys, httplib -import sha, base64, hmac -import xml.dom.minidom - -from hodlib.Common.util import * - -class goldAllocationManager: - def __init__(self, cfg, log): - self.__GOLD_SECRET_KEY_FILE = cfg['auth-file'] - (self.__goldHost, self.__goldPort) = (cfg['allocation-manager-address'][0], - cfg['allocation-manager-address'][1]) - self.cfg = cfg - self.log = log - - def getQuote(self, user, project, ignoreErrors=True): - # Get Secret Key from File - secret = '' - try: - secretFile = open(self.__GOLD_SECRET_KEY_FILE) - secret = secretFile.readline() - except Exception, e: - self.log.error("Unable to open file %s" % self.__GOLD_SECRET_KEY_FILE) - self.log.debug(get_exception_string()) - return (ignoreErrors or False) - secretFile.close() - secret = secret.rstrip() - - # construct the SSRMAP request body - body = 'Job%s%s10' % (project, user) - - # compute digest - message = sha.new() - message.update(body) - digest = message.digest() - digestStr = base64.b64encode(digest) - - # compute signature - message = hmac.new(secret, digest, sha) - signatureStr = base64.b64encode(message.digest()) - - # construct the SSSRMAP Message - sssrmapRequest = '\ -%s%s%s' % (body, digestStr, signatureStr) - self.log.info('sssrmapRequest: %s' % sssrmapRequest) - - try: - # post message to GOLD server - webservice = httplib.HTTP(self.__goldHost, self.__goldPort) - webservice.putrequest("POST", "/SSSRMAP3 HTTP/1.1") - webservice.putheader("Content-Type", "text/xml; charset=\"utf-8\"") - webservice.putheader("Transfer-Encoding", "chunked") - webservice.endheaders() - webservice.send("%X" % len(sssrmapRequest) + "\r\n" + sssrmapRequest + '0\r\n') - - # handle the response - statusCode, statusmessage, header = webservice.getreply() - responseStr = webservice.getfile().read() - self.log.debug("httpStatusCode: %d" % statusCode) - self.log.info('responseStr: %s' % responseStr) - - # parse XML response - if (statusCode == 200): - responseArr = responseStr.split("\n") - responseBody = responseArr[2] - try: - doc = xml.dom.minidom.parseString(responseBody) - responseVal = doc.getElementsByTagName("Value")[0].firstChild.nodeValue - self.log.info("responseVal: %s" % responseVal) - if (responseVal == 'Success'): - return True - else: - return False - except Exception, e: - self.log.error("Unable to parse GOLD responseBody XML \"(%s)\" to get responseVal" % (responseBody)) - self.log.debug(get_exception_string()) - return (ignoreErrors or False) - else: - self.log.error("Invalid HTTP statusCode %d" % statusCode) - except Exception, e: - self.log.error("Unable to POST message to GOLD server (%s, %d)" % - (self.__goldHost, self.__goldPort)) - self.log.debug(get_exception_string()) - return (ignoreErrors or False) - - return True - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/__init__.py deleted file mode 100644 index 12c2f1e1da..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/__init__.py +++ /dev/null @@ -1,15 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/allocationManagerUtil.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/allocationManagerUtil.py deleted file mode 100644 index 515e875070..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/allocationManagerUtil.py +++ /dev/null @@ -1,27 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -"""defines Allocation Manager Utilities""" - -# -*- python -*- -from hodlib.allocationManagers.goldAllocationManager import goldAllocationManager - -class allocationManagerUtil: - def getAllocationManager(name, cfg, log): - """returns a concrete instance of the specified AllocationManager""" - if name == 'gold': - return goldAllocationManager(cfg, log) - - getAllocationManager = staticmethod(getAllocationManager) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/desc.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/desc.py deleted file mode 100644 index 013e3bde02..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/desc.py +++ /dev/null @@ -1,298 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -"""manage component descriptors""" -# -*- python -*- - -import random - -from sets import Set -from pprint import pformat -from hodlib.Common.util import local_fqdn -from hodlib.Common.tcp import tcpSocket, tcpError - -class Schema: - """the primary class for describing - schema's """ - STRING, LIST, MAP = range(3) - - def __init__(self, name, type = STRING, delim=','): - self.name = name - self.type = type - self.delim = delim - - def getName(self): - return self.name - - def getType(self): - return self.type - - def getDelim(self): - return self.delim - -class _Merger: - """A class to merge lists and add key/value - pairs to a dictionary""" - def mergeList(x, y, uniq=True): - l = [] - l.extend(x) - l.extend(y) - if not uniq: - return l - - s = Set(l) - l = list(s) - return l - - mergeList = staticmethod(mergeList) - - def mergeMap(to, add): - - for k in add: - to.setdefault(k, add[k]) - - return to - - mergeMap = staticmethod(mergeMap) - -class NodePoolDesc: - """a schema for describing - Nodepools""" - def __init__(self, dict): - self.dict = dict.copy() - - self.dict.setdefault('attrs', {}) - - self._checkRequired() - - if 'options' in dict: self.dict['attrs'] = dict['options'] - - def _checkRequired(self): - - if not 'id' in self.dict: - raise ValueError, "nodepool needs 'id'" - - if self.getPkgDir() == None: - raise ValueError, "nodepool %s needs 'pkgs'" % (self.getName()) - - def getName(self): - return self.dict['id'] - - def getPkgDir(self): - return self.dict['batch-home'] - - def getAttrs(self): - return self.dict['attrs'] - - def getSchema(): - schema = {} - - s = Schema('id') - schema[s.getName()] = s - - s = Schema('batch-home', Schema.LIST, ':') - schema[s.getName()] = s - - s = Schema('attrs', Schema.MAP) - schema[s.getName()] = s - - return schema - - getSchema = staticmethod(getSchema) - -class ServiceDesc: - """A schema for describing services""" - def __init__(self, dict): - self.dict = dict.copy() - - self.dict.setdefault('external', False) - self.dict.setdefault('attrs', {}) - self.dict.setdefault('envs', {}) - self.dict.setdefault('host',None) - self.dict.setdefault('port',None) - self.dict.setdefault('tar', None) - self.dict.setdefault('pkgs', '') - self.dict.setdefault('final-attrs', {}) - self._checkRequired() - if self.dict.has_key('hadoop-tar-ball'): - self.dict['tar'] = self.dict['hadoop-tar-ball'] - - def _checkRequired(self): - - if not 'id' in self.dict: - raise ValueError, "service description needs 'id'" - -# if len(self.getPkgDirs()) <= 0: -# raise ValueError, "service description %s needs 'pkgs'" % (self.getName()) - - def getName(self): - return self.dict['id'] - - def isExternal(self): - """True if the service is outside hod. - e.g. connect to existing HDFS""" - - return self.dict['external'] - - def getPkgDirs(self): - return self.dict['pkgs'] - - def getTar(self): - return self.dict['tar'] - - def getAttrs(self): - return self.dict['attrs'] - - def getfinalAttrs(self): - return self.dict['final-attrs'] - - def getEnvs(self): - return self.dict['envs'] - - def getSchema(): - schema = {} - - s = Schema('id') - schema[s.getName()] = s - - s = Schema('external') - schema[s.getName()] = s - - s = Schema('pkgs', Schema.LIST, ':') - schema[s.getName()] = s - - s = Schema('tar', Schema.LIST, ":") - schema[s.getName()] = s - - s = Schema('attrs', Schema.MAP) - schema[s.getName()] = s - - s = Schema('final-attrs', Schema.MAP) - schema[s.getName()] = s - - s = Schema('envs', Schema.MAP) - schema[s.getName()] = s - - return schema - - getSchema = staticmethod(getSchema) - -class CommandDesc: - - def __init__(self, dict): - """a class for how a command is described""" - self.dict = dict - - def __repr__(self): - return pformat(self.dict) - - def _getName(self): - """return the name of the command to be run""" - return self.dict['name'] - - def _getProgram(self): - """return where the program is """ - return self.dict['program'] - - def _getArgv(self): - """return the arguments for the command to be run""" - return self.dict['argv'] - - def _getEnvs(self): - """return the environment in which the command is to be run""" - return self.dict['envs'] - - def _getPkgDirs(self): - """return the packages for this command""" - return self.dict['pkgdirs'] - - def _getWorkDirs(self): - """return the working directories for this command""" - return self.dict['workdirs'] - - def _getAttrs(self): - """return the list of attributes for this command""" - return self.dict['attrs'] - - def _getfinalAttrs(self): - """return the final xml params list for this command""" - return self.dict['final-attrs'] - - def _getForeground(self): - """return if the command is to be run in foreground or not""" - return self.dict['fg'] - - def _getStdin(self): - return self.dict['stdin'] - - def toString(cmdDesc): - """return a string representation of this command""" - row = [] - row.append('name=%s' % (cmdDesc._getName())) - row.append('program=%s' % (cmdDesc._getProgram())) - row.append('pkgdirs=%s' % CommandDesc._csv(cmdDesc._getPkgDirs(), ':')) - - if 'argv' in cmdDesc.dict: - row.append('argv=%s' % CommandDesc._csv(cmdDesc._getArgv())) - - if 'envs' in cmdDesc.dict: - envs = cmdDesc._getEnvs() - list = [] - for k in envs: - v = envs[k] - list.append('%s=%s' % (k, v)) - row.append('envs=%s' % CommandDesc._csv(list)) - - if 'workdirs' in cmdDesc.dict: - row.append('workdirs=%s' % CommandDesc._csv(cmdDesc._getWorkDirs(), ':')) - - if 'attrs' in cmdDesc.dict: - attrs = cmdDesc._getAttrs() - list = [] - for k in attrs: - v = attrs[k] - list.append('%s=%s' % (k, v)) - row.append('attrs=%s' % CommandDesc._csv(list)) - - if 'final-attrs' in cmdDesc.dict: - fattrs = cmdDesc._getAttrs() - list = [] - for k in fattrs: - v = fattrs[k] - list.append('%s=%s' % (k, v)) - row.append('final-attrs=%s' % CommandDesc._cvs(list)) - - if 'fg' in cmdDesc.dict: - row.append('fg=%s' % (cmdDesc._getForeground())) - - if 'stdin' in cmdDesc.dict: - row.append('stdin=%s' % (cmdDesc._getStdin())) - - return CommandDesc._csv(row) - - toString = staticmethod(toString) - - def _csv(row, delim=','): - """return a string in csv format""" - import cStringIO - import csv - - queue = cStringIO.StringIO() - writer = csv.writer(queue, delimiter=delim, escapechar='\\', quoting=csv.QUOTE_NONE, - doublequote=False, lineterminator='\n') - writer.writerow(row) - return queue.getvalue().rstrip('\n') - - _csv = staticmethod(_csv) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/descGenerator.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/descGenerator.py deleted file mode 100644 index 03852cca64..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/descGenerator.py +++ /dev/null @@ -1,72 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -"""manage hod configuration""" -# -*- python -*- - -import sys, csv, os -from optparse import Option, OptionParser -from xml.dom import minidom -from sets import Set -from select import select, poll, POLLIN - -from hodlib.Common.desc import * - -class DescGenerator: - """Contains the conversion to descriptors and other method calls - to config""" - def __init__(self, hodConfig): - """parse all the descriptors""" - - self.hodConfig = hodConfig - - def initializeDesc(self): - self.hodConfig['nodepooldesc'] = self.createNodePoolDesc() - self.hodConfig['servicedesc'] = self.createServiceDescDict() - - return self.hodConfig - - def getServices(self): - """get all the services from the config""" - - sdd = {} - for keys in self.hodConfig: - if keys.startswith('gridservice-'): - str = keys.split('-') - dict = self.hodConfig[keys] - if 'server-params' in dict: dict['attrs'] = dict['server-params'] - if 'final-server-params' in dict: dict['final-attrs'] = dict['final-server-params'] - dict['id'] = str[1] - desc = ServiceDesc(dict) - sdd[desc.getName()] = desc - - return sdd - - def createNodePoolDesc(self): - """ create a node pool descriptor and store - it in hodconfig""" - - desc = NodePoolDesc(self.hodConfig['resource_manager']) - return desc - - def createServiceDescDict(self): - """create a service descriptor for - all the services and store it in the - hodconfig""" - - sdd = self.getServices() - return sdd - - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/hodsvc.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/hodsvc.py deleted file mode 100644 index e042fe13b7..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/hodsvc.py +++ /dev/null @@ -1,228 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -# $Id:setup.py 5158 2007-04-09 00:14:35Z zim $ -# -#------------------------------------------------------------------------------ -import os, time, shutil, xmlrpclib, socket, pprint - -from signal import * - -from hodlib.Common.logger import hodLog, hodDummyLogger -from hodlib.Common.socketServers import hodXMLRPCServer -from hodlib.Common.util import local_fqdn -from hodlib.Common.xmlrpc import hodXRClient - -class hodBaseService: - """hodBaseService class - This class provides service registration, logging, - and configuration access methods. It also provides an XML-RPC server. - This class should be extended to create hod services. Methods beginning - with _xr_method will automatically be added to instances of this class. - """ - def __init__(self, name, config, xrtype='threaded'): - """ Initialization requires a name string and a config object of type - hodlib.Common.setup.options or hodlib.Common.setup.config.""" - - self.name = name - self.hostname = local_fqdn() - self._cfg = config - self._xrc = None - self.logs = {} - self._baseLogger = None - self._serviceID = os.getenv('PBS_JOBID') - - self.__logDir = None - self.__svcrgy = None - self.__stop = False - self.__xrtype = xrtype - - self._init_logging() - - if name != 'serviceRegistry': self._init_signals() - self._init_xrc_server() - - def __set_logging_level(self, level): - self.logs['main'].info("Setting log level to %s." % level) - for loggerName in self.loggers.keys(): - self.logs['main'].set_logger_level(loggerName, level) - - def __get_logging_level(self): - if self._cfg.has_key('stream'): - return self.loggers['main'].get_level('stream', 'main') - elif self._cfg.has_key('log-dir'): - return self.loggers['main'].get_level('file', 'main') - else: - return 0 - - def _xr_method_stop(self, *args): - """XML-RPC method, calls stop() on ourselves.""" - - return self.stop() - - def _xr_method_status(self, *args): - """XML-RPC method, calls status() on ourselves.""" - - return self.status() - - def _init_logging(self): - if self._cfg.has_key('debug'): - if self._cfg['debug'] > 0: - self._baseLogger = hodLog(self.name) - self.logs['main'] = self._baseLogger.add_logger('main') - - if self._cfg.has_key('stream'): - if self._cfg['stream']: - self._baseLogger.add_stream(level=self._cfg['debug'], - addToLoggerNames=('main',)) - - if self._cfg.has_key('log-dir'): - if self._serviceID: - self.__logDir = os.path.join(self._cfg['log-dir'], "%s.%s" % ( - self._cfg['userid'], self._serviceID)) - else: - self.__logDir = os.path.join(self._cfg['log-dir'], - self._cfg['userid']) - if not os.path.exists(self.__logDir): - os.mkdir(self.__logDir) - - self._baseLogger.add_file(logDirectory=self.__logDir, - level=self._cfg['debug'], addToLoggerNames=('main',)) - - if self._cfg.has_key('syslog-address'): - self._baseLogger.add_syslog(self._cfg['syslog-address'], - level=self._cfg['debug'], addToLoggerNames=('main',)) - - if not self.logs.has_key('main'): - self.logs['main'] = hodDummyLogger() - else: - self.logs['main'] = hodDummyLogger() - else: - self.logs['main'] = hodDummyLogger() - - def _init_signals(self): - def sigStop(sigNum, handler): - self.sig_wrapper(sigNum, self.stop) - - def toggleLevel(): - currentLevel = self.__get_logging_level() - if currentLevel == 4: - self.__set_logging_level(1) - else: - self.__set_logging_level(currentLevel + 1) - - def sigStop(sigNum, handler): - self._sig_wrapper(sigNum, self.stop) - - def sigDebug(sigNum, handler): - self.sig_wrapper(sigNum, toggleLevel) - - signal(SIGTERM, sigStop) - signal(SIGQUIT, sigStop) - signal(SIGINT, sigStop) - signal(SIGUSR2, sigDebug) - - def _sig_wrapper(self, sigNum, handler, *args): - self.logs['main'].info("Caught signal %s." % sigNum) - - if args: - handler(args) - else: - handler() - - def _init_xrc_server(self): - host = None - ports = None - if self._cfg.has_key('xrs-address'): - (host, port) = (self._cfg['xrs-address'][0], self._cfg['xrs-address'][1]) - ports = (port,) - elif self._cfg.has_key('xrs-port-range'): - host = '' - ports = self._cfg['xrs-port-range'] - - if host != None: - if self.__xrtype == 'threaded': - self._xrc = hodXMLRPCServer(host, ports) - elif self.__xrtype == 'twisted': - try: - from socketServers import twistedXMLRPCServer - self._xrc = twistedXMLRPCServer(host, ports, self.logs['main']) - except ImportError: - self.logs['main'].error("Twisted XML-RPC server not available, " - + "falling back on threaded server.") - self._xrc = hodXMLRPCServer(host, ports) - for attr in dir(self): - if attr.startswith('_xr_method_'): - self._xrc.register_function(getattr(self, attr), - attr[11:]) - - self._xrc.register_introspection_functions() - - def _register_service(self, port=None, installSignalHandlers=1): - if self.__svcrgy: - self.logs['main'].info( - "Registering service with service registery %s... " % self.__svcrgy) - svcrgy = hodXRClient(self.__svcrgy, None, None, 0, 0, installSignalHandlers) - - if self._xrc and self._http: - svcrgy.registerService(self._cfg['userid'], self._serviceID, - self.hostname, self.name, 'hod', { - 'xrs' : "http://%s:%s" % ( - self._xrc.server_address[0], - self._xrc.server_address[1]),'http' : - "http://%s:%s" % (self._http.server_address[0], - self._http.server_address[1])}) - elif self._xrc: - svcrgy.registerService(self._cfg['userid'], self._serviceID, - self.hostname, self.name, 'hod', { - 'xrs' : "http://%s:%s" % ( - self._xrc.server_address[0], - self._xrc.server_address[1]),}) - elif self._http: - svcrgy.registerService(self._cfg['userid'], self._serviceID, - self.hostname, self.name, 'hod', {'http' : - "http://%s:%s" % (self._http.server_address[0], - self._http.server_address[1]),}) - else: - svcrgy.registerService(self._cfg['userid'], self._serviceID, - self.hostname, name, 'hod', {} ) - - def start(self): - """ Start XML-RPC server and register service.""" - - self.logs['main'].info("Starting HOD service: %s ..." % self.name) - - if self._xrc: self._xrc.serve_forever() - if self._cfg.has_key('register') and self._cfg['register']: - self._register_service() - - def stop(self): - """ Stop XML-RPC server, unregister service and set stop flag. """ - - self.logs['main'].info("Stopping service...") - if self._xrc: self._xrc.stop() - self.__stop = True - - return True - - def status(self): - """Returns true, should be overriden.""" - - return True - - def wait(self): - """Wait until stop method is called.""" - - while not self.__stop: - time.sleep(.1) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/logger.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/logger.py deleted file mode 100644 index 3101ab2cde..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/logger.py +++ /dev/null @@ -1,788 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -"""hodLogger provides a customized interface to Python's core logging package. -""" - -import sys, os, re, logging, logging.handlers, inspect, pprint, types -from tcp import get_address_tuple - -fileFormatString = '[%(asctime)s] %(levelname)s/%(levelno)s \ -%(module)s:%(lineno)s - %(message)s' - -streamFormatString = '%(levelname)s - %(message)s' - -debugStreamFormatString = '[%(asctime)s] %(levelname)s/%(levelno)s \ -%(module)s:%(lineno)s - %(message)s' - -syslogFormatString = '(%(process)d) %(levelname)s/%(levelno)s \ -%(module)s:%(lineno)s - %(message)s' - -smtpFormatString = '[%(asctime)s] %(levelname)s/%(levelno)s \ -%(module)s:%(lineno)s\n\n%(message)s' - -fileFormater = logging.Formatter(fileFormatString) -streamFormater = logging.Formatter(streamFormatString) -debugStreamFormater = logging.Formatter(debugStreamFormatString) -syslogFormater = logging.Formatter(syslogFormatString) -smtpFormater = logging.Formatter(smtpFormatString) - -defaultFileLevel = 3 -defaultStreamLevel = 4 -defaultSyslogLevel = 3 -defaultSmtpLevel = 0 - -hodLogLevelMap = { 0 : logging.CRITICAL, - 1 : logging.ERROR, - 2 : logging.WARNING, - 3 : logging.INFO, - 4 : logging.DEBUG } - -hodStreamFormatMap = { 0 : streamFormater, - 1 : streamFormater, - 2 : streamFormater, - 3 : streamFormater, - 4 : debugStreamFormater } - -rehodLogLevelMap = {} -for key in hodLogLevelMap.keys(): - rehodLogLevelMap[hodLogLevelMap[key]] = key - - -reModule = re.compile("^(.*)\..*$") - -hodLogs = {} - -class hodRotatingFileHandler(logging.handlers.RotatingFileHandler): - """ This class needs to be used in place of RotatingFileHandler when - the 2.4.0 Python interpreter is used.""" - - def emit(self, record): - """ - Emit a record. - - If a formatter is specified, it is used to format the record. - The record is then written to the stream with a trailing newline - [N.B. this may be removed depending on feedback]. If exception - information is present, it is formatted using - traceback.print_exception and appended to the stream. - - ***** - - THIS IS A HACK, when instances of hodLogger get passed to the child of - a child thread for some reason self.stream gets closed. This version - of emit re-opens self.stream if it is closed. After testing it appears - that self.stream is only closed once after the second thread is - initialized so there is not performance penalty to this hack. This - problem only exists in python 2.4. - - ***** - """ - try: - if self.shouldRollover(record): - self.doRollover() - try: - msg = self.format(record) - fs = "%s\n" - if not hasattr(types, "UnicodeType"): #if no unicode support... - self.stream.write(fs % msg) - else: - try: - self.stream.write(fs % msg) - except UnicodeError: - self.stream.write(fs % msg.encode("UTF-8")) - except ValueError: - self.stream = open(self.baseFilename, self.mode) - self.stream.write(fs % msg) - - self.flush() - except: - self.handleError(record) - except: - self.handleError(record) - - def shouldRollover(self, record): - """ - Determine if rollover should occur. - - Basically, see if the supplied record would cause the file to exceed - the size limit we have. - - ***** - - THIS IS A HACK, when instances of hodLogger get passed to the child of - a child thread for some reason self.stream gets closed. This version - of emit re-opens self.stream if it is closed. After testing it appears - that self.stream is only closed once after the second thread is - initialized so there is not performance penalty to this hack. This - problem only exists in python 2.4. - - ***** - """ - if self.maxBytes > 0: # are we rolling over? - msg = "%s\n" % self.format(record) - - try: - #due to non-posix-compliant Windows feature - self.stream.seek(0, 2) - except ValueError: - self.stream = open(self.baseFilename, self.mode) - self.stream.seek(0, 2) - - if self.stream.tell() + len(msg) >= self.maxBytes: - return 1 - return 0 - -class hodCustomLoggingLogger(logging.Logger): - """ Slight extension of the logging.Logger class used by the hodLog class. - """ - def findCaller(self): - """ findCaller() is supposed to return the callers file name and line - number of the caller. This was broken when the logging package was - wrapped by hodLog. We should return much more relevant info now. - """ - - callerModule = '' - callerLine = 0 - - currentModule = os.path.basename(__file__) - currentModule = reModule.sub("\g<1>", currentModule) - - frames = inspect.stack() - for i in range(len(frames)): - frameModule = os.path.basename(frames[i][1]) - frameModule = reModule.sub("\g<1>", frameModule) - if frameModule == currentModule: - previousFrameModule = os.path.basename(frames[i+1][1]) - previousFrameModule = reModule.sub("\g<1>", - previousFrameModule) - callerFile = frames[i+1][1] - callerLine = frames[i+1][2] - continue - - returnValues = (callerFile, callerLine) - if sys.version.startswith('2.4.4') or sys.version.startswith('2.5'): - returnValues = (callerFile, callerLine, None) - - return returnValues - -class hodLog: - """ Cluster management logging class. - - logging levels: 0 - log only critical messages - 1 - log critical and error messages - 2 - log critical, error, and warning messages - 3 - log critical, error, warning, and info messages - 4 - log critical, error, warning, info, and debug - messages""" - - def __init__(self, appName): - """Constructs a hodLogger object. - - appName - name of logging application, log filenames will be - prepended with this name""" - - self.__appName = appName - - # initialize a dictionary to hold loggerNames - self.__loggerNames = {} - - # initialize a dictionary to track log handlers and handler classes - self.__logObjs = { 'file' : {}, 'smtp' : {}, - 'syslog' : {}, 'strm' : {} } - - # use a custom logging.Logger class - logging.setLoggerClass(hodCustomLoggingLogger) - - # get the root app logger - self.__logger = logging.getLogger(appName) - self.__logger.setLevel(logging.DEBUG) - - hodLogs[self.__appName] = self - - def __attr__(self, attrname): - """loggerNames - list of defined logger names""" - - if attrname == "loggerNames": return self.__loggerNames.keys() - else: raise AttributeError, attrname - - def __repr__(self): - """Returns a string representation of a hodLog object of the form: - - LOG_NAME - file: FILENAME (level LEVEL) - smtp: SMTP_SERVER from FROM_ADDRESS (level LEVEL) - strm: STRM_OBJECT (level LEVEL) - ... """ - - hodLogString = "hodLog: %s\n\n" % self.__appName - for loggerName in self.__loggerNames.keys(): - hodLogString = "%s logger: %s\n" % (hodLogString, loggerName) - handlerClasses = self.__logObjs.keys() - handlerClasses.sort() - for handlerClass in handlerClasses: - try: - loggerLevelName = logging.getLevelName( - self.__logObjs[handlerClass][loggerName]['level']) - hodLogString = "%s %s: %s (level %s)\n" % ( - hodLogString, handlerClass, - self.__logObjs[handlerClass][loggerName]['data'], - loggerLevelName) - except: - hodLogString = "%s %s: none\n" % ( - hodLogString, handlerClass) - hodLogString = "%s\n" % hodLogString - - return hodLogString - - # 'private' method which adds handlers to self.__logObjs - def __add_to_handlers(self, handlerClass, loggerName, handler, data, - level): - self.__logObjs[handlerClass][loggerName] = {} - self.__logObjs[handlerClass][loggerName]['handler'] = handler - self.__logObjs[handlerClass][loggerName]['data'] = data - self.__logObjs[handlerClass][loggerName]['level'] = level - - # 'private' method which determines whether a hod log level is valid and - # returns a valid logging.Logger level - def __get_logging_level(self, level, defaultLevel): - loggingLevel = '' - try: - loggingLevel = hodLogLevelMap[int(level)] - except: - loggingLevel = hodLogLevelMap[defaultLevel] - - return loggingLevel - - # make a logging.logger name rootLogger.childLogger in our case the - # appName.componentName - def __get_logging_logger_name(self, loggerName): - return "%s.%s" % (self.__appName, loggerName) - - def add_logger(self, loggerName): - """Adds a logger of name loggerName. - - loggerName - name of component of a given application doing the - logging - - Returns a hodLogger object for the just added logger.""" - - try: - self.__loggerNames[loggerName] - except: - loggingLoggerName = self.__get_logging_logger_name(loggerName) - logging.getLogger(loggingLoggerName) - - self.__loggerNames[loggerName] = 1 - - return hodLogger(self.__appName, loggingLoggerName) - - def add_file(self, logDirectory, maxBytes=0, backupCount=0, - level=defaultFileLevel, addToLoggerNames=None): - """Adds a file handler to all defined loggers or a specified set of - loggers. Each log file will be located in logDirectory and have a - name of the form appName-loggerName.log. - - logDirectory - logging directory - maxBytes - maximum log size to write in bytes before rotate - backupCount - number of rotated logs to keep - level - cluster management log level - addToLoggerNames - list of logger names to which stream handling - will be added""" - - def add_file_handler(loggerName): - if not self.__logObjs['file'].has_key(loggerName): - loggingLevel = self.__get_logging_level(level, - defaultFileLevel) - - logFile = os.path.join(logDirectory, "%s-%s.log" % ( - self.__appName, loggerName)) - - logFilePresent = False - if(os.path.exists(logFile)): - logFilePresent = True - - if sys.version.startswith('2.4'): - fileHandler = hodRotatingFileHandler(logFile, - maxBytes=maxBytes, backupCount=backupCount) - else: - fileHandler = logging.handlers.RotatingFileHandler(logFile, - maxBytes=maxBytes, backupCount=backupCount) - if logFilePresent and backupCount: - fileHandler.doRollover() - - fileHandler.setLevel(loggingLevel) - fileHandler.setFormatter(fileFormater) - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - aLogger = logging.getLogger(loggingLoggerName) - aLogger.addHandler(fileHandler) - - fileData = "%s" % logFile - self.__add_to_handlers('file', loggerName, fileHandler, - fileData, loggingLevel) - - if addToLoggerNames: - for loggerName in addToLoggerNames: - add_file_handler(loggerName) - else: - for loggerName in self.__loggerNames: - add_file_handler(loggerName) - - def add_stream(self, stream=sys.stderr, level=defaultStreamLevel, - addToLoggerNames=None): - """Adds a stream handler to all defined loggers or a specified set of - loggers. - - stream - a stream such as sys.stderr or sys.stdout - level - cluster management log level - addToLoggerNames - tupple of logger names to which stream handling - will be added""" - - def add_stream_handler(loggerName): - if not self.__logObjs['strm'].has_key(loggerName): - loggingLevel = self.__get_logging_level(level, - defaultStreamLevel) - - streamHandler = logging.StreamHandler(stream) - - streamHandler.setLevel(loggingLevel) - - streamHandler.setFormatter(hodStreamFormatMap[int(level)]) - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - aLogger = logging.getLogger(loggingLoggerName) - aLogger.addHandler(streamHandler) - - streamData = "%s" % stream - self.__add_to_handlers('strm', loggerName, streamHandler, - streamData, loggingLevel) - - if addToLoggerNames: - for loggerName in addToLoggerNames: - add_stream_handler(loggerName) - else: - for loggerName in self.__loggerNames: - add_stream_handler(loggerName) - - def add_syslog(self, address, level=defaultSyslogLevel, - addToLoggerNames=None): - def add_syslog_handler(loggerName): - if not self.__logObjs['syslog'].has_key(loggerName): - loggingLevel = self.__get_logging_level(level, - defaultSyslogLevel) - - address[1] = int(address[1]) - syslogHandler = logging.handlers.SysLogHandler(tuple(address), - 9) - - syslogHandler.setLevel(loggingLevel) - - syslogHandler.setFormatter(syslogFormater) - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - aLogger = logging.getLogger(loggingLoggerName) - aLogger.addHandler(syslogHandler) - - syslogData = "%s:%s" % (address[0], address[1]) - self.__add_to_handlers('syslog', loggerName, syslogHandler, - syslogData, loggingLevel) - - if addToLoggerNames: - for loggerName in addToLoggerNames: - add_syslog_handler(loggerName) - else: - for loggerName in self.__loggerNames: - add_syslog_handler(loggerName) - - - def add_smtp(self, mailHost, fromAddress, toAddresses, - level=defaultSmtpLevel, addToLoggerNames=None): - """Adds an SMTP handler to all defined loggers or a specified set of - loggers. - - mailHost - SMTP server to used when sending mail - fromAddress - email address to use as the from address when - sending mail - toAdresses - comma seperated list of email address to which - mail will be sent - level - cluster management log level - addToLoggerNames - tupple of logger names to which smtp handling - will be added""" - - def add_email_handler(loggerName): - if not self.__logObjs['smtp'].has_key(loggerName): - loggingLevel = self.__get_logging_level(level, - defaultSmtpLevel) - - subject = loggerName - if loggingLevel == 50: - subject = "%s - a critical error has occured." % subject - elif loggingLevel == 40: - subject = "%s - an error has occured." % subject - elif loggingLevel == 30: - subject = "%s - warning message." % subject - elif loggingLevel == 20: - subject = "%s - information message." % subject - elif loggingLevel == 10: - subject = "%s - debugging message." % subject - - mailHostTuple = get_address_tuple(mailHost) - emailHandler = logging.handlers.SMTPHandler(mailHostTuple, - fromAddress, toAddresses, subject) - - emailHandler.setFormatter(smtpFormater) - emailHandler.setLevel(loggingLevel) - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - aLogger = logging.getLogger(loggingLoggerName) - aLogger.addHandler(emailHandler) - - emailData = "%s from %s" % (mailHost, fromAddress) - self.__add_to_handlers('smtp', loggerName, emailHandler, - emailData, loggingLevel) - - if addToLoggerNames: - for loggerName in addToLoggerNames: - add_email_handler(loggerName) - else: - for loggerName in self.__loggerNames: - add_email_handler(loggerName) - - def status(self): - statusStruct = {} - for loggerName in self.__loggerNames.keys(): - statusStruct[loggerName] = [] - for handlerClass in self.__logObjs.keys(): - loggerDict = {} - try: - level = self.__logObjs[handlerClass][loggerName]['level'] - level = rehodLogLevelMap[level] - - loggerDict['handler'] = handlerClass - loggerDict['level'] = level - loggerDict['data'] = \ - self.__logObjs[handlerClass][loggerName]['data'] - except: - pass - else: - statusStruct[loggerName].append(loggerDict) - - return statusStruct - - def lock_handlers(self): - for handlerClass in self.__logObjs.keys(): - for loggerName in self.__logObjs[handlerClass].keys(): - self.__logObjs[handlerClass][loggerName]['handler'].acquire() - - def release_handlers(self): - for handlerClass in self.__logObjs.keys(): - for loggerName in self.__logObjs[handlerClass].keys(): - self.__logObjs[handlerClass][loggerName]['handler'].release() - - def get_level(self, handler, loggerName): - return rehodLogLevelMap[self.__logObjs[handler][loggerName]['level']] - - def set_level(self, handler, loggerName, level): - """Sets the logging level of a particular logger and logger handler. - - handler - handler (smtp, file, or stream) - loggerName - logger to set level on - level - level to set logger - """ - - level = self.__get_logging_level(level, defaultFileLevel) - self.__logObjs[handler][loggerName]['handler'].setLevel(level) - self.__logObjs[handler][loggerName]['level'] = level - - if handler == 'stream': - self.__logObjs[handler][loggerName]['handler'].setFormatter( - hodStreamFormatMap[int(level)]) - - def set_logger_level(self, loggerName, level): - status = 0 - for handlerClass in self.__logObjs.keys(): - if self.__logObjs[handlerClass].has_key(loggerName): - self.set_level(handlerClass, loggerName, level) - else: - status = 1 - - return status - - def rollover(self, loggerName): - status = 0 - if self.__logObjs['file'].has_key(loggerName): - if self.__logObjs['file'][loggerName]['handler'].shouldRollover(): - self.__logObjs['file'][loggerName]['handler'].doRollover() - else: - status = 1 - - return status - - def set_max_bytes(self, maxBytes): - status = 0 - if self.__logObjs.has_key('file'): - for loggerName in self.__logObjs['file'].keys(): - self.__logObjs['file'][loggerName]['handler'].maxBytes = 0 - else: - status = 1 - - return status - - def get_logger(self, loggerName): - """ Returns a hodLogger object for a logger by name. """ - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - return hodLogger(self.__appName, loggingLoggerName) - - def critical(self, loggerName, msg): - """Logs a critical message and flushes log buffers. This method really - should only be called upon a catastrophic failure. - - loggerName - logger to use - msg - message to be logged""" - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - logger = logging.getLogger(loggingLoggerName) - logger.critical(msg) - self.flush() - - def error(self, loggerName, msg): - """Logs an error message and flushes log buffers. - - loggerName - logger to use - msg - message to be logged""" - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - logger = logging.getLogger(loggingLoggerName) - logger.error(msg) - self.flush() - - def warn(self, loggerName, msg): - """Logs a warning message. - - loggerName - logger to use - msg - message to be logged""" - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - logger = logging.getLogger(loggingLoggerName) - logger.warn(msg) - - def info(self, loggerName, msg): - """Logs an information message. - - loggerName - logger to use - msg - message to be logged""" - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - logger = logging.getLogger(loggingLoggerName) - logger.info(msg) - - def debug(self, loggerName, msg): - """Logs a debugging message. - - loggerName - logger to use - msg - message to be logged""" - - loggingLoggerName = self.__get_logging_logger_name(loggerName) - logger = logging.getLogger(loggingLoggerName) - logger.debug(msg) - - def flush(self): - """Flush all log handlers.""" - - for handlerClass in self.__logObjs.keys(): - for loggerName in self.__logObjs[handlerClass].keys(): - self.__logObjs[handlerClass][loggerName]['handler'].flush() - - def shutdown(self): - """Shutdown all logging, flushing all buffers.""" - - for handlerClass in self.__logObjs.keys(): - for loggerName in self.__logObjs[handlerClass].keys(): - self.__logObjs[handlerClass][loggerName]['handler'].flush() - # Causes famous 'ValueError: I/O operation on closed file' - # self.__logObjs[handlerClass][loggerName]['handler'].close() - -class hodLogger: - """ Encapsulates a particular logger from a hodLog object. """ - def __init__(self, appName, loggingLoggerName): - """Constructs a hodLogger object (a particular logger in a hodLog - object). - - loggingLoggerName - name of a logger in hodLog object""" - - self.__appName = appName - self.__loggerName = loggingLoggerName - self.__logger = logging.getLogger(self.__loggerName) - - def __repr__(self): - """Returns a string representation of a hodComponentLog object.""" - - return "%s hodLog" % self.__loggerName - - def __call__(self): - pass - - def set_logger_level(self, loggerName, level): - - return hodLogs[self.__appName].set_logger_level(loggerName, level) - - def set_max_bytes(self, maxBytes): - - return hodLogs[self.__appName].set_max_bytes(maxBytes) - - def rollover(self): - return hodLogs[self.__appName].rollover(self.__loggerName) - - def get_level(self, handler, loggerName): - - return hodLogs[self.__appName].get_level(handler, loggerName) - - def critical(self, msg): - """Logs a critical message and calls sys.exit(1). - - msg - message to be logged""" - - self.__logger.critical(msg) - - def error(self, msg): - """Logs an error message. - - msg - message to be logged""" - - self.__logger.error(msg) - - def warn(self, msg): - """Logs a warning message. - - msg - message to be logged""" - - self.__logger.warn(msg) - - def info(self, msg): - """Logs an information message. - - msg - message to be logged""" - - self.__logger.info(msg) - - def debug(self, msg): - """Logs a debugging message. - - msg - message to be logged""" - - self.__logger.debug(msg) - -class hodDummyLogger: - """ Dummy hodLogger class. Other hod classes requiring a hodLogger default - to this hodLogger if no logger is passed.""" - - def __init__(self): - """pass""" - - pass - - def __repr__(self): - return "dummy hodLogger" - - def __call__(self): - """pass""" - - pass - - def set_logger_level(self, loggerName, level): - - return 0 - - def set_max_bytes(self, loggerName, maxBytes): - - return 0 - - def get_level(self, handler, loggerName): - - return 4 - - def rollover(self): - - return 0 - - def critical(self, msg): - """pass""" - - pass - - def error(self, msg): - """pass""" - - pass - - def warn(self, msg): - """pass""" - - pass - - def info(self, msg): - """pass""" - - pass - - def debug(self, msg): - """pass""" - - pass - -def ensureLogDir(logDir): - """Verify that the passed in log directory exists, and if it doesn't - create it.""" - if not os.path.exists(logDir): - try: - old_mask = os.umask(0) - os.makedirs(logDir, 01777) - os.umask(old_mask) - except Exception, e: - print >>sys.stderr, "Could not create log directories %s. Exception: %s. Stack Trace: %s" % (logDir, get_exception_error_string(), get_exception_string()) - raise e - -def getLogger(cfg, logName): - if cfg['debug'] > 0: - user = cfg['userid'] - baseLogger = hodLog(logName) - log = baseLogger.add_logger('main') - - if cfg.has_key('log-dir'): - serviceId = os.getenv('PBS_JOBID') - if serviceId: - logDir = os.path.join(cfg['log-dir'], "%s.%s" % (user, serviceId)) - else: - logDir = os.path.join(cfg['log-dir'], user) - if not os.path.exists(logDir): - os.mkdir(logDir) - - baseLogger.add_file(logDirectory=logDir, level=cfg['debug'], - addToLoggerNames=('main',)) - - try: - if cfg.has_key('stream') and cfg['stream']: - baseLogger.add_stream(level=cfg['debug'], addToLoggerNames=('main',)) - - if cfg.has_key('syslog-address'): - baseLogger.add_syslog(cfg['syslog-address'], - level=cfg['debug'], addToLoggerNames=('main',)) - except Exception,e: - # Caught an exception while initialising logger - log.critical("%s Logger failed to initialise. Reason : %s" % (logName, e)) - pass - return log diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/miniHTMLParser.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/miniHTMLParser.py deleted file mode 100644 index 34a0fd0124..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/miniHTMLParser.py +++ /dev/null @@ -1,45 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -import urllib, urlparse, re - -from HTMLParser import HTMLParser - -class miniHTMLParser( HTMLParser ): - - viewedQueue = [] - instQueue = [] - - def setBaseUrl(self, url): - self.baseUrl = url - - def getNextLink( self ): - if self.instQueue == []: - return None - else: - return self.instQueue.pop(0) - - def handle_starttag( self, tag, attrs ): - if tag == 'a': - newstr = urlparse.urljoin(self.baseUrl, str(attrs[0][1])) - if re.search('mailto', newstr) != None: - return - - if (newstr in self.viewedQueue) == False: - self.instQueue.append( newstr ) - self.viewedQueue.append( newstr ) - - - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/nodepoolutil.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/nodepoolutil.py deleted file mode 100644 index d733780ec1..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/nodepoolutil.py +++ /dev/null @@ -1,26 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -from hodlib.NodePools.torque import TorquePool - -class NodePoolUtil: - def getNodePool(nodePoolDesc, cfg, log): - """returns a concrete instance of NodePool as configured by 'cfg'""" - npd = nodePoolDesc - name = npd.getName() - if name == 'torque': - return TorquePool(npd, cfg, log) - - getNodePool = staticmethod(getNodePool) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/setup.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/setup.py deleted file mode 100644 index 791b095c9b..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/setup.py +++ /dev/null @@ -1,1058 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -# $Id:setup.py 5158 2007-04-09 00:14:35Z zim $ -# $Id:setup.py 5158 2007-04-09 00:14:35Z zim $ -# -#------------------------------------------------------------------------------ - -"""'setup' provides for reading and verifing configuration files based on - Python's SafeConfigParser class.""" - -import sys, os, re, pprint - -from ConfigParser import SafeConfigParser -from optparse import OptionParser, IndentedHelpFormatter, OptionGroup -from util import get_perms, replace_escapes -from types import typeValidator, typeValidatorInstance, is_valid_type, \ - typeToString -from hodlib.Hod.hod import hodHelp - -reEmailAddress = re.compile("^.*@.*$") -reEmailDelimit = re.compile("@") -reComma = re.compile("\s*,\s*") -reDot = re.compile("\.") -reCommentHack = re.compile("^.*?\s+#|;.*", flags=re.S) -reCommentNewline = re.compile("\n|\r$") -reKeyVal = r"(? 1: - statusMsgs.append( "%s: %s problems found." % ( - errorPrefix, errorCount)) - self.var_error_suggest(statusMsgs) - status = False - elif errorCount > 0: - statusMsgs.append( "%s: %s problem found." % ( - errorPrefix, errorCount)) - self.var_error_suggest(statusMsgs) - status = False - - self.__validated = True - - if self.__originalDir: - os.chdir(oldDir) - - return status,statusMsgs - - def normalizeValue(self, section, option) : - return typeValidatorInstance.normalize( - self._configDef[section][option]['type'], - self[section][option]) - - def validateValue(self, section, option): - # Validates a section.option and exits on error - valueInfo = typeValidatorInstance.verify( - self._configDef[section][option]['type'], - self[section][option]) - if valueInfo['isValid'] == 1: - return [] - else: - if valueInfo['errorData']: - return self.var_error(section, option, valueInfo['errorData']) - else: - return self.var_error(section, option) - -class config(SafeConfigParser, baseConfig): - def __init__(self, configFile, configDef=None, originalDir=None, - options=None, checkPerms=False): - """Constructs config object. - - configFile - configuration file to read - configDef - definition object - options - options object - checkPerms - check file permission on config file, 0660 - - sample configuration file: - - [snis] - modules_dir = modules/ ; location of infoModules - md5_defs_dir = etc/md5_defs ; location of infoTree md5 defs - info_store = var/info ; location of nodeInfo store - cam_daemon = localhost:8200 ; cam daemon address""" - - - SafeConfigParser.__init__(self) - baseConfig.__init__(self, configDef, originalDir) - - if(os.path.exists(configFile)): - self.configFile = configFile - else: - raise IOError - - self._options = options - - ## UNUSED CODE : checkPerms is never True - ## zim: this code is used if one instantiates config() with checkPerms set to - ## True. - if checkPerms: self.__check_perms() - - self.read(configFile) - - self._configDef = configDef - if not self._configDef: - self._mySections = self.sections() - - self.__initialize_config_dict() - - def __initialize_config_dict(self): - """ build a dictionary of config vars keyed by section name defined in - configDef, if options defined override config""" - - for section in self._mySections: - items = self.items(section) - self._dict[section] = {} - - # First fill self._dict with whatever is given in hodrc. - # Going by this, options given at the command line either override - # options in hodrc, or get appended to the list, like for - # hod.client-params. Note that after this dict has _only_ hodrc - # params - for keyValuePair in items: - # stupid commenting bug in ConfigParser class, lines without an - # option value pair or section required that ; or # are at the - # beginning of the line, :( - newValue = reCommentHack.sub("", keyValuePair[1]) - newValue = reCommentNewline.sub("", newValue) - self._dict[section][keyValuePair[0]] = newValue - # end of filling with options given in hodrc - # now start filling in command line options - if self._options: - for option in self._configDef[section].keys(): - if self._options[section].has_key(option): - # the user has given an option - compoundOpt = "%s.%s" %(section,option) - if ( compoundOpt == \ - 'gridservice-mapred.final-server-params' \ - or compoundOpt == \ - 'gridservice-hdfs.final-server-params' \ - or compoundOpt == \ - 'gridservice-mapred.server-params' \ - or compoundOpt == \ - 'gridservice-hdfs.server-params' \ - or compoundOpt == \ - 'hod.client-params' ): - - if ( compoundOpt == \ - 'gridservice-mapred.final-server-params' \ - or compoundOpt == \ - 'gridservice-hdfs.final-server-params' ): - overwrite = False - else: overwrite = True - - # Append to the current list of values in self._dict - if not self._dict[section].has_key(option): - self._dict[section][option] = "" - dictOpts = reKeyValList.split(self._dict[section][option]) - dictOptsKeyVals = {} - for opt in dictOpts: - if opt != '': - # when dict _has_ params from hodrc - if reKeyVal.search(opt): - (key, val) = reKeyVal.split(opt,1) - # we only consider the first '=' for splitting - # we do this to support passing params like - # mapred.child.java.opts=-Djava.library.path=some_dir - # Even in case of an invalid error like unescaped '=', - # we don't want to fail here itself. We leave such errors - # to be caught during validation which happens after this - dictOptsKeyVals[key] = val - else: - # this means an invalid option. Leaving it - #for config.verify to catch - dictOptsKeyVals[opt] = None - - cmdLineOpts = reKeyValList.split(self._options[section][option]) - - for opt in cmdLineOpts: - if reKeyVal.search(opt): - # Same as for hodrc options. only consider - # the first = - ( key, val ) = reKeyVal.split(opt,1) - else: - key = opt - val = None - # whatever is given at cmdline overrides - # what is given in hodrc only for non-final params - if dictOptsKeyVals.has_key(key): - if overwrite: - dictOptsKeyVals[key] = val - else: dictOptsKeyVals[key] = val - - self._dict[section][option] = "" - for key in dictOptsKeyVals: - if self._dict[section][option] == "": - if dictOptsKeyVals[key]: - self._dict[section][option] = key + "=" + \ - dictOptsKeyVals[key] - else: #invalid option. let config.verify catch - self._dict[section][option] = key - else: - if dictOptsKeyVals[key]: - self._dict[section][option] = \ - self._dict[section][option] + "," + key + \ - "=" + dictOptsKeyVals[key] - else: #invalid option. let config.verify catch - self._dict[section][option] = \ - self._dict[section][option] + "," + key - - else: - # for rest of the options, that don't need - # appending business. - # options = cmdline opts + defaults - # dict = hodrc opts only - # only non default opts can overwrite any opt - # currently in dict - if not self._dict[section].has_key(option): - # options not mentioned in hodrc - self._dict[section][option] = \ - self._options[section][option] - elif self._configDef[section][option]['default'] != \ - self._options[section][option]: - # option mentioned in hodrc but user has given a - # non-default option - self._dict[section][option] = \ - self._options[section][option] - - ## UNUSED METHOD - ## zim: is too :) - def __check_perms(self): - perms = None - if self._options: - try: - perms = get_perms(self.configFile) - except OSError, data: - self._options.print_help() - raise Exception("*** could not find config file: %s" % data) - sys.exit(1) - else: - perms = get_perms(self.configFile) - - if perms != requiredPerms: - error = "*** '%s' has invalid permission: %s should be %s\n" % \ - (self.configFile, perms, requiredPerms) - raise Exception( error) - sys.exit(1) - - def replace_escape_seqs(self): - """ replace any escaped characters """ - replace_escapes(self) - -class formatter(IndentedHelpFormatter): - def format_option_strings(self, option): - """Return a comma-separated list of option strings & metavariables.""" - if option.takes_value(): - metavar = option.metavar or option.dest.upper() - short_opts = [sopt - for sopt in option._short_opts] - long_opts = [self._long_opt_fmt % (lopt, metavar) - for lopt in option._long_opts] - else: - short_opts = option._short_opts - long_opts = option._long_opts - - if self.short_first: - opts = short_opts + long_opts - else: - opts = long_opts + short_opts - - return ", ".join(opts) - -class options(OptionParser, baseConfig): - - def __init__(self, optionDef, usage, version, originalDir=None, - withConfig=False, defaultConfig=None, defaultLocation=None, - name=None): - """Constructs and options object. - - optionDef - definition object - usage - usage statement - version - version string - withConfig - used in conjunction with a configuration file - defaultConfig - default configuration file - - """ - OptionParser.__init__(self, usage=usage) - baseConfig.__init__(self, optionDef, originalDir) - - self.formatter = formatter(4, max_help_position=100, width=180, - short_first=1) - - self.__name = name - self.__version = version - self.__withConfig = withConfig - self.__defaultConfig = defaultConfig - self.__defaultLoc = defaultLocation - self.args = [] - self.__optionList = [] - self.__compoundOpts = [] - self.__shortMap = {} - self.__alphaString = 'abcdefghijklmnopqrstuvxyzABCDEFGHIJKLMNOPQRSTUVXYZ1234567890' - self.__alpha = [] - self.__parsedOptions = {} - self.__reserved = [ 'h' ] - - self.__orig_grps = [] - self.__orig_grp_lists = {} - self.__orig_option_list = [] - - self.__display_grps = [] - self.__display_grp_lists = {} - self.__display_option_list = [] - - self.config = None - - if self.__withConfig: - self.__reserved.append('c') - self.__reserved.append('v') - - self.__gen_alpha() - - # build self.__optionList, so it contains all the options that are - # possible. the list elements are of the form section.option - for section in self._mySections: - if self.__withConfig and section == 'config': - raise Exception( - "withConfig set 'config' cannot be used as a section name") - for option in self._configDef[section].keys(): - if '.' in option: - raise Exception("Options cannot contain: '.'") - elif self.__withConfig and option == 'config': - raise Exception( - "With config set, option config is not allowed.") - elif self.__withConfig and option == 'verbose-help': - raise Exception( - "With config set, option verbose-help is not allowed.") - self.__optionList.append(self.__splice_compound(section, - option)) - self.__build_short_map() - self.__add_options() - self.__init_display_options() - - (self.__parsedOptions, self.args) = self.parse_args() - - # Now process the positional arguments only for the client side - if self.__name == 'hod': - - hodhelp = hodHelp() - - _operation = getattr(self.__parsedOptions,'hod.operation') - _script = getattr(self.__parsedOptions, 'hod.script') - nArgs = self.args.__len__() - if _operation: - # -o option is given - if nArgs != 0: - self.error('invalid syntax : command and operation(-o) cannot coexist') - elif nArgs == 0 and _script: - # for a script option, without subcommand: hod -s script ... - pass - elif nArgs == 0: - print "Usage: ",hodhelp.help() - sys.exit(0) - else: - # subcommand is given - cmdstr = self.args[0] # the subcommand itself - cmdlist = hodhelp.ops - if cmdstr not in cmdlist: - print "Usage: ", hodhelp.help() - sys.exit(2) - - numNodes = None - clusterDir = None - # Check which subcommand. cmdstr = subcommand itself now. - if cmdstr == "allocate": - clusterDir = getattr(self.__parsedOptions, 'hod.clusterdir') - numNodes = getattr(self.__parsedOptions, 'hod.nodecount') - - if not clusterDir or not numNodes: - print hodhelp.usage(cmdstr) - sys.exit(3) - - cmdstr = cmdstr + ' ' + clusterDir + ' ' + numNodes - - setattr(self.__parsedOptions,'hod.operation', cmdstr) - - elif cmdstr == "deallocate" or cmdstr == "info": - clusterDir = getattr(self.__parsedOptions, 'hod.clusterdir') - - if not clusterDir: - print hodhelp.usage(cmdstr) - sys.exit(3) - - cmdstr = cmdstr + ' ' + clusterDir - setattr(self.__parsedOptions,'hod.operation', cmdstr) - - elif cmdstr == "list": - setattr(self.__parsedOptions,'hod.operation', cmdstr) - pass - - elif cmdstr == "script": - clusterDir = getattr(self.__parsedOptions, 'hod.clusterdir') - numNodes = getattr(self.__parsedOptions, 'hod.nodecount') - originalDir = getattr(self.__parsedOptions, 'hod.original-dir') - - if originalDir and clusterDir: - self.remove_exit_code_file(originalDir, clusterDir) - - if not _script or not clusterDir or not numNodes: - print hodhelp.usage(cmdstr) - sys.exit(3) - pass - - elif cmdstr == "help": - if nArgs == 1: - self.print_help() - sys.exit(0) - elif nArgs != 2: - self.print_help() - sys.exit(3) - elif self.args[1] == 'options': - self.print_options() - sys.exit(0) - cmdstr = cmdstr + ' ' + self.args[1] - setattr(self.__parsedOptions,'hod.operation', cmdstr) - - # end of processing for arguments on the client side - - if self.__withConfig: - self.config = self.__parsedOptions.config - if not self.config: - self.error("configuration file must be specified") - if not os.path.isabs(self.config): - # A relative path. Append the original directory which would be the - # current directory at the time of launch - try: - origDir = getattr(self.__parsedOptions, 'hod.original-dir') - if origDir is not None: - self.config = os.path.join(origDir, self.config) - self.__parsedOptions.config = self.config - except AttributeError, e: - self.error("hod.original-dir is not defined.\ - Cannot get current directory") - if not os.path.exists(self.config): - if self.__defaultLoc and not re.search("/", self.config): - self.__parsedOptions.config = os.path.join( - self.__defaultLoc, self.config) - self.__build_dict() - - def norm_cluster_dir(self, orig_dir, directory): - directory = os.path.expanduser(directory) - if not os.path.isabs(directory): - directory = os.path.join(orig_dir, directory) - directory = os.path.abspath(directory) - - return directory - - def remove_exit_code_file(self, orig_dir, dir): - try: - dir = self.norm_cluster_dir(orig_dir, dir) - if os.path.exists(dir): - exit_code_file = os.path.join(dir, "script.exitcode") - if os.path.exists(exit_code_file): - os.remove(exit_code_file) - except: - print >>sys.stderr, "Could not remove the script.exitcode file." - - def __init_display_options(self): - self.__orig_option_list = self.option_list[:] - optionListTitleMap = {} - for option in self.option_list: - optionListTitleMap[option._long_opts[0]] = option - - self.__orig_grps = self.option_groups[:] - for group in self.option_groups: - self.__orig_grp_lists[group.title] = group.option_list[:] - - groupTitleMap = {} - optionTitleMap = {} - for group in self.option_groups: - groupTitleMap[group.title] = group - optionTitleMap[group.title] = {} - for option in group.option_list: - (sectionName, optionName) = \ - self.__split_compound(option._long_opts[0]) - optionTitleMap[group.title][optionName] = option - - for section in self._mySections: - for option in self._configDef[section]: - if self._configDef[section][option]['help']: - if groupTitleMap.has_key(section): - if not self.__display_grp_lists.has_key(section): - self.__display_grp_lists[section] = [] - self.__display_grp_lists[section].append( - optionTitleMap[section][option]) - - try: - self.__display_option_list.append( - optionListTitleMap["--" + self.__splice_compound( - section, option)]) - except KeyError: - pass - try: - self.__display_option_list.append(optionListTitleMap['--config']) - except KeyError: - pass - - self.__display_option_list.append(optionListTitleMap['--help']) - self.__display_option_list.append(optionListTitleMap['--verbose-help']) - self.__display_option_list.append(optionListTitleMap['--version']) - - self.__display_grps = self.option_groups[:] - for section in self._mySections: - if self.__display_grp_lists.has_key(section): - self.__orig_grp_lists[section] = \ - groupTitleMap[section].option_list - else: - try: - self.__display_grps.remove(groupTitleMap[section]) - except KeyError: - pass - - def __gen_alpha(self): - assignedOptions = [] - for section in self._configDef: - for option in self._configDef[section]: - if self._configDef[section][option]['short']: - assignedOptions.append( - self._configDef[section][option]['short']) - - for symbol in self.__alphaString: - if not symbol in assignedOptions: - self.__alpha.append(symbol) - - def __splice_compound(self, section, option): - return "%s.%s" % (section, option) - - def __split_compound(self, compound): - return compound.split('.') - - def __build_short_map(self): - """ build a short_map of parametername : short_option. This is done - only for those parameters that don't have short options already - defined in configDef. - If possible, the first letter in the option that is not already - used/reserved as a short option is allotted. Otherwise the first - letter in __alpha that isn't still used is allotted. - e.g. { 'hodring.java-home': 'T', 'resource_manager.batch-home': 'B' } - """ - - optionsKey = {} - for compound in self.__optionList: - (section, option) = self.__split_compound(compound) - if not optionsKey.has_key(section): - optionsKey[section] = [] - optionsKey[section].append(option) - - for section in self._configDef.sections(): - options = optionsKey[section] - options.sort() - for option in options: - if not self._configDef[section][option]['short']: - compound = self.__splice_compound(section, option) - shortOptions = self.__shortMap.values() - for i in range(0, len(option)): - letter = option[i] - letter = letter.lower() - if letter in self.__alpha: - if not letter in shortOptions and \ - not letter in self.__reserved: - self.__shortMap[compound] = letter - break - if not self.__shortMap.has_key(compound): - for i in range(0, len(self.__alpha)): - letter = self.__alpha[i] - if not letter in shortOptions and \ - not letter in self.__reserved: - self.__shortMap[compound] = letter - - def __add_option(self, config, compoundOpt, section, option, group=None): - addMethod = self.add_option - if group: addMethod=group.add_option - - self.__compoundOpts.append(compoundOpt) - - if compoundOpt == 'gridservice-mapred.final-server-params' or \ - compoundOpt == 'gridservice-hdfs.final-server-params' or \ - compoundOpt == 'gridservice-mapred.server-params' or \ - compoundOpt == 'gridservice-hdfs.server-params' or \ - compoundOpt == 'hod.client-params': - _action = 'append' - elif config[section][option]['type'] == 'bool': - _action = 'store_true' - else: - _action = 'store' - - if self.__shortMap.has_key(compoundOpt): - addMethod("-" + self.__shortMap[compoundOpt], - "--" + compoundOpt, dest=compoundOpt, - action= _action, - metavar=config[section][option]['type'], - default=config[section][option]['default'], - help=config[section][option]['desc']) - else: - if config[section][option]['short']: - addMethod("-" + config[section][option]['short'], - "--" + compoundOpt, dest=compoundOpt, - action= _action, - metavar=config[section][option]['type'], - default=config[section][option]['default'], - help=config[section][option]['desc']) - else: - addMethod('', "--" + compoundOpt, dest=compoundOpt, - action= _action, - metavar=config[section][option]['type'], - default=config[section][option]['default'], - help=config[section][option]['desc']) - - def __add_options(self): - if self.__withConfig: - self.add_option("-c", "--config", dest='config', - action='store', default=self.__defaultConfig, - metavar='config_file', - help="Full path to configuration file.") - - self.add_option("", "--verbose-help", - action='help', default=None, - metavar='flag', - help="Display verbose help information.") - - self.add_option("-v", "--version", - action='version', default=None, - metavar='flag', - help="Display version information.") - - self.version = self.__version - - if len(self._mySections) > 1: - for section in self._mySections: - group = OptionGroup(self, section) - for option in self._configDef[section]: - compoundOpt = self.__splice_compound(section, option) - self.__add_option(self._configDef, compoundOpt, section, - option, group) - self.add_option_group(group) - else: - for section in self._mySections: - for option in self._configDef[section]: - compoundOpt = self.__splice_compound(section, option) - self.__add_option(self._configDef, compoundOpt, section, - option) - - def __build_dict(self): - if self.__withConfig: - self._dict['config'] = str(getattr(self.__parsedOptions, 'config')) - for compoundOption in dir(self.__parsedOptions): - if compoundOption in self.__compoundOpts: - (section, option) = self.__split_compound(compoundOption) - if not self._dict.has_key(section): - self._dict[section] = {} - - if getattr(self.__parsedOptions, compoundOption): - _attr = getattr(self.__parsedOptions, compoundOption) - # when we have multi-valued parameters passed separately - # from command line, python optparser pushes them into a - # list. So converting all such lists to strings - if type(_attr) == type([]): - import string - _attr = string.join(_attr,',') - self._dict[section][option] = _attr - - for section in self._configDef: - for option in self._configDef[section]: - if self._configDef[section][option]['type'] == 'bool': - compoundOption = self.__splice_compound(section, option) - if not self._dict.has_key(section): - self._dict[section] = {} - - if option not in self._dict[section]: - self._dict[section][option] = False - - def __set_display_groups(self): - if not '--verbose-help' in sys.argv: - self.option_groups = self.__display_grps - self.option_list = self.__display_option_list - for group in self.option_groups: - group.option_list = self.__display_grp_lists[group.title] - - def __unset_display_groups(self): - if not '--verbose-help' in sys.argv: - self.option_groups = self.__orig_grps - self.option_list = self.__orig_option_list - for group in self.option_groups: - group.option_list = self.__orig_grp_lists[group.title] - - def print_help(self, file=None): - self.__set_display_groups() - OptionParser.print_help(self, file) - self.__unset_display_groups() - - def print_options(self): - _usage = self.usage - self.set_usage('') - self.print_help() - self.set_usage(_usage) - - def verify(self): - return baseConfig.verify(self) - - def replace_escape_seqs(self): - replace_escapes(self) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/socketServers.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/socketServers.py deleted file mode 100644 index 72dbd69569..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/socketServers.py +++ /dev/null @@ -1,621 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -# Various socket server and helper classes. -# -# -import os, sys, socket, threading, pprint, re, xmlrpclib, time - -from select import select -from SocketServer import ThreadingMixIn, ForkingMixIn -from BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer -from SimpleXMLRPCServer import SimpleXMLRPCRequestHandler, SimpleXMLRPCServer -from SimpleHTTPServer import SimpleHTTPRequestHandler -from random import Random -from urlparse import urlparse - -Fault = xmlrpclib.Fault - -from hodlib.Common.util import local_fqdn -from hodlib.Common.logger import hodDummyLogger - -class hodHTTPHandler(BaseHTTPRequestHandler): - port = -1 - - def __init__(self, request, client_address, server, registerService): - self.registerService = registerService - BaseHTTPRequestHandler.__init__(self, request, client_address, server) - - def log_message(self, *args): - """Forget logging for now.""" - - pass - - def do_GET(self): - self.fullUrl = "http://%s:%s%s" % (self.server.server_address[0], - self.server.server_address[1], - self.path) - - parsedUrl = urlparse(self.fullUrl) - self.writeHeaders() - self.writeData(parsedUrl) - - def w(self, string): - self.wfile.write("%s\n" % string) - - def writeHeaders(self): - self.send_response(200, 'OK') - self.send_header('Content-type', 'text/html') - self.end_headers() - - def sendWrongPage(self, userJob): - self.w('') - if userJob == False: - self.w('invalid URL specified') - elif re.match("^\d+$", userJob): - self.w('invalid URL specified, job %s does not exist' % userJob) - elif re.match("^\w+$", userJob): - self.w('invalid URL specified, user %s does not exist' % userJob) - self.w('') - - def getServiceHosts(self, serviceInfo): - hostInfo = { 'long' : {}, 'short' : {} } - for user in serviceInfo: - for job in serviceInfo[user]: - for host in serviceInfo[user][job]: - for serviceItem in serviceInfo[user][job][host]: - serviceName = serviceItem.keys() - serviceName = serviceName[0] - if isinstance(serviceItem[serviceName], str): - hostInfo['short'][self.getJobKey(user, job, host)] = True - hostInfo['long'][self.getJobKey(user, job, host)] = True - - return hostInfo - - def getJobInfo(self, job, serviceInfo): - jobInfo = {} - - for user in serviceInfo.keys(): - for someJob in serviceInfo[user].keys(): - if job == someJob: - jobInfo[user] = { job : serviceInfo[user][job] } - - return jobInfo - - def getJobKey(self, user, job, host): - return "%s-%s-%s" % (user, job, host) - - def writeData(self, parsedUrl): - options = parsedUrl[4] - serviceInfo = self.server.service.getServiceInfo() - users = serviceInfo.keys() - users.sort() - - self.w("") - self.w("") - self.w("") - self.writeCSS() - self.w("") - self.w('HOD Service Registry Information') - if serviceInfo == {}: - self.w('

  No HOD clusters configured.') - else: - if parsedUrl[2] == '/': - self.w('   ') - count = 0 - for user in users: - self.writeUserData(user, options, serviceInfo, count) - count = count + 1 - elif parsedUrl[2][1:] in serviceInfo: - self.w('   
') - self.writeUserData(parsedUrl[2][1:], options, serviceInfo, 0) - elif re.match("^\d+$", parsedUrl[2][1:]): - jobInfo = self.getJobInfo(parsedUrl[2][1:], serviceInfo) - if jobInfo.keys(): - self.w('   
') - for user in jobInfo.keys(): - self.writeUserData(user, options, jobInfo, 0) - else: - self.sendWrongPage(parsedUrl[2][1:]) - self.w('   
') - count = 0 - for user in users: - self.writeUserData(user, options, serviceInfo, count) - count = count + 1 - elif re.match("^\w+$", parsedUrl[2][1:]): - self.sendWrongPage(parsedUrl[2][1:]) - self.w('   
') - count = 0 - for user in users: - self.writeUserData(user, options, serviceInfo, count) - count = count + 1 - else: - self.sendWrongPage(False) - self.w('   
') - count = 0 - for user in users: - self.writeUserData(user, options, serviceInfo, count) - count = count + 1 - - self.w('
') - self.w("") - self.w("") - self.w("") - - def writeCSS(self): - self.w('") - - def writeUserData(self, user, options, serviceInfo, count): - hostInfo = self.getServiceHosts(serviceInfo) - hostKey = 'short' - if options == 'display=long': - hostKey = 'long' - - if count == 0: - self.w('') - self.w('') - self.w('Active Users') - self.w('') - self.w('') - self.w('') - self.w('%s' % user) - self.w('') - jobIDs = serviceInfo[user].keys() - jobIDs.sort() - for jobID in jobIDs: - self.w('') - if count == 0: - self.w('') - self.w('') - self.w('') - self.w('') - self.w('' % jobID) - self.w('') - self.w('') - self.w('
') - self.w('PBS Job Identifiers') - self.w('
%s') - hosts = serviceInfo[user][jobID].keys() - hosts.sort() - for host in hosts: - if hostInfo[hostKey].has_key(self.getJobKey(user, jobID, host)): - self.w('') - if count == 0: - self.w('') - self.w('') - self.w('') - self.w('') - self.w('' % host) - self.w('') - self.w('') - self.w('
') - self.w('Hosts Running Services') - self.w('
%s') - self.w('') - self.w('') - self.w('') - self.w('') - for serviceItem in serviceInfo[user][jobID][host]: - serviceName = serviceItem.keys() - serviceName = serviceName[0] - if isinstance(serviceItem[serviceName], dict) and \ - options == 'display=long': - self.w('') - self.w('' % serviceName) - self.w('') - self.w('') - elif isinstance(serviceItem[serviceName], str): - self.w('') - self.w('' % serviceName) - self.w('') - self.w('') - self.w('
') - self.w('Service Information') - self.w('
%s') - self.w('') - for key in serviceItem[serviceName]: - self.w('') - self.w('' % key) - self.w('' % serviceItem[serviceName][key]) - self.w('') - self.w('
%s%s
') - self.w('
%s') - (host, port) = serviceItem[serviceName].split(':') - hostnameInfo = socket.gethostbyname_ex(host) - if serviceName.startswith('mapred'): - self.w('Hadoop Job Tracker' % (hostnameInfo[0], port)) - elif serviceName.startswith('hdfs'): - self.w('HDFS Name Node ' % (hostnameInfo[0], port)) - else: - self.w('%s' % serviceItem[serviceName]) - self.w('
') - self.w('
') - count = count + 1 - self.w('
') - count = count + 1 - self.w('') - self.w('') -# self.w("
")
-#    self.w(pprint.pformat(serviceInfo))
-#    self.w("
") - -class baseSocketServer: - def __init__(self, host, ports): - self.host = host - self.ports = ports - self.__stopForever = threading.Event() - self.__stopForever.clear() - self.__run = threading.Event() - self.__run.set() - self.server_address = () - self.mThread = None - - def server_bind(self): - """server_bind() method binds to a random range of ports.""" - - self.socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) - - if len(self.ports) > 1: - randomPort = Random(os.getpid()) - portSequence = range(self.ports[0], self.ports[1]) - - maxTryCount = abs(self.ports[0] - self.ports[1]) - tryCount = 0 - while True: - somePort = randomPort.choice(portSequence) - self.server_address = (self.host, somePort) - try: - self.socket.bind(self.server_address) - except socket.gaierror, errData: - raise socket.gaierror, errData - except: - tryCount = tryCount + 1 - if tryCount > maxTryCount: - bindError = "bind failure for port range %s:%d" % ( - self.ports) - - raise socket.error, bindError - else: - break - else: - self.server_address = (self.host, int(self.ports[0])) - self.socket.bind(self.server_address) - - if self.host == '': - self.server_address = (local_fqdn(), self.server_address[1]) - - def _serve_forever(self): - """Replacement for serve_forever loop. - - All baseSocketServers run within a master thread; that thread - imitates serve_forever, but checks an event (self.__stopForever) - before processing new connections. - """ - - while not self.__stopForever.isSet(): - (rlist, wlist, xlist) = select([self.socket], [], [], - 1) - - if (len(rlist) > 0 and self.socket == rlist[0]): - self.handle_request() - - while not self.__run.isSet(): - if self.__stopForever.isSet(): - break - time.sleep(1) - - self.server_close() - - return True - - def serve_forever(self): - """Handle requests until stopForever event flag indicates stop.""" - - self.mThread = threading.Thread(name="baseSocketServer", - target=self._serve_forever) - self.mThread.start() - - return self.mThread - - def pause(self): - """Temporarily stop servicing requests.""" - - self.__run.clear() - - def cont(self): - """Resume servicing requests.""" - - self.__run.set() - - def stop(self): - """Set the stopForever flag to tell serve_forever() to exit.""" - - self.__stopForever.set() - if self.mThread: self.mThread.join() - return True - - def is_alive(self): - if self.mThread != None: - return self.mThread.isAlive() - else: - return False - -class threadedHTTPServer(baseSocketServer, ThreadingMixIn, HTTPServer): - def __init__(self, host, ports): - baseSocketServer.__init__(self, host, ports) - HTTPServer.__init__(self, self.server_address, SimpleHTTPRequestHandler) - -class forkingHTTPServer(baseSocketServer, ForkingMixIn, HTTPServer): - def __init__(self, host, ports): - baseSocketServer.__init__(self, host, ports) - HTTPServer.__init__(self, self.server_address, SimpleHTTPRequestHandler) - -class hodHTTPServer(baseSocketServer, ThreadingMixIn, HTTPServer): - service = None - def __init__(self, host, ports, serviceobj = None): - self.service = serviceobj - baseSocketServer.__init__(self, host, ports) - HTTPServer.__init__(self, self.server_address, hodHTTPHandler) - - def finish_request(self, request, client_address): - self.RequestHandlerClass(request, client_address, self, self.service) - -class hodXMLRPCServer(baseSocketServer, ThreadingMixIn, SimpleXMLRPCServer): - def __init__(self, host, ports, - requestHandler=SimpleXMLRPCRequestHandler, - logRequests=False, allow_none=False, encoding=None): - baseSocketServer.__init__(self, host, ports) - SimpleXMLRPCServer.__init__(self, self.server_address, requestHandler, - logRequests) - - self.register_function(self.stop, 'stop') - -try: - from twisted.web import server, xmlrpc - from twisted.internet import reactor, defer - from twisted.internet.threads import deferToThread - from twisted.python import log - - class twistedXMLRPC(xmlrpc.XMLRPC): - def __init__(self, logger): - xmlrpc.XMLRPC.__init__(self) - - self.__XRMethods = {} - self.__numRequests = 0 - self.__logger = logger - self.__pause = False - - def render(self, request): - request.content.seek(0, 0) - args, functionPath = xmlrpclib.loads(request.content.read()) - try: - function = self._getFunction(functionPath) - except Fault, f: - self._cbRender(f, request) - else: - request.setHeader("content-type", "text/xml") - defer.maybeDeferred(function, *args).addErrback( - self._ebRender).addCallback(self._cbRender, request) - - return server.NOT_DONE_YET - - def _cbRender(self, result, request): - if isinstance(result, xmlrpc.Handler): - result = result.result - if not isinstance(result, Fault): - result = (result,) - try: - s = xmlrpclib.dumps(result, methodresponse=1) - except: - f = Fault(self.FAILURE, "can't serialize output") - s = xmlrpclib.dumps(f, methodresponse=1) - request.setHeader("content-length", str(len(s))) - request.write(s) - request.finish() - - def _ebRender(self, failure): - if isinstance(failure.value, Fault): - return failure.value - log.err(failure) - return Fault(self.FAILURE, "error") - - def _getFunction(self, methodName): - while self.__pause: - time.sleep(1) - - self.__numRequests = self.__numRequests + 1 - function = None - try: - def defer_function(*args): - return deferToThread(self.__XRMethods[methodName], - *args) - function = defer_function - self.__logger.info( - "[%s] processing defered XML-RPC call to: %s ..." % - (self.__numRequests, methodName)) - except KeyError: - self.__logger.warn( - "[%s] fault %s on XML-RPC call to %s, method not found." % ( - self.__numRequests, self.NOT_FOUND, methodName)) - raise xmlrpc.NoSuchFunction(self.NOT_FOUND, - "method %s not found" % methodName) - - return function - - def register_function(self, functionRef, methodName): - self.__XRMethods[methodName] = functionRef - - def list_methods(self): - return self.__XRMethods.keys() - - def num_requests(self): - return self.__numRequests - - def pause(self): - self.__pause = True - - def cont(self): - self.__pause = False - - class twistedXMLRPCServer: - def __init__(self, host, ports, logger=None, threadPoolSize=100): - self.__host = host - self.__ports = ports - - if logger == None: - logger = hodDummyLogger() - - self.__logger = logger - - self.server_address = ['', ''] - reactor.suggestThreadPoolSize(threadPoolSize) - - self.__stopForever = threading.Event() - self.__stopForever.clear() - self.__mThread = None - - self.__xmlrpc = twistedXMLRPC(self.__logger) - - def _serve_forever(self): - if len(self.__ports) > 1: - randomPort = Random(os.getpid()) - portSequence = range(self.__ports[0], self.__ports[1]) - - maxTryCount = abs(self.__ports[0] - self.__ports[1]) - tryCount = 0 - while True: - somePort = randomPort.choice(portSequence) - self.server_address = (self.__host, int(somePort)) - if self.__host == '': - self.server_address = (local_fqdn(), self.server_address[1]) - try: - reactor.listenTCP(int(somePort), server.Site( - self.__xmlrpc), interface=self.__host) - reactor.run(installSignalHandlers=0) - except: - self.__logger.debug("Failed to bind to: %s:%s." % ( - self.__host, somePort)) - tryCount = tryCount + 1 - if tryCount > maxTryCount: - self.__logger.warn("Failed to bind to: %s:%s" % ( - self.__host, self.__ports)) - sys.exit(1) - else: - break - else: - try: - self.server_address = (self.__host, int(self.__ports[0])) - if self.__host == '': - self.server_address = (local_fqdn(), self.server_address[1]) - reactor.listenTCP(int(self.__ports[0]), server.Site(self.__xmlrpc), - interface=self.__host) - reactor.run(installSignalHandlers=0) - except: - self.__logger.warn("Failed to bind to: %s:%s."% ( - self.__host, self.__ports[0])) - sys.exit(1) - - def serve_forever(self): - """Handle requests until stopForever event flag indicates stop.""" - - self.__mThread = threading.Thread(name="XRServer", - target=self._serve_forever) - self.__mThread.start() - - if not self.__mThread.isAlive(): - raise Exception("Twisted XMLRPC server thread dead.") - - def register_function(self, functionRef, methodName): - self.__xmlrpc.register_function(functionRef, methodName) - - def register_introspection_functions(self): - pass - - def register_instance(self, instance): - for method in dir(instance): - if not method.startswith('_'): - self.register_function(getattr(instance, method), method) - - def pause(self): - self.__xmlrpc.pause() - - def cont(self): - self.__xmlrpc.cont() - - def stop(self): - def stop_thread(): - time.sleep(2) - reactor.stop() - - self.__stopForever.set() - - stopThread = threading.Thread(name='XRStop', target=stop_thread) - stopThread.start() - - return True - - def is_alive(self): - status = False - if reactor.running == 1: - status = True - - return status - - def status(self): - """Return status information on running XMLRPC Server.""" - stat = { 'XR server address' : self.server_address, - 'XR methods' : self.system_listMethods(), - 'XR server alive' : self.is_alive(), - 'XR requests processed' : self.__xmlrpc.num_requests(), - 'XR server stop flag' : self.__stopForever.isSet()} - return(stat) - - def system_listMethods(self): - return self.__xmlrpc.list_methods() - - def get_server_address(self): - waitCount = 0 - while self.server_address == '': - if waitCount == 9: - break - time.sleep(1) - waitCount = waitCount + 1 - - return self.server_address -except ImportError: - pass diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/tcp.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/tcp.py deleted file mode 100644 index a118a67f9c..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/tcp.py +++ /dev/null @@ -1,176 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -# $Id:tcp.py 6172 2007-05-22 20:26:54Z zim $ -# -#------------------------------------------------------------------------------ - -""" TCP related classes. """ - -import socket, re, string -reAddress = re.compile(":") -reMayBeIp = re.compile("^\d+\.\d+\.\d+\.\d+$") -reValidPort = re.compile("^\d+$") - -class Error(Exception): - def __init__(self, msg=''): - self.message = msg - Exception.__init__(self, msg) - - def __repr__(self): - return self.message - -class tcpError(Error): - def __init__(self, message): - Error.__init__(self, message) - -class tcpSocket: - def __init__(self, address, timeout=30, autoflush=0): - """Constructs a tcpSocket object. - - address - standard tcp address (HOST:PORT) - timeout - socket timeout""" - - self.address = address - self.__autoFlush = autoflush - self.__remoteSock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - self.__remoteSock.settimeout(timeout) - self.host = None - self.port = None - splitAddress = address - if isinstance(address, (tuple, list)): - self.host = address[0] - self.port = int(address[1]) - else: - splitAddress = get_address_tuple(address) - if not splitAddress[0]: - self.host = 'localhost' - else: - self.host = splitAddress[0] - - self.port = int(splitAddress[1]) - - self.__fileObjectOut = '' - self.__fileObjectIn = '' - - def __repr__(self): - return self.address - - def __iter__(self): - return self - - def next(self): - sockLine = self.read() - if not sockLine: - raise StopIteration - - return sockLine - - def open(self): - """Attempts to open a socket to the specified address.""" - - socketAddress = (self.host, self.port) - - try: - self.__remoteSock.connect(socketAddress) - if self.__autoFlush: - self.__fileObjectOut = self.__remoteSock.makefile('wb', 0) - else: - self.__fileObjectOut = self.__remoteSock.makefile('wb') - - self.__fileObjectIn = self.__remoteSock.makefile('rb', 0) - except: - raise tcpError, "connection failure: %s" % self.address - - def flush(self): - """Flushes write buffer.""" - self.__fileObjectOut.flush() - - def close(self): - """Attempts to close and open socket connection""" - - try: - self.__remoteSock.close() - self.__fileObjectOut.close() - self.__fileObjectIn.close() - except socket.error, exceptionObject: - exceptionMessage = "close failure %s %s" % (self.address, - exceptionObject.__str__()) - raise tcpError, exceptionMessage - - def verify(self): - """Verifies that a given IP address/host and port are valid. This - method will not attempt to open a socket to the specified address. - """ - - isValidAddress = False - if reMayBeIp.match(self.host): - if check_ip_address(self.host): - if reValidPort.match(str(self.port)): - isValidAddress = True - else: - if reValidPort.match(str(self.port)): - isValidAddress = True - - return(isValidAddress) - - def read(self): - """Reads a line off of the active socket.""" - - return self.__fileObjectIn.readline() - - def write(self, string): - """Writes a string to the active socket.""" - - print >> self.__fileObjectOut, string - -def check_net_address(address): - valid = True - pieces = string.split(address, '.') - if len(pieces) != 4: - valid = False - else: - for piece in pieces: - if int(piece) < 0 or int(piece) > 255: - valid = False - - return valid - -def check_ip_address(address): - valid = True - pieces = string.split(address, '.') - if len(pieces) != 4: - valid = False - else: - if int(pieces[0]) < 1 or int(pieces[0]) > 254: - valid = False - for i in range(1,4): - if int(pieces[i]) < 0 or int(pieces[i]) > 255: - valid = False - - return valid - -def get_address_tuple(address): - """ Returns an address tuple for TCP address. - - address - TCP address of the form host:port - - returns address tuple (host, port) - """ - - addressList = reAddress.split(address) - addressTuple = (addressList[0], int(addressList[1])) - - return addressTuple diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/threads.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/threads.py deleted file mode 100644 index 0d19042074..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/threads.py +++ /dev/null @@ -1,389 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. - -import threading, time, os, sys, pprint - -from popen2 import Popen4, Popen3, MAXFD -from signal import SIGTERM, SIGKILL - -class baseThread(threading.Thread): - """Base CAM threading class. The run method should be overridden.""" - - def __init__(self, name): - threading.Thread.__init__(self, name=name) - self.stopFlag = threading.Event() - self.stopFlag.clear() - self.running = threading.Event() - self.running.set() - self.isFinished = threading.Event() - self.isFinished.clear() - - def join(self, timeout=None): - self.stopFlag.set() - threading.Thread.join(self, timeout) - - def pause(self): - """Pause thread.""" - - self.running.clear() - - def cont(self): - """Resume thread operation.""" - - self.running.set() - -class simpleCommand(baseThread): - """Command execution object. Command output and exit status are captured. - - Public class attributes: - - cmdString - command to be executed - outputBuffer - command output, stdout + stderr - status - exit status, as returned by wait - - stdin - standard input for command - stdout - standard output of command when buffer == False - stderr - standard error of command when mode == 3 and buffer == False - - """ - - def __init__(self, name, cmdString, env=os.environ, mode=4, buffer=True, - wait=True, chdir=None): - """Class initialization. - - name - thread name to use when running the command - cmdString - command string to execute - inputString - string to print to command's stdin - env - shell environment dictionary - mode - 3 for popen3 and 4 for popen4 - buffer - out put to be retrieved with output() method - wait - return immediately after start() is called and output - command results as they come to stdout""" - - baseThread.__init__(self, name=name) - - self.cmdString = cmdString - self.__mode = mode - self.__buffer = buffer - self.__wait = wait - self.__chdir = chdir - self.__outputBuffer = [] - self.__status = None - self.__pid = None - self.__isFinished = threading.Event() - self.__isFinished.clear() - - self.stdin = None - self.stdout = None - self.stderr = None - - self.__env = env - - def run(self): - """ Overridden run method. Most of the work happens here. start() - should be called in place of this method.""" - - oldDir = None - if self.__chdir: - if os.path.exists(self.__chdir): - oldDir = os.getcwd() - os.chdir(self.__chdir) - else: - raise Exception( - "simpleCommand: invalid chdir specified: %s" % - self.__chdir) - - cmd = None - if self.__mode == 3: - cmd = _Popen3Env(self.cmdString, env=self.__env) - else: - cmd = _Popen4Env(self.cmdString, env=self.__env) - self.__pid = cmd.pid - - self.stdin = cmd.tochild - - if self.__mode == 3: - self.stderr = cmd.childerr - - while cmd.fromchild == None: - time.sleep(1) - - if self.__buffer == True: - output = cmd.fromchild.readline() - while output != '': - while not self.running.isSet(): - if self.stopFlag.isSet(): - break - time.sleep(1) - self.__outputBuffer.append(output) - output = cmd.fromchild.readline() - - elif self.__wait == False: - output = cmd.fromchild.readline() - while output != '': - while not self.running.isSet(): - if self.stopFlag.isSet(): - break - time.sleep(1) - print output, - if self.stopFlag.isSet(): - break - output = cmd.fromchild.readline() - else: - self.stdout = cmd.fromchild - - self.__status = cmd.poll() - while self.__status == -1: - while not self.running.isSet(): - if self.stopFlag.isSet(): - break - time.sleep(1) - - self.__status = cmd.poll() - time.sleep(1) - - if oldDir: - os.chdir(oldDir) - - self.__isFinished.set() - - sys.exit(0) - - def getPid(self): - """return pid of the launches process""" - return self.__pid - - def output(self): - return self.__outputBuffer[:] - - def wait(self): - """Wait blocking until command execution completes.""" - - self.__isFinished.wait() - - return os.WEXITSTATUS(self.__status) - - def is_running(self): - """Returns boolean, are we running?""" - - status = True - if self.__isFinished.isSet(): - status = False - - return status - - def exit_code(self): - """ Returns process exit code.""" - - if self.__status != None: - return os.WEXITSTATUS(self.__status) - else: - return None - - def exit_status_string(self): - """Return a string representation of the command's exit status.""" - - statusString = None - if self.__status: - exitStatus = os.WEXITSTATUS(self.__status) - exitSignal = os.WIFSIGNALED(self.__status) - coreDump = os.WCOREDUMP(self.__status) - - statusString = "exit code: %s | signal: %s | core %s" % \ - (exitStatus, exitSignal, coreDump) - - return(statusString) - - def stop(self): - """Stop the running command and join it's execution thread.""" - - self.join() - - def kill(self): - count = 0 - while self.is_running(): - try: - if count > 20: - os.kill(self.__pid, SIGKILL) - break - else: - os.kill(self.__pid, SIGTERM) - except: - break - - time.sleep(.1) - count = count + 1 - - self.stop() - -class _Popen3Env(Popen3): - def __init__(self, cmd, capturestderr=False, bufsize=-1, env=os.environ): - self._env = env - Popen3.__init__(self, cmd, capturestderr, bufsize) - - def _run_child(self, cmd): - if isinstance(cmd, basestring): - cmd = ['/bin/sh', '-c', cmd] - for i in xrange(3, MAXFD): - try: - os.close(i) - except OSError: - pass - - try: - os.execvpe(cmd[0], cmd, self._env) - finally: - os._exit(1) - -class _Popen4Env(_Popen3Env, Popen4): - childerr = None - - def __init__(self, cmd, bufsize=-1, env=os.environ): - self._env = env - Popen4.__init__(self, cmd, bufsize) - -class loop(baseThread): - """ A simple extension of the threading.Thread class which continuously - executes a block of code until join(). - """ - - def __init__(self, name, functionRef, functionArgs=None, sleep=1, wait=0, - offset=False): - """Initialize a loop object. - - name - thread name - functionRef - a function reference - functionArgs - function arguments in the form of a tuple, - sleep - time to wait between function execs - wait - time to wait before executing the first time - offset - set true to sleep as an offset of the start of the - last func exec instead of the end of the last func - exec - """ - - self.__functionRef = functionRef - self.__functionArgs = functionArgs - self.__sleep = sleep - self.__wait = wait - self.__offset = offset - - baseThread.__init__(self, name=name) - - def run(self): - """Do not call this directly. Call self.start().""" - - startTime = None - while not self.stopFlag.isSet(): - sleep = self.__sleep - if self.__wait > 0: - startWaitCount = 0 - while not self.stopFlag.isSet(): - while not self.running.isSet(): - if self.stopFlag.isSet(): - break - time.sleep(1) - time.sleep(0.5) - startWaitCount = startWaitCount + .5 - if startWaitCount >= self.__wait: - self.__wait = 0 - break - startTime = time.time() - - if not self.stopFlag.isSet(): - if self.running.isSet(): - if self.__functionArgs: - self.__functionRef(self.__functionArgs) - else: - self.__functionRef() - endTime = time.time() - - while not self.running.isSet(): - time.sleep(1) - - while not self.stopFlag.isSet(): - while not self.running.isSet(): - if self.stopFlag.isSet(): - break - time.sleep(1) - - currentTime = time.time() - if self.__offset: - elapsed = time.time() - startTime - else: - elapsed = time.time() - endTime - - if elapsed >= self.__sleep: - break - - time.sleep(0.5) - - self.isFinished.set() - - def set_sleep(self, sleep, wait=None, offset=None): - """Modify loop frequency paramaters. - - sleep - time to wait between function execs - wait - time to wait before executing the first time - offset - set true to sleep as an offset of the start of the - last func exec instead of the end of the last func - exec - """ - - self.__sleep = sleep - if wait != None: - self.__wait = wait - if offset != None: - self.__offset = offset - - def get_sleep(self): - """Get loop frequency paramaters. - Returns a dictionary with sleep, wait, offset. - """ - - return { - 'sleep' : self.__sleep, - 'wait' : self.__wait, - 'offset' : self.__offset, - } - -class func(baseThread): - """ A simple extension of the threading.Thread class which executes - a function in a separate thread. - """ - - def __init__(self, name, functionRef, functionArgs=None): - """Initialize a func object. - - name - thread name - functionRef - a function reference - functionArgs - function arguments in the form of a tuple, - """ - - self.__functionRef = functionRef - self.__functionArgs = functionArgs - - baseThread.__init__(self, name=name) - - def run(self): - """Do not call this directly. Call self.start().""" - - if not self.stopFlag.isSet(): - if self.running.isSet(): - if self.__functionArgs: - self.__functionRef(self.__functionArgs) - else: - self.__functionRef() - sys.exit(0) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/types.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/types.py deleted file mode 100644 index 9612ce4313..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/types.py +++ /dev/null @@ -1,1266 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -# $Id:types.py 6172 2007-05-22 20:26:54Z zim $ -# -#------------------------------------------------------------------------------ - -""" Higher level data types and type related classes. - - Supported Types (Verification and Display): - - address - validates ip:port and host:port tcp addresses - ip_address - validates and IP address - net_address - validates an IP like address, ie netmask - hostname - validates a hostname with DNS - eaddress - validates a single email address or a comma - seperated list of email addresses - http_version - validates a value is a http version (1.0/1.1) - tcp_port - validates a value to be a valid tcp port (2-65535) - bool - validates value is (0, 1, true, false) / converts - true -> 1 and false -> 0 - directory - validates a values is a directory / resolves path to - absolute path - file - validates a value is a file / resolves path to absolute - path - float - validates a value is a float, converts string to float - pos_float - validates a value is a float and >= 0, converts string - to float - pos_num - same as pos_float - neg_float - validates a value is a float and < 0, converts string to - float - int - validates a value is an integer, converts string to - integer - pos_int - validates a value is an integer and >= 0, converts - string to integer - neg_int - validates a values is an integer and < 0, converts - striing to integer - freq - frequency, positive integer - size - validates a size in bytes, kb, mb, kb, and tb - (int > 0 post fixed with K, M, G, or T) also converts - value to integer bytes - range - numeric range, x-y normalized to a tuple, if a single - number is supplie a single element tuple is returned - timestamp - utc timestamp of the form YYYYMMDDHHMMSS - user_account - UNIX user account name - user_group - UNIX group name - string - arbitrarily long string - list - comma seperated list of strings of arbitrary length, - keyval - comma seperated list of key=value pairs, key does not - need to be unique. - uri - a uri """ - -import sys, os, socket, pwd, grp, stat, re, re, string, pprint, urlparse - -from tcp import tcpSocket, check_net_address, check_ip_address -from util import check_timestamp - -types = { 'directory' : { 'db' : 'string', - 'units' : None }, - - 'address' : { 'db' : 'string', - 'units' : None }, - - 'ip_address' : { 'db' : 'string', - 'units' : None }, - - 'net_address' : { 'db' : 'string', - 'units' : None }, - - 'bool' : { 'db' : 'bool', - 'units' : None }, - - 'int' : { 'db' : 'integer', - 'units' : None }, - - 'float' : { 'db' : 'float', - 'units' : None }, - - 'pos_int' : { 'db' : 'integer', - 'units' : None }, - - 'neg_int' : { 'db' : 'integer', - 'units' : None }, - - 'pos_num' : { 'db' : 'float', - 'units' : None }, - - 'pos_float' : { 'db' : 'float', - 'units' : None }, - - 'neg_float' : { 'db' : 'float', - 'units' : None }, - - 'string' : { 'db' : 'string', - 'units' : None }, - - 'list' : { 'db' : 'string', - 'units' : None }, - - 'file' : { 'db' : 'string', - 'units' : None }, - - 'size' : { 'db' : 'integer', - 'units' : 'bytes' }, - - 'freq' : { 'db' : 'integer', - 'units' : 'hz' }, - - 'eaddress' : { 'db' : 'string', - 'units' : None }, - - 'tcp_port' : { 'db' : 'integer', - 'units' : None }, - - 'http_version' : { 'db' : 'float', - 'units' : None }, - - 'range' : { 'db' : 'string', - 'units' : None }, - - 'hostname' : { 'db' : 'string', - 'units' : None }, - - 'user_account' : { 'db' : 'string', - 'units' : None }, - - 'user_group' : { 'db' : 'string', - 'units' : None }, - - 'timestamp' : { 'db' : 'timestamp', - 'units' : None }, - - 'keyval' : { 'db' : 'string', - 'units' : None }, - - 'uri' : { 'db' : 'string', - 'units' : None }, - - '' : { 'db' : 'string', - 'units' : None }} - -dbTypes = { 'string' : { 'type' : 'varchar', - 'store' : 'type_strings_0', - 'table' : True }, - - 'integer' : { 'type' : 'bigint', - 'store' : 'integers', - 'table' : False }, - - 'float' : { 'type' : 'real', - 'store' : 'floats', - 'table' : False }, - - 'bool' : { 'type' : 'boolean', - 'store' : 'bools', - 'table' : False }, - - 'timestamp' : { 'type' : 'timestamp(0)', - 'store' : 'timestamps', - 'table' : False }} - -reSizeFormat = re.compile("^(\d+)(k|m|g|t|p|kb|mb|gb|tb|pb)$", flags=2) -reDash = re.compile("\s*-\s*") - -sizeFactors = { 'b' : 1, - 'bytes' : 1, - 'k' : 1024, - 'kb' : 1024, - 'm' : 1048576, - 'mb' : 1048576, - 'g' : 1073741824, - 'gb' : 1073741824, - 't' : 1099511627776, - 'tb' : 1099511627776, - 'p' : 1125899906842624, - 'pb' : 1125899906842624 } - -freqFactors = { 'hz' : 1, - 'khz' : 1000, - 'mhz' : 1000000, - 'ghz' : 1000000000, - 'thz' : 1000000000000, - 'phz' : 1000000000000000 } - -sizeMap = [ { 'factor' : sizeFactors['b'], - 'long' : 'byte', - 'short' : 'byte' }, - - { 'factor' : sizeFactors['k'], - 'long' : 'Kilobyte', - 'short' : 'KB' }, - - { 'factor' : sizeFactors['m'], - 'long' : 'Megabyte', - 'short' : 'MB' }, - - { 'factor' : sizeFactors['g'], - 'long' : 'Gigabyte', - 'short' : 'GB' }, - - { 'factor' : sizeFactors['t'], - 'long' : 'Terabyte', - 'short' : 'TB' }, - - { 'factor' : sizeFactors['p'], - 'long' : 'Petabyte', - 'short' : 'PB' } ] - -freqMap = [ { 'factor' : freqFactors['hz'], - 'long' : 'Hertz', - 'short' : 'Hz' }, - - { 'factor' : freqFactors['khz'], - 'long' : 'Kilohertz', - 'short' : 'KHz' }, - - { 'factor' : freqFactors['mhz'], - 'long' : 'Megahertz', - 'short' : 'MHz' }, - - { 'factor' : freqFactors['ghz'], - 'long' : 'Gigahertz', - 'short' : 'GHz' }, - - { 'factor' : freqFactors['thz'], - 'long' : 'Terahertz', - 'short' : 'THz' }, - - { 'factor' : freqFactors['phz'], - 'long' : 'Petahertz', - 'short' : 'PHz' } ] - -reListString = r"(?, - 'name' : 'SA_COMMON.old_xml_dir', - 'value': 'var/data/old' }, - - { 'func' : , - 'name' : 'SA_COMMON.log_level', - 'value': '4' } ] - - validatedInfo = [ { # name supplied to add() - 'name' : 'SA_COMMON.tmp_xml_dir', - - # is valid or not - 'isValid' : 1 - - # normalized value - 'normalized' : /var/data/tmp, - - # error string ? - 'errorData' : 0 }, - - { 'name' : 'SA_COMMON.new_xml_dir', - 'isValid' : 1 - 'normalized' : /var/data/new, - 'errorData' : 0 } ]""" - - if attrname == "validateList": - return self.validateList # list of items to be validated - elif attrname == "validatedInfo": - return self.validatedInfo # list of validation results - else: raise AttributeError, attrname - - def __build_verify_functions(self): - functions = {} - for function in dir(self): - functions[function] = 1 - - for type in types.keys(): - # kinda bad, need to find out how to know the name of the class - # I'm in. But it works. - functionName = "_typeValidator__verify_%s" % type - if functions.has_key(functionName): - self.verifyFunctions[type] = getattr(self, functionName) - else: - if type == '': - self.verifyFunctions[type] = self.__verify_nothing - else: - error = "Verify function %s for type %s does not exist." \ - % (functionName, type) - raise Exception(error) - sys.exit(1) - - def __get_value_info(self): - valueInfo = { 'isValid' : 0, 'normalized' : 0, 'errorData' : 0 } - - return valueInfo - - def __set_value_info(self, valueInfo, **valueData): - try: - valueInfo['normalized'] = valueData['normalized'] - valueInfo['isValid'] = 1 - except KeyError: - valueInfo['isValid'] = 0 - try: - valueInfo['errorData'] = valueData['errorData'] - except: - pass - - # start of 'private' verification methods, each one should correspond to a - # type string (see self.verify_config()) - def __verify_directory(self, type, value): - valueInfo = self.__get_value_info() - - if os.path.isdir(value): - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - else: - self.__set_value_info(valueInfo) - - return valueInfo - - def __norm_directory(self, value): - return self.__normalizedPath(value) - - def __verify_address(self, type, value): - valueInfo = self.__get_value_info() - - try: - socket = tcpSocket(value) - if socket.verify(): - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - else: - self.__set_value_info(valueInfo) - except: - self.__set_value_info(valueInfo) - - return valueInfo - - def __norm_address(self, value): - return value.split(':') - - def __verify_ip_address(self, type, value): - valueInfo = self.__get_value_info() - - if check_ip_address(value): - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - else: - self.__set_value_info(valueInfo) - - return valueInfo - - def __verify_net_address(self, type, value): - valueInfo = self.__get_value_info() - - if check_net_address(value): - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - else: - self.__set_value_info(valueInfo) - - return valueInfo - - def __verify_bool(self, type, value): - valueInfo = self.__get_value_info() - - value = str(value) - if re.match("^false|0|f|no$", value, 2): - self.__set_value_info(valueInfo, normalized=False) - elif re.match("^true|1|t|yes$", value, 2): - self.__set_value_info(valueInfo, normalized=True) - else: - self.__set_value_info(valueInfo) - - return valueInfo - - def __norm_bool(self, value): - value = str(value) - norm = "" - if re.match("^false|0|f|no$", value, 2): - norm = False - elif re.match("^true|1|t|yes$", value, 2): - norm = True - else: - raise Exception("invalid bool specified: %s" % value) - - return norm - - def __verify_int(self, type, value): - valueInfo = self.__get_value_info() - - try: - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - except: - self.__set_value_info(valueInfo) - - return valueInfo - - def __norm_int(self, value): - return int(value) - - def __verify_float(self, type, value): - valueInfo = self.__get_value_info() - - try: - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - except: - self.__set_value_info(valueInfo) - - return valueInfo - - def __norm_float(self, value): - return float(value) - - def __verify_pos_int(self, type, value): - valueInfo = self.__get_value_info() - - try: - value = self.normalize(type, value) - except: - self.__set_value_info(valueInfo) - else: - self.__set_value_info(valueInfo, normalized=value) - - return valueInfo - - def __norm_pos_int(self, value): - value = int(value) - if value < 0: - raise Exception("value is not positive: %s" % value) - - return value - - def __verify_neg_int(self, type, value): - valueInfo = self.__get_value_info() - - try: - value = self.normalize(type, value) - except: - self.__set_value_info(valueInfo) - else: - self.__set_value_info(valueInfo, normalized=value) - - return valueInfo - - def __norm_neg_int(self, type, value): - value = int(value) - if value > 0: - raise Exception("value is not negative: %s" % value) - - return value - - def __verify_freq(self, type, value): - return self.__verify_pos_int(type, value) - - def __norm_freq(self, value): - return self.__norm_pos_int(value) - - def __verify_pos_float(self, type, value): - valueInfo = self.__get_value_info() - - try: - value = self.normalize(type, value) - except: - self.__set_value_info(valueInfo) - else: - self.__set_value_info(valueInfo, normalized=value) - - return valueInfo - - def __norm_pos_float(self, value): - value = float(value) - if value < 0: - raise Exception("value is not positive: %s" % value) - - return value - - def __verify_pos_num(self, type, value): - return self.__verify_pos_float(value) - - def __norm_pos_num(self, value): - return self.__norm_pos_float(value) - - def __verify_neg_float(self, type, value): - valueInfo = self.__get_value_info() - - try: - value = self.normalize(type, value) - except: - self.__set_value_info(valueInfo) - else: - self.__set_value_info(valueInfo, normalized=value) - - return valueInfo - - def __norm_neg_float(self, value): - value = float(value) - if value >= 0: - raise Exception("value is not negative: %s" % value) - - return value - - def __verify_string(self, type, value): - valueInfo = self.__get_value_info() - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - - return valueInfo - - def __norm_string(self, value): - return str(value) - - def __verify_keyval(self, type, value): - valueInfo = self.__get_value_info() - - if reKeyVal.search(value): - try: - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - except: - self.__set_value_info(valueInfo, errorData = \ - "invalid list of key-value pairs : [ %s ]" % value) - else: - msg = "No key value pairs found?" - self.__set_value_info(valueInfo, errorData=msg) - - return valueInfo - - def __norm_keyval(self, value): - list = self.__norm_list(value) - keyValue = {} - for item in list: - (key, value) = reKeyVal.split(item) - #if not keyValue.has_key(key): - # keyValue[key] = [] - #keyValue[key].append(value) - keyValue[key] = value - return keyValue - - def __verify_list(self, type, value): - valueInfo = self.__get_value_info() - - self.__set_value_info(valueInfo, normalized=self.normalize(type,value)) - - return valueInfo - - def __norm_list(self, value): - norm = [] - if reList.search(value): - norm = reList.split(value) - else: - norm = [value,] - - return norm - - def __verify_file(self, type, value): - valueInfo = self.__get_value_info() - - if os.path.isfile(value): - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - else: - self.__set_value_info(valueInfo) - - return valueInfo - - def __norm_file(self, value): - return self.__normalizedPath(value) - - def __verify_size(self, type, value): - valueInfo = self.__get_value_info() - - value = str(value) - if reSizeFormat.match(value): - numberPart = int(reSizeFormat.sub("\g<1>", value)) - factorPart = reSizeFormat.sub("\g<2>", value) - try: - normalized = normalize_size(numberPart, factorPart) - self.__set_value_info(valueInfo, - normalized=normalized) - except: - self.__set_value_info(valueInfo) - else: - try: - value = int(value) - except: - self.__set_value_info(valueInfo) - else: - if value >= 0: - self.__set_value_info(valueInfo, normalized=value) - else: - self.__set_value_info(valueInfo) - - return valueInfo - - def __norm_size(self, file): - norm = None - if reSizeFormat.match(value): - numberPart = int(reSizeFormat.sub("\g<1>", value)) - factorPart = reSizeFormat.sub("\g<2>", value) - norm = normalize_size(numberPart, factorPart) - else: - norm = int(value) - - return norm - - - def __verify_eaddress(self, type, value): - valueInfo = self.__get_value_info() - - emailList = reComma.split(value) - - for emailAddress in emailList: - if reEmailAddress.match(emailAddress): - emailParts = reEmailDelimit.split(emailAddress) - try: - socket.gethostbyname(emailParts[1]) - self.__set_value_info(valueInfo, normalized=self.normalize( - type, value)) - except: - errorString = "%s is invalid (domain lookup failed)" % \ - emailAddress - self.__set_value_info(valueInfo, errorData=errorString) - else: - errorString = "%s is invalid" % emailAddress - self.__set_value_info(valueInfo, errorData=errorString) - - return valueInfo - - def __verify_tcp_port(self, type, value): - valueInfo = self.__get_value_info() - - try: - value = self.__norm_tcp_port(value) - except: - self.__set_value_info(valueInfo) - else: - if value in range(2, 65536): - self.__set_value_info(valueInfo, normalized=value) - else: - self.__set_value_info(valueInfo) - - return valueInfo - - def __norm_tcp_port(self, value): - return int(value) - - def __verify_http_version(self, type, value): - valueInfo = self.__get_value_info() - - if value in ('1.0', '1.1'): - self.__set_value_info(valueInfo, normalized=float(value)) - else: - self.__set_value_info(valueInfo) - - return valueInfo - - def __verify_range(self, type, value): - valueInfo = self.__get_value_info() - - range = reDash.split(value) - - try: - if len(range) > 1: - start = int(range[0]) - end = int(range[1]) - else: - start = int(range[0]) - end = None - except: - self.__set_value_info(valueInfo) - else: - if end: - if end - start != 0: - self.__set_value_info(valueInfo, normalized=(start, end)) - else: - self.__set_value_info(valueInfo) - else: - self.__set_value_info(valueInfo, normalized=(start,)) - - return valueInfo - - def __norm_range(self, value): - range = reDash.split(value) - if len(range) > 1: - start = int(range[0]) - end = int(range[1]) - else: - start = int(range[0]) - end = None - - return (start, end) - - def __verify_uri(self, type, value): - valueInfo = self.__get_value_info() - - _norm = None - try: - uriComponents = urlparse.urlparse(value) - if uriComponents[0] == '' or uriComponents[0] == 'file': - # if scheme is '' or 'file' - if not os.path.isfile(uriComponents[2]) and \ - not os.path.isdir(uriComponents[2]): - raise Exception("Invalid local URI") - else: - self.__set_value_info(valueInfo, normalized=self.normalize( - type,value)) - else: - # other schemes - # currently not checking anything. TODO - self.__set_value_info(valueInfo, normalized=self.normalize( - type,value)) - except: - errorString = "%s is an invalid uri" % value - self.__set_value_info(valueInfo, errorData=errorString) - - return valueInfo - - def __norm_uri(self, value): - uriComponents = list(urlparse.urlparse(value)) - if uriComponents[0] == '': - # if scheme is ''' - return self.__normalizedPath(uriComponents[2]) - elif uriComponents[0] == 'file': - # if scheme is 'file' - normalizedPath = self.__normalizedPath(uriComponents[2]) - return urlparse.urlunsplit(uriComponents[0:1] + [normalizedPath] + uriComponents[3:]) - - # Not dealing with any other case right now - return value - - def __verify_timestamp(self, type, value): - valueInfo = self.__get_value_info() - - if check_timestamp(value): - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - else: - self.__set_value_info(valueInfo) - - return valueInfo - - def __verify_hostname(self, type, value): - valueInfo = self.__get_value_info() - - try: - socket.gethostbyname(value) - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - except: - errorString = "%s is invalid (domain lookup failed)" % value - self.__set_value_info(valueInfo, errorData=errorString) - - return valueInfo - - def __verify_user_account(self, type, value): - valueInfo = self.__get_value_info() - - try: - pwd.getpwnam(value) - except: - errorString = "'%s' user account does not exist" % value - self.__set_value_info(valueInfo, errorData=errorString) - else: - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - - return valueInfo - - def __verify_user_group(self, type, value): - valueInfo = self.__get_value_info() - - try: - grp.getgrnam(value) - except: - errorString = "'%s' group does not exist" % value - self.__set_value_info(valueInfo, errorData=errorString) - else: - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - - return valueInfo - - def __verify_nothing(self, type, value): - valueInfo = self.__get_value_info() - - self.__set_value_info(valueInfo, normalized=self.normalize(type, - value)) - - return valueInfo - - #-------------------------------------------------------------------------- - - def normalize(self, type, value): - try: - normFunc = getattr(self, "_typeValidator__norm_%s" % type) - return normFunc(value) - except AttributeError, A: - # this exception should occur only when we don't have corresponding normalize function - return value - - def verify(self, type, value, allowNone=False): - """Verifies a value based on its type. - - type - supported configValidator type - value - data to be validated - allowNone - don't freak out if None or '' is supplied - - returns a valueInfo dictionary: - - valueInfo = { 'isValid' : 1, 'normalized' : 5, 'errorData' : 0 } - - where: - - isValid - true or false (0/1) - normalized - the normalized value - errorData - if invalid an error string - - supported types: - - see top level""" - - result = None - if allowNone: - if value == '' or value == None: - result = self.__verify_nothing(None, None) - result['normalized'] = None - else: - result = self.verifyFunctions[type](type, value) - else: - result = self.verifyFunctions[type](type, value) - - return result - - def is_valid_type(self, type): - """Returns true if type is valid.""" - - return types.has_key(type) - - def type_info(self, type): - """Returns type info dictionary.""" - - dbInfo = dbTypes[types[type]['db']] - typeInfo = types[type].copy() - typeInfo['db'] = dbInfo - - return typeInfo - - def add(self, name, type, value): - """Adds a value and type by name to the configValidate object to be - verified using validate(). - - name - name used to key values and access the results of the - validation - type - configValidator type - value - data to be verified""" - - self.validateList.append({ 'name' : name, - 'type' : type, - 'value': value }) - - def validate(self, allowNone=False): - """Validates configValidate object populating validatedInfo with - valueInfo dictionaries for each value added to the object.""" - - for valItem in self.validateList: - valueInfo = self.verify(valItem['type'], valItem['value'], - allowNone) - if valueInfo: - valueInfo['name'] = valItem['name'] - self.validatedInfo.append(valueInfo) - else: - raise Exception("\nMissing a return value: valueInfo\n%s" % \ - self.verifyFunctions[valItem['type']](valItem['value'])) - - def __normalizedPath(self, value): - oldWd = os.getcwd() - if self.__originalDir: - os.chdir(self.__originalDir) - normPath = os.path.realpath(value) - os.chdir(oldWd) - return normPath - - -class display: - def __init__(self): - self.displayFunctions = {} - self.__build_dispaly_functions() - - def __build_dispaly_functions(self): - functions = {} - for function in dir(self): - functions[function] = 1 - - for type in types.keys(): - # kinda bad, need to find out how to know the name of the class - # I'm in. But it works. - functionName = "_cisplay__display_%s" % type - if functions.has_key(functionName): - self.displayFunctions[type] = getattr(self, functionName) - else: - if type == '': - self.displayFunctions[type] = self.__display_default - else: - error = "Display function %s for type %s does not exist." \ - % (functionName, type) - raise Exception(error) - sys.exit(1) - - def __display_default(self, value, style): - return value - - def __display_generic_number(self, value): - displayNumber = '' - splitNum = string.split(str(value), sep='.') - numList = list(str(splitNum[0])) - numList.reverse() - length = len(numList) - counter = 0 - for char in numList: - counter = counter + 1 - if counter % 3 or counter == length: - displayNumber = "%s%s" % (char, displayNumber) - else: - displayNumber = ",%s%s" % (char, displayNumber) - - if len(splitNum) > 1: - displayNumber = "%s.%s" % (displayNumber, splitNum[1]) - - return displayNumber - - def __display_generic_mappable(self, map, value, style, plural=True): - displayValue = '' - length = len(str(value)) - if length > 3: - for factorSet in map: - displayValue = float(value) / factorSet['factor'] - if len(str(int(displayValue))) <= 3 or \ - factorSet['factor'] == map[-1]['factor']: - displayValue = "%10.2f" % displayValue - if displayValue[-1] == '0': - if displayValue > 1 and style != 'short' and plural: - displayValue = "%s %ss" % (displayValue[:-1], - factorSet[style]) - else: - displayValue = "%s %s" % (displayValue[:-1], - factorSet[style]) - else: - if displayValue > 1 and style != 'short' and plural: - displayValue = "%s %ss" % (displayValue, - factorSet[style]) - else: - displayValue = "%s %s" % (displayValue, - factorSet[style]) - break - - return displayValue - - def __display_directory(self, value, style): - return self.__display_default(value, style) - - def __display_address(self, value, style): - return self.__display_default(value, style) - - def __display_ip_address(self, value, style): - return self.__display_default(value, style) - - def __display_net_address(self, value, style): - return self.__display_default(value, style) - - def __display_bool(self, value, style): - displayValue = value - - if not isinstance(displayValue, bool): - if re.match("^false|0|f|no$", value, 2): - displayValue=False - elif re.match("^true|1|t|yes$", value, 2): - displayValue=True - - return displayValue - - def __display_int(self, value, style): - return self.__display_generic_number(value) - - def __display_float(self, value, style): - return self.__display_generic_number(value) - - def __display_pos_int(self, value, style): - return self.__display_generic_number(value) - - def __display_neg_int(self, value, style): - return self.__display_generic_number(value) - - def __display_pos_num(self, value, style): - return self.__display_generic_number(value) - - def __display_pos_float(self, value, style): - return self.__display_generic_number(value) - - def __display_neg_float(self, value, style): - return self.__display_generic_number(value) - - def __display_string(self, value, style): - return self.__display_default(value, style) - - def __display_list(self, value, style): - value = value.rstrip() - return value.rstrip(',') - - def __display_keyval(self, value, style): - value = value.rstrip() - return value.rstrip(',') - - def __display_file(self, value, style): - return self.__display_default(value, style) - - def __display_size(self, value, style): - return self.__display_generic_mappable(sizeMap, value, style) - - def __display_freq(self, value, style): - return self.__display_generic_mappable(freqMap, value, style, False) - - def __display_eaddress(self, value, style): - return self.__display_default(value, style) - - def __display_tcp_port(self, value, style): - return self.__display_default(value, style) - - def __display_http_version(self, value, style): - return self.__display_default(value, style) - - def __display_range(self, value, style): - return self.__display_default(value, style) - - def __display_hostname(self, value, style): - return self.__display_default(value, style) - - def __display_user_account(self, value, style): - return self.__display_default(value, style) - - def __display_user_group(self, value, style): - return self.__display_default(value, style) - - def __display_timestamp(self, value, style): - return self.__display_default(value, style) - - def display(self, type, value, style='short'): - displayValue = value - if value != None: - displayValue = self.displayFunctions[type](value, style) - - return displayValue - -typeValidatorInstance = typeValidator() - -def is_valid_type(type): - """Returns true if type is valid.""" - - return typeValidatorInstance.is_valid_type(type) - -def type_info(type): - """Returns type info dictionary.""" - - return typeValidatorInstance.type_info(type) - -def verify(type, value, allowNone=False): - """Returns a normalized valueInfo dictionary.""" - - return typeValidatorInstance.verify(type, value, allowNone) - -def __normalize(map, val, factor): - normFactor = string.lower(factor) - normVal = float(val) - return int(normVal * map[normFactor]) - -def normalize_size(size, factor): - """ Normalize a size to bytes. - - size - number of B, KB, MB, GB, TB, or PB - factor - size factor (case insensitive): - b | bytes - bytes - k | kb - kilobytes - m | mb - megabytes - g | gb - gigabytes - t | tb - terabytes - p | pb - petabytes - """ - - return __normalize(sizeFactors, size, factor) - -def normalize_freq(freq, factor): - """ Normalize a frequency to hertz. - - freq - number of Hz, Khz, Mhz, Ghz, Thz, or Phz - factor - size factor (case insensitive): - Hz - Hertz - Mhz - Megahertz - Ghz - Gigahertz - Thz - Terahertz - Phz - Petahertz - """ - - return __normalize(freqFactors, freq, factor) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/util.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/util.py deleted file mode 100644 index 3d5cb6fade..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/util.py +++ /dev/null @@ -1,309 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -import errno, sys, os, traceback, stat, socket, re, warnings, signal - -from hodlib.Common.tcp import tcpSocket, tcpError -from hodlib.Common.threads import simpleCommand - -setUGV = { 'S_ISUID' : 2, 'S_ISGID' : 1, 'S_ISVTX' : 0 } -reEscapeSeq = r"\\(.)?" -reEscapeSeq = re.compile(reEscapeSeq) - -HOD_INTERRUPTED_CODE = 127 -HOD_INTERRUPTED_MESG = "Hod interrupted. Cleaning up and exiting" -TORQUE_USER_LIMITS_COMMENT_FIELD = "User-limits exceeded. " + \ - "Requested:([0-9]*) Used:([0-9]*) MaxLimit:([0-9]*)" -TORQUE_USER_LIMITS_EXCEEDED_MSG = "Requested number of nodes exceeded " + \ - "maximum user limits. " - -class AlarmException(Exception): - def __init__(self, msg=''): - self.message = msg - Exception.__init__(self, msg) - - def __repr__(self): - return self.message - -def isProcessRunning(pid): - '''Check if a process is running, by sending it a 0 signal, and checking for errors''' - # This method is documented in some email threads on the python mailing list. - # For e.g.: http://mail.python.org/pipermail/python-list/2002-May/144522.html - try: - os.kill(pid, 0) - return True - except OSError, err: - return err.errno == errno.EPERM - -def untar(file, targetDir): - status = False - command = 'tar -C %s -zxf %s' % (targetDir, file) - commandObj = simpleCommand('untar', command) - commandObj.start() - commandObj.wait() - commandObj.join() - if commandObj.exit_code() == 0: - status = True - - return status - -def tar(tarFile, tarDirectory, tarList): - currentDir = os.getcwd() - os.chdir(tarDirectory) - status = False - command = 'tar -czf %s ' % (tarFile) - - for file in tarList: - command = "%s%s " % (command, file) - - commandObj = simpleCommand('tar', command) - commandObj.start() - commandObj.wait() - commandObj.join() - if commandObj.exit_code() == 0: - status = True - else: - status = commandObj.exit_status_string() - - os.chdir(currentDir) - - return status - -def to_http_url(list): - """convert [hostname, port] to a http url""" - str = '' - str = "http://%s:%s" % (list[0], list[1]) - - return str - -def get_exception_string(): - (type, value, tb) = sys.exc_info() - exceptList = traceback.format_exception(type, value, tb) - exceptString = '' - for line in exceptList: - exceptString = "%s%s" % (exceptString, line) - - return exceptString - -def get_exception_error_string(): - (type, value, tb) = sys.exc_info() - if value: - exceptString = "%s %s" % (type, value) - else: - exceptString = type - - return exceptString - -def check_timestamp(timeStamp): - """ Checks the validity of a timeStamp. - - timeStamp - (YYYY-MM-DD HH:MM:SS in UTC) - - returns True or False - """ - isValid = True - - try: - timeStruct = time.strptime(timeStamp, "%Y-%m-%d %H:%M:%S") - except: - isValid = False - - return isValid - -def sig_wrapper(sigNum, handler, *args): - if args: - handler(args) - else: - handler() - -def get_perms(filename): - mode = stat.S_IMODE(os.stat(filename)[stat.ST_MODE]) - permsString = '' - permSet = 0 - place = 2 - for who in "USR", "GRP", "OTH": - for what in "R", "W", "X": - if mode & getattr(stat,"S_I"+what+who): - permSet = permSet + 2**place - place = place - 1 - - permsString = "%s%s" % (permsString, permSet) - permSet = 0 - place = 2 - - permSet = 0 - for permFlag in setUGV.keys(): - if mode & getattr(stat, permFlag): - permSet = permSet + 2**setUGV[permFlag] - - permsString = "%s%s" % (permSet, permsString) - - return permsString - -def local_fqdn(): - """Return a system's true FQDN rather than any aliases, which are - occasionally returned by socket.gethostname.""" - - fqdn = None - me = os.uname()[1] - nameInfo=socket.gethostbyname_ex(me) - nameInfo[1].append(nameInfo[0]) - for name in nameInfo[1]: - if name.count(".") and name.startswith(me): - fqdn = name - if fqdn == None: - fqdn = me - return(fqdn) - -def need_to_allocate(allocated, config, command): - status = True - - if allocated.isSet(): - status = False - elif re.search("\s*dfs.*$", command) and \ - config['gridservice-hdfs']['external']: - status = False - elif config['gridservice-mapred']['external']: - status = False - - return status - -def filter_warnings(): - warnings.filterwarnings('ignore', - message=".*?'with' will become a reserved keyword.*") - -def args_to_string(list): - """return a string argument space seperated""" - arg = '' - for item in list: - arg = "%s%s " % (arg, item) - return arg[:-1] - -def replace_escapes(object): - """ replace any escaped character. e.g \, with , \= with = and so on """ - # here object is either a config object or a options object - for section in object._mySections: - for option in object._configDef[section].keys(): - if object[section].has_key(option): - if object._configDef[section][option]['type'] == 'keyval': - keyValDict = object[section][option] - object[section][option] = {} - for (key,value) in keyValDict.iteritems(): - match = reEscapeSeq.search(value) - if match: - value = reEscapeSeq.sub(r"\1", value) - object[section][option][key] = value - -def hadoopVersion(hadoopDir, java_home, log): - # Determine the version of hadoop being used by executing the - # hadoop version command. Code earlier in idleTracker.py - hadoopVersion = { 'major' : None, 'minor' : None } - hadoopPath = os.path.join(hadoopDir, 'bin', 'hadoop') - cmd = "%s version" % hadoopPath - log.debug('Executing command %s to find hadoop version' % cmd) - env = os.environ - env['JAVA_HOME'] = java_home - hadoopVerCmd = simpleCommand('HadoopVersion', cmd, env) - hadoopVerCmd.start() - hadoopVerCmd.wait() - hadoopVerCmd.join() - if hadoopVerCmd.exit_code() == 0: - verLine = hadoopVerCmd.output()[0] - log.debug('Version from hadoop command: %s' % verLine) - hadoopVerRegExp = re.compile("Hadoop ([0-9]+)\.([0-9]+).*") - verMatch = hadoopVerRegExp.match(verLine) - if verMatch != None: - hadoopVersion['major'] = verMatch.group(1) - hadoopVersion['minor'] = verMatch.group(2) - return hadoopVersion - - -def get_cluster_status(hdfsAddress, mapredAddress): - """Determine the status of the cluster based on socket availability - of HDFS and Map/Reduce.""" - status = 0 - - mapredSocket = tcpSocket(mapredAddress) - try: - mapredSocket.open() - mapredSocket.close() - except tcpError: - status = 14 - - hdfsSocket = tcpSocket(hdfsAddress) - try: - hdfsSocket.open() - hdfsSocket.close() - except tcpError: - if status > 0: - status = 10 - else: - status = 13 - - return status - -def parseEquals(list): - # takes in a list of keyval pairs e.g ['a=b','c=d'] and returns a - # dict e.g {'a'='b','c'='d'}. Used in GridService/{mapred.py/hdfs.py} and - # HodRing/hodring.py. No need for specially treating escaped =. as in \=, - # since all keys are generated by hod and don't contain such anomalies - dict = {} - for elems in list: - splits = elems.split('=') - dict[splits[0]] = splits[1] - return dict - -def getMapredSystemDirectory(mrSysDirRoot, userid, jobid): - return os.path.join(mrSysDirRoot, userid, 'mapredsystem', jobid) - -class HodInterrupt: - def __init__(self): - self.HodInterruptFlag = False - self.log = None - - def set_log(self, log): - self.log = log - - def init_signals(self): - - def sigStop(sigNum, handler): - sig_wrapper(sigNum, self.setFlag) - - signal.signal(signal.SIGTERM, sigStop) # 15 : software termination signal - signal.signal(signal.SIGQUIT, sigStop) # 3 : Quit program - signal.signal(signal.SIGINT, sigStop) # 2 ^C : Interrupt program - - def sig_wrapper(sigNum, handler, *args): - self.log.critical("Caught signal %s." % sigNum ) - - if args: - handler(args) - else: - handler() - - def setFlag(self, val = True): - self.HodInterruptFlag = val - - def isSet(self): - return self.HodInterruptFlag - -class HodInterruptException(Exception): - def __init__(self, value = ""): - self.value = value - - def __str__(self): - return repr(self.value) - -hodInterrupt = HodInterrupt() diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/xmlrpc.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/xmlrpc.py deleted file mode 100644 index bb7ef8b60c..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Common/xmlrpc.py +++ /dev/null @@ -1,57 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -import xmlrpclib, time, random, signal -from hodlib.Common.util import hodInterrupt, HodInterruptException - -class hodXRClient(xmlrpclib.ServerProxy): - def __init__(self, uri, transport=None, encoding=None, verbose=0, - allow_none=0, installSignalHandlers=1, retryRequests=True, timeOut=15): - xmlrpclib.ServerProxy.__init__(self, uri, transport, encoding, verbose, - allow_none) - self.__retryRequests = retryRequests - self.__timeOut = timeOut - if (installSignalHandlers!=0): - self.__set_alarm() - - def __set_alarm(self): - def alarm_handler(sigNum, sigHandler): - raise Exception("XML-RPC socket timeout.") - - signal.signal(signal.SIGALRM, alarm_handler) - - def __request(self, methodname, params): - response = None - retryWaitTime = 5 + random.randint(0, 5) - for i in range(0, 30): - signal.alarm(self.__timeOut) - try: - response = self._ServerProxy__request(methodname, params) - signal.alarm(0) - break - except Exception: - if self.__retryRequests: - if hodInterrupt.isSet(): - raise HodInterruptException() - time.sleep(retryWaitTime) - else: - raise Exception("hodXRClientTimeout") - - return response - - def __getattr__(self, name): - # magic method dispatcher - return xmlrpclib._Method(self.__request, name) - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/__init__.py deleted file mode 100644 index 52138f2f8a..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/__init__.py +++ /dev/null @@ -1,18 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. - -from mapred import MapReduce, MapReduceExternal -from hdfs import Hdfs, HdfsExternal diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/hdfs.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/hdfs.py deleted file mode 100644 index 11efd116c3..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/hdfs.py +++ /dev/null @@ -1,310 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -"""define Hdfs as subclass of Service""" - -# -*- python -*- - -import os - -from service import * -from hodlib.Hod.nodePool import * -from hodlib.Common.desc import CommandDesc -from hodlib.Common.util import get_exception_string, parseEquals - -class HdfsExternal(MasterSlave): - """dummy proxy to external HDFS instance""" - - def __init__(self, serviceDesc, workDirs, version): - MasterSlave.__init__(self, serviceDesc, workDirs,None) - self.launchedMaster = True - self.masterInitialized = True - self.version = version - - def getMasterRequest(self): - return None - - def getMasterCommands(self, serviceDict): - return [] - - def getAdminCommands(self, serviceDict): - return [] - - def getWorkerCommands(self, serviceDict): - return [] - - def getMasterAddrs(self): - attrs = self.serviceDesc.getfinalAttrs() - addr = attrs['fs.default.name'] - return [addr] - - def setMasterParams(self, dict): - self.serviceDesc.dict['final-attrs']['fs.default.name'] = "%s:%s" % \ - (dict['host'], dict['fs_port']) - - if self.version < 16: - self.serviceDesc.dict['final-attrs']['dfs.info.port'] = \ - str(self.serviceDesc.dict['info_port']) - else: - # After Hadoop-2185 - self.serviceDesc.dict['final-attrs']['dfs.http.address'] = "%s:%s" % \ - (dict['host'], dict['info_port']) - - def getInfoAddrs(self): - attrs = self.serviceDesc.getfinalAttrs() - if self.version < 16: - addr = attrs['fs.default.name'] - k,v = addr.split( ":") - infoaddr = k + ':' + attrs['dfs.info.port'] - else: - # After Hadoop-2185 - infoaddr = attrs['dfs.http.address'] - return [infoaddr] - -class Hdfs(MasterSlave): - - def __init__(self, serviceDesc, nodePool, required_node, version, \ - format=True, upgrade=False, - workers_per_ring = 1): - MasterSlave.__init__(self, serviceDesc, nodePool, required_node) - self.masterNode = None - self.masterAddr = None - self.runAdminCommands = True - self.infoAddr = None - self._isLost = False - self.format = format - self.upgrade = upgrade - self.workers = [] - self.version = version - self.workers_per_ring = workers_per_ring - - def getMasterRequest(self): - req = NodeRequest(1, [], False) - return req - - def getMasterCommands(self, serviceDict): - - masterCommands = [] - if self.format: - masterCommands.append(self._getNameNodeCommand(True)) - - if self.upgrade: - masterCommands.append(self._getNameNodeCommand(False, True)) - else: - masterCommands.append(self._getNameNodeCommand(False)) - - return masterCommands - - def getAdminCommands(self, serviceDict): - - adminCommands = [] - if self.upgrade and self.runAdminCommands: - adminCommands.append(self._getNameNodeAdminCommand('-safemode wait')) - adminCommands.append(self._getNameNodeAdminCommand('-finalizeUpgrade', - True, True)) - - self.runAdminCommands = False - return adminCommands - - def getWorkerCommands(self, serviceDict): - workerCmds = [] - for id in range(1, self.workers_per_ring + 1): - workerCmds.append(self._getDataNodeCommand(str(id))) - - return workerCmds - - def setMasterNodes(self, list): - node = list[0] - self.masterNode = node - - def getMasterAddrs(self): - return [self.masterAddr] - - def getInfoAddrs(self): - return [self.infoAddr] - - def getWorkers(self): - return self.workers - - def setMasterParams(self, list): - dict = self._parseEquals(list) - self.masterAddr = dict['fs.default.name'] - k,v = self.masterAddr.split( ":") - self.masterNode = k - if self.version < 16: - self.infoAddr = self.masterNode + ':' + dict['dfs.info.port'] - else: - # After Hadoop-2185 - self.infoAddr = dict['dfs.http.address'] - - def _parseEquals(self, list): - return parseEquals(list) - - def _setWorkDirs(self, workDirs, envs, attrs, parentDirs, subDir): - namedir = None - hadooptmpdir = None - datadir = [] - - for p in parentDirs: - workDirs.append(p) - workDirs.append(os.path.join(p, subDir)) - dir = os.path.join(p, subDir, 'dfs-data') - datadir.append(dir) - if not hadooptmpdir: - # Not used currently, generating hadooptmpdir just in case - hadooptmpdir = os.path.join(p, subDir, 'hadoop-tmp') - - if not namedir: - namedir = os.path.join(p, subDir, 'dfs-name') - - workDirs.append(namedir) - workDirs.extend(datadir) - - # FIXME!! use csv - attrs['dfs.name.dir'] = namedir - attrs['hadoop.tmp.dir'] = hadooptmpdir - attrs['dfs.data.dir'] = ','.join(datadir) - envs['HADOOP_ROOT_LOGGER'] = "INFO,DRFA" - - - def _getNameNodeCommand(self, format=False, upgrade=False): - sd = self.serviceDesc - - parentDirs = self.workDirs - workDirs = [] - attrs = sd.getfinalAttrs().copy() - envs = sd.getEnvs().copy() - - if 'fs.default.name' not in attrs: - attrs['fs.default.name'] = 'fillinhostport' - - if self.version < 16: - if 'dfs.info.port' not in attrs: - attrs['dfs.info.port'] = 'fillinport' - else: - # Addressing Hadoop-2185, added the following. Earlier versions don't - # care about this - if 'dfs.http.address' not in attrs: - attrs['dfs.http.address'] = 'fillinhostport' - - self._setWorkDirs(workDirs, envs, attrs, parentDirs, 'hdfs-nn') - - dict = { 'name' : 'namenode' } - dict['program'] = os.path.join('bin', 'hadoop') - argv = ['namenode'] - if format: - argv.append('-format') - elif upgrade: - argv.append('-upgrade') - dict['argv'] = argv - dict['envs'] = envs - dict['pkgdirs'] = sd.getPkgDirs() - dict['workdirs'] = workDirs - dict['final-attrs'] = attrs - dict['attrs'] = sd.getAttrs() - if format: - dict['fg'] = 'true' - dict['stdin'] = 'Y' - cmd = CommandDesc(dict) - return cmd - - def _getNameNodeAdminCommand(self, adminCommand, wait=True, ignoreFailures=False): - sd = self.serviceDesc - - parentDirs = self.workDirs - workDirs = [] - attrs = sd.getfinalAttrs().copy() - envs = sd.getEnvs().copy() - nn = self.masterAddr - - if nn == None: - raise ValueError, "Can't get namenode address" - - attrs['fs.default.name'] = nn - - self._setWorkDirs(workDirs, envs, attrs, parentDirs, 'hdfs-nn') - - dict = { 'name' : 'dfsadmin' } - dict['program'] = os.path.join('bin', 'hadoop') - argv = ['dfsadmin'] - argv.append(adminCommand) - dict['argv'] = argv - dict['envs'] = envs - dict['pkgdirs'] = sd.getPkgDirs() - dict['workdirs'] = workDirs - dict['final-attrs'] = attrs - dict['attrs'] = sd.getAttrs() - if wait: - dict['fg'] = 'true' - dict['stdin'] = 'Y' - if ignoreFailures: - dict['ignorefailures'] = 'Y' - cmd = CommandDesc(dict) - return cmd - - def _getDataNodeCommand(self, id): - - sd = self.serviceDesc - - parentDirs = self.workDirs - workDirs = [] - attrs = sd.getfinalAttrs().copy() - envs = sd.getEnvs().copy() - nn = self.masterAddr - - if nn == None: - raise ValueError, "Can't get namenode address" - - attrs['fs.default.name'] = nn - - if self.version < 16: - if 'dfs.datanode.port' not in attrs: - attrs['dfs.datanode.port'] = 'fillinport' - if 'dfs.datanode.info.port' not in attrs: - attrs['dfs.datanode.info.port'] = 'fillinport' - else: - # Adding the following. Hadoop-2185 - if 'dfs.datanode.address' not in attrs: - attrs['dfs.datanode.address'] = 'fillinhostport' - if 'dfs.datanode.http.address' not in attrs: - attrs['dfs.datanode.http.address'] = 'fillinhostport' - - if self.version >= 18: - # After HADOOP-3283 - # TODO: check for major as well as minor versions - attrs['dfs.datanode.ipc.address'] = 'fillinhostport' - - # unique workdirs in case of multiple datanodes per hodring - pd = [] - for dir in parentDirs: - dir = dir + "-" + id - pd.append(dir) - parentDirs = pd - # end of unique workdirs - - self._setWorkDirs(workDirs, envs, attrs, parentDirs, 'hdfs-dn') - - dict = { 'name' : 'datanode' } - dict['program'] = os.path.join('bin', 'hadoop') - dict['argv'] = ['datanode'] - dict['envs'] = envs - dict['pkgdirs'] = sd.getPkgDirs() - dict['workdirs'] = workDirs - dict['final-attrs'] = attrs - dict['attrs'] = sd.getAttrs() - - cmd = CommandDesc(dict) - return cmd - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/mapred.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/mapred.py deleted file mode 100644 index 086f052fda..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/mapred.py +++ /dev/null @@ -1,272 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -"""define MapReduce as subclass of Service""" - -# -*- python -*- - -import os, copy, time - -from service import * -from hodlib.Hod.nodePool import * -from hodlib.Common.desc import CommandDesc -from hodlib.Common.util import get_exception_string, parseEquals - -class MapReduceExternal(MasterSlave): - """dummy proxy to external MapReduce instance""" - - def __init__(self, serviceDesc, workDirs, version): - MasterSlave.__init__(self, serviceDesc, workDirs,None) - self.launchedMaster = True - self.masterInitialized = True - self.version = version - - def getMasterRequest(self): - return None - - def getMasterCommands(self, serviceDict): - return [] - - def getAdminCommands(self, serviceDict): - return [] - - def getWorkerCommands(self, serviceDict): - return [] - - def getMasterAddrs(self): - attrs = self.serviceDesc.getfinalAttrs() - addr = attrs['mapred.job.tracker'] - return [addr] - - def needsMore(self): - return 0 - - def needsLess(self): - return 0 - - def setMasterParams(self, dict): - self.serviceDesc['final-attrs']['mapred.job.tracker'] = "%s:%s" % (dict['host'], - dict['tracker_port']) - - if self.version < 16: - self.serviceDesc.dict['final-attrs']['mapred.job.tracker.info.port'] = \ - str(self.serviceDesc.dict['info_port']) - else: - # After Hadoop-2185 - self.serviceDesc['final-attrs']['mapred.job.tracker.http.address'] = \ - "%s:%s" %(dict['host'], dict['info_port']) - - def getInfoAddrs(self): - attrs = self.serviceDesc.getfinalAttrs() - if self.version < 16: - addr = attrs['mapred.job.tracker'] - k,v = addr.split( ":") - infoaddr = k + ':' + attrs['mapred.job.tracker.info.port'] - else: - # After Hadoop-2185 - # Note: earlier,we never respected mapred.job.tracker.http.address - infoaddr = attrs['mapred.job.tracker.http.address'] - return [infoaddr] - -class MapReduce(MasterSlave): - - def __init__(self, serviceDesc, workDirs,required_node, version, - workers_per_ring = 1): - MasterSlave.__init__(self, serviceDesc, workDirs,required_node) - - self.masterNode = None - self.masterAddr = None - self.infoAddr = None - self.workers = [] - self.required_node = required_node - self.version = version - self.workers_per_ring = workers_per_ring - - def isLaunchable(self, serviceDict): - hdfs = serviceDict['hdfs'] - if (hdfs.isMasterInitialized()): - return True - return False - - def getMasterRequest(self): - req = NodeRequest(1, [], False) - return req - - def getMasterCommands(self, serviceDict): - - hdfs = serviceDict['hdfs'] - - cmdDesc = self._getJobTrackerCommand(hdfs) - return [cmdDesc] - - def getAdminCommands(self, serviceDict): - return [] - - def getWorkerCommands(self, serviceDict): - - hdfs = serviceDict['hdfs'] - - workerCmds = [] - for id in range(1, self.workers_per_ring + 1): - workerCmds.append(self._getTaskTrackerCommand(str(id), hdfs)) - - return workerCmds - - def setMasterNodes(self, list): - node = list[0] - self.masterNode = node - - def getMasterAddrs(self): - return [self.masterAddr] - - def getInfoAddrs(self): - return [self.infoAddr] - - def getWorkers(self): - return self.workers - - def requiredNode(self): - return self.required_host - - def setMasterParams(self, list): - dict = self._parseEquals(list) - self.masterAddr = dict['mapred.job.tracker'] - k,v = self.masterAddr.split(":") - self.masterNode = k - if self.version < 16: - self.infoAddr = self.masterNode + ':' + dict['mapred.job.tracker.info.port'] - else: - # After Hadoop-2185 - self.infoAddr = dict['mapred.job.tracker.http.address'] - - def _parseEquals(self, list): - return parseEquals(list) - - def _setWorkDirs(self, workDirs, envs, attrs, parentDirs, subDir): - local = [] - system = None - temp = None - hadooptmpdir = None - dfsclient = [] - - for p in parentDirs: - workDirs.append(p) - workDirs.append(os.path.join(p, subDir)) - dir = os.path.join(p, subDir, 'mapred-local') - local.append(dir) - if not system: - system = os.path.join(p, subDir, 'mapred-system') - if not temp: - temp = os.path.join(p, subDir, 'mapred-temp') - if not hadooptmpdir: - # Not used currently, generating hadooptmpdir just in case - hadooptmpdir = os.path.join(p, subDir, 'hadoop-tmp') - dfsclientdir = os.path.join(p, subDir, 'dfs-client') - dfsclient.append(dfsclientdir) - workDirs.append(dfsclientdir) - # FIXME!! use csv - attrs['mapred.local.dir'] = ','.join(local) - attrs['mapred.system.dir'] = 'fillindir' - attrs['mapred.temp.dir'] = temp - attrs['hadoop.tmp.dir'] = hadooptmpdir - - - envs['HADOOP_ROOT_LOGGER'] = "INFO,DRFA" - - - def _getJobTrackerCommand(self, hdfs): - sd = self.serviceDesc - - parentDirs = self.workDirs - workDirs = [] - attrs = sd.getfinalAttrs().copy() - envs = sd.getEnvs().copy() - - if 'mapred.job.tracker' not in attrs: - attrs['mapred.job.tracker'] = 'fillinhostport' - - if self.version < 16: - if 'mapred.job.tracker.info.port' not in attrs: - attrs['mapred.job.tracker.info.port'] = 'fillinport' - else: - # Addressing Hadoop-2185, - if 'mapred.job.tracker.http.address' not in attrs: - attrs['mapred.job.tracker.http.address'] = 'fillinhostport' - - attrs['fs.default.name'] = hdfs.getMasterAddrs()[0] - - self._setWorkDirs(workDirs, envs, attrs, parentDirs, 'mapred-jt') - - dict = { 'name' : 'jobtracker' } - dict['version'] = self.version - dict['program'] = os.path.join('bin', 'hadoop') - dict['argv'] = ['jobtracker'] - dict['envs'] = envs - dict['pkgdirs'] = sd.getPkgDirs() - dict['workdirs'] = workDirs - dict['final-attrs'] = attrs - dict['attrs'] = sd.getAttrs() - cmd = CommandDesc(dict) - return cmd - - def _getTaskTrackerCommand(self, id, hdfs): - - sd = self.serviceDesc - - parentDirs = self.workDirs - workDirs = [] - attrs = sd.getfinalAttrs().copy() - envs = sd.getEnvs().copy() - jt = self.masterAddr - - if jt == None: - raise ValueError, "Can't get job tracker address" - - attrs['mapred.job.tracker'] = jt - attrs['fs.default.name'] = hdfs.getMasterAddrs()[0] - - if self.version < 16: - if 'tasktracker.http.port' not in attrs: - attrs['tasktracker.http.port'] = 'fillinport' - # earlier to 16, tasktrackers always took ephemeral port 0 for - # tasktracker.report.bindAddress - else: - # Adding the following. Hadoop-2185 - if 'mapred.task.tracker.report.address' not in attrs: - attrs['mapred.task.tracker.report.address'] = 'fillinhostport' - if 'mapred.task.tracker.http.address' not in attrs: - attrs['mapred.task.tracker.http.address'] = 'fillinhostport' - - # unique parentDirs in case of multiple tasktrackers per hodring - pd = [] - for dir in parentDirs: - dir = dir + "-" + id - pd.append(dir) - parentDirs = pd - # end of unique workdirs - - self._setWorkDirs(workDirs, envs, attrs, parentDirs, 'mapred-tt') - - dict = { 'name' : 'tasktracker' } - dict['program'] = os.path.join('bin', 'hadoop') - dict['argv'] = ['tasktracker'] - dict['envs'] = envs - dict['pkgdirs'] = sd.getPkgDirs() - dict['workdirs'] = workDirs - dict['final-attrs'] = attrs - dict['attrs'] = sd.getAttrs() - cmd = CommandDesc(dict) - return cmd - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/service.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/service.py deleted file mode 100644 index f0c7f5cbbf..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/GridServices/service.py +++ /dev/null @@ -1,266 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -"""defines Service as abstract interface""" - -# -*- python -*- -import random, socket - -class Service: - """ the service base class that all the - other services inherit from. """ - def __init__(self, serviceDesc, workDirs): - self.serviceDesc = serviceDesc - self.workDirs = workDirs - - def getName(self): - return self.serviceDesc.getName() - - def getInfoAddrs(self): - """Return a list of addresses that provide - information about the servie""" - return [] - - def isLost(self): - """True if the service is down""" - raise NotImplementedError - - def addNodes(self, nodeList): - """add nodeSet""" - raise NotImplementedError - - def removeNodes(self, nodeList): - """remove a nodeset""" - raise NotImplementedError - - def getWorkers(self): - raise NotImplementedError - - def needsMore(self): - """return number of nodes the service wants to add""" - raise NotImplementedError - - def needsLess(self): - """return number of nodes the service wants to remove""" - raise NotImplementedError - -class MasterSlave(Service): - """ the base class for a master slave - service architecture. """ - def __init__(self, serviceDesc, workDirs,requiredNode): - Service.__init__(self, serviceDesc, workDirs) - self.launchedMaster = False - self.masterInitialized = False - self.masterAddress = 'none' - self.requiredNode = requiredNode - self.failedMsg = None - self.masterFailureCount = 0 - - def getRequiredNode(self): - return self.requiredNode - - def getMasterRequest(self): - """ the number of master you need - to run for this service. """ - raise NotImplementedError - - def isLaunchable(self, serviceDict): - """ if your service does not depend on - other services. is set to true by default. """ - return True - - def getMasterCommands(self, serviceDict): - """ a list of master commands you - want to run for this service. """ - raise NotImplementedError - - def getAdminCommands(self, serviceDict): - """ a list of admin commands you - want to run for this service. """ - raise NotImplementedError - - def getWorkerCommands(self, serviceDict): - """ a list of worker commands you want to - run for this service. """ - raise NotImplementedError - - def setMasterNodes(self, list): - """ set the status of master nodes - after they start running on a node cluster. """ - raise NotImplementedError - - def addNodes(self, list): - """ add nodes to a service. Not implemented - currently. """ - raise NotImplementedError - - def getMasterAddrs(self): - """ return the addresses of master. the - hostname:port to which worker nodes should - connect. """ - raise NotImplementedError - - def setMasterParams(self, list): - """ set the various master params - depending on what each hodring set - the master params to. """ - raise NotImplementedError - - def setlaunchedMaster(self): - """ set the status of master launched - to true. """ - self.launchedMaster = True - - def isMasterLaunched(self): - """ return if a master has been launched - for the service or not. """ - return self.launchedMaster - - def isMasterInitialized(self): - """ return if a master if launched - has been initialized or not. """ - return self.masterInitialized - - def setMasterInitialized(self): - """ set the master initialized to - true. """ - self.masterInitialized = True - # Reset failure related variables, as master is initialized successfully. - self.masterFailureCount = 0 - self.failedMsg = None - - def getMasterAddress(self): - """ it needs to change to reflect - more that one masters. Currently it - keeps a knowledge of where the master - was launched and to keep track if it was actually - up or not. """ - return self.masterAddress - - def setMasterAddress(self, addr): - self.masterAddress = addr - - def isExternal(self): - return self.serviceDesc.isExternal() - - def setMasterFailed(self, err): - """Sets variables related to Master failure""" - self.masterFailureCount += 1 - self.failedMsg = err - # When command is sent to HodRings, this would have been set to True. - # Reset it to reflect the correct status. - self.launchedMaster = False - - def getMasterFailed(self): - return self.failedMsg - - def getMasterFailureCount(self): - return self.masterFailureCount - -class NodeRequest: - """ A class to define - a node request. """ - def __init__(self, n, required = [], preferred = [], isPreemptee = True): - self.numNodes = n - self.preferred = preferred - self.isPreemptee = isPreemptee - self.required = required - - def setNumNodes(self, n): - self.numNodes = n - - def setPreferredList(self, list): - self.preferred = list - - def setIsPreemptee(self, flag): - self.isPreemptee = flag - - -class ServiceUtil: - """ this class should be moved out of - service.py to a util file""" - localPortUsed = {} - - def getUniqRandomPort(h=None, low=50000, high=60000, retry=900, log=None): - """This allocates a randome free port between low and high""" - # We use a default value of 900 retries, which takes an agreeable - # time limit of ~ 6.2 seconds to check 900 ports, in the worse case - # of no available port in those 900. - - while retry > 0: - n = random.randint(low, high) - if n in ServiceUtil.localPortUsed: - continue - s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - if not h: - h = socket.gethostname() - avail = False - if log: log.debug("Trying to see if port %s is available"% n) - try: - s.bind((h, n)) - if log: log.debug("Yes, port %s is available" % n) - avail = True - except socket.error,e: - if log: log.debug("Could not bind to the port %s. Reason %s" % (n,e)) - retry -= 1 - pass - # The earlier code that used to be here had syntax errors. The code path - # couldn't be followd anytime, so the error remained uncaught. - # This time I stumbled upon the error - s.close() - - if avail: - ServiceUtil.localPortUsed[n] = True - return n - raise ValueError, "Can't find unique local port between %d and %d" % (low, high) - - getUniqRandomPort = staticmethod(getUniqRandomPort) - - def getUniqPort(h=None, low=40000, high=60000, retry=900, log=None): - """get unique port on a host that can be used by service - This and its consumer code should disappear when master - nodes get allocatet by nodepool""" - - # We use a default value of 900 retries, which takes an agreeable - # time limit of ~ 6.2 seconds to check 900 ports, in the worse case - # of no available port in those 900. - - n = low - while retry > 0: - n = n + 1 - if n in ServiceUtil.localPortUsed: - continue - s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - if not h: - h = socket.gethostname() - avail = False - if log: log.debug("Trying to see if port %s is available"% n) - try: - s.bind((h, n)) - if log: log.debug("Yes, port %s is available" % n) - avail = True - except socket.error,e: - if log: log.debug("Could not bind to the port %s. Reason %s" % (n,e)) - retry -= 1 - pass - s.close() - - if avail: - ServiceUtil.localPortUsed[n] = True - return n - - raise ValueError, "Can't find unique local port between %d and %d" % (low, high) - - getUniqPort = staticmethod(getUniqPort) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/__init__.py deleted file mode 100644 index 12c2f1e1da..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/__init__.py +++ /dev/null @@ -1,15 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/hadoop.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/hadoop.py deleted file mode 100644 index 616d775803..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/hadoop.py +++ /dev/null @@ -1,747 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -"""define WorkLoad as abstract interface for user job""" -# -*- python -*- - -import os, time, sys, shutil, exceptions, re, threading, signal, urllib, pprint, math - -from HTMLParser import HTMLParser - -import xml.dom.minidom -import xml.dom.pulldom -from xml.dom import getDOMImplementation - -from hodlib.Common.util import * -from hodlib.Common.xmlrpc import hodXRClient -from hodlib.Common.miniHTMLParser import miniHTMLParser -from hodlib.Common.nodepoolutil import NodePoolUtil -from hodlib.Common.tcp import tcpError, tcpSocket - -reCommandDelimeterString = r"(?> sitefile, topElement.toxml() - sitefile.close() - -class hadoopCluster: - def __init__(self, cfg, log): - self.__cfg = cfg - self.__log = log - self.__changedClusterParams = [] - - self.__hostname = local_fqdn() - self.__svcrgyClient = None - self.__nodePool = NodePoolUtil.getNodePool(self.__cfg['nodepooldesc'], - self.__cfg, self.__log) - self.__hadoopCfg = hadoopConfig() - self.jobId = None - self.mapredInfo = None - self.hdfsInfo = None - self.ringmasterXRS = None - - def __get_svcrgy_client(self): - svcrgyUrl = to_http_url(self.__cfg['hod']['xrs-address']) - return hodXRClient(svcrgyUrl) - - def __get_service_status(self): - serviceData = self.__get_service_data() - - status = True - hdfs = False - mapred = False - - for host in serviceData.keys(): - for item in serviceData[host]: - service = item.keys() - if service[0] == 'hdfs.grid' and \ - self.__cfg['gridservice-hdfs']['external'] == False: - hdfs = True - elif service[0] == 'mapred.grid': - mapred = True - - if not mapred: - status = "mapred" - - if not hdfs and self.__cfg['gridservice-hdfs']['external'] == False: - if status != True: - status = "mapred and hdfs" - else: - status = "hdfs" - - return status - - def __get_service_data(self): - registry = to_http_url(self.__cfg['hod']['xrs-address']) - serviceData = self.__svcrgyClient.getServiceInfo( - self.__cfg['hod']['userid'], self.__setup.np.getNodePoolId()) - - return serviceData - - def __check_job_status(self): - failureCount = 0 - status = False - state = 'Q' - userLimitsFirstFlag = True - - while (state=='Q') or (state==False): - if hodInterrupt.isSet(): - raise HodInterruptException() - - jobInfo = self.__nodePool.getJobInfo() - state = jobInfo['job_state'] - self.__log.debug('job state %s' % state) - if state == False: - failureCount += 1 - if (failureCount >= self.__cfg['hod']['job-status-query-failure-retries']): - self.__log.debug('Number of retries reached max limit while querying job status') - break - time.sleep(self.__cfg['hod']['job-command-failure-interval']) - elif state!='Q': - break - else: - self.__log.debug('querying for job status after job-status-query-interval') - time.sleep(self.__cfg['hod']['job-status-query-interval']) - - if self.__cfg['hod'].has_key('job-feasibility-attr') and \ - self.__cfg['hod']['job-feasibility-attr']: - (status, msg) = self.__isJobFeasible() - if status == "Never": - self.__log.critical(TORQUE_USER_LIMITS_EXCEEDED_MSG + msg + \ - "This cluster cannot be allocated now.") - return -1 - elif status == False: - if userLimitsFirstFlag: - self.__log.critical(TORQUE_USER_LIMITS_EXCEEDED_MSG + msg + \ - "This cluster allocation will succeed only after other " + \ - "clusters are deallocated.") - userLimitsFirstFlag = False - - if state and state != 'C': - status = True - - return status - - def __isJobFeasible(self): - return self.__nodePool.isJobFeasible() - - def __get_ringmaster_client(self): - ringmasterXRS = None - - ringList = self.__svcrgyClient.getServiceInfo( - self.__cfg['ringmaster']['userid'], self.__nodePool.getServiceId(), - 'ringmaster', 'hod') - - if ringList and len(ringList): - if isinstance(ringList, list): - ringmasterXRS = ringList[0]['xrs'] - else: - count = 0 - waitTime = self.__cfg['hod']['allocate-wait-time'] - - while count < waitTime: - if hodInterrupt.isSet(): - raise HodInterruptException() - - ringList = self.__svcrgyClient.getServiceInfo( - self.__cfg['ringmaster']['userid'], self.__nodePool.getServiceId(), - 'ringmaster', - 'hod') - - if ringList and len(ringList): - if isinstance(ringList, list): - ringmasterXRS = ringList[0]['xrs'] - - if ringmasterXRS is not None: - break - else: - time.sleep(1) - count = count + 1 - # check to see if the job exited by any chance in that time: - if (count % self.__cfg['hod']['job-status-query-interval'] == 0): - if not self.__check_job_status(): - break - return ringmasterXRS - - def __init_hadoop_service(self, serviceName, xmlrpcClient): - status = True - serviceAddress = None - serviceInfo = None - - for i in range(0, 250): - try: - if hodInterrupt.isSet(): - raise HodInterruptException() - - serviceAddress = xmlrpcClient.getServiceAddr(serviceName) - if serviceAddress: - if serviceAddress == 'not found': - time.sleep(1) - # check to see if the job exited by any chance in that time: - if ((i+1) % self.__cfg['hod']['job-status-query-interval'] == 0): - if not self.__check_job_status(): - break - else: - serviceInfo = xmlrpcClient.getURLs(serviceName) - break - except HodInterruptException,h : - raise h - except: - self.__log.critical("'%s': ringmaster xmlrpc error." % serviceName) - self.__log.debug(get_exception_string()) - status = False - break - - if serviceAddress == 'not found' or not serviceAddress: - self.__log.critical("Failed to retrieve '%s' service address." % - serviceName) - status = False - elif serviceAddress.startswith("Error: "): - errs = serviceAddress[len("Error: "):] - self.__log.critical("Cluster could not be allocated because of the following errors.\n%s" % \ - errs) - status = False - else: - try: - self.__svcrgyClient.registerService(self.__cfg['hodring']['userid'], - self.jobId, self.__hostname, - serviceName, 'grid', serviceInfo) - - except HodInterruptException, h: - raise h - except: - self.__log.critical("'%s': registry xmlrpc error." % serviceName) - self.__log.debug(get_exception_string()) - status = False - - return status, serviceAddress, serviceInfo - - def __collect_jobtracker_ui(self, dir): - - link = self.mapredInfo + "/jobtracker.jsp" - parser = miniHTMLParser() - parser.setBaseUrl(self.mapredInfo) - node_cache = {} - - self.__log.debug("collect_jobtracker_ui seeded with " + link) - - def alarm_handler(number, stack): - raise AlarmException("timeout") - - signal.signal(signal.SIGALRM, alarm_handler) - - input = None - while link: - self.__log.debug("link: %s" % link) - # taskstats.jsp,taskdetails.jsp not included since too many to collect - if re.search( - "jobfailures\.jsp|jobtracker\.jsp|jobdetails\.jsp|jobtasks\.jsp", - link): - - for i in range(1,5): - if hodInterrupt.isSet(): - raise HodInterruptException() - try: - input = urllib.urlopen(link) - break - except: - self.__log.debug(get_exception_string()) - time.sleep(1) - - if input: - out = None - - self.__log.debug("collecting " + link + "...") - filename = re.sub(self.mapredInfo, "", link) - filename = dir + "/" + filename - filename = re.sub("http://","", filename) - filename = re.sub("[\?\&=:]","_",filename) - filename = filename + ".html" - - try: - tempdir, tail = os.path.split(filename) - if not os.path.exists(tempdir): - os.makedirs(tempdir) - except: - self.__log.debug(get_exception_string()) - - out = open(filename, 'w') - - bufSz = 8192 - - signal.alarm(10) - - try: - self.__log.debug("Starting to grab: %s" % link) - buf = input.read(bufSz) - - while len(buf) > 0: - # Feed the file into the HTML parser - parser.feed(buf) - - # Re-write the hrefs in the file - p = re.compile("\?(.+?)=(.+?)") - buf = p.sub(r"_\1_\2",buf) - p= re.compile("&(.+?)=(.+?)") - buf = p.sub(r"_\1_\2",buf) - p = re.compile("http://(.+?):(\d+)?") - buf = p.sub(r"\1_\2/",buf) - buf = re.sub("href=\"/","href=\"",buf) - p = re.compile("href=\"(.+?)\"") - buf = p.sub(r"href=\1.html",buf) - - out.write(buf) - buf = input.read(bufSz) - - signal.alarm(0) - input.close() - if out: - out.close() - - self.__log.debug("Finished grabbing: %s" % link) - except AlarmException: - if hodInterrupt.isSet(): - raise HodInterruptException() - if out: out.close() - if input: input.close() - - self.__log.debug("Failed to retrieve: %s" % link) - else: - self.__log.debug("Failed to retrieve: %s" % link) - - # Get the next link in level traversal order - link = parser.getNextLink() - - parser.close() - - def check_cluster(self, clusterInfo): - status = 0 - - if 'mapred' in clusterInfo: - mapredAddress = clusterInfo['mapred'][7:] - hdfsAddress = clusterInfo['hdfs'][7:] - status = get_cluster_status(hdfsAddress, mapredAddress) - if status == 0: - status = 12 - else: - status = 15 - - return status - - def is_cluster_deallocated(self, jobId): - """Returns True if the JobId that represents this cluster - is in the Completed or exiting state.""" - jobInfo = self.__nodePool.getJobInfo(jobId) - state = None - if jobInfo is not None and jobInfo.has_key('job_state'): - state = jobInfo['job_state'] - return ((state == 'C') or (state == 'E')) - - def cleanup(self): - if self.__nodePool: self.__nodePool.finalize() - - def get_job_id(self): - return self.jobId - - def delete_job(self, jobId): - '''Delete a job given it's ID''' - ret = 0 - if self.__nodePool: - ret = self.__nodePool.deleteJob(jobId) - else: - raise Exception("Invalid state: Node pool is not initialized to delete the given job.") - return ret - - def is_valid_account(self): - """Verify if the account being used to submit the job is a valid account. - This code looks for a file /bin/verify-account. - If the file is present, it executes the file, passing as argument - the account name. It returns the exit code and output from the - script on non-zero exit code.""" - - accountValidationScript = os.path.abspath('./verify-account') - if not os.path.exists(accountValidationScript): - return (0, None) - - account = self.__nodePool.getAccountString() - exitCode = 0 - errMsg = None - try: - accountValidationCmd = simpleCommand('Account Validation Command',\ - '%s %s' % (accountValidationScript, - account)) - accountValidationCmd.start() - accountValidationCmd.wait() - accountValidationCmd.join() - exitCode = accountValidationCmd.exit_code() - self.__log.debug('account validation script is run %d' \ - % exitCode) - errMsg = None - if exitCode is not 0: - errMsg = accountValidationCmd.output() - except Exception, e: - exitCode = 0 - self.__log.warn('Error executing account script: %s ' \ - 'Accounting is disabled.' \ - % get_exception_error_string()) - self.__log.debug(get_exception_string()) - return (exitCode, errMsg) - - def allocate(self, clusterDir, min, max=None): - status = 0 - failureCount = 0 - self.__svcrgyClient = self.__get_svcrgy_client() - - self.__log.debug("allocate %s %s %s" % (clusterDir, min, max)) - - if min < 3: - self.__log.critical("Minimum nodes must be greater than 2.") - status = 2 - else: - nodeSet = self.__nodePool.newNodeSet(min) - walltime = None - if self.__cfg['hod'].has_key('walltime'): - walltime = self.__cfg['hod']['walltime'] - self.jobId, exitCode = self.__nodePool.submitNodeSet(nodeSet, walltime) - # if the job submission returned an error other than no resources - # retry a couple of times - while (self.jobId is False) and (exitCode != 188): - if hodInterrupt.isSet(): - raise HodInterruptException() - - failureCount += 1 - if (failureCount >= self.__cfg['hod']['job-status-query-failure-retries']): - self.__log.debug("failed submitting job more than the retries. exiting") - break - else: - # wait a bit before retrying - time.sleep(self.__cfg['hod']['job-command-failure-interval']) - if hodInterrupt.isSet(): - raise HodInterruptException() - self.jobId, exitCode = self.__nodePool.submitNodeSet(nodeSet, walltime) - - if self.jobId: - jobStatus = None - try: - jobStatus = self.__check_job_status() - except HodInterruptException, h: - self.__log.info(HOD_INTERRUPTED_MESG) - self.delete_job(self.jobId) - self.__log.info("Cluster %s removed from queue." % self.jobId) - raise h - else: - if jobStatus == -1: - self.delete_job(self.jobId); - status = 4 - return status - - if jobStatus: - self.__log.info("Cluster Id %s" \ - % self.jobId) - try: - self.ringmasterXRS = self.__get_ringmaster_client() - - self.__log.debug("Ringmaster at : %s" % self.ringmasterXRS ) - ringClient = None - if self.ringmasterXRS: - ringClient = hodXRClient(self.ringmasterXRS) - - hdfsStatus, hdfsAddr, self.hdfsInfo = \ - self.__init_hadoop_service('hdfs', ringClient) - - if hdfsStatus: - self.__log.info("HDFS UI at http://%s" % self.hdfsInfo) - - mapredStatus, mapredAddr, self.mapredInfo = \ - self.__init_hadoop_service('mapred', ringClient) - - if mapredStatus: - self.__log.info("Mapred UI at http://%s" % self.mapredInfo) - - if self.__cfg['hod'].has_key('update-worker-info') \ - and self.__cfg['hod']['update-worker-info']: - workerInfoMap = {} - workerInfoMap['HDFS UI'] = 'http://%s' % self.hdfsInfo - workerInfoMap['Mapred UI'] = 'http://%s' % self.mapredInfo - # Ringmaster URL sample format : http://hostname:port/ - workerInfoMap['RM RPC Port'] = '%s' % self.ringmasterXRS.split(":")[2].strip("/") - if mapredAddr.find(':') != -1: - workerInfoMap['Mapred RPC Port'] = mapredAddr.split(':')[1] - ret = self.__nodePool.updateWorkerInfo(workerInfoMap, self.jobId) - if ret != 0: - self.__log.warn('Could not update HDFS and Mapred information.' \ - 'User Portal may not show relevant information.' \ - 'Error code=%s' % ret) - - self.__cfg.replace_escape_seqs() - - # Go generate the client side hadoop-site.xml now - # adding final-params as well, just so that conf on - # client-side and server-side are (almost) the same - clientParams = None - serverParams = {} - finalServerParams = {} - - # client-params - if self.__cfg['hod'].has_key('client-params'): - clientParams = self.__cfg['hod']['client-params'] - - # server-params - if self.__cfg['gridservice-mapred'].has_key('server-params'): - serverParams.update(\ - self.__cfg['gridservice-mapred']['server-params']) - if self.__cfg['gridservice-hdfs'].has_key('server-params'): - # note that if there are params in both mapred and hdfs - # sections, the ones in hdfs overwirte the ones in mapred - serverParams.update(\ - self.__cfg['gridservice-hdfs']['server-params']) - - # final-server-params - if self.__cfg['gridservice-mapred'].has_key(\ - 'final-server-params'): - finalServerParams.update(\ - self.__cfg['gridservice-mapred']['final-server-params']) - if self.__cfg['gridservice-hdfs'].has_key( - 'final-server-params'): - finalServerParams.update(\ - self.__cfg['gridservice-hdfs']['final-server-params']) - - clusterFactor = self.__cfg['hod']['cluster-factor'] - tempDir = self.__cfg['hod']['temp-dir'] - if not os.path.exists(tempDir): - os.makedirs(tempDir) - tempDir = os.path.join( tempDir, self.__cfg['hod']['userid']\ - + "." + self.jobId ) - mrSysDir = getMapredSystemDirectory(self.__cfg['hodring']['mapred-system-dir-root'],\ - self.__cfg['hod']['userid'], self.jobId) - self.__hadoopCfg.gen_site_conf(clusterDir, tempDir, min,\ - hdfsAddr, mrSysDir, mapredAddr, clientParams,\ - serverParams, finalServerParams,\ - clusterFactor) - self.__log.info("hadoop-site.xml at %s" % clusterDir) - # end of hadoop-site.xml generation - else: - status = 8 - else: - status = 7 - else: - status = 6 - if status != 0: - self.__log.debug("Cleaning up cluster id %s, as cluster could not be allocated." % self.jobId) - if ringClient is None: - self.delete_job(self.jobId) - else: - self.__log.debug("Calling rm.stop()") - ringClient.stopRM() - self.__log.debug("Returning from rm.stop()") - except HodInterruptException, h: - self.__log.info(HOD_INTERRUPTED_MESG) - if self.ringmasterXRS: - if ringClient is None: - ringClient = hodXRClient(self.ringmasterXRS) - self.__log.debug("Calling rm.stop()") - ringClient.stopRM() - self.__log.debug("Returning from rm.stop()") - self.__log.info("Cluster Shutdown by informing ringmaster.") - else: - self.delete_job(self.jobId) - self.__log.info("Cluster %s removed from queue directly." % self.jobId) - raise h - else: - self.__log.critical("No cluster found, ringmaster failed to run.") - status = 5 - - elif self.jobId == False: - if exitCode == 188: - self.__log.critical("Request execeeded maximum resource allocation.") - else: - self.__log.critical("Job submission failed with exit code %s" % exitCode) - status = 4 - else: - self.__log.critical("Scheduler failure, allocation failed.\n\n") - status = 4 - - if status == 5 or status == 6: - ringMasterErrors = self.__svcrgyClient.getRMError() - if ringMasterErrors: - self.__log.critical("Cluster could not be allocated because" \ - " of the following errors on the "\ - "ringmaster host %s.\n%s" % \ - (ringMasterErrors[0], ringMasterErrors[1])) - self.__log.debug("Stack trace on ringmaster: %s" % ringMasterErrors[2]) - return status - - def __isRingMasterAlive(self, rmAddr): - ret = True - rmSocket = tcpSocket(rmAddr) - try: - rmSocket.open() - rmSocket.close() - except tcpError: - ret = False - - return ret - - def deallocate(self, clusterDir, clusterInfo): - status = 0 - - nodeSet = self.__nodePool.newNodeSet(clusterInfo['min'], - id=clusterInfo['jobid']) - self.mapredInfo = clusterInfo['mapred'] - self.hdfsInfo = clusterInfo['hdfs'] - - try: - if self.__cfg['hod'].has_key('hadoop-ui-log-dir'): - clusterStatus = self.check_cluster(clusterInfo) - if clusterStatus != 14 and clusterStatus != 10: - # If JT is still alive - self.__collect_jobtracker_ui(self.__cfg['hod']['hadoop-ui-log-dir']) - else: - self.__log.debug('hadoop-ui-log-dir not specified. Skipping Hadoop UI log collection.') - except HodInterruptException, h: - # got an interrupt. just pass and proceed to qdel - pass - except: - self.__log.info("Exception in collecting Job tracker logs. Ignoring.") - - rmAddr = None - if clusterInfo.has_key('ring'): - # format is http://host:port/ We need host:port - rmAddr = clusterInfo['ring'][7:] - if rmAddr.endswith('/'): - rmAddr = rmAddr[:-1] - - if (rmAddr is None) or (not self.__isRingMasterAlive(rmAddr)): - # Cluster is already dead, don't try to contact ringmaster. - self.__nodePool.finalize() - status = 10 # As cluster is dead, we just set the status to 'cluster dead'. - else: - xrsAddr = clusterInfo['ring'] - rmClient = hodXRClient(xrsAddr) - self.__log.debug('calling rm.stop') - rmClient.stopRM() - self.__log.debug('completed rm.stop') - - # cleanup hod temp dirs - tempDir = os.path.join( self.__cfg['hod']['temp-dir'], \ - self.__cfg['hod']['userid'] + "." + clusterInfo['jobid'] ) - if os.path.exists(tempDir): - shutil.rmtree(tempDir) - - return status - -class hadoopScript: - def __init__(self, conf, execDir): - self.__environ = os.environ.copy() - self.__environ['HADOOP_CONF_DIR'] = conf - self.__execDir = execDir - - def run(self, script): - scriptThread = simpleCommand(script, script, self.__environ, 4, False, - False, self.__execDir) - scriptThread.start() - scriptThread.wait() - scriptThread.join() - - return scriptThread.exit_code() diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/hod.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/hod.py deleted file mode 100644 index b2587bb77a..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/hod.py +++ /dev/null @@ -1,754 +0,0 @@ -#Licensed to the Apache Software Foundation (ASF) under one -#or more contributor license agreements. See the NOTICE file -#distributed with this work for additional information -#regarding copyright ownership. The ASF licenses this file -#to you under the Apache License, Version 2.0 (the -#"License"); you may not use this file except in compliance -#with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -#Unless required by applicable law or agreed to in writing, software -#distributed under the License is distributed on an "AS IS" BASIS, -#WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -#See the License for the specific language governing permissions and -#limitations under the License. -# -*- python -*- - -import sys, os, getpass, pprint, re, cPickle, random, shutil, time, errno - -import hodlib.Common.logger - -from hodlib.ServiceRegistry.serviceRegistry import svcrgy -from hodlib.Common.xmlrpc import hodXRClient -from hodlib.Common.util import to_http_url, get_exception_string -from hodlib.Common.util import get_exception_error_string -from hodlib.Common.util import hodInterrupt, HodInterruptException -from hodlib.Common.util import HOD_INTERRUPTED_CODE - -from hodlib.Common.nodepoolutil import NodePoolUtil -from hodlib.Hod.hadoop import hadoopCluster, hadoopScript - -CLUSTER_DATA_FILE = 'clusters' -INVALID_STATE_FILE_MSGS = \ - [ - - "Requested operation cannot be performed. Cannot read %s: " + \ - "Permission denied.", - - "Requested operation cannot be performed. " + \ - "Cannot write to %s: Permission denied.", - - "Requested operation cannot be performed. " + \ - "Cannot read/write to %s: Permission denied.", - - "Cannot update %s: Permission denied. " + \ - "Cluster is deallocated, but info and list " + \ - "operations might show incorrect information.", - - ] - -class hodState: - def __init__(self, store): - self.__store = store - self.__stateFile = None - self.__init_store() - self.__STORE_EXT = ".state" - - def __init_store(self): - if not os.path.exists(self.__store): - os.mkdir(self.__store) - - def __set_state_file(self, id=None): - if id: - self.__stateFile = os.path.join(self.__store, "%s%s" % (id, - self.__STORE_EXT)) - else: - for item in os.listdir(self.__store): - if item.endswith(self.__STORE_EXT): - self.__stateFile = os.path.join(self.__store, item) - - def get_state_file(self): - return self.__stateFile - - def checkStateFile(self, id=None, modes=(os.R_OK,)): - # is state file exists/readable/writable/both? - self.__set_state_file(id) - - # return true if file doesn't exist, because HOD CAN create - # state file and so WILL have permissions to read and/or write - try: - os.stat(self.__stateFile) - except OSError, err: - if err.errno == errno.ENOENT: # error 2 (no such file) - return True - - # file exists - ret = True - for mode in modes: - ret = ret and os.access(self.__stateFile, mode) - return ret - - def read(self, id=None): - info = {} - - self.__set_state_file(id) - - if self.__stateFile: - if os.path.isfile(self.__stateFile): - stateFile = open(self.__stateFile, 'r') - try: - info = cPickle.load(stateFile) - except EOFError: - pass - - stateFile.close() - - return info - - def write(self, id, info): - self.__set_state_file(id) - if not os.path.exists(self.__stateFile): - self.clear(id) - - stateFile = open(self.__stateFile, 'w') - cPickle.dump(info, stateFile) - stateFile.close() - - def clear(self, id=None): - self.__set_state_file(id) - if self.__stateFile and os.path.exists(self.__stateFile): - os.remove(self.__stateFile) - else: - for item in os.listdir(self.__store): - if item.endswith(self.__STORE_EXT): - os.remove(item) - -class hodRunner: - - def __init__(self, cfg, log=None, cluster=None): - self.__hodhelp = hodHelp() - self.__ops = self.__hodhelp.ops - self.__cfg = cfg - self.__npd = self.__cfg['nodepooldesc'] - self.__opCode = 0 - self.__user = getpass.getuser() - self.__registry = None - self.__baseLogger = None - # Allowing to pass in log object to help testing - a stub can be passed in - if log is None: - self.__setup_logger() - else: - self.__log = log - - self.__userState = hodState(self.__cfg['hod']['user_state']) - - self.__clusterState = None - self.__clusterStateInfo = { 'env' : None, 'hdfs' : None, 'mapred' : None } - - # Allowing to pass in log object to help testing - a stib can be passed in - if cluster is None: - self.__cluster = hadoopCluster(self.__cfg, self.__log) - else: - self.__cluster = cluster - - def __setup_logger(self): - self.__baseLogger = hodlib.Common.logger.hodLog('hod') - self.__log = self.__baseLogger.add_logger(self.__user ) - - if self.__cfg['hod']['stream']: - self.__baseLogger.add_stream(level=self.__cfg['hod']['debug'], - addToLoggerNames=(self.__user ,)) - - if self.__cfg['hod'].has_key('syslog-address'): - self.__baseLogger.add_syslog(self.__cfg['hod']['syslog-address'], - level=self.__cfg['hod']['debug'], - addToLoggerNames=(self.__user ,)) - - def get_logger(self): - return self.__log - - def __setup_cluster_logger(self, directory): - self.__baseLogger.add_file(logDirectory=directory, level=4, - backupCount=self.__cfg['hod']['log-rollover-count'], - addToLoggerNames=(self.__user ,)) - - def __setup_cluster_state(self, directory): - self.__clusterState = hodState(directory) - - def __norm_cluster_dir(self, directory): - directory = os.path.expanduser(directory) - if not os.path.isabs(directory): - directory = os.path.join(self.__cfg['hod']['original-dir'], directory) - directory = os.path.abspath(directory) - - return directory - - def __setup_service_registry(self): - cfg = self.__cfg['hod'].copy() - cfg['debug'] = 0 - self.__registry = svcrgy(cfg, self.__log) - self.__registry.start() - self.__log.debug(self.__registry.getXMLRPCAddr()) - self.__cfg['hod']['xrs-address'] = self.__registry.getXMLRPCAddr() - self.__cfg['ringmaster']['svcrgy-addr'] = self.__cfg['hod']['xrs-address'] - - def __set_cluster_state_info(self, env, hdfs, mapred, ring, jobid, min, max): - self.__clusterStateInfo['env'] = env - self.__clusterStateInfo['hdfs'] = "http://%s" % hdfs - self.__clusterStateInfo['mapred'] = "http://%s" % mapred - self.__clusterStateInfo['ring'] = ring - self.__clusterStateInfo['jobid'] = jobid - self.__clusterStateInfo['min'] = min - self.__clusterStateInfo['max'] = max - - def __set_user_state_info(self, info): - userState = self.__userState.read(CLUSTER_DATA_FILE) - for key in info.keys(): - userState[key] = info[key] - - self.__userState.write(CLUSTER_DATA_FILE, userState) - - def __remove_cluster(self, clusterDir): - clusterInfo = self.__userState.read(CLUSTER_DATA_FILE) - if clusterDir in clusterInfo: - del(clusterInfo[clusterDir]) - self.__userState.write(CLUSTER_DATA_FILE, clusterInfo) - - def __cleanup(self): - if self.__registry: self.__registry.stop() - - def __check_operation(self, operation): - opList = operation.split() - - if not opList[0] in self.__ops: - self.__log.critical("Invalid hod operation specified: %s" % operation) - self._op_help(None) - self.__opCode = 2 - - return opList - - def __adjustMasterFailureCountConfig(self, nodeCount): - # This method adjusts the ringmaster.max-master-failures variable - # to a value that is bounded by the a function of the number of - # nodes. - - maxFailures = self.__cfg['ringmaster']['max-master-failures'] - # Count number of masters required - depends on which services - # are external - masters = 0 - if not self.__cfg['gridservice-hdfs']['external']: - masters += 1 - if not self.__cfg['gridservice-mapred']['external']: - masters += 1 - - # So, if there are n nodes and m masters, we look atleast for - # all masters to come up. Therefore, atleast m nodes should be - # good, which means a maximum of n-m master nodes can fail. - maxFailedNodes = nodeCount - masters - - # The configured max number of failures is now bounded by this - # number. - self.__cfg['ringmaster']['max-master-failures'] = \ - min(maxFailures, maxFailedNodes) - - def _op_allocate(self, args): - operation = "allocate" - argLength = len(args) - min = 0 - max = 0 - errorFlag = False - errorMsgs = [] - - if argLength == 3: - nodes = args[2] - clusterDir = self.__norm_cluster_dir(args[1]) - - if not os.path.exists(clusterDir): - try: - os.makedirs(clusterDir) - except OSError, err: - errorFlag = True - errorMsgs.append("Could not create cluster directory. %s" \ - % (str(err))) - elif not os.path.isdir(clusterDir): - errorFlag = True - errorMsgs.append( \ - "Invalid cluster directory (--hod.clusterdir or -d) : " + \ - clusterDir + " : Not a directory") - - if int(nodes) < 3 : - errorFlag = True - errorMsgs.append("Invalid nodecount (--hod.nodecount or -n) : " + \ - "Must be >= 3. Given nodes: %s" % nodes) - if errorFlag: - for msg in errorMsgs: - self.__log.critical(msg) - self.__opCode = 3 - return - - if not self.__userState.checkStateFile(CLUSTER_DATA_FILE, \ - (os.R_OK, os.W_OK)): - self.__log.critical(INVALID_STATE_FILE_MSGS[2] % \ - self.__userState.get_state_file()) - self.__opCode = 1 - return - - clusterList = self.__userState.read(CLUSTER_DATA_FILE) - if clusterDir in clusterList.keys(): - self.__setup_cluster_state(clusterDir) - clusterInfo = self.__clusterState.read() - # Check if the job is not running. Only then can we safely - # allocate another cluster. Otherwise the user would need - # to deallocate and free up resources himself. - if clusterInfo.has_key('jobid') and \ - self.__cluster.is_cluster_deallocated(clusterInfo['jobid']): - self.__log.warn("Found a dead cluster at cluster directory '%s'. Deallocating it to allocate a new one." % (clusterDir)) - self.__remove_cluster(clusterDir) - self.__clusterState.clear() - else: - self.__log.critical("Found a previously allocated cluster at cluster directory '%s'. HOD cannot determine if this cluster can be automatically deallocated. Deallocate the cluster if it is unused." % (clusterDir)) - self.__opCode = 12 - return - - self.__setup_cluster_logger(clusterDir) - - (status, message) = self.__cluster.is_valid_account() - if status is not 0: - if message: - for line in message: - self.__log.critical("verify-account output: %s" % line) - self.__log.critical("Cluster cannot be allocated because account verification failed. " \ - + "verify-account returned exit code: %s." % status) - self.__opCode = 4 - return - else: - self.__log.debug("verify-account returned zero exit code.") - if message: - self.__log.debug("verify-account output: %s" % message) - - if re.match('\d+-\d+', nodes): - (min, max) = nodes.split("-") - min = int(min) - max = int(max) - else: - try: - nodes = int(nodes) - min = nodes - max = nodes - except ValueError: - print self.__hodhelp.help(operation) - self.__log.critical( - "%s operation requires a pos_int value for n(nodecount)." % - operation) - self.__opCode = 3 - else: - self.__setup_cluster_state(clusterDir) - clusterInfo = self.__clusterState.read() - self.__opCode = self.__cluster.check_cluster(clusterInfo) - if self.__opCode == 0 or self.__opCode == 15: - self.__setup_service_registry() - if hodInterrupt.isSet(): - self.__cleanup() - raise HodInterruptException() - self.__log.debug("Service Registry started.") - - self.__adjustMasterFailureCountConfig(nodes) - - try: - allocateStatus = self.__cluster.allocate(clusterDir, min, max) - except HodInterruptException, h: - self.__cleanup() - raise h - # Allocation has gone through. - # Don't care about interrupts any more - - try: - if allocateStatus == 0: - self.__set_cluster_state_info(os.environ, - self.__cluster.hdfsInfo, - self.__cluster.mapredInfo, - self.__cluster.ringmasterXRS, - self.__cluster.jobId, - min, max) - self.__setup_cluster_state(clusterDir) - self.__clusterState.write(self.__cluster.jobId, - self.__clusterStateInfo) - # Do we need to check for interrupts here ?? - - self.__set_user_state_info( - { clusterDir : self.__cluster.jobId, } ) - self.__opCode = allocateStatus - except Exception, e: - # Some unknown problem. - self.__cleanup() - self.__cluster.deallocate(clusterDir, self.__clusterStateInfo) - self.__opCode = 1 - raise Exception(e) - elif self.__opCode == 12: - self.__log.critical("Cluster %s already allocated." % clusterDir) - elif self.__opCode == 10: - self.__log.critical("dead\t%s\t%s" % (clusterInfo['jobid'], - clusterDir)) - elif self.__opCode == 13: - self.__log.warn("hdfs dead\t%s\t%s" % (clusterInfo['jobid'], - clusterDir)) - elif self.__opCode == 14: - self.__log.warn("mapred dead\t%s\t%s" % (clusterInfo['jobid'], - clusterDir)) - - if self.__opCode > 0 and self.__opCode != 15: - self.__log.critical("Cannot allocate cluster %s" % clusterDir) - else: - print self.__hodhelp.help(operation) - self.__log.critical("%s operation requires two arguments. " % operation - + "A cluster directory and a nodecount.") - self.__opCode = 3 - - def _is_cluster_allocated(self, clusterDir): - if os.path.isdir(clusterDir): - self.__setup_cluster_state(clusterDir) - clusterInfo = self.__clusterState.read() - if clusterInfo != {}: - return True - return False - - def _op_deallocate(self, args): - operation = "deallocate" - argLength = len(args) - if argLength == 2: - clusterDir = self.__norm_cluster_dir(args[1]) - if os.path.isdir(clusterDir): - self.__setup_cluster_state(clusterDir) - clusterInfo = self.__clusterState.read() - if clusterInfo == {}: - self.__handle_invalid_cluster_directory(clusterDir, cleanUp=True) - else: - self.__opCode = \ - self.__cluster.deallocate(clusterDir, clusterInfo) - # irrespective of whether deallocate failed or not\ - # remove the cluster state. - self.__clusterState.clear() - if not self.__userState.checkStateFile(CLUSTER_DATA_FILE, (os.W_OK,)): - self.__log.critical(INVALID_STATE_FILE_MSGS[3] % \ - self.__userState.get_state_file()) - self.__opCode = 1 - return - self.__remove_cluster(clusterDir) - else: - self.__handle_invalid_cluster_directory(clusterDir, cleanUp=True) - else: - print self.__hodhelp.help(operation) - self.__log.critical("%s operation requires one argument. " % operation - + "A cluster path.") - self.__opCode = 3 - - def _op_list(self, args): - operation = 'list' - clusterList = self.__userState.read(CLUSTER_DATA_FILE) - for path in clusterList.keys(): - if not os.path.isdir(path): - self.__log.info("cluster state unknown\t%s\t%s" % (clusterList[path], path)) - continue - self.__setup_cluster_state(path) - clusterInfo = self.__clusterState.read() - if clusterInfo == {}: - # something wrong with the cluster directory. - self.__log.info("cluster state unknown\t%s\t%s" % (clusterList[path], path)) - continue - clusterStatus = self.__cluster.check_cluster(clusterInfo) - if clusterStatus == 12: - self.__log.info("alive\t%s\t%s" % (clusterList[path], path)) - elif clusterStatus == 10: - self.__log.info("dead\t%s\t%s" % (clusterList[path], path)) - elif clusterStatus == 13: - self.__log.info("hdfs dead\t%s\t%s" % (clusterList[path], path)) - elif clusterStatus == 14: - self.__log.info("mapred dead\t%s\t%s" % (clusterList[path], path)) - - def _op_info(self, args): - operation = 'info' - argLength = len(args) - if argLength == 2: - clusterDir = self.__norm_cluster_dir(args[1]) - if os.path.isdir(clusterDir): - self.__setup_cluster_state(clusterDir) - clusterInfo = self.__clusterState.read() - if clusterInfo == {}: - # something wrong with the cluster directory. - self.__handle_invalid_cluster_directory(clusterDir) - else: - clusterStatus = self.__cluster.check_cluster(clusterInfo) - if clusterStatus == 12: - self.__print_cluster_info(clusterInfo) - self.__log.info("hadoop-site.xml at %s" % clusterDir) - elif clusterStatus == 10: - self.__log.critical("%s cluster is dead" % clusterDir) - elif clusterStatus == 13: - self.__log.warn("%s cluster hdfs is dead" % clusterDir) - elif clusterStatus == 14: - self.__log.warn("%s cluster mapred is dead" % clusterDir) - - if clusterStatus != 12: - if clusterStatus == 15: - self.__log.critical("Cluster %s not allocated." % clusterDir) - else: - self.__print_cluster_info(clusterInfo) - self.__log.info("hadoop-site.xml at %s" % clusterDir) - - self.__opCode = clusterStatus - else: - self.__handle_invalid_cluster_directory(clusterDir) - else: - print self.__hodhelp.help(operation) - self.__log.critical("%s operation requires one argument. " % operation - + "A cluster path.") - self.__opCode = 3 - - def __handle_invalid_cluster_directory(self, clusterDir, cleanUp=False): - if not self.__userState.checkStateFile(CLUSTER_DATA_FILE, (os.R_OK,)): - self.__log.critical(INVALID_STATE_FILE_MSGS[0] % \ - self.__userState.get_state_file()) - self.__opCode = 1 - return - - clusterList = self.__userState.read(CLUSTER_DATA_FILE) - if clusterDir in clusterList.keys(): - # previously allocated cluster. - self.__log.critical("Cannot find information for cluster with id '%s' in previously allocated cluster directory '%s'." % (clusterList[clusterDir], clusterDir)) - if cleanUp: - self.__cluster.delete_job(clusterList[clusterDir]) - self.__log.critical("Freeing resources allocated to the cluster.") - if not self.__userState.checkStateFile(CLUSTER_DATA_FILE, (os.W_OK,)): - self.__log.critical(INVALID_STATE_FILE_MSGS[1] % \ - self.__userState.get_state_file()) - self.__opCode = 1 - return - self.__remove_cluster(clusterDir) - self.__opCode = 3 - else: - if not os.path.exists(clusterDir): - self.__log.critical( \ - "Invalid hod.clusterdir(--hod.clusterdir or -d). " + \ - clusterDir + " : No such directory") - elif not os.path.isdir(clusterDir): - self.__log.critical( \ - "Invalid hod.clusterdir(--hod.clusterdir or -d). " + \ - clusterDir + " : Not a directory") - else: - self.__log.critical( \ - "Invalid hod.clusterdir(--hod.clusterdir or -d). " + \ - clusterDir + " : Not tied to any allocated cluster.") - self.__opCode = 15 - - def __print_cluster_info(self, clusterInfo): - keys = clusterInfo.keys() - - _dict = { - 'jobid' : 'Cluster Id', 'min' : 'Nodecount', - 'hdfs' : 'HDFS UI at' , 'mapred' : 'Mapred UI at' - } - - for key in _dict.keys(): - if clusterInfo.has_key(key): - self.__log.info("%s %s" % (_dict[key], clusterInfo[key])) - - if clusterInfo.has_key('ring'): - self.__log.debug("%s\t%s" % ('Ringmaster at ', clusterInfo['ring'])) - - if self.__cfg['hod']['debug'] == 4: - for var in clusterInfo['env'].keys(): - self.__log.debug("%s = %s" % (var, clusterInfo['env'][var])) - - def _op_help(self, arg): - if arg == None or arg.__len__() != 2: - print "hod commands:\n" - for op in self.__ops: - print self.__hodhelp.help(op) - else: - if arg[1] not in self.__ops: - print self.__hodhelp.help('help') - self.__log.critical("Help requested for invalid operation : %s"%arg[1]) - self.__opCode = 3 - else: print self.__hodhelp.help(arg[1]) - - def operation(self): - operation = self.__cfg['hod']['operation'] - try: - opList = self.__check_operation(operation) - if self.__opCode == 0: - if not self.__userState.checkStateFile(CLUSTER_DATA_FILE, (os.R_OK,)): - self.__log.critical(INVALID_STATE_FILE_MSGS[0] % \ - self.__userState.get_state_file()) - self.__opCode = 1 - return self.__opCode - getattr(self, "_op_%s" % opList[0])(opList) - except HodInterruptException, h: - self.__log.critical("op: %s failed because of a process interrupt." \ - % operation) - self.__opCode = HOD_INTERRUPTED_CODE - except: - self.__log.critical("op: %s failed: %s" % (operation, - get_exception_error_string())) - self.__log.debug(get_exception_string()) - - self.__cleanup() - - self.__log.debug("return code: %s" % self.__opCode) - - return self.__opCode - - def script(self): - errorFlag = False - errorMsgs = [] - scriptRet = 0 # return from the script, if run - - script = self.__cfg['hod']['script'] - nodes = self.__cfg['hod']['nodecount'] - clusterDir = self.__cfg['hod']['clusterdir'] - - if not os.path.exists(script): - errorFlag = True - errorMsgs.append("Invalid script file (--hod.script or -s) : " + \ - script + " : No such file") - elif not os.path.isfile(script): - errorFlag = True - errorMsgs.append("Invalid script file (--hod.script or -s) : " + \ - script + " : Not a file.") - else: - isExecutable = os.access(script, os.X_OK) - if not isExecutable: - errorFlag = True - errorMsgs.append("Invalid script file (--hod.script or -s) : " + \ - script + " : Not an executable.") - - if not os.path.exists(clusterDir): - try: - os.makedirs(clusterDir) - except OSError, err: - errorFlag = True - errorMsgs.append("Could not create cluster directory. %s" % (str(err))) - elif not os.path.isdir(clusterDir): - errorFlag = True - errorMsgs.append( \ - "Invalid cluster directory (--hod.clusterdir or -d) : " + \ - clusterDir + " : Not a directory") - - if int(self.__cfg['hod']['nodecount']) < 3 : - errorFlag = True - errorMsgs.append("Invalid nodecount (--hod.nodecount or -n) : " + \ - "Must be >= 3. Given nodes: %s" % nodes) - - if errorFlag: - for msg in errorMsgs: - self.__log.critical(msg) - self.handle_script_exit_code(scriptRet, clusterDir) - sys.exit(3) - - try: - self._op_allocate(('allocate', clusterDir, str(nodes))) - if self.__opCode == 0: - if self.__cfg['hod'].has_key('script-wait-time'): - time.sleep(self.__cfg['hod']['script-wait-time']) - self.__log.debug('Slept for %d time. Now going to run the script' % self.__cfg['hod']['script-wait-time']) - if hodInterrupt.isSet(): - self.__log.debug('Hod interrupted - not executing script') - else: - scriptRunner = hadoopScript(clusterDir, - self.__cfg['hod']['original-dir']) - self.__opCode = scriptRunner.run(script) - scriptRet = self.__opCode - self.__log.info("Exit code from running the script: %d" % self.__opCode) - else: - self.__log.critical("Error %d in allocating the cluster. Cannot run the script." % self.__opCode) - - if hodInterrupt.isSet(): - # Got interrupt while executing script. Unsetting it for deallocating - hodInterrupt.setFlag(False) - if self._is_cluster_allocated(clusterDir): - self._op_deallocate(('deallocate', clusterDir)) - except HodInterruptException, h: - self.__log.critical("Script failed because of a process interrupt.") - self.__opCode = HOD_INTERRUPTED_CODE - except: - self.__log.critical("script: %s failed: %s" % (script, - get_exception_error_string())) - self.__log.debug(get_exception_string()) - - self.__cleanup() - - self.handle_script_exit_code(scriptRet, clusterDir) - - return self.__opCode - - def handle_script_exit_code(self, scriptRet, clusterDir): - # We want to give importance to a failed script's exit code, and write out exit code to a file separately - # so users can easily get it if required. This way they can differentiate between the script's exit code - # and hod's exit code. - if os.path.exists(clusterDir): - exit_code_file_name = (os.path.join(clusterDir, 'script.exitcode')) - if scriptRet != 0: - exit_code_file = open(exit_code_file_name, 'w') - print >>exit_code_file, scriptRet - exit_code_file.close() - self.__opCode = scriptRet - else: - #ensure script exit code file is not there: - if (os.path.exists(exit_code_file_name)): - os.remove(exit_code_file_name) - -class hodHelp: - def __init__(self): - self.ops = ['allocate', 'deallocate', 'info', 'list','script', 'help'] - - self.usage_strings = \ - { - 'allocate' : 'hod allocate -d -n [OPTIONS]', - 'deallocate' : 'hod deallocate -d [OPTIONS]', - 'list' : 'hod list [OPTIONS]', - 'info' : 'hod info -d [OPTIONS]', - 'script' : - 'hod script -d -n -s