diff --git a/Makefile b/Makefile index 1e49402a1b0e335dfa5153bb01bafe70aaa34259..c5d004fb108e785581def5ca35961bd07ad8e2c5 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 e0c7603632f1b37d1e275f539285812c915ca91f..f084f22a1f9dc5891bc4340c0ceb14ca9152a745 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 <class> <params>. 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 <host> 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 0000000000000000000000000000000000000000..b61e8163b57ef77ebb9a228bb09016bedfe66714 --- /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 0000000000000000000000000000000000000000..d72dbadc3904f327effddf99594045067be2f529 --- /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 0000000000000000000000000000000000000000..77f9cb69b96724905aa065203238b22e9121d454 --- /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 00696b2c1ab02d1599712f28b363cdbe7a103363..f28b39af9b084e84b3a77b51dda3777593694b2a 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 ee847cfff03ebb9a78017f078a287ecc4341949c..0f9b9b1ece1a27bef97be6fe9c705a5ee9e6632f 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 40c2be8f6d26def3bb459200edabb803f9d356d7..776401341368be940402691e726cf4db9e86f6c4 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 7c4dce586960e1db8123dfe7fadb199a4e32d736..6fae3c0940cf5a147c4b3b2e32a5131d67175b0b 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 0ae2cbc6e8d16cff6df992e965f8deb63cb4a86d..e9fbdca7523a36e8067b1d34e69960b35039d410 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 3e4cd4935a19cce8599b0dac05046b5fc7966bae..ee93d3c85c0e9e318d8f0a1449a9fe81f4d6da07 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 b2114eb15107a6d50d80599482fe0501b632d3df..23ab9b345c1768abd09c617db77fc48cba2d1b0f 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 8037434c387ad0777774519fb33471e989f6813b..0e0b3954d4db9703da58957b19b5d0af18f3e6dc 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 679a61f3c0b9670814262d55340a5c963b75f85a..be73aae541d6822f2fdea62350aa09b1ee4e1a5f 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 6aa0e2233894666cff75b18a22e841f518f5b298..8637c6e30aa47bf08b441bc5fe6fad2214e1cfc1 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 b33f3c863ef414d2f8475edd00265c3b60bb92a0..20954a1224243b68565cdbb63d420656ab345dbd 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 0000000000000000000000000000000000000000..2d1feebbb1ede64fe2899d6a9608b6477e965991 --- /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/NexusScheduler.scala b/src/scala/spark/MesosScheduler.scala similarity index 50% rename from src/scala/spark/NexusScheduler.scala rename to src/scala/spark/MesosScheduler.scala index a5343039ef174f4c7393003efe0490a61309e757..873a97c59c14436bad326ac985c746e5b1750508 100644 --- a/src/scala/spark/NexusScheduler.scala +++ b/src/scala/spark/MesosScheduler.scala @@ -3,11 +3,14 @@ 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 nexus.{Scheduler => NScheduler} -import nexus._ +import mesos.{Scheduler => NScheduler} +import mesos._ -// The main Scheduler implementation, which talks to Nexus. Clients are expected +// 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 @@ -18,9 +21,9 @@ import nexus._ // 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( +private class MesosScheduler( master: String, frameworkName: String, execArg: Array[Byte]) -extends NScheduler with spark.Scheduler +extends NScheduler with spark.Scheduler with Logging { // Lock used by runTasks to ensure only one thread can be in it val runTasksMutex = new Object() @@ -30,7 +33,16 @@ extends NScheduler with spark.Scheduler val registeredLock = new Object() // Current callback object (may be null) - var activeOp: ParallelOperation = 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 @@ -41,15 +53,15 @@ extends NScheduler with spark.Scheduler return id } - // Driver for talking to Nexus + // Driver for talking to Mesos 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) + val ns = MesosScheduler.this + ns.driver = new MesosSchedulerDriver(ns, master) ns.driver.run() } }.start @@ -61,31 +73,35 @@ extends NScheduler with spark.Scheduler 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) + var opId = 0 + waitForRegister() + this.synchronized { + opId = newOpId() + } + val myOp = new SimpleParallelOperation(this, tasks, opId) - try { - this.synchronized { - this.activeOp = myOp - } - driver.reviveOffers(); - myOp.join(); - } finally { - this.synchronized { - this.activeOp = null - } + 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 } + + 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) + override def registered(d: SchedulerDriver, frameworkId: String) { + logInfo("Registered as framework ID " + frameworkId) registeredLock.synchronized { isRegistered = true registeredLock.notifyAll() @@ -100,23 +116,34 @@ extends NScheduler with spark.Scheduler } override def resourceOffer( - d: SchedulerDriver, oid: Long, offers: SlaveOfferVector) { + d: SchedulerDriver, oid: String, offers: java.util.List[SlaveOffer]) { synchronized { - val tasks = new TaskDescriptionVector - if (activeOp != null) { - try { + 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) { - activeOp.slaveOffer(offers.get(i)) match { - case Some(task) => tasks.add(task) - case None => {} + 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) } } - } catch { - case e: Exception => e.printStackTrace } - } - val params = new StringMap - params.set("timeout", "1") + } + val params = new java.util.HashMap[String, String] + params.put("timeout", "1") d.replyToOffer(oid, tasks, params) // TODO: use smaller timeout } } @@ -124,26 +151,33 @@ extends NScheduler with spark.Scheduler override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { synchronized { try { - if (activeOp != null) { - activeOp.statusUpdate(status) + 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 => e.printStackTrace + case e: Exception => logError("Exception in statusUpdate", e) } } } override def error(d: SchedulerDriver, code: Int, message: String) { synchronized { - if (activeOp != null) { - try { - activeOp.error(code, message) - } catch { - case e: Exception => e.printStackTrace + if (activeOps.size > 0) { + for ((opId, activeOp) <- activeOps) { + try { + activeOp.error(code, message) + } catch { + case e: Exception => logError("Exception in error callback", e) + } } } else { - val msg = "Nexus error: %s (error code: %d)".format(message, code) - System.err.println(msg) + logError("Mesos error: %s (error code: %d)".format(message, code)) System.exit(1) } } @@ -153,24 +187,27 @@ extends NScheduler with spark.Scheduler 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): Option[TaskDescription] + 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: NexusScheduler, tasks: Array[Task[T]]) -extends ParallelOperation + 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", "1000").toLong + val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong val callingThread = currentThread val numTasks = tasks.length @@ -204,7 +241,7 @@ extends ParallelOperation } } - def slaveOffer(offer: SlaveOffer): Option[TaskDescription] = { + def slaveOffer(offer: SlaveOffer, availableCpus: Int, availableMem: Int): Option[TaskDescription] = { if (tasksLaunched < numTasks) { var checkPrefVals: Array[Boolean] = Array(true) val time = System.currentTimeMillis @@ -212,9 +249,8 @@ extends ParallelOperation 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) + val desiredMem = 500 + if ((availableCpus < desiredCpus) || (availableMem < desiredMem)) return None for (checkPref <- checkPrefVals; i <- 0 until numTasks) { if (!launched(i) && (!checkPref || @@ -222,19 +258,23 @@ extends ParallelOperation tasks(i).preferredLocations.isEmpty)) { val taskId = sched.newTaskId() + sched.taskIdToOpId(taskId) = opId 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") + 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 StringMap - params.set("cpus", "" + desiredCpus) - params.set("mem", "" + desiredMem) + 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)) } @@ -259,24 +299,40 @@ extends ParallelOperation 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() + 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 - println("Lost TID " + tid) - launched(tidToIndex(tid)) = false - tasksLaunched -= 1 + 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) { diff --git a/src/scala/spark/ParallelArray.scala b/src/scala/spark/ParallelArray.scala index 39ca867cb9afae242871abd3d10851f43d8b2a8d..a01904d61c5d6ebb927f4e530a7a958e348c0f7f 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 f9a16ed7820c7f4a0bdaca0dbfef3a4f36060394..803c0638653c0aba4677e2f68e28883a6f72b053 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 27cf48e9d28cac8157a3b056d15b59cf949abd96..b9f3128c822c8fea988e1d723574db4b42b63781 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 50d8e4908a8dbfb40cc8d444658debf5cdc5822b..216ea4c0a926e8229ac1708ac94967e50091e7ad 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 0000000000000000000000000000000000000000..0f7a21354ddd152ed618269d77619ce5bcdf081a --- /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 efb864472dea6323da97f03d54d52945e152fda8..6e94009f6ee972a24e180d43881fa2dadd3b142e 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 52bcb89f003fc226c93c8c74d6e2d9bc36efa49f..27d73aefbd69420f0e6aca4ed33e7cb339676cf7 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 0000000000000000000000000000000000000000..6a40d92765b922cd9ca72fd9d91e13a7667288ad --- /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 7d91b20e792e1c0290cfc9433268768ddf77410a..13d81ec1cf096befaff8b7e9901b783faa17212e 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", "<init>", "()V") diff --git a/src/scala/spark/repl/SparkInterpreter.scala b/src/scala/spark/repl/SparkInterpreter.scala index 6237c83625453d19ba288f35911e3bb0357c7e60..ae2e7e8a681a5d9c466c8e0db804ad9c1052aa6d 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 26361fdc255942f284c4cad81c779539b54c8e1f..5bad0a37daaee8c7ac87d66870f4fe166cd2e816 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 bdca83a2d534f6ab881b6c218cf4384511a15f43..0000000000000000000000000000000000000000 --- 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 6854acd6a5c198ae2e7f75484b79bf5e23dc3c69..0000000000000000000000000000000000000000 --- 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 153542f8de9fc89a3511653586725e2dce253e93..0000000000000000000000000000000000000000 --- 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 328b73c8288bb395223660f795540b00a74d76cd..0000000000000000000000000000000000000000 --- 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 9ce0fd4f442fb2c8a1aaae48fe1cba88b03e8de6..0000000000000000000000000000000000000000 --- 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 d6fd3f01819b7eef1a1e7a928cfd6110a799b3db..0000000000000000000000000000000000000000 --- 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 0000000000000000000000000000000000000000..3f9d847618bcec9c81f36ad6ee76a0f53816987f 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/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 36aa65a06900cbe15977d1f48f0896aec6d2141c..0000000000000000000000000000000000000000 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/google-collect-1.0-rc5/COPYING b/third_party/guava-r06/COPYING similarity index 100% rename from third_party/google-collect-1.0-rc5/COPYING rename to third_party/guava-r06/COPYING diff --git a/third_party/guava-r06/README b/third_party/guava-r06/README new file mode 100644 index 0000000000000000000000000000000000000000..a0e832dd542ed0937a580b77f7c0fc43ea9d0fa2 --- /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 0000000000000000000000000000000000000000..8ff3a81748d5a96a1605a2c73ef0a29cf48d3ef5 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 95cf0710fe626688b56e3c68f990eb8026a41ee0..0000000000000000000000000000000000000000 --- 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 aaa7d35c3e87e1b2663611393d9cbd1d3d4140b7..0000000000000000000000000000000000000000 --- 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 5a03fb737b381f0b07b830ba4bb6e87342a7a914..0000000000000000000000000000000000000000 --- 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 5f9f92f1661f2f587a71138d249f9bd1f7df4296..0000000000000000000000000000000000000000 --- 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 e87b2764dbbf132fcd1c129ffcb14e985bba7393..0000000000000000000000000000000000000000 --- 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 51613eae0a1139b0fc85f53b23df2572ba8bcd7a..0000000000000000000000000000000000000000 --- 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 1bb891c54084cdc3679bf29fc70507ed85eac1d5..0000000000000000000000000000000000000000 --- 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 fc194f6d27e6194f121d718168f35b09a635174c..0000000000000000000000000000000000000000 --- 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 65aa79ab44edd2ff74ad2bbb902681ef0e835c17..0000000000000000000000000000000000000000 --- 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 e16b36dacffd4fbfaf81f155e21685082d96024c..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/build.xml +++ /dev/null @@ -1,81 +0,0 @@ -<?xml version="1.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. ---> - -<project name="hod" default="compile"> - <import file="../build-contrib.xml"/> - <target name="compile"> - <mkdir dir="${build.dir}"/> - <copy todir="${build.dir}"> - <fileset dir="${basedir}"> - <exclude name="**/VERSION"/> - </fileset> - </copy> - <exec executable="echo" output="${build.dir}/bin/VERSION"> - <arg line="${version}" /> - </exec> - </target> - <target name="package" depends="compile"> - <mkdir dir="${dist.dir}/contrib/${name}"/> - <copy todir="${dist.dir}/contrib/${name}"> - <fileset dir="${build.dir}"/> - </copy> - <chmod dir="${dist.dir}/contrib/${name}/bin" perm="a+x" includes="*"/> - </target> - - <target name="test" depends="compile" description="Run HOD unit tests"> - <antcall target="python.pathcheck"/> - <antcall target="checkAndRunTests"/> - </target> - - <target name="checkAndRunTests" if="python.home"> - <!-- Check python version now --> - <exec executable="/bin/sh" outputproperty="hodtest.pythonVersion"> - <arg value="-c" /> - <arg value="${python.home}/python -V" /> - </exec> - <condition property="python.versionmatched"> - <!--- Currently check for only 2.5.1 --> - <equals arg1="${hodtest.pythonVersion}" arg2="Python 2.5.1" /> - </condition> - <antcall target="python.versioncheck"/> - <antcall target="runtests"/> - </target> - - <target name="python.pathcheck" unless="python.home"> - <echo message="'python.home' is not defined. Please pass -Dpython.home=<Path to Python> to Ant on the command-line."/> - </target> - - <target name="runtests" if="python.versionmatched"> - <echo message="Using Python at : ${python.home}" /> - <echo message="Version : ${hodtest.pythonVersion}"/> - <exec executable="/bin/sh" resultproperty="hodtest.failedTests"> - <arg value="-c" /> - <arg value="${python.home}/python ${build.dir}/testing/main.py" /> - </exec> - <condition property="hodtest.success"> - <equals arg1="${hodtest.failedTests}" arg2="0"/> - </condition> - <fail message="TestCases failed. ${hodtest.failedTests} failed to run successfully." unless="hodtest.success"/> - </target> - - <target name="python.versioncheck" unless="python.versionmatched"> - <echo message="Need Python version 2.5.1. You specified ${hodtest.pythonVersion}"/> - </target> - -</project> 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 bc2866d4c887712434218225ee92596cedd8ee75..0000000000000000000000000000000000000000 --- 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 ca894a702abcd684de4edec98fc4dc3c076604f8..0000000000000000000000000000000000000000 --- 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 - <install-location>/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 ae2b0738f9df4c6a060e4488cf54afdd868ae080..0000000000000000000000000000000000000000 --- 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 <install dir>/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 56759d796307be4bb5fc9ff6b4ec461937c483fd..0000000000000000000000000000000000000000 --- 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 2794c50354bed44441066106960e4a06b5590d3b..0000000000000000000000000000000000000000 --- 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 = '<Body><Request action="Quote" actor="hod"><Object>Job</Object><Data><Job><ProjectId>%s</ProjectId><UserId>%s</UserId><WallDuration>10</WallDuration></Job></Data></Request></Body>' % (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 = '<?xml version="1.0" encoding="UTF-8"?>\ -<Envelope>%s<Signature><DigestValue>%s</DigestValue><SignatureValue>%s</SignatureValue><SecurityToken type="Symmetric"></SecurityToken></Signature></Envelope>' % (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 12c2f1e1da706ef7b66c7ec27e762b2287340d9f..0000000000000000000000000000000000000000 --- 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 515e875070156ef44ce3b50a06b18dabcb3baa83..0000000000000000000000000000000000000000 --- 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 013e3bde02100f3f092203a181350dc710b6cdf7..0000000000000000000000000000000000000000 --- 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 03852cca64e703c00f015597a3b5408c474d8ba7..0000000000000000000000000000000000000000 --- 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 e042fe13b730e0b23e424b052865b17c486def9d..0000000000000000000000000000000000000000 --- 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 3101ab2cde8c89830dfff18dc184a89102918fe8..0000000000000000000000000000000000000000 --- 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 34a0fd012415808c91d3e75157abe3d34f164e88..0000000000000000000000000000000000000000 --- 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 d733780ec1fa3323fc7e05ec0c18abc5bf573b61..0000000000000000000000000000000000000000 --- 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 791b095c9bae6bc79ae15f682ee6cfa5c8d3a41b..0000000000000000000000000000000000000000 --- 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"(?<!\\)=" -reKeyVal = re.compile(reKeyVal) -reKeyValList = r"(?<!\\)," -reKeyValList = re.compile(reKeyValList) - -errorPrefix = 'error' -requiredPerms = '0660' - -class definition: - def __init__(self): - """Generates a configuration definition object.""" - self.__def = {} - self.__defOrder = [] - - def __repr__(self): - return pprint.pformat(self.__def) - - def __getitem__(self, section): - return self.__def[section] - - def __iter__(self): - return iter(self.__def) - - def sections(self): - """Returns a list of sections/groups.""" - - if len(self.__defOrder): - return self.__defOrder - else: - return self.__def.keys() - - def add_section(self, section): - """Add a configuration section / option group.""" - - if self.__def.has_key(section): - raise Exception("Section already exists: '%s'" % section) - else: - self.__def[section] = {} - - def add_def(self, section, var, type, desc, help = True, default = None, - req = True, validate = True, short = None): - """ Add a variable definition. - - section - section name - var - variable name - type - valid hodlib.types - desc - description of variable - help - display help for this variable - default - default value - req - bool, requried? - validate - bool, validate type value? - short - short symbol (1 character), - help - bool, display help?""" - - if self.__def.has_key(section): - if not is_valid_type(type): - raise Exception("Type (type) is invalid: %s.%s - '%s'" % (section, var, - type)) - if not isinstance(desc, str): - raise Exception("Description (desc) must be a string: %s.%s - '%s'" % ( - section, var, desc)) - if not isinstance(req, bool): - raise Exception("Required (req) must be a bool: %s.%s - '%s'" % (section, - var, - req)) - if not isinstance(validate, bool): - raise Exception("Validate (validate) must be a bool: %s.%s - '%s'" % ( - section, var, validate)) - - if self.__def[section].has_key(var): - raise Exception("Variable name already defined: '%s'" % var) - else: - self.__def[section][var] = { 'type' : type, - 'desc' : desc, - 'help' : help, - 'default' : default, - 'req' : req, - 'validate' : validate, - 'short' : short } - else: - raise Exception("Section does not exist: '%s'" % section) - - def add_defs(self, defList, defOrder=None): - """ Add a series of definitions. - - defList = { section0 : ((name0, - type0, - desc0, - help0, - default0, - req0, - validate0, - short0), - .... - (nameN, - typeN, - descN, - helpN, - defaultN, - reqN, - validateN, - shortN)), - .... - - sectionN : ... } - - Where the short synmbol is optional and can only be one char.""" - - for section in defList.keys(): - self.add_section(section) - for defTuple in defList[section]: - if isinstance(defTuple, tuple): - if len(defTuple) < 7: - raise Exception( - "section %s is missing an element: %s" % ( - section, pprint.pformat(defTuple))) - else: - raise Exception("section %s of defList is not a tuple" % - section) - - if len(defTuple) == 7: - self.add_def(section, defTuple[0], defTuple[1], - defTuple[2], defTuple[3], defTuple[4], - defTuple[5], defTuple[6]) - else: - self.add_def(section, defTuple[0], defTuple[1], - defTuple[2], defTuple[3], defTuple[4], - defTuple[5], defTuple[6], defTuple[7]) - if defOrder: - for section in defOrder: - if section in self.__def: - self.__defOrder.append(section) - - for section in self.__def: - if not section in defOrder: - raise Exception( - "section %s is missing from specified defOrder." % - section) - -class baseConfig: - def __init__(self, configDef, originalDir=None): - self.__toString = typeToString() - self.__validated = False - self._configDef = configDef - self._options = None - self._mySections = [] - self._dict = {} - self.configFile = None - self.__originalDir = originalDir - - if self._configDef: - self._mySections = configDef.sections() - - def __repr__(self): - """Returns a string representation of a config object including all - normalizations.""" - - print_string = ''; - for section in self._mySections: - print_string = "%s[%s]\n" % (print_string, section) - options = self._dict[section].keys() - for option in options: - print_string = "%s%s = %s\n" % (print_string, option, - self._dict[section][option]) - - print_string = "%s\n" % (print_string) - - print_string = re.sub("\n\n$", "", print_string) - - return print_string - - def __getitem__(self, section): - """ Returns a dictionary of configuration name and values by section. - """ - return self._dict[section] - - def __setitem__(self, section, value): - self._dict[section] = value - - def __iter__(self): - return iter(self._dict) - - def has_key(self, section): - status = False - if section in self._dict: - status = True - - return status - - # Prints configuration error messages - def var_error(self, section, option, *addData): - errorStrings = [] - if not self._dict[section].has_key(option): - self._dict[section][option] = None - errorStrings.append("%s: invalid '%s' specified in section %s (--%s.%s): %s" % ( - errorPrefix, option, section, section, option, self._dict[section][option])) - - if addData: - errorStrings.append("%s: additional info: %s\n" % (errorPrefix, - addData[0])) - return errorStrings - - def var_error_suggest(self, errorStrings): - if self.configFile: - errorStrings.append("Check your command line options and/or " + \ - "your configuration file %s" % self.configFile) - - def __get_args(self, section): - def __dummyToString(type, value): - return value - - toString = __dummyToString - if self.__validated: - toString = self.__toString - - args = [] - if isinstance(self._dict[section], dict): - for option in self._dict[section]: - if section in self._configDef and \ - option in self._configDef[section]: - if self._configDef[section][option]['type'] == 'bool': - if self._dict[section][option] == 'True' or \ - self._dict[section][option] == True: - args.append("--%s.%s" % (section, option)) - else: - args.append("--%s.%s" % (section, option)) - args.append(toString( - self._configDef[section][option]['type'], - self._dict[section][option])) - else: - if section in self._configDef: - if self._configDef[section][option]['type'] == 'bool': - if self._dict[section] == 'True' or \ - self._dict[section] == True: - args.append("--%s" % section) - else: - if self._dict[section] != 'config': - args.append("--%s" % section) - args.append(toString(self._configDef[section]['type'], - self._dict[section])) - - return args - - def values(self): - return self._dict.values() - - def keys(self): - return self._dict.keys() - - def get_args(self, exclude=None, section=None): - """Retrieve a tuple of config arguments.""" - - args = [] - if section: - args = self.__get_args(section) - else: - for section in self._dict: - if exclude: - if not section in exclude: - args.extend(self.__get_args(section)) - else: - args.extend(self.__get_args(section)) - - return tuple(args) - - def verify(self): - """Verifies each configuration variable, using the configValidator - class, based on its type as defined by the dictionary configDef. - Upon encountering a problem an error is printed to STDERR and - false is returned.""" - - oldDir = os.getcwd() - if self.__originalDir: - os.chdir(self.__originalDir) - - status = True - statusMsgs = [] - - if self._configDef: - errorCount = 0 - configValidator = typeValidator(self.__originalDir) - - # foreach section and option by type string as defined in configDef - # add value to be validated to validator - for section in self._mySections: - for option in self._configDef[section].keys(): - configVarName = "%s.%s" % (section, option) - - if self._dict[section].has_key(option): - if self._configDef[section][option].has_key('validate'): - if self._configDef[section][option]['validate']: - # is the section.option needed to be validated? - configValidator.add(configVarName, - self._configDef[section][option]['type'], - self._dict[section][option]) - else: - # If asked not to validate, just normalize - self[section][option] = \ - configValidator.normalize( - self._configDef[section][option]['type'], - self._dict[section][option]) - if self._configDef[section][option]['default'] != \ - None: - self._configDef[section][option]['default'] = \ - configValidator.normalize( - self._configDef[section][option]['type'], - self._configDef[section][option]['default'] - ) - self._configDef[section][option]['default'] = \ - self.__toString( - self._configDef[section][option]['type'], - self._configDef[section][option]['default'] - ) - else: - # This should not happen. Just in case, take this as 'to be validated' case. - configValidator.add(configVarName, - self._configDef[section][option]['type'], - self._dict[section][option]) - elif self._configDef[section][option]['req']: - statusMsgs.append("%s: %s.%s is not defined." - % (errorPrefix, section, option)) - errorCount = errorCount + 1 - - configValidator.validate() - - for valueInfo in configValidator.validatedInfo: - sectionsOptions = reDot.split(valueInfo['name']) - - if valueInfo['isValid'] == 1: - self._dict[sectionsOptions[0]][sectionsOptions[1]] = \ - valueInfo['normalized'] - else: - if valueInfo['errorData']: - statusMsgs.extend(self.var_error(sectionsOptions[0], - sectionsOptions[1], valueInfo['errorData'])) - else: - statusMsgs.extend(self.var_error(sectionsOptions[0], - sectionsOptions[1])) - errorCount = errorCount + 1 - - if errorCount > 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 72dbd695698f07524f49332ddd5ebcd3d74696a9..0000000000000000000000000000000000000000 --- 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('<font class="alert">') - if userJob == False: - self.w('invalid URL specified') - elif re.match("^\d+$", userJob): - self.w('invalid URL specified, job <b>%s</b> does not exist' % userJob) - elif re.match("^\w+$", userJob): - self.w('invalid URL specified, user <b>%s</b> does not exist' % userJob) - self.w('</font>') - - 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("<html>") - self.w("<body>") - self.w("<head>") - self.writeCSS() - self.w("</head>") - self.w('<font class="header2">HOD Service Registry Information</font>') - if serviceInfo == {}: - self.w('<br><br><font class="header"> No HOD clusters configured.</font>') - else: - if parsedUrl[2] == '/': - self.w(' <table class="main">') - count = 0 - for user in users: - self.writeUserData(user, options, serviceInfo, count) - count = count + 1 - elif parsedUrl[2][1:] in serviceInfo: - self.w(' <table class="main">') - 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(' <table class="main">') - for user in jobInfo.keys(): - self.writeUserData(user, options, jobInfo, 0) - else: - self.sendWrongPage(parsedUrl[2][1:]) - self.w(' <table class="main">') - 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(' <table class="main">') - count = 0 - for user in users: - self.writeUserData(user, options, serviceInfo, count) - count = count + 1 - else: - self.sendWrongPage(False) - self.w(' <table class="main">') - count = 0 - for user in users: - self.writeUserData(user, options, serviceInfo, count) - count = count + 1 - - self.w('</table>') - self.w("</pre>") - self.w("</body>") - self.w("</html>") - - def writeCSS(self): - self.w('<style type="text/css">') - - self.w('table.main { border: 0px; padding: 1; background-color: #E1ECE0; width: 70%; margin: 10; }') - self.w('table.sub1 { background-color: #F1F1F1; padding: 0; }') - self.w('table.sub2 { background-color: #FFFFFF; padding: 0; }') - self.w('table.sub3 { border: 1px solid #EEEEEE; background-color: #FFFFFF; padding: 0; }') - self.w('td.header { border-bottom: 1px solid #CCCCCC; padding: 2;}') - self.w('td.service1 { border: 0px; background-color: #FFFFFF; padding: 2; width: 10%}') - self.w('td.service2 { border: 0px; background-color: #FFFFFF; padding: 2; width: 90%}') - self.w('td { vertical-align: top; padding: 0; }') - self.w('td.noborder { border-style: none; border-collapse: collapse; }') - self.w('tr.colored { background-color: #F1F1F1; }') - self.w('font { font-family: Helvetica, Arial, sans-serif; font-size: 10pt; color: #666666; }') - self.w('font.header { font-family: Helvetica, Arial, sans-serif; font-size: 10pt; color: #333333; font-style: bold }') - self.w('font.header2 { font-family: Helvetica, Arial, sans-serif; font-size: 16pt; color: #333333; }') - self.w('font.sml { font-family: Helvetica, Arial, sans-serif; font-size: 8pt; color: #666666; }') - self.w('font.alert { font-family: Helvetica, Arial, sans-serif; font-size: 9pt; color: #FF7A22; }') - self.w('a { font-family: Helvetica, Arial, sans-serif; text-decoration:none; font-size: 10pt; color: #111111; }') - self.w('a:visited { font-family: Helvetica, Arial, sans-serif; color:#2D4628; text-decoration:none; font-size: 10pt; }') - self.w('a:hover { font-family: Helvetica, Arial, sans-serif; color:#00A033; text-decoration:none; font-size: 10pt; }') - self.w('a.small { font-family: Helvetica, Arial, sans-serif; text-decoration:none; font-size: 8pt }') - self.w('a.small:hover { color:#822499; text-decoration:none; font-size: 8pt }') - - self.w("</style>") - - def writeUserData(self, user, options, serviceInfo, count): - hostInfo = self.getServiceHosts(serviceInfo) - hostKey = 'short' - if options == 'display=long': - hostKey = 'long' - - if count == 0: - self.w('<tr>') - self.w('<td class="header" colspan="2">') - self.w('<font class="header">Active Users</font>') - self.w('</td>') - self.w('</tr>') - self.w('<tr>') - self.w('<td><font>%s</font></td>' % user) - self.w('<td>') - jobIDs = serviceInfo[user].keys() - jobIDs.sort() - for jobID in jobIDs: - self.w('<table class="sub1" width="100%">') - if count == 0: - self.w('<tr>') - self.w('<td class="header" colspan="2">') - self.w('<font class="header">PBS Job Identifiers</font>') - self.w('</td>') - self.w('</tr>') - self.w('<tr>') - self.w('<td><font>%s</font></td>' % jobID) - self.w('<td>') - hosts = serviceInfo[user][jobID].keys() - hosts.sort() - for host in hosts: - if hostInfo[hostKey].has_key(self.getJobKey(user, jobID, host)): - self.w('<table class="sub2" width="100%">') - if count == 0: - self.w('<tr>') - self.w('<td class="header" colspan="2">') - self.w('<font class="header">Hosts Running Services</font>') - self.w('</td>') - self.w('</tr>') - self.w('<tr>') - self.w('<td><font>%s</font></td>' % host) - self.w('<td>') - self.w('<table class="sub3" width="100%">') - self.w('<tr>') - self.w('<td colspan="2">') - self.w('<font class="header">Service Information</font>') - self.w('</td>') - self.w('</tr>') - for serviceItem in serviceInfo[user][jobID][host]: - serviceName = serviceItem.keys() - serviceName = serviceName[0] - if isinstance(serviceItem[serviceName], dict) and \ - options == 'display=long': - self.w('<tr class="colored">') - self.w('<td><font>%s</font></td>' % serviceName) - self.w('<td>') - self.w('<table width="100%">') - for key in serviceItem[serviceName]: - self.w('<tr>') - self.w('<td class="service1"><font>%s</font></td>' % key) - self.w('<td class="service2"><font>%s</font></td>' % serviceItem[serviceName][key]) - self.w('</tr>') - self.w('</table>') - self.w('</td>') - self.w('</tr>') - elif isinstance(serviceItem[serviceName], str): - self.w('<tr class="colored">') - self.w('<td><font class="service1">%s</font></td>' % serviceName) - self.w('<td>') - (host, port) = serviceItem[serviceName].split(':') - hostnameInfo = socket.gethostbyname_ex(host) - if serviceName.startswith('mapred'): - self.w('<a href="http://%s:%s">Hadoop Job Tracker</a>' % (hostnameInfo[0], port)) - elif serviceName.startswith('hdfs'): - self.w('<a href="http://%s:%s">HDFS Name Node</a> ' % (hostnameInfo[0], port)) - else: - self.w('<font class="service2">%s</font>' % serviceItem[serviceName]) - self.w('</td>') - self.w('</tr>') - self.w('</table>') - self.w('</td>') - self.w('</tr>') - self.w('</table>') - count = count + 1 - self.w('</td>') - self.w('</tr>') - self.w('</table>') - count = count + 1 - self.w('</td>') - self.w('</tr>') -# self.w("<pre>") -# self.w(pprint.pformat(serviceInfo)) -# self.w("</pre>") - -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 a118a67f9ccb5201ddfc957b24a42dc76532d18b..0000000000000000000000000000000000000000 --- 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 0d19042074401272125b7a6c06d75b3f2bfa5b61..0000000000000000000000000000000000000000 --- 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 9612ce43132474d7beea6f5442d991f9233fb7aa..0000000000000000000000000000000000000000 --- 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"(?<!\\)," -reList = re.compile(reListString) - -reKeyVal = r"(?<!\\)=" -reKeyVal = re.compile(reKeyVal) - -class typeToString: - """Provides method for converting normalized types to strings.""" - def __init__(self): - self.toStringFunctions = {} - self.__build_to_string_functions() - - def __call__(self, type, value): - return self.toStringFunctions[type](value) - - def __build_to_string_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 = "_typeToString__tostring_%s" % type - if functions.has_key(functionName): - self.toStringFunctions[type] = getattr(self, functionName) - else: - if type == '': - self.toStringFunctions[type] = self.__tostring_nothing - else: - error = "To string function %s for type %s does not exist." \ - % (functionName, type) - raise Exception(error) - sys.exit(1) - - def __tostring(self, value): - return str(value) - - def __tostring_directory(self, value): - return self.__tostring(value) - - def __tostring_address(self, value): - return "%s:%s" % (value[0], value[1]) - - def __tostring_ip_address(self, value): - return self.__tostring(value) - - def __tostring_net_address(self, value): - return self.__tostring(value) - - def __tostring_bool(self, value): - if value == False: - return 'false' - elif value == True: - return 'true' - else: - return str(value) - - def __tostring_int(self, value): - return self.__tostring(value) - - def __tostring_float(self, value): - return self.__tostring(value) - - def __tostring_pos_int(self, value): - return self.__tostring(value) - - def __tostring_neg_int(self, value): - return self.__tostring(value) - - def __tostring_freq(self, value): - return self.__tostring(value) - - def __tostring_pos_float(self, value): - return self.__tostring(value) - - def __tostring_pos_num(self, value): - return self.__tostring(value) - - def __tostring_neg_float(self, value): - return self.__tostring(value) - - def __tostring_string(self, value): - return value - - def __tostring_keyval(self, value): - string = '"' # to protect from shell escapes - for key in value: - # for item in value[key]: - # string = "%s%s=%s," % (string, key, item) - # Quotes still cannot protect Double-slashes. - # Dealing with them separately - val = re.sub(r"\\\\",r"\\\\\\\\",value[key]) - - string = "%s%s=%s," % (string, key, val) - - return string[:-1] + '"' - - def __tostring_list(self, value): - string = '' - for item in value: - string = "%s%s," % (string, item) - - return string[:-1] - - def __tostring_file(self, value): - return self.__tostring(value) - - def __tostring_size(self, value): - return self.__tostring(value) - - def __tostring_eaddress(self, value): - return self.__tostring(value) - - def __tostring_tcp_port(self, value): - return self.__tostring(value) - - def __tostring_http_version(self, value): - return self.__tostring(value) - - def __tostring_range(self, value): - if len(value) < 2: - return value[0] - else: - return "%s-%s" % (value[0], value[1]) - - def __tostring_timestamp(self, value): - return self.__tostring(value) - - def __tostring_hostname(self, value): - return self.__tostring(value) - - def __tostring_user_account(self, value): - return self.__tostring(value) - - def __tostring_user_group(self, value): - return self.__tostring(value) - - def __tostring_uri(self, value): - return self.__tostring(value) - - def __tostring_nothing(self, value): - return value - -class typeValidator: - """Type validation class used to normalize values or validated - single/large sets of values by type.""" - - def __init__(self, originalDir=None): - self.verifyFunctions = {} - self.__build_verify_functions() - - self.validateList = [] - self.validatedInfo = [] - self.__originalDir = originalDir - - def __getattr__(self, attrname): - """validateList = [ { 'func' : <bound method configValidator>, - 'name' : 'SA_COMMON.old_xml_dir', - 'value': 'var/data/old' }, - - { 'func' : <bound method configValidator>, - '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 3d5cb6fade874088fba1883a212ce48659833041..0000000000000000000000000000000000000000 --- 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 bb7ef8b60c971edaff52a9efba8452ea7620edfd..0000000000000000000000000000000000000000 --- 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 52138f2f8a30e48547588eaa29f834c50ad38898..0000000000000000000000000000000000000000 --- 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 11efd116c3856bc001277339f31055bc2ae9de38..0000000000000000000000000000000000000000 --- 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 086f052fda91cd5133e8f311e3aa3af2c8dbbcc8..0000000000000000000000000000000000000000 --- 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 f0c7f5cbbf8571dcb2ba9eab70d2d633a8fabb2b..0000000000000000000000000000000000000000 --- 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 12c2f1e1da706ef7b66c7ec27e762b2287340d9f..0000000000000000000000000000000000000000 --- 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 616d7758035656fedd9a6babf9570064a4285b63..0000000000000000000000000000000000000000 --- 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"(?<!\\);" -reCommandDelimeter = re.compile(reCommandDelimeterString) - -class hadoopConfig: - def __create_xml_element(self, doc, name, value, description, final = False): - prop = doc.createElement("property") - nameP = doc.createElement("name") - string = doc.createTextNode(name) - nameP.appendChild(string) - valueP = doc.createElement("value") - string = doc.createTextNode(value) - valueP.appendChild(string) - if final: - finalP = doc.createElement("final") - string = doc.createTextNode("true") - finalP.appendChild(string) - desc = doc.createElement("description") - string = doc.createTextNode(description) - desc.appendChild(string) - prop.appendChild(nameP) - prop.appendChild(valueP) - if final: - prop.appendChild(finalP) - prop.appendChild(desc) - - return prop - - def gen_site_conf(self, confDir, tempDir, numNodes, hdfsAddr, mrSysDir,\ - mapredAddr=None, clientParams=None, serverParams=None,\ - finalServerParams=None, clusterFactor=None): - if not mapredAddr: - mapredAddr = "dummy:8181" - - implementation = getDOMImplementation() - doc = implementation.createDocument('', 'configuration', None) - comment = doc.createComment( - "This is an auto generated hadoop-site.xml, do not modify") - topElement = doc.documentElement - topElement.appendChild(comment) - - description = {} - paramsDict = { 'mapred.job.tracker' : mapredAddr , \ - 'fs.default.name' : "hdfs://" + hdfsAddr, \ - 'hadoop.tmp.dir' : tempDir, \ - } - - paramsDict['mapred.system.dir'] = mrSysDir - - # mapred-default.xml is no longer used now. - numred = int(math.floor(clusterFactor * (int(numNodes) - 1))) - paramsDict['mapred.reduce.tasks'] = str(numred) - # end - - # for all the above vars generated, set the description - for k, v in paramsDict.iteritems(): - description[k] = 'Hod generated parameter' - - # finalservelParams - if finalServerParams: - for k, v in finalServerParams.iteritems(): - if not description.has_key(k): - description[k] = "final server parameter" - paramsDict[k] = v - - # servelParams - if serverParams: - for k, v in serverParams.iteritems(): - if not description.has_key(k): - # if no final value for same param is mentioned - description[k] = "server parameter" - paramsDict[k] = v - - # clientParams - if clientParams: - for k, v in clientParams.iteritems(): - if not description.has_key(k) or description[k] == "server parameter": - # Just add, if no final value for same param is mentioned. - # Replace even if server param is mentioned for same config variable - description[k] = "client-side parameter" - paramsDict[k] = v - - # generate the xml elements - for k,v in paramsDict.iteritems(): - if ( description[k] == "final server parameter" or \ - description[k] == "Hod generated parameter" ): - final = True - else: final = False - prop = self.__create_xml_element(doc, k, v, description[k], final) - topElement.appendChild(prop) - - siteName = os.path.join(confDir, "hadoop-site.xml") - sitefile = file(siteName, 'w') - print >> 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 <install-dir>/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 b2587bb77a437a8d527219d1be44e4d2c994ec48..0000000000000000000000000000000000000000 --- 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 <clusterdir> -n <nodecount> [OPTIONS]', - 'deallocate' : 'hod deallocate -d <clusterdir> [OPTIONS]', - 'list' : 'hod list [OPTIONS]', - 'info' : 'hod info -d <clusterdir> [OPTIONS]', - 'script' : - 'hod script -d <clusterdir> -n <nodecount> -s <script> [OPTIONS]', - 'help' : 'hod help <OPERATION>', - } - - self.description_strings = \ - { - 'allocate' : "Allocates a cluster of n nodes using the specified \n" + \ - " cluster directory to store cluster state \n" + \ - " information. The Hadoop site XML is also stored \n" + \ - " in this location.\n", - - 'deallocate' : "Deallocates a cluster using the specified \n" + \ - " cluster directory. This operation is also \n" + \ - " required to clean up a dead cluster.\n", - - 'list' : "List all clusters currently allocated by a user, \n" + \ - " along with limited status information and the \n" + \ - " cluster ID.\n", - - 'info' : "Provide detailed information on an allocated cluster.\n", - - 'script' : "Allocates a cluster of n nodes with the given \n" +\ - " cluster directory, runs the specified script \n" + \ - " using the allocated cluster, and then \n" + \ - " deallocates the cluster.\n", - - 'help' : "Print help for the operation and exit.\n" + \ - "Available operations : %s.\n" % self.ops, - } - - def usage(self, op): - return "Usage : " + self.usage_strings[op] + "\n" + \ - "For full description: hod help " + op + ".\n" - - def help(self, op=None): - if op is None: - return "hod <operation> [ARGS] [OPTIONS]\n" + \ - "Available operations : %s\n" % self.ops + \ - "For help on a particular operation : hod help <operation>.\n" + \ - "For all options : hod help options." - else: - return "Usage : " + self.usage_strings[op] + "\n" + \ - "Description : " + self.description_strings[op] + \ - "For all options : hod help options.\n" diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/nodePool.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/nodePool.py deleted file mode 100644 index 4eafddb0c93c5766998e19f0802c7492506fa3cf..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Hod/nodePool.py +++ /dev/null @@ -1,128 +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 nodepool and nodeset as abstract interface for batch system""" -# -*- python -*- - -from hodlib.GridServices.service import * - -class NodeSet: - """a set of nodes as one allocation unit""" - - PENDING, COMMITTED, COMPLETE = range(3) - - def __init__(self, id, numNodes, preferredList, isPreemptee): - self.id = id - self.numNodes = numNodes - self.isPreemptee = isPreemptee - self.preferredList = preferredList - self.cmdDescSet = [] - - def getId(self): - """returns a unique id of the nodeset""" - return self.id - - def registerCommand(self, cmdDesc): - """register a command to the nodeset""" - self.cmdDescSet.append(cmdDesc) - - def getAddrList(self): - """get list of node host names - May return empty list if node set is not allocated yet""" - raise NotImplementedError - - def _getNumNodes(self): - return self.numNodes - - def _isPreemptee(self): - return self.isPreemptee - - def _getPreferredList(self): - return self.preferredList - - def _getCmdSet(self): - return self.cmdDescSet - -class NodePool: - """maintains a collection of node sets as they get allocated. - Also the base class for all kinds of nodepools. """ - - def __init__(self, nodePoolDesc, cfg, log): - self.nodePoolDesc = nodePoolDesc - self.nodeSetDict = {} - self._cfg = cfg - self.nextNodeSetId = 0 - self._log = log - - - def newNodeSet(self, numNodes, preferred=[], isPreemptee=True, id=None): - """create a nodeset possibly with asked properties""" - raise NotImplementedError - - def submitNodeSet(self, nodeSet, walltime = None, qosLevel = None, - account = None, resourcelist = None): - """submit the nodeset request to nodepool - return False if error happened""" - raise NotImplementedError - - def pollNodeSet(self, nodeSet): - """return status of node set""" - raise NotImplementedError - - def getWorkers(self): - """return the hosts that comprise this nodepool""" - raise NotImplementedError - - def runWorkers(self, nodeSet = None, args = []): - """Run node set workers.""" - - raise NotImplementedError - - def freeNodeSet(self, nodeset): - """free a node set""" - raise NotImplementedError - - def finalize(self): - """cleans up all nodesets""" - raise NotImplementedError - - def getServiceId(self): - raise NotImplementedError - - def getJobInfo(self, jobId=None): - raise NotImplementedError - - def deleteJob(self, jobId): - """Delete a job, given it's id""" - raise NotImplementedError - - def isJobFeasible(self): - """Check if job can run by looking at any user/job limits""" - raise NotImplementedError - - def updateWorkerInfo(self, workerInfoMap, jobId): - """Update information about the workers started by this NodePool.""" - raise NotImplementedError - - def getAccountString(self): - """Return the account string for this job""" - raise NotImplementedError - - def getNextNodeSetId(self): - id = self.nextNodeSetId - self.nextNodeSetId += 1 - - return id - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/HodRing/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/HodRing/__init__.py deleted file mode 100644 index 12c2f1e1da706ef7b66c7ec27e762b2287340d9f..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/HodRing/__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/HodRing/hodRing.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/HodRing/hodRing.py deleted file mode 100644 index 02d6dbfca0438bfdefb077dc653338b475df3d8a..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/HodRing/hodRing.py +++ /dev/null @@ -1,928 +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. -#!/usr/bin/env python -"""hodring launches hadoop commands on work node and - cleans up all the work dirs afterward -""" -# -*- python -*- -import os, sys, time, shutil, getpass, xml.dom.minidom, xml.dom.pulldom -import socket, sets, urllib, csv, signal, pprint, random, re, httplib - -from xml.dom import getDOMImplementation -from pprint import pformat -from optparse import OptionParser -from urlparse import urlparse -from hodlib.Common.util import local_fqdn, parseEquals, getMapredSystemDirectory, isProcessRunning -from hodlib.Common.tcp import tcpSocket, tcpError - -binfile = sys.path[0] -libdir = os.path.dirname(binfile) -sys.path.append(libdir) - -import hodlib.Common.logger - -from hodlib.GridServices.service import * -from hodlib.Common.util import * -from hodlib.Common.socketServers import threadedHTTPServer -from hodlib.Common.hodsvc import hodBaseService -from hodlib.Common.threads import simpleCommand -from hodlib.Common.xmlrpc import hodXRClient - -mswindows = (sys.platform == "win32") -originalcwd = os.getcwd() - -reHdfsURI = re.compile("hdfs://(.*?:\d+)(.*)") - -class CommandDesc: - """A class that represents the commands that - are run by hodring""" - def __init__(self, dict, log): - self.log = log - self.log.debug("In command desc") - self.log.debug("Done in command desc") - dict.setdefault('argv', []) - dict.setdefault('version', None) - dict.setdefault('envs', {}) - dict.setdefault('workdirs', []) - dict.setdefault('attrs', {}) - dict.setdefault('final-attrs', {}) - dict.setdefault('fg', False) - dict.setdefault('ignorefailures', False) - dict.setdefault('stdin', None) - - self.log.debug("Printing dict") - self._checkRequired(dict) - self.dict = dict - - def _checkRequired(self, dict): - if 'name' not in dict: - raise ValueError, "Command description lacks 'name'" - if 'program' not in dict: - raise ValueError, "Command description lacks 'program'" - if 'pkgdirs' not in dict: - raise ValueError, "Command description lacks 'pkgdirs'" - - def getName(self): - return self.dict['name'] - - def getProgram(self): - return self.dict['program'] - - def getArgv(self): - return self.dict['argv'] - - def getVersion(self): - return self.dict['version'] - - def getEnvs(self): - return self.dict['envs'] - - def getPkgDirs(self): - return self.dict['pkgdirs'] - - def getWorkDirs(self): - return self.dict['workdirs'] - - def getAttrs(self): - return self.dict['attrs'] - - def getfinalAttrs(self): - return self.dict['final-attrs'] - - def isForeground(self): - return self.dict['fg'] - - def isIgnoreFailures(self): - return self.dict['ignorefailures'] - - def getStdin(self): - return self.dict['stdin'] - - def parseDesc(str): - - dict = CommandDesc._parseMap(str) - - dict['argv'] = CommandDesc._parseList(dict['argv']) - dict['envs'] = CommandDesc._parseMap(dict['envs']) - dict['pkgdirs'] = CommandDesc._parseList(dict['pkgdirs'], ':') - dict['workdirs'] = CommandDesc._parseList(dict['workdirs'], ':') - dict['attrs'] = CommandDesc._parseMap(dict['attrs']) - dict['final-attrs'] = CommandDesc._parseMap(dict['final-attrs']) - - return CommandDesc(dict) - - parseDesc = staticmethod(parseDesc) - - def _parseList(str, delim = ','): - list = [] - for row in csv.reader([str], delimiter=delim, escapechar='\\', - quoting=csv.QUOTE_NONE, doublequote=False): - list.extend(row) - return list - - _parseList = staticmethod(_parseList) - - def _parseMap(str): - """Parses key value pairs""" - dict = {} - for row in csv.reader([str], escapechar='\\', quoting=csv.QUOTE_NONE, doublequote=False): - for f in row: - [k, v] = f.split('=', 1) - dict[k] = v - return dict - - _parseMap = staticmethod(_parseMap) - -class MRSystemDirectoryManager: - """Class that is responsible for managing the MapReduce system directory""" - - def __init__(self, jtPid, mrSysDir, fsName, hadoopPath, log, retries=120): - self.__jtPid = jtPid - self.__mrSysDir = mrSysDir - self.__fsName = fsName - self.__hadoopPath = hadoopPath - self.__log = log - self.__retries = retries - - def toCleanupArgs(self): - return " --jt-pid %s --mr-sys-dir %s --fs-name %s --hadoop-path %s " \ - % (self.__jtPid, self.__mrSysDir, self.__fsName, self.__hadoopPath) - - def removeMRSystemDirectory(self): - - jtActive = isProcessRunning(self.__jtPid) - count = 0 # try for a max of a minute for the process to end - while jtActive and (count<self.__retries): - time.sleep(0.5) - jtActive = isProcessRunning(self.__jtPid) - count += 1 - - if count == self.__retries: - self.__log.warn('Job Tracker did not exit even after a minute. Not going to try and cleanup the system directory') - return - - self.__log.debug('jt is now inactive') - - cmd = "%s dfs -fs hdfs://%s -rmr %s" % (self.__hadoopPath, self.__fsName, \ - self.__mrSysDir) - self.__log.debug('Command to run to remove system directory: %s' % (cmd)) - try: - hadoopCommand = simpleCommand('mr-sys-dir-cleaner', cmd) - hadoopCommand.start() - hadoopCommand.wait() - hadoopCommand.join() - ret = hadoopCommand.exit_code() - if ret != 0: - self.__log.warn("Error in removing MapReduce system directory '%s' from '%s' using path '%s'" \ - % (self.__mrSysDir, self.__fsName, self.__hadoopPath)) - self.__log.warn(pprint.pformat(hadoopCommand.output())) - else: - self.__log.info("Removed MapReduce system directory successfully.") - except: - self.__log.error('Exception while cleaning up MapReduce system directory. May not be cleaned up. %s', \ - get_exception_error_string()) - self.__log.debug(get_exception_string()) - - -def createMRSystemDirectoryManager(dict, log): - keys = [ 'jt-pid', 'mr-sys-dir', 'fs-name', 'hadoop-path' ] - for key in keys: - if (not dict.has_key(key)) or (dict[key] is None): - return None - - mrSysDirManager = MRSystemDirectoryManager(int(dict['jt-pid']), dict['mr-sys-dir'], \ - dict['fs-name'], dict['hadoop-path'], log) - return mrSysDirManager - -class HadoopCommand: - """Runs a single hadoop command""" - - def __init__(self, id, desc, tempdir, tardir, log, javahome, - mrSysDir, restart=False): - self.desc = desc - self.log = log - self.javahome = javahome - self.__mrSysDir = mrSysDir - self.program = desc.getProgram() - self.name = desc.getName() - self.workdirs = desc.getWorkDirs() - self.hadoopdir = tempdir - self.confdir = os.path.join(self.hadoopdir, '%d-%s' % (id, self.name), - "confdir") - self.logdir = os.path.join(self.hadoopdir, '%d-%s' % (id, self.name), - "logdir") - self.out = os.path.join(self.logdir, '%s.out' % self.name) - self.err = os.path.join(self.logdir, '%s.err' % self.name) - - self.child = None - self.restart = restart - self.filledInKeyVals = [] - self._createWorkDirs() - self._createHadoopSiteXml() - self._createHadoopLogDir() - self.__hadoopThread = None - self.stdErrContents = "" # store list of contents for returning to user - - def _createWorkDirs(self): - for dir in self.workdirs: - if os.path.exists(dir): - if not os.access(dir, os.F_OK | os.R_OK | os.W_OK | os.X_OK): - raise ValueError, "Workdir %s does not allow rwx permission." % (dir) - continue - try: - os.makedirs(dir) - except: - pass - - def getFilledInKeyValues(self): - return self.filledInKeyVals - - def createXML(self, doc, attr, topElement, final): - for k,v in attr.iteritems(): - self.log.debug('_createHadoopSiteXml: ' + str(k) + " " + str(v)) - if ( v == "fillinport" ): - v = "%d" % (ServiceUtil.getUniqRandomPort(low=50000, log=self.log)) - - keyvalpair = '' - if isinstance(v, (tuple, list)): - for item in v: - keyvalpair = "%s%s=%s," % (keyvalpair, k, item) - keyvalpair = keyvalpair[:-1] - else: - keyvalpair = k + '=' + v - - self.filledInKeyVals.append(keyvalpair) - if(k == "mapred.job.tracker"): # total hack for time's sake - keyvalpair = k + "=" + v - self.filledInKeyVals.append(keyvalpair) - - if ( v == "fillinhostport"): - port = "%d" % (ServiceUtil.getUniqRandomPort(low=50000, log=self.log)) - self.log.debug('Setting hostname to: %s' % local_fqdn()) - v = local_fqdn() + ':' + port - - keyvalpair = '' - if isinstance(v, (tuple, list)): - for item in v: - keyvalpair = "%s%s=%s," % (keyvalpair, k, item) - keyvalpair = keyvalpair[:-1] - else: - keyvalpair = k + '=' + v - - self.filledInKeyVals.append(keyvalpair) - if ( v == "fillindir"): - v = self.__mrSysDir - pass - - prop = None - if isinstance(v, (tuple, list)): - for item in v: - prop = self._createXmlElement(doc, k, item, "No description", final) - topElement.appendChild(prop) - else: - if k == 'fs.default.name': - prop = self._createXmlElement(doc, k, "hdfs://" + v, "No description", final) - else: - prop = self._createXmlElement(doc, k, v, "No description", final) - topElement.appendChild(prop) - - def _createHadoopSiteXml(self): - if self.restart: - if not os.path.exists(self.confdir): - os.makedirs(self.confdir) - else: - assert os.path.exists(self.confdir) == False - os.makedirs(self.confdir) - - implementation = getDOMImplementation() - doc = implementation.createDocument('', 'configuration', None) - comment = doc.createComment("This is an auto generated hadoop-site.xml, do not modify") - topElement = doc.documentElement - topElement.appendChild(comment) - - finalAttr = self.desc.getfinalAttrs() - self.createXML(doc, finalAttr, topElement, True) - attr = {} - attr1 = self.desc.getAttrs() - for k,v in attr1.iteritems(): - if not finalAttr.has_key(k): - attr[k] = v - self.createXML(doc, attr, topElement, False) - - - siteName = os.path.join(self.confdir, "hadoop-site.xml") - sitefile = file(siteName, 'w') - print >> sitefile, topElement.toxml() - sitefile.close() - self.log.debug('created %s' % (siteName)) - - def _createHadoopLogDir(self): - if self.restart: - if not os.path.exists(self.logdir): - os.makedirs(self.logdir) - else: - assert os.path.exists(self.logdir) == False - os.makedirs(self.logdir) - - def _createXmlElement(self, doc, name, value, description, final): - prop = doc.createElement("property") - nameP = doc.createElement("name") - string = doc.createTextNode(name) - nameP.appendChild(string) - valueP = doc.createElement("value") - string = doc.createTextNode(value) - valueP.appendChild(string) - desc = doc.createElement("description") - string = doc.createTextNode(description) - desc.appendChild(string) - prop.appendChild(nameP) - prop.appendChild(valueP) - prop.appendChild(desc) - if (final): - felement = doc.createElement("final") - string = doc.createTextNode("true") - felement.appendChild(string) - prop.appendChild(felement) - pass - - return prop - - def getMRSystemDirectoryManager(self): - return MRSystemDirectoryManager(self.__hadoopThread.getPid(), self.__mrSysDir, \ - self.desc.getfinalAttrs()['fs.default.name'], \ - self.path, self.log) - - def run(self, dir): - status = True - args = [] - desc = self.desc - - self.log.debug(pprint.pformat(desc.dict)) - - - self.log.debug("Got package dir of %s" % dir) - - self.path = os.path.join(dir, self.program) - - self.log.debug("path: %s" % self.path) - args.append(self.path) - args.extend(desc.getArgv()) - envs = desc.getEnvs() - fenvs = os.environ - - for k, v in envs.iteritems(): - fenvs[k] = v - - if envs.has_key('HADOOP_OPTS'): - fenvs['HADOOP_OPTS'] = envs['HADOOP_OPTS'] - self.log.debug("HADOOP_OPTS : %s" % fenvs['HADOOP_OPTS']) - - fenvs['JAVA_HOME'] = self.javahome - fenvs['HADOOP_CONF_DIR'] = self.confdir - fenvs['HADOOP_LOG_DIR'] = self.logdir - - self.log.info(pprint.pformat(fenvs)) - - hadoopCommand = '' - for item in args: - hadoopCommand = "%s%s " % (hadoopCommand, item) - - # Redirecting output and error to self.out and self.err - hadoopCommand = hadoopCommand + ' 1>%s 2>%s ' % (self.out, self.err) - - self.log.debug('running command: %s' % (hadoopCommand)) - self.log.debug('hadoop env: %s' % fenvs) - self.log.debug('Command stdout will be redirected to %s ' % self.out + \ - 'and command stderr to %s' % self.err) - - self.__hadoopThread = simpleCommand('hadoop', hadoopCommand, env=fenvs) - self.__hadoopThread.start() - - while self.__hadoopThread.stdin == None: - time.sleep(.2) - self.log.debug("hadoopThread still == None ...") - - input = desc.getStdin() - self.log.debug("hadoop input: %s" % input) - if input: - if self.__hadoopThread.is_running(): - print >>self.__hadoopThread.stdin, input - else: - self.log.error("hadoop command failed to start") - - self.__hadoopThread.stdin.close() - - self.log.debug("isForground: %s" % desc.isForeground()) - if desc.isForeground(): - self.log.debug("Waiting on hadoop to finish...") - self.__hadoopThread.wait() - - self.log.debug("Joining hadoop thread...") - self.__hadoopThread.join() - if self.__hadoopThread.exit_code() != 0: - status = False - else: - status = self.getCommandStatus() - - self.log.debug("hadoop run status: %s" % status) - - if status == False: - self.handleFailedCommand() - - if (status == True) or (not desc.isIgnoreFailures()): - return status - else: - self.log.error("Ignoring Failure") - return True - - def kill(self): - self.__hadoopThread.kill() - if self.__hadoopThread: - self.__hadoopThread.join() - - def addCleanup(self, list): - list.extend(self.workdirs) - list.append(self.confdir) - - def getCommandStatus(self): - status = True - ec = self.__hadoopThread.exit_code() - if (ec != 0) and (ec != None): - status = False - return status - - def handleFailedCommand(self): - self.log.error('hadoop error: %s' % ( - self.__hadoopThread.exit_status_string())) - # read the contents of redirected stderr to print information back to user - if os.path.exists(self.err): - f = None - try: - f = open(self.err) - lines = f.readlines() - # format - for line in lines: - self.stdErrContents = "%s%s" % (self.stdErrContents, line) - finally: - if f is not None: - f.close() - self.log.error('See %s.out and/or %s.err for details. They are ' % \ - (self.name, self.name) + \ - 'located at subdirectories under either ' + \ - 'hodring.work-dirs or hodring.log-destination-uri.') - -class HodRing(hodBaseService): - """The main class for hodring that - polls the commands it runs""" - def __init__(self, config): - hodBaseService.__init__(self, 'hodring', config['hodring']) - self.log = self.logs['main'] - self._http = None - self.__pkg = None - self.__pkgDir = None - self.__tempDir = None - self.__running = {} - self.__hadoopLogDirs = [] - self.__init_temp_dir() - - def __init_temp_dir(self): - self.__tempDir = os.path.join(self._cfg['temp-dir'], - "%s.%s.hodring" % (self._cfg['userid'], - self._cfg['service-id'])) - if not os.path.exists(self.__tempDir): - os.makedirs(self.__tempDir) - os.chdir(self.__tempDir) - - def __fetch(self, url, spath): - retry = 3 - success = False - while (retry != 0 and success != True): - try: - input = urllib.urlopen(url) - bufsz = 81920 - buf = input.read(bufsz) - out = open(spath, 'w') - while len(buf) > 0: - out.write(buf) - buf = input.read(bufsz) - input.close() - out.close() - success = True - except: - self.log.debug("Failed to copy file") - retry = retry - 1 - if (retry == 0 and success != True): - raise IOError, "Failed to copy the files" - - - def __get_name(self, addr): - parsedUrl = urlparse(addr) - path = parsedUrl[2] - split = path.split('/', 1) - return split[1] - - def __get_dir(self, name): - """Return the root directory inside the tarball - specified by name. Assumes that the tarball begins - with a root directory.""" - import tarfile - myTarFile = tarfile.open(name) - hadoopPackage = myTarFile.getnames()[0] - self.log.debug("tarball name : %s hadoop package name : %s" %(name,hadoopPackage)) - return hadoopPackage - - def getRunningValues(self): - return self.__running.values() - - def getTempDir(self): - return self.__tempDir - - def getHadoopLogDirs(self): - return self.__hadoopLogDirs - - def __download_package(self, ringClient): - self.log.debug("Found download address: %s" % - self._cfg['download-addr']) - try: - addr = 'none' - downloadTime = self._cfg['tarball-retry-initial-time'] # download time depends on tarball size and network bandwidth - - increment = 0 - - addr = ringClient.getTarList(self.hostname) - - while(addr == 'none'): - rand = self._cfg['tarball-retry-initial-time'] + increment + \ - random.uniform(0,self._cfg['tarball-retry-interval']) - increment = increment + 1 - self.log.debug("got no tarball. Retrying again in %s seconds." % rand) - time.sleep(rand) - addr = ringClient.getTarList(self.hostname) - - - self.log.debug("got this address %s" % addr) - - tarName = self.__get_name(addr) - self.log.debug("tar package name: %s" % tarName) - - fetchPath = os.path.join(os.getcwd(), tarName) - self.log.debug("fetch path: %s" % fetchPath) - - self.__fetch(addr, fetchPath) - self.log.debug("done fetching") - - tarUrl = "http://%s:%d/%s" % (self._http.server_address[0], - self._http.server_address[1], - tarName) - try: - ringClient.registerTarSource(self.hostname, tarUrl,addr) - #ringClient.tarDone(addr) - except KeyError, e: - self.log.error("registerTarSource and tarDone failed: ", e) - raise KeyError(e) - - check = untar(fetchPath, os.getcwd()) - - if (check == False): - raise IOError, "Untarring failed." - - self.__pkg = self.__get_dir(tarName) - self.__pkgDir = os.path.join(os.getcwd(), self.__pkg) - except Exception, e: - self.log.error("Failed download tar package: %s" % - get_exception_error_string()) - raise Exception(e) - - def __run_hadoop_commands(self, restart=True): - id = 0 - for desc in self._cfg['commanddesc']: - self.log.debug(pprint.pformat(desc.dict)) - mrSysDir = getMapredSystemDirectory(self._cfg['mapred-system-dir-root'], - self._cfg['userid'], self._cfg['service-id']) - self.log.debug('mrsysdir is %s' % mrSysDir) - cmd = HadoopCommand(id, desc, self.__tempDir, self.__pkgDir, self.log, - self._cfg['java-home'], mrSysDir, restart) - - self.__hadoopLogDirs.append(cmd.logdir) - self.log.debug("hadoop log directory: %s" % self.__hadoopLogDirs) - - try: - # if the tarball isn't there, we use the pkgs dir given. - if self.__pkgDir == None: - pkgdir = desc.getPkgDirs() - else: - pkgdir = self.__pkgDir - - self.log.debug('This is the packcage dir %s ' % (pkgdir)) - if not cmd.run(pkgdir): - addnInfo = "" - if cmd.stdErrContents is not "": - addnInfo = " Information from stderr of the command:\n%s" % (cmd.stdErrContents) - raise Exception("Could not launch the %s using %s/bin/hadoop.%s" % (desc.getName(), pkgdir, addnInfo)) - except Exception, e: - self.log.debug("Exception running hadoop command: %s\n%s" % (get_exception_error_string(), get_exception_string())) - self.__running[id] = cmd - raise Exception(e) - - id += 1 - if desc.isForeground(): - continue - self.__running[id-1] = cmd - - # ok.. now command is running. If this HodRing got jobtracker, - # Check if it is ready for accepting jobs, and then only return - self.__check_jobtracker(desc, id-1, pkgdir) - - def __check_jobtracker(self, desc, id, pkgdir): - # Check jobtracker status. Return properly if it is ready to accept jobs. - # Currently Checks for Jetty to come up, the last thing that can be checked - # before JT completes initialisation. To be perfectly reliable, we need - # hadoop support - name = desc.getName() - if name == 'jobtracker': - # Yes I am the Jobtracker - self.log.debug("Waiting for jobtracker to initialise") - version = desc.getVersion() - self.log.debug("jobtracker version : %s" % version) - hadoopCmd = self.getRunningValues()[id] - attrs = hadoopCmd.getFilledInKeyValues() - attrs = parseEquals(attrs) - jobTrackerAddr = attrs['mapred.job.tracker'] - self.log.debug("jobtracker rpc server : %s" % jobTrackerAddr) - if version < 16: - jettyAddr = jobTrackerAddr.split(':')[0] + ':' + \ - attrs['mapred.job.tracker.info.port'] - else: - jettyAddr = attrs['mapred.job.tracker.http.address'] - self.log.debug("Jobtracker jetty : %s" % jettyAddr) - - # Check for Jetty to come up - # For this do a http head, and then look at the status - defaultTimeout = socket.getdefaulttimeout() - # socket timeout isn`t exposed at httplib level. Setting explicitly. - socket.setdefaulttimeout(1) - sleepTime = 0.5 - jettyStatus = False - jettyStatusmsg = "" - while sleepTime <= 32: - # There is a possibility that the command might fail after a while. - # This code will check if the command failed so that a better - # error message can be returned to the user. - if not hadoopCmd.getCommandStatus(): - self.log.critical('Hadoop command found to have failed when ' \ - 'checking for jobtracker status') - hadoopCmd.handleFailedCommand() - addnInfo = "" - if hadoopCmd.stdErrContents is not "": - addnInfo = " Information from stderr of the command:\n%s" \ - % (hadoopCmd.stdErrContents) - raise Exception("Could not launch the %s using %s/bin/hadoop.%s" \ - % (desc.getName(), pkgdir, addnInfo)) - - try: - jettyConn = httplib.HTTPConnection(jettyAddr) - jettyConn.request("HEAD", "/jobtracker.jsp") - # httplib inherently retries the following till socket timeout - resp = jettyConn.getresponse() - if resp.status != 200: - # Some problem? - jettyStatus = False - jettyStatusmsg = "Jetty gave a non-200 response to a HTTP-HEAD" +\ - " request. HTTP Status (Code, Msg): (%s, %s)" % \ - ( resp.status, resp.reason ) - break - else: - self.log.info("Jetty returned a 200 status (%s)" % resp.reason) - self.log.info("JobTracker successfully initialised") - return - except socket.error: - self.log.debug("Jetty gave a socket error. Sleeping for %s" \ - % sleepTime) - time.sleep(sleepTime) - sleepTime = sleepTime * 2 - except Exception, e: - jettyStatus = False - jettyStatusmsg = ("Process(possibly other than jetty) running on" + \ - " port assigned to jetty is returning invalid http response") - break - socket.setdefaulttimeout(defaultTimeout) - if not jettyStatus: - self.log.critical("Jobtracker failed to initialise.") - if jettyStatusmsg: - self.log.critical( "Reason: %s" % jettyStatusmsg ) - else: self.log.critical( "Reason: Jetty failed to give response") - raise Exception("JobTracker failed to initialise") - - def stop(self): - self.log.debug("Entered hodring stop.") - if self._http: - self.log.debug("stopping http server...") - self._http.stop() - - self.log.debug("call hodsvcrgy stop...") - hodBaseService.stop(self) - - def _xr_method_clusterStart(self, initialize=True): - return self.clusterStart(initialize) - - def _xr_method_clusterStop(self): - return self.clusterStop() - - def start(self): - """Run and maintain hodring commands""" - - try: - if self._cfg.has_key('download-addr'): - self._http = threadedHTTPServer('', self._cfg['http-port-range']) - self.log.info("Starting http server...") - self._http.serve_forever() - self.log.debug("http://%s:%d" % (self._http.server_address[0], - self._http.server_address[1])) - - hodBaseService.start(self) - - ringXRAddress = None - if self._cfg.has_key('ringmaster-xrs-addr'): - ringXRAddress = "http://%s:%s/" % (self._cfg['ringmaster-xrs-addr'][0], - self._cfg['ringmaster-xrs-addr'][1]) - self.log.debug("Ringmaster at %s" % ringXRAddress) - - self.log.debug("Creating service registry XML-RPC client.") - serviceClient = hodXRClient(to_http_url( - self._cfg['svcrgy-addr'])) - if ringXRAddress == None: - self.log.info("Did not get ringmaster XML-RPC address. Fetching information from service registry.") - ringList = serviceClient.getServiceInfo(self._cfg['userid'], - self._cfg['service-id'], 'ringmaster', 'hod') - - self.log.debug(pprint.pformat(ringList)) - - if len(ringList): - if isinstance(ringList, list): - ringXRAddress = ringList[0]['xrs'] - - count = 0 - while (ringXRAddress == None and count < 3000): - ringList = serviceClient.getServiceInfo(self._cfg['userid'], - self._cfg['service-id'], 'ringmaster', 'hod') - - if len(ringList): - if isinstance(ringList, list): - ringXRAddress = ringList[0]['xrs'] - - count = count + 1 - time.sleep(.2) - - if ringXRAddress == None: - raise Exception("Could not get ringmaster XML-RPC server address.") - - self.log.debug("Creating ringmaster XML-RPC client.") - ringClient = hodXRClient(ringXRAddress) - - id = self.hostname + "_" + str(os.getpid()) - - if 'download-addr' in self._cfg: - self.__download_package(ringClient) - else: - self.log.debug("Did not find a download address.") - - cmdlist = [] - firstTime = True - increment = 0 - hadoopStartupTime = 2 - - cmdlist = ringClient.getCommand(id) - - while (cmdlist == []): - if firstTime: - sleepTime = increment + self._cfg['cmd-retry-initial-time'] + hadoopStartupTime\ - + random.uniform(0,self._cfg['cmd-retry-interval']) - firstTime = False - else: - sleepTime = increment + self._cfg['cmd-retry-initial-time'] + \ - + random.uniform(0,self._cfg['cmd-retry-interval']) - self.log.debug("Did not get command list. Waiting for %s seconds." % (sleepTime)) - time.sleep(sleepTime) - increment = increment + 1 - cmdlist = ringClient.getCommand(id) - - self.log.debug(pformat(cmdlist)) - cmdDescs = [] - for cmds in cmdlist: - cmdDescs.append(CommandDesc(cmds['dict'], self.log)) - - self._cfg['commanddesc'] = cmdDescs - - self.log.info("Running hadoop commands...") - - self.__run_hadoop_commands(False) - - masterParams = [] - for k, cmd in self.__running.iteritems(): - masterParams.extend(cmd.filledInKeyVals) - - self.log.debug("printing getparams") - self.log.debug(pformat(id)) - self.log.debug(pformat(masterParams)) - # when this is on a required host, the ringMaster already has our masterParams - if(len(masterParams) > 0): - ringClient.addMasterParams(id, masterParams) - except Exception, e: - raise Exception(e) - - def clusterStart(self, initialize=True): - """Start a stopped mapreduce/dfs cluster""" - if initialize: - self.log.debug('clusterStart Method Invoked - Initialize') - else: - self.log.debug('clusterStart Method Invoked - No Initialize') - try: - self.log.debug("Creating service registry XML-RPC client.") - serviceClient = hodXRClient(to_http_url(self._cfg['svcrgy-addr']), - None, None, 0, 0, 0) - - self.log.info("Fetching ringmaster information from service registry.") - count = 0 - ringXRAddress = None - while (ringXRAddress == None and count < 3000): - ringList = serviceClient.getServiceInfo(self._cfg['userid'], - self._cfg['service-id'], 'ringmaster', 'hod') - if len(ringList): - if isinstance(ringList, list): - ringXRAddress = ringList[0]['xrs'] - count = count + 1 - - if ringXRAddress == None: - raise Exception("Could not get ringmaster XML-RPC server address.") - - self.log.debug("Creating ringmaster XML-RPC client.") - ringClient = hodXRClient(ringXRAddress, None, None, 0, 0, 0) - - id = self.hostname + "_" + str(os.getpid()) - - cmdlist = [] - if initialize: - if 'download-addr' in self._cfg: - self.__download_package(ringClient) - else: - self.log.debug("Did not find a download address.") - while (cmdlist == []): - cmdlist = ringClient.getCommand(id) - else: - while (cmdlist == []): - cmdlist = ringClient.getAdminCommand(id) - - self.log.debug(pformat(cmdlist)) - cmdDescs = [] - for cmds in cmdlist: - cmdDescs.append(CommandDesc(cmds['dict'], self.log)) - - self._cfg['commanddesc'] = cmdDescs - - if initialize: - self.log.info("Running hadoop commands again... - Initialize") - self.__run_hadoop_commands() - masterParams = [] - for k, cmd in self.__running.iteritems(): - self.log.debug(cmd) - masterParams.extend(cmd.filledInKeyVals) - - self.log.debug("printing getparams") - self.log.debug(pformat(id)) - self.log.debug(pformat(masterParams)) - # when this is on a required host, the ringMaster already has our masterParams - if(len(masterParams) > 0): - ringClient.addMasterParams(id, masterParams) - else: - self.log.info("Running hadoop commands again... - No Initialize") - self.__run_hadoop_commands() - - except: - self.log.error(get_exception_string()) - - return True - - def clusterStop(self): - """Stop a running mapreduce/dfs cluster without stopping the hodring""" - self.log.debug('clusterStop Method Invoked') - try: - for cmd in self.__running.values(): - cmd.kill() - self.__running = {} - except: - self.log.error(get_exception_string()) - - return True diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/NodePools/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/NodePools/__init__.py deleted file mode 100644 index 12c2f1e1da706ef7b66c7ec27e762b2287340d9f..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/NodePools/__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/NodePools/torque.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/NodePools/torque.py deleted file mode 100644 index 49b03dcfbe8d7851434014a8129d3a13a7492864..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/NodePools/torque.py +++ /dev/null @@ -1,334 +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. -"""Maui/Torque implementation of NodePool""" -# -*- python -*- - -import os, sys, csv, socket, time, re, pprint - -from hodlib.Hod.nodePool import * -from hodlib.Schedulers.torque import torqueInterface -from hodlib.Common.threads import simpleCommand -from hodlib.Common.util import get_exception_string, args_to_string, local_fqdn, \ - TORQUE_USER_LIMITS_COMMENT_FIELD - -class TorqueNodeSet(NodeSet): - def __init__(self, id, numNodes, preferredList, isPreemptee): - NodeSet.__init__(self, id, numNodes, preferredList, isPreemptee) - self.qsubId = None - self.addrList = [] - - def _setQsubId(self, qsubId): - self.qsubId = qsubId - - def _setAddrList(self, addrList): - self.addrList = addrList - - def getAddrList(self): - return self.addrList - -class TorquePool(NodePool): - def __init__(self, nodePoolDesc, cfg, log): - NodePool.__init__(self, nodePoolDesc, cfg, log) - - environ = os.environ.copy() - - if self._cfg['resource_manager'].has_key('pbs-server'): - environ['PBS_DEFAULT'] = self._cfg['resource_manager']['pbs-server'] - - self.__torque = torqueInterface( - self._cfg['resource_manager']['batch-home'], environ, self._log) - - def getAccountString(self): - account = '' - if self._cfg['resource_manager'].has_key('pbs-account'): - account = self._cfg['resource_manager']['pbs-account'] - return account - - def __gen_submit_params(self, nodeSet, walltime = None, qosLevel = None, - account = None): - argList = [] - stdinList = [] - - npd = self.nodePoolDesc - - def gen_stdin_list(): - # Here we are basically generating the standard input for qsub. - # Specifically a script to exec ringmaster. - stdinList.append('#!/bin/sh') - - ringBin = os.path.join(self._cfg['hod']['base-dir'], 'bin', - 'ringmaster') - ringArgs = [ringBin,] - ringArgs.extend(self._cfg.get_args(exclude=('hod'))) - - ringMasterCommand = args_to_string(ringArgs) - - self._log.debug("ringmaster cmd: %s" % ringMasterCommand) - - stdinList.append(ringMasterCommand) - - def gen_arg_list(): - def process_qsub_attributes(): - rawAttributes = self.nodePoolDesc.getAttrs() - - # 'W:x' is used to specify torque management extentensions ie -W x= ... - resourceManagementExtensions = '' - if 'W:x' in rawAttributes: - resourceManagementExtensions = rawAttributes['W:x'] - - if qosLevel: - if len(resourceManagementExtensions) > 0: - resourceManagementExtensions += ';' - resourceManagementExtensions += 'QOS:%s' % (qosLevel) - - rawAttributes['W:x'] = resourceManagementExtensions - - hostname = local_fqdn() - - # key values are expected to have string values. - rawAttributes['l:nodes'] = "%s" % nodeSet._getNumNodes() - - if walltime: - rawAttributes['l:walltime'] = "%s" % walltime - - #create a dict of dictionaries for - # various arguments of torque - cmds = {} - for key in rawAttributes: - value = rawAttributes[key] - - if key.find(':') == -1: - raise ValueError, 'Syntax error: missing colon after %s in %s=%s' % ( - key, key, value) - - [option, subOption] = key.split(':', 1) - if not option in cmds: - cmds[option] = {} - cmds[option][subOption] = value - - opts = [] - #create a string from this - #dictionary of dictionaries createde above - for k in cmds: - csv = [] - nv = cmds[k] - for n in nv: - v = nv[n] - if len(n) == 0: - csv.append(v) - else: - csv.append('%s=%s' % (n, v)) - opts.append('-%s' % (k)) - opts.append(','.join(csv)) - - for option in cmds: - commandList = [] - for subOption in cmds[option]: - value = cmds[option][subOption] - if len(subOption) == 0: - commandList.append(value) - else: - commandList.append("%s=%s" % (subOption, value)) - opts.append('-%s' % option) - opts.append(','.join(commandList)) - - return opts - - pkgdir = npd.getPkgDir() - - qsub = os.path.join(pkgdir, 'bin', 'qsub') - sdd = self._cfg['servicedesc'] - - gsvc = None - for key in sdd: - gsvc = sdd[key] - break - - argList.extend(process_qsub_attributes()) - - argList.extend(('-N', '"' + self._cfg['hod']['title'] + '"')) - argList.extend(('-r','n')) - - if 'pbs-user' in self._cfg['resource_manager']: - argList.extend(('-u', self._cfg['resource_manager']['pbs-user'])) - - argList.extend(('-d','/tmp/')) - if 'queue' in self._cfg['resource_manager']: - queue = self._cfg['resource_manager']['queue'] - argList.extend(('-q',queue)) - - # In HOD 0.4, we pass in an account string only if it is mentioned. - # Also, we don't append userid to the account string, as HOD jobs run as the - # user running them, not as 'HOD' user. - if self._cfg['resource_manager'].has_key('pbs-account'): - argList.extend(('-A', (self._cfg['resource_manager']['pbs-account']))) - - if 'env-vars' in self._cfg['resource_manager']: - qsub_envs = self._cfg['resource_manager']['env-vars'] - argList.extend(('-v', self.__keyValToString(qsub_envs))) - - gen_arg_list() - gen_stdin_list() - - return argList, stdinList - - def __keyValToString(self, keyValList): - ret = "" - for key in keyValList: - ret = "%s%s=%s," % (ret, key, keyValList[key]) - return ret[:-1] - - def newNodeSet(self, numNodes, preferred=[], isPreemptee=True, id=None): - if not id: - id = self.getNextNodeSetId() - - nodeSet = TorqueNodeSet(id, numNodes, preferred, isPreemptee) - - self.nodeSetDict[nodeSet.getId()] = nodeSet - - return nodeSet - - def submitNodeSet(self, nodeSet, walltime = None, qosLevel = None, - account = None): - - argList, stdinList = self.__gen_submit_params(nodeSet, walltime, qosLevel, - account) - - jobId, exitCode = self.__torque.qsub(argList, stdinList) - - ## UNUSED CODE: LINE ## - nodeSet.qsubId = jobId - - return jobId, exitCode - - def freeNodeSet(self, nodeSet): - - exitCode = self.deleteJob(nodeSet.getId()) - - del self.nodeSetDict[nodeSet.getId()] - - return exitCode - - def finalize(self): - status = 0 - exitCode = 0 - for nodeSet in self.nodeSetDict.values(): - exitCode = self.freeNodeSet(nodeSet) - - if exitCode > 0 and exitCode != 153: - status = 4 - - return status - - ## UNUSED METHOD ?? ## - def getWorkers(self): - hosts = [] - - qstatInfo = self.__torque(self.getServiceId()) - if qstatInfo: - hosts = qstatInfop['exec_host'] - - return hosts - - ## UNUSED METHOD ?? ## - def pollNodeSet(self, nodeSet): - status = NodeSet.COMPLETE - nodeSet = self.nodeSetDict[0] - - qstatInfo = self.__torque(self.getServiceId()) - - if qstatMap: - jobstate = qstatMap['job_state'] - exechost = qstatMap['exec_host'] - - if jobstate == 'Q': - status = NodeSet.PENDING - elif exechost == None: - status = NodeSet.COMMITTED - else: - nodeSet._setAddrList(exec_host) - - return status - - def getServiceId(self): - id = None - - nodeSets = self.nodeSetDict.values() - if len(nodeSets): - id = nodeSets[0].qsubId - - if id == None: - id = os.getenv('PBS_JOBID') - - return id - - def getJobInfo(self, jobId=None): - - jobNonExistentErrorCode = 153 - self.__jobInfo = { 'job_state' : False } - - if jobId == None: - jobId = self.getServiceId() - - qstatInfo, exitCode = self.__torque.qstat(jobId) - if exitCode == 0: - self.__jobInfo = qstatInfo - elif exitCode == jobNonExistentErrorCode: - # This really means that the job completed - # However, setting only job_state for now, not - # any other attributes, as none seem required. - self.__jobInfo = { 'job_state' : 'C' } - - return self.__jobInfo - - def deleteJob(self, jobId): - exitCode = self.__torque.qdel(jobId) - return exitCode - - def isJobFeasible(self): - comment = None - msg = None - if self.__jobInfo.has_key('comment'): - comment = self.__jobInfo['comment'] - try: - if comment: - commentField = re.compile(self._cfg['hod']['job-feasibility-attr']) - match = commentField.search(comment) - if match: - reqUsage = int(match.group(1)) - currentUsage = int(match.group(2)) - maxUsage = int(match.group(3)) - msg = "Current Usage:%s, Requested:%s, Maximum Limit:%s " % \ - (currentUsage, reqUsage, maxUsage) - if reqUsage > maxUsage: - return "Never", msg - if reqUsage + currentUsage > maxUsage: - return False, msg - except Exception, e: - self._log.error("Error in isJobFeasible : %s" %e) - raise Exception(e) - return True, msg - - def runWorkers(self, args): - return self.__torque.pbsdsh(args) - - def updateWorkerInfo(self, workerInfoMap, jobId): - workerInfoStr = '' - for key in workerInfoMap.keys(): - workerInfoStr = '%s,%s:%s' % (workerInfoStr, key, workerInfoMap[key]) - exitCode = self.__torque.qalter("notes", workerInfoStr[1:], jobId) - return exitCode - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/__init__.py deleted file mode 100644 index 12c2f1e1da706ef7b66c7ec27e762b2287340d9f..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/__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/RingMaster/idleJobTracker.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/idleJobTracker.py deleted file mode 100644 index 33f145baab34ece28de27a8663ee01f8afae4410..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/idleJobTracker.py +++ /dev/null @@ -1,218 +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 os, re, time -from hodlib.Common.threads import loop, func -from hodlib.Common.threads import simpleCommand -from hodlib.Common.util import get_exception_string, hadoopVersion - -class HadoopJobStatus: - """This class represents the status of a single Hadoop job""" - - def __init__(self, jobId, status): - self.__jobId = jobId - self.__status = status - - def getJobId(self): - return self.__jobId - - def getStatus(self): - return self.__status - -class HadoopClientException(Exception): - """This class represents an exception that is raised when we fail in - running the job client.""" - - def __init__(self, errorCode): - self.errorCode = errorCode - -class JobTrackerMonitor: - """This class monitors the JobTracker of an allocated cluster - periodically to detect whether it is idle. If it is found - to be idle for more than a configured limit, it calls back - registered handlers who can act upon the idle cluster.""" - - def __init__(self, log, idleJTHandler, interval, limit, - hadoopDir, javaHome, servInfoProvider): - self.__log = log - self.__idlenessLimit = limit - self.__idleJobTrackerHandler = idleJTHandler - self.__hadoopDir = hadoopDir - hadoopPath = os.path.join(self.__hadoopDir, "bin", "hadoop") - #hadoop directory can be from pkgs or a temp location like tarball. Verify once. - if not os.path.exists(hadoopPath): - raise Exception('Invalid Hadoop path specified: %s' % hadoopPath) - self.__javaHome = javaHome - # Note that when this object is created, we don't yet know the JT URL. - # The service info provider will be polled until we get the URL. - self.__serviceInfoProvider = servInfoProvider - self.__jobCountRegExp = re.compile("([0-9]+) jobs currently running.*") - self.__jobStatusRegExp = re.compile("(\S+)\s+(\d)\s+\d+\s+\S+$") - self.__firstIdleTime = 0 - self.__hadoop15Version = { 'major' : '0', 'minor' : '15' } - #Assumption: we are not going to support versions older than 0.15 for Idle Job tracker. - if not self.__isCompatibleHadoopVersion(self.__hadoop15Version): - raise Exception('Incompatible Hadoop Version: Cannot check status') - self.__stopFlag = False - self.__jtURLFinderThread = func(name='JTURLFinderThread', functionRef=self.getJobTrackerURL) - self.__jtMonitorThread = loop(name='JTMonitorThread', functionRef=self.monitorJobTracker, - sleep=interval) - self.__jobTrackerURL = None - - def start(self): - """This method starts a thread that will determine the JobTracker URL""" - self.__jtURLFinderThread.start() - - def stop(self): - self.__log.debug('Joining the monitoring thread.') - self.__stopFlag = True - if self.__jtMonitorThread.isAlive(): - self.__jtMonitorThread.join() - self.__log.debug('Joined the monitoring thread.') - - def getJobTrackerURL(self): - """This method periodically checks the service info provider for the JT URL""" - self.__jobTrackerURL = self.__serviceInfoProvider.getServiceAddr('mapred') - while not self.__stopFlag and not self.__isValidJobTrackerURL(): - time.sleep(10) - if not self.__stopFlag: - self.__jobTrackerURL = self.__serviceInfoProvider.getServiceAddr('mapred') - else: - break - - if self.__isValidJobTrackerURL(): - self.__log.debug('Got URL %s. Starting monitoring' % self.__jobTrackerURL) - self.__jtMonitorThread.start() - - def monitorJobTracker(self): - """This method is periodically called to monitor the JobTracker of the cluster.""" - try: - if self.__isIdle(): - if self.__idleJobTrackerHandler: - self.__log.info('Detected cluster as idle. Calling registered callback handler.') - self.__idleJobTrackerHandler.handleIdleJobTracker() - except: - self.__log.debug('Exception while monitoring job tracker. %s' % get_exception_string()) - - def getJobsStatus(self): - """This method should return the status of all jobs that are run on the HOD allocated - hadoop cluster""" - jobStatusList = [] - try: - hadoop16Version = { 'major' : '0', 'minor' : '16' } - if self.__isCompatibleHadoopVersion(hadoop16Version): - jtStatusCommand = self.__initStatusCommand(option='-list all') - jtStatusCommand.start() - jtStatusCommand.wait() - jtStatusCommand.join() - if jtStatusCommand.exit_code() == 0: - for line in jtStatusCommand.output(): - jobStatus = self.__extractJobStatus(line) - if jobStatus is not None: - jobStatusList.append(jobStatus) - except: - self.__log.debug('Exception while getting job statuses. %s' % get_exception_string()) - return jobStatusList - - def __isValidJobTrackerURL(self): - """This method checks that the passed in URL is not one of the special case strings - returned by the getServiceAddr API""" - return ((self.__jobTrackerURL != None) and (self.__jobTrackerURL != 'not found') \ - and (not self.__jobTrackerURL.startswith('Error'))) - - def __extractJobStatus(self, line): - """This method parses an output line from the job status command and creates - the JobStatus object if there is a match""" - jobStatus = None - line = line.strip() - jsMatch = self.__jobStatusRegExp.match(line) - if jsMatch: - jobStatus = HadoopJobStatus(jsMatch.group(1), int(jsMatch.group(2))) - return jobStatus - - def __isIdle(self): - """This method checks if the JobTracker is idle beyond a certain limit.""" - jobCount = 0 - err = False - - try: - jobCount = self.__getJobCount() - except HadoopClientException, hce: - self.__log.debug('HadoopClientException handled in getting job count. \ - Error code: %s' % hce.errorCode) - err = True - - if (jobCount==0) or err: - if self.__firstIdleTime == 0: - #detecting idleness for the first time - self.__firstIdleTime = time.time() - else: - if ((time.time()-self.__firstIdleTime) >= self.__idlenessLimit): - self.__log.info('Idleness limit crossed for cluster') - return True - else: - # reset idleness time - self.__firstIdleTime = 0 - - return False - - def __getJobCount(self): - """This method executes the hadoop job -list command and parses the output to detect - the number of running jobs.""" - - # We assume here that the poll interval is small enough to detect running jobs. - # If jobs start and stop within the poll interval, the cluster would be incorrectly - # treated as idle. Hadoop 2266 will provide a better mechanism than this. - jobs = -1 - jtStatusCommand = self.__initStatusCommand() - jtStatusCommand.start() - jtStatusCommand.wait() - jtStatusCommand.join() - if jtStatusCommand.exit_code() == 0: - for line in jtStatusCommand.output(): - match = self.__jobCountRegExp.match(line) - if match: - jobs = int(match.group(1)) - elif jtStatusCommand.exit_code() == 1: - # for now, exit code 1 comes for any exception raised by JobClient. If hadoop gets - # to differentiate and give more granular exit codes, we can check for those errors - # corresponding to network errors etc. - raise HadoopClientException(jtStatusCommand.exit_code()) - return jobs - - def __isCompatibleHadoopVersion(self, expectedVersion): - """This method determines whether the version of hadoop being used is one that - is higher than the expectedVersion. - This can be used for checking if a particular feature is available or not""" - ver = hadoopVersion(self.__hadoopDir, self.__javaHome, self.__log) - ret = False - - if (ver['major']!=None) and (int(ver['major']) >= int(expectedVersion['major'])) \ - and (ver['minor']!=None) and (int(ver['minor']) >= int(expectedVersion['minor'])): - ret = True - return ret - - def __initStatusCommand(self, option="-list"): - """This method initializes the command to run to check the JT status""" - cmd = None - hadoopPath = os.path.join(self.__hadoopDir, 'bin', 'hadoop') - cmdStr = "%s job -jt %s" % (hadoopPath, self.__jobTrackerURL) - cmdStr = "%s %s" % (cmdStr, option) - self.__log.debug('cmd str %s' % cmdStr) - env = os.environ - env['JAVA_HOME'] = self.__javaHome - cmd = simpleCommand('HadoopStatus', cmdStr, env) - return cmd - diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/ringMaster.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/ringMaster.py deleted file mode 100644 index a289d95d75d9b6ba4f28fd0bc88641db9ed95ccd..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/RingMaster/ringMaster.py +++ /dev/null @@ -1,1019 +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. -#!/usr/bin/env python -"""manages services and nodepool""" -# -*- python -*- - -import os, sys, random, time, sets, shutil, threading -import urllib, urlparse, re, getpass, pprint, signal, shutil - -from pprint import pformat -from HTMLParser import HTMLParser - -binfile = sys.path[0] -libdir = os.path.dirname(binfile) -sys.path.append(libdir) - -import hodlib.Common.logger -from hodlib.RingMaster.idleJobTracker import JobTrackerMonitor, HadoopJobStatus - -from hodlib.Common.threads import func - -from hodlib.Hod.nodePool import * -from hodlib.Common.util import * -from hodlib.Common.nodepoolutil import NodePoolUtil -from hodlib.Common.socketServers import hodXMLRPCServer -from hodlib.Common.socketServers import threadedHTTPServer -from hodlib.NodePools import * -from hodlib.NodePools.torque import * -from hodlib.GridServices import * -from hodlib.Common.descGenerator import * -from hodlib.Common.xmlrpc import hodXRClient -from hodlib.Common.miniHTMLParser import miniHTMLParser -from hodlib.Common.threads import simpleCommand - -class ringMasterServer: - """The RPC server that exposes all the master config - changes. Also, one of these RPC servers runs as a proxy - and all the hodring instances register with this proxy""" - instance = None - xmlrpc = None - - def __init__(self, cfg, log, logMasterSources, retry=5): - try: - from hodlib.Common.socketServers import twistedXMLRPCServer - ringMasterServer.xmlrpc = twistedXMLRPCServer("", - cfg['ringmaster']['xrs-port-range']) - except ImportError: - log.info("Twisted interface not found. Using hodXMLRPCServer.") - ringMasterServer.xmlrpc = hodXMLRPCServer("", - cfg['ringmaster']['xrs-port-range']) - - ringMasterServer.xmlrpc.register_instance(logMasterSources) - self.logMasterSources = logMasterSources - ringMasterServer.xmlrpc.serve_forever() - - while not ringMasterServer.xmlrpc.is_alive(): - time.sleep(.5) - - log.debug('Ringmaster RPC Server at %d' % - ringMasterServer.xmlrpc.server_address[1]) - - def startService(ss, cfg, np, log, rm): - logMasterSources = _LogMasterSources(ss, cfg, np, log, rm) - ringMasterServer.instance = ringMasterServer(cfg, log, logMasterSources) - - def stopService(): - ringMasterServer.xmlrpc.stop() - - def getPort(): - return ringMasterServer.instance.port - - def getAddress(): - return 'http://%s:%d/' % (socket.gethostname(), - ringMasterServer.xmlrpc.server_address[1]) - - startService = staticmethod(startService) - stopService = staticmethod(stopService) - getPort = staticmethod(getPort) - getAddress = staticmethod(getAddress) - -class _LogMasterSources: - """All the methods that are run by the RPC server are - added into this class """ - - def __init__(self, serviceDict, cfg, np, log, rm): - self.serviceDict = serviceDict - self.tarSource = [] - self.tarSourceLock = threading.Lock() - self.dict = {} - self.count = {} - self.logsourceList = [] - self.logsourceListLock = threading.Lock() - self.masterParam = [] - self.masterParamLock = threading.Lock() - self.verify = 'none' - self.cmdLock = threading.Lock() - self.cfg = cfg - self.log = log - self.np = np - self.rm = rm - self.hdfsHost = None - self.mapredHost = None - self.maxconnect = self.cfg['ringmaster']['max-connect'] - self.log.debug("Using max-connect value %s"%self.maxconnect) - - - def registerTarSource(self, hostname, url, addr=None): - self.log.debug("registering: " + url) - lock = self.tarSourceLock - lock.acquire() - self.dict[url] = url - self.count[url] = 0 - # addr is None when ringMaster himself invokes this method - if addr: - c = self.count[addr] - self.count[addr] = c - 1 - lock.release() - if addr: - str = "%s is done" % (addr) - self.log.debug(str) - return url - - def getTarList(self,hodring): # this looks useful - lock = self.tarSourceLock - lock.acquire() - leastkey = None - leastval = -1 - for k, v in self.count.iteritems(): - if (leastval == -1): - leastval = v - pass - if (v <= leastval and v < self.maxconnect): - leastkey = k - leastval = v - if (leastkey == None): - url = 'none' - else: - url = self.dict[leastkey] - self.count[leastkey] = leastval + 1 - self.log.debug("%s %d" % (leastkey, self.count[leastkey])) - lock.release() - self.log.debug('sending url ' + url+" to "+hodring) # this looks useful - return url - - def tarDone(self, uri): - str = "%s is done" % (uri) - self.log.debug(str) - lock = self.tarSourceLock - lock.acquire() - c = self.count[uri] - self.count[uri] = c - 1 - lock.release() - return uri - - def status(self): - return True - -# FIXME: this code is broken, it relies on a central service registry -# -# def clusterStart(self, changedClusterParams=[]): -# self.log.debug("clusterStart method invoked.") -# self.dict = {} -# self.count = {} -# try: -# if (len(changedClusterParams) > 0): -# self.log.debug("Updating config.") -# for param in changedClusterParams: -# (key, sep1, val) = param.partition('=') -# (i1, sep2, i2) = key.partition('.') -# try: -# prev = self.cfg[i1][i2] -# self.rm.cfg[i1][i2] = val -# self.cfg[i1][i2] = val -# self.log.debug("\nModified [%s][%s]=%s to [%s][%s]=%s" % (i1, i2, prev, i1, i2, val)) -# except KeyError, e: -# self.log.info("Skipping %s as no such config parameter found in ringmaster" % param) -# self.log.debug("Regenerating Service Description.") -# dGen = DescGenerator(self.rm.cfg) -# self.rm.cfg['servicedesc'] = dGen.createServiceDescDict() -# self.cfg['servicedesc'] = self.rm.cfg['servicedesc'] -# -# self.rm.tar = None -# if self.rm.cfg['ringmaster'].has_key('hadoop-tar-ball'): -# self.rm.download = True -# self.rm.tar = self.rm.cfg['ringmaster']['hadoop-tar-ball'] -# self.log.debug("self.rm.tar=%s" % self.rm.tar) -# -# self.rm.cd_to_tempdir() -# -# self.rm.tarAddress = None -# hostname = socket.gethostname() -# if (self.rm.download): -# self.rm.basename = os.path.basename(self.rm.tar) -# dest = os.path.join(os.getcwd(), self.rm.basename) -# src = self.rm.tar -# self.log.debug("cp %s -> %s" % (src, dest)) -# shutil.copy(src, dest) -# self.rm.tarAddress = "%s%s" % (self.rm.httpAddress, self.rm.basename) -# self.registerTarSource(hostname, self.rm.tarAddress) -# self.log.debug("Registered new tarAddress %s" % self.rm.tarAddress) -# else: -# self.log.debug("Download not set.") -# -# if (self.rm.tar != None): -# self.cfg['hodring']['download-addr'] = self.rm.tarAddress -# self.rm.cfg['hodring']['download-addr'] = self.rm.tarAddress -# -# sdl = self.rm.cfg['servicedesc'] -# workDirs = self.rm.getWorkDirs(self.rm.cfg, True) -# hdfsDesc = sdl['hdfs'] -# hdfs = None -# if hdfsDesc.isExternal(): -# hdfs = HdfsExternal(hdfsDesc, workDirs) -# else: -# hdfs = Hdfs(hdfsDesc, workDirs, 0, False, True) -# -# self.rm.serviceDict[hdfs.getName()] = hdfs -# mrDesc = sdl['mapred'] -# mr = None -# if mrDesc.isExternal(): -# mr = MapReduceExternal(mrDesc, workDirs) -# else: -# mr = MapReduce(mrDesc, workDirs, 1) -# self.rm.serviceDict[mr.getName()] = mr -# -# ringList = self.rm.serviceClient.getServiceInfo(self.cfg['hodring']['userid'], -# self.np.getServiceId(), 'hodring', 'hod') -# -# slaveList = ringList -# hdfsringXRAddress = None -# # Start HDFS Master - Step 1 -# if not hdfsDesc.isExternal(): -# masterFound = False -# for ring in ringList: -# ringXRAddress = ring['xrs'] -# if ringXRAddress == None: -# raise Exception("Could not get hodring XML-RPC server address.") -# if (ringXRAddress.find(self.hdfsHost) != -1): -# ringClient = hodXRClient(ringXRAddress, None, None, 0, 0, 0, False, 0) -# hdfsringXRAddress = ringXRAddress -# self.log.debug("Invoking clusterStart on " + ringXRAddress + " (HDFS Master)") -# ringClient.clusterStart() -# masterFound = True -# slaveList.remove(ring) -# break -# if not masterFound: -# raise Exception("HDFS Master host not found") -# while hdfs.getInfoAddrs() == None: -# self.log.debug("Waiting for HDFS Master (Name Node) to register dfs.info.port") -# time.sleep(1) -# -# # Start MAPRED Master - Step 2 -# if not mrDesc.isExternal(): -# masterFound = False -# for ring in ringList: -# ringXRAddress = ring['xrs'] -# if ringXRAddress == None: -# raise Exception("Could not get hodring XML-RPC server address.") -# if (not mrDesc.isExternal() and ringXRAddress.find(self.mapredHost) != -1): -# ringClient = hodXRClient(ringXRAddress, None, None, 0, 0, 0, False, 0) -# self.log.debug("Invoking clusterStart on " + ringXRAddress + " (MAPRED Master)") -# ringClient.clusterStart() -# masterFound = True -# slaveList.remove(ring) -# break -# if not masterFound: -# raise Excpetion("MAPRED Master host not found") -# while mr.getInfoAddrs() == None: -# self.log.debug("Waiting for MAPRED Master (Job Tracker) to register \ -# mapred.job.tracker.info.port") -# time.sleep(1) -# -# # Start Slaves - Step 3 -# for ring in slaveList: -# ringXRAddress = ring['xrs'] -# if ringXRAddress == None: -# raise Exception("Could not get hodring XML-RPC server address.") -# ringClient = hodXRClient(ringXRAddress, None, None, 0, 0, 0, False, 0) -# self.log.debug("Invoking clusterStart on " + ringXRAddress + " (Slaves)") -# ringThread = func(name='hodring_slaves_start', functionRef=ringClient.clusterStart()) -# ring['thread'] = ringThread -# ringThread.start() -# -# for ring in slaveList: -# ringThread = ring['thread'] -# if ringThread == None: -# raise Exception("Could not get hodring thread (Slave).") -# ringThread.join() -# self.log.debug("Completed clusterStart on " + ring['xrs'] + " (Slave)") -# -# # Run Admin Commands on HDFS Master - Step 4 -# if not hdfsDesc.isExternal(): -# if hdfsringXRAddress == None: -# raise Exception("HDFS Master host not found (to Run Admin Commands)") -# ringClient = hodXRClient(hdfsringXRAddress, None, None, 0, 0, 0, False, 0) -# self.log.debug("Invoking clusterStart(False) - Admin on " -# + hdfsringXRAddress + " (HDFS Master)") -# ringClient.clusterStart(False) -# -# except: -# self.log.debug(get_exception_string()) -# return False -# -# self.log.debug("Successfully started cluster.") -# return True -# -# def clusterStop(self): -# self.log.debug("clusterStop method invoked.") -# try: -# hdfsAddr = self.getServiceAddr('hdfs') -# if hdfsAddr.find(':') != -1: -# h, p = hdfsAddr.split(':', 1) -# self.hdfsHost = h -# self.log.debug("hdfsHost: " + self.hdfsHost) -# mapredAddr = self.getServiceAddr('mapred') -# if mapredAddr.find(':') != -1: -# h, p = mapredAddr.split(':', 1) -# self.mapredHost = h -# self.log.debug("mapredHost: " + self.mapredHost) -# ringList = self.rm.serviceClient.getServiceInfo(self.cfg['hodring']['userid'], -# self.np.getServiceId(), -# 'hodring', 'hod') -# for ring in ringList: -# ringXRAddress = ring['xrs'] -# if ringXRAddress == None: -# raise Exception("Could not get hodring XML-RPC server address.") -# ringClient = hodXRClient(ringXRAddress, None, None, 0, 0, 0, False) -# self.log.debug("Invoking clusterStop on " + ringXRAddress) -# ringThread = func(name='hodring_stop', functionRef=ringClient.clusterStop()) -# ring['thread'] = ringThread -# ringThread.start() -# -# for ring in ringList: -# ringThread = ring['thread'] -# if ringThread == None: -# raise Exception("Could not get hodring thread.") -# ringThread.join() -# self.log.debug("Completed clusterStop on " + ring['xrs']) -# -# except: -# self.log.debug(get_exception_string()) -# return False -# -# self.log.debug("Successfully stopped cluster.") -# -# return True - - def getCommand(self, addr): - """This method is called by the - hodrings to get commands from - the ringmaster""" - lock = self.cmdLock - cmdList = [] - lock.acquire() - try: - try: - for v in self.serviceDict.itervalues(): - if (not v.isExternal()): - if v.isLaunchable(self.serviceDict): - # If a master is still not launched, or the number of - # retries for launching master is not reached, - # launch master - if not v.isMasterLaunched() and \ - (v.getMasterFailureCount() <= \ - self.cfg['ringmaster']['max-master-failures']): - cmdList = v.getMasterCommands(self.serviceDict) - v.setlaunchedMaster() - v.setMasterAddress(addr) - break - if cmdList == []: - for s in self.serviceDict.itervalues(): - if (not v.isExternal()): - if s.isMasterInitialized(): - cl = s.getWorkerCommands(self.serviceDict) - cmdList.extend(cl) - else: - cmdList = [] - break - except: - self.log.debug(get_exception_string()) - finally: - lock.release() - pass - - cmd = addr + pformat(cmdList) - self.log.debug("getCommand returning " + cmd) - return cmdList - - def getAdminCommand(self, addr): - """This method is called by the - hodrings to get admin commands from - the ringmaster""" - lock = self.cmdLock - cmdList = [] - lock.acquire() - try: - try: - for v in self.serviceDict.itervalues(): - cmdList = v.getAdminCommands(self.serviceDict) - if cmdList != []: - break - except Exception, e: - self.log.debug(get_exception_string()) - finally: - lock.release() - pass - cmd = addr + pformat(cmdList) - self.log.debug("getAdminCommand returning " + cmd) - return cmdList - - def addMasterParams(self, addr, vals): - """This method is called by - hodring to update any parameters - its changed for the commands it was - running""" - self.log.debug('Comment: adding master params from %s' % addr) - self.log.debug(pformat(vals)) - lock = self.masterParamLock - lock.acquire() - try: - for v in self.serviceDict.itervalues(): - if v.isMasterLaunched(): - if (v.getMasterAddress() == addr): - v.setMasterParams(vals) - v.setMasterInitialized() - except: - self.log.debug(get_exception_string()) - pass - lock.release() - - return addr - - def setHodRingErrors(self, addr, errors): - """This method is called by the hodrings to update errors - it encountered while starting up""" - self.log.critical("Hodring at %s failed with following errors:\n%s" \ - % (addr, errors)) - lock = self.masterParamLock - lock.acquire() - try: - for v in self.serviceDict.itervalues(): - if v.isMasterLaunched(): - if (v.getMasterAddress() == addr): - # strip the PID part. - idx = addr.rfind('_') - if idx is not -1: - addr = addr[:idx] - v.setMasterFailed("Hodring at %s failed with following" \ - " errors:\n%s" % (addr, errors)) - except: - self.log.debug(get_exception_string()) - pass - lock.release() - return True - - def getKeys(self): - lock= self.masterParamLock - lock.acquire() - keys = self.serviceDict.keys() - lock.release() - - return keys - - def getServiceAddr(self, name): - addr = 'not found' - self.log.debug("getServiceAddr name: %s" % name) - lock= self.masterParamLock - lock.acquire() - try: - service = self.serviceDict[name] - except KeyError: - pass - else: - self.log.debug("getServiceAddr service: %s" % service) - # Check if we should give up ! If the limit on max failures is hit, - # give up. - err = service.getMasterFailed() - if (err is not None) and \ - (service.getMasterFailureCount() > \ - self.cfg['ringmaster']['max-master-failures']): - self.log.critical("Detected errors (%s) beyond allowed number"\ - " of failures (%s). Flagging error to client" \ - % (service.getMasterFailureCount(), \ - self.cfg['ringmaster']['max-master-failures'])) - addr = "Error: " + err - elif (service.isMasterInitialized()): - addr = service.getMasterAddrs()[0] - else: - addr = 'not found' - lock.release() - self.log.debug("getServiceAddr addr %s: %s" % (name, addr)) - - return addr - - def getURLs(self, name): - addr = 'none' - lock = self.masterParamLock - lock.acquire() - - try: - service = self.serviceDict[name] - except KeyError: - pass - else: - if (service.isMasterInitialized()): - addr = service.getInfoAddrs()[0] - - lock.release() - - return addr - - def stopRM(self): - """An XMLRPC call which will spawn a thread to stop the Ringmaster program.""" - # We spawn a thread here because we want the XMLRPC call to return. Calling - # stop directly from here will also stop the XMLRPC server. - try: - self.log.debug("inside xml-rpc call to stop ringmaster") - rmStopperThread = func('RMStopper', self.rm.stop) - rmStopperThread.start() - self.log.debug("returning from xml-rpc call to stop ringmaster") - return True - except: - self.log.debug("Exception in stop: %s" % get_exception_string()) - return False - -class RingMaster: - def __init__(self, cfg, log, **kwds): - """starts nodepool and services""" - self.download = False - self.httpServer = None - self.cfg = cfg - self.log = log - self.__hostname = local_fqdn() - self.workDirs = None - - # ref to the idle job tracker object. - self.__jtMonitor = None - self.__idlenessDetected = False - self.__stopInProgress = False - self.__isStopped = False # to let main exit - self.__exitCode = 0 # exit code with which the ringmaster main method should return - - self.workers_per_ring = self.cfg['ringmaster']['workers_per_ring'] - - self.__initialize_signal_handlers() - - sdd = self.cfg['servicedesc'] - gsvc = None - for key in sdd: - gsvc = sdd[key] - break - - npd = self.cfg['nodepooldesc'] - self.np = NodePoolUtil.getNodePool(npd, cfg, log) - - self.log.debug("Getting service ID.") - - self.serviceId = self.np.getServiceId() - - self.log.debug("Got service ID: %s" % self.serviceId) - - self.tarSrcLoc = None - if self.cfg['ringmaster'].has_key('hadoop-tar-ball'): - self.download = True - self.tarSrcLoc = self.cfg['ringmaster']['hadoop-tar-ball'] - - self.cd_to_tempdir() - - if (self.download): - self.__copy_tarball(os.getcwd()) - self.basename = self.__find_tarball_in_dir(os.getcwd()) - if self.basename is None: - raise Exception('Did not find tarball copied from %s in %s.' - % (self.tarSrcLoc, os.getcwd())) - - self.serviceAddr = to_http_url(self.cfg['ringmaster']['svcrgy-addr']) - - self.log.debug("Service registry @ %s" % self.serviceAddr) - - self.serviceClient = hodXRClient(self.serviceAddr) - self.serviceDict = {} - try: - sdl = self.cfg['servicedesc'] - - workDirs = self.getWorkDirs(cfg) - - hdfsDesc = sdl['hdfs'] - hdfs = None - - # Determine hadoop Version - hadoopVers = hadoopVersion(self.__getHadoopDir(), \ - self.cfg['hodring']['java-home'], self.log) - - if (hadoopVers['major']==None) or (hadoopVers['minor']==None): - raise Exception('Could not retrive the version of Hadoop.' - + ' Check the Hadoop installation or the value of the hodring.java-home variable.') - if hdfsDesc.isExternal(): - hdfs = HdfsExternal(hdfsDesc, workDirs, version=int(hadoopVers['minor'])) - hdfs.setMasterParams( self.cfg['gridservice-hdfs'] ) - else: - hdfs = Hdfs(hdfsDesc, workDirs, 0, version=int(hadoopVers['minor']), - workers_per_ring = self.workers_per_ring) - - self.serviceDict[hdfs.getName()] = hdfs - - mrDesc = sdl['mapred'] - mr = None - if mrDesc.isExternal(): - mr = MapReduceExternal(mrDesc, workDirs, version=int(hadoopVers['minor'])) - mr.setMasterParams( self.cfg['gridservice-mapred'] ) - else: - mr = MapReduce(mrDesc, workDirs,1, version=int(hadoopVers['minor']), - workers_per_ring = self.workers_per_ring) - - self.serviceDict[mr.getName()] = mr - except: - self.log.critical("Exception in creating Hdfs and Map/Reduce descriptor objects: \ - %s." % get_exception_error_string()) - self.log.debug(get_exception_string()) - raise - - # should not be starting these in a constructor - ringMasterServer.startService(self.serviceDict, cfg, self.np, log, self) - - self.rpcserver = ringMasterServer.getAddress() - - self.httpAddress = None - self.tarAddress = None - hostname = socket.gethostname() - if (self.download): - self.httpServer = threadedHTTPServer(hostname, - self.cfg['ringmaster']['http-port-range']) - - self.httpServer.serve_forever() - self.httpAddress = "http://%s:%d/" % (self.httpServer.server_address[0], - self.httpServer.server_address[1]) - self.tarAddress = "%s%s" % (self.httpAddress, self.basename) - - ringMasterServer.instance.logMasterSources.registerTarSource(hostname, - self.tarAddress) - else: - self.log.debug("Download not set.") - - self.log.debug("%s %s %s %s %s" % (self.cfg['ringmaster']['userid'], - self.serviceId, self.__hostname, 'ringmaster', 'hod')) - - if self.cfg['ringmaster']['register']: - if self.httpAddress: - self.serviceClient.registerService(self.cfg['ringmaster']['userid'], - self.serviceId, self.__hostname, 'ringmaster', 'hod', { - 'xrs' : self.rpcserver, 'http' : self.httpAddress }) - else: - self.serviceClient.registerService(self.cfg['ringmaster']['userid'], - self.serviceId, self.__hostname, 'ringmaster', 'hod', { - 'xrs' : self.rpcserver, }) - - self.log.debug("Registered with serivce registry: %s." % self.serviceAddr) - - hodRingPath = os.path.join(cfg['ringmaster']['base-dir'], 'bin', 'hodring') - hodRingWorkDir = os.path.join(cfg['hodring']['temp-dir'], 'hodring' + '_' - + getpass.getuser()) - - self.cfg['hodring']['hodring'] = [hodRingWorkDir,] - self.cfg['hodring']['svcrgy-addr'] = self.cfg['ringmaster']['svcrgy-addr'] - self.cfg['hodring']['service-id'] = self.np.getServiceId() - - self.cfg['hodring']['ringmaster-xrs-addr'] = self.__url_to_addr(self.rpcserver) - - if (self.tarSrcLoc != None): - cfg['hodring']['download-addr'] = self.tarAddress - - self.__init_job_tracker_monitor(ringMasterServer.instance.logMasterSources) - - def __init_job_tracker_monitor(self, logMasterSources): - hadoopDir = self.__getHadoopDir() - self.log.debug('hadoopdir=%s, java-home=%s' % \ - (hadoopDir, self.cfg['hodring']['java-home'])) - try: - self.__jtMonitor = JobTrackerMonitor(self.log, self, - self.cfg['ringmaster']['jt-poll-interval'], - self.cfg['ringmaster']['idleness-limit'], - hadoopDir, self.cfg['hodring']['java-home'], - logMasterSources) - self.log.debug('starting jt monitor') - self.__jtMonitor.start() - except: - self.log.critical('Exception in running idle job tracker. This cluster cannot be deallocated if idle.\ - Exception message: %s' % get_exception_error_string()) - self.log.debug('Exception details: %s' % get_exception_string()) - - - def __getHadoopDir(self): - hadoopDir = None - if self.cfg['ringmaster'].has_key('hadoop-tar-ball'): - tarFile = os.path.join(os.getcwd(), self.basename) - ret = untar(tarFile, os.getcwd()) - if not ret: - raise Exception('Untarring tarfile %s to directory %s failed. Cannot find hadoop directory.' \ - % (tarFile, os.getcwd())) - hadoopDir = os.path.join(os.getcwd(), self.__get_dir(tarFile)) - else: - hadoopDir = self.cfg['gridservice-mapred']['pkgs'] - self.log.debug('Returning Hadoop directory as: %s' % hadoopDir) - return hadoopDir - - def __get_dir(self, name): - """Return the root directory inside the tarball - specified by name. Assumes that the tarball begins - with a root directory.""" - import tarfile - myTarFile = tarfile.open(name) - hadoopPackage = myTarFile.getnames()[0] - self.log.debug("tarball name : %s hadoop package name : %s" %(name,hadoopPackage)) - return hadoopPackage - - def __find_tarball_in_dir(self, dir): - """Find the tarball among files specified in the given - directory. We need this method because how the tarball - source URI is given depends on the method of copy and - we can't get the tarball name from that. - This method will fail if there are multiple tarballs - in the directory with the same suffix.""" - files = os.listdir(dir) - for file in files: - if self.tarSrcLoc.endswith(file): - return file - return None - - def __copy_tarball(self, destDir): - """Copy the hadoop tar ball from a remote location to the - specified destination directory. Based on the URL it executes - an appropriate copy command. Throws an exception if the command - returns a non-zero exit code.""" - # for backwards compatibility, treat the default case as file:// - url = '' - if self.tarSrcLoc.startswith('/'): - url = 'file:/' - src = '%s%s' % (url, self.tarSrcLoc) - if src.startswith('file://'): - src = src[len('file://')-1:] - cpCmd = '/bin/cp' - cmd = '%s %s %s' % (cpCmd, src, destDir) - self.log.debug('Command to execute: %s' % cmd) - copyProc = simpleCommand('remote copy', cmd) - copyProc.start() - copyProc.wait() - copyProc.join() - ret = copyProc.exit_code() - self.log.debug('Completed command execution. Exit Code: %s.' % ret) - - if ret != 0: - output = copyProc.output() - raise Exception('Could not copy tarball using command %s. Exit code: %s. Output: %s' - % (cmd, ret, output)) - else: - raise Exception('Unsupported URL for file: %s' % src) - -# input: http://hostname:port/. output: [hostname,port] - def __url_to_addr(self, url): - addr = url.rstrip('/') - if addr.startswith('http://'): - addr = addr.replace('http://', '', 1) - addr_parts = addr.split(':') - return [addr_parts[0], int(addr_parts[1])] - - def __initialize_signal_handlers(self): - def sigStop(sigNum, handler): - sig_wrapper(sigNum, self.stop) - - signal.signal(signal.SIGTERM, sigStop) - signal.signal(signal.SIGINT, sigStop) - signal.signal(signal.SIGQUIT, sigStop) - - def __clean_up(self): - tempDir = self.__get_tempdir() - os.chdir(os.path.split(tempDir)[0]) - if os.path.exists(tempDir): - shutil.rmtree(tempDir, True) - - self.log.debug("Cleaned up temporary dir: %s" % tempDir) - - def __get_tempdir(self): - dir = os.path.join(self.cfg['ringmaster']['temp-dir'], - "%s.%s.ringmaster" % (self.cfg['ringmaster']['userid'], - self.np.getServiceId())) - return dir - - def getWorkDirs(self, cfg, reUse=False): - - if (not reUse) or (self.workDirs == None): - import math - frand = random.random() - while math.ceil(frand) != math.floor(frand): - frand = frand * 100 - - irand = int(frand) - uniq = '%s-%d-%s' % (socket.gethostname(), os.getpid(), irand) - dirs = [] - parentDirs = cfg['ringmaster']['work-dirs'] - for p in parentDirs: - dir = os.path.join(p, uniq) - dirs.append(dir) - self.workDirs = dirs - - return self.workDirs - - def _fetchLink(self, link, parentDir): - parser = miniHTMLParser() - self.log.debug("Checking link %s" %link) - while link: - - # Get the file from the site and link - input = urllib.urlopen(link) - out = None - contentType = input.info().gettype() - isHtml = contentType == 'text/html' - - #print contentType - if isHtml: - parser.setBaseUrl(input.geturl()) - else: - parsed = urlparse.urlparse(link) - hp = parsed[1] - h = hp - p = None - if hp.find(':') != -1: - h, p = hp.split(':', 1) - path = parsed[2] - path = path.split('/') - file = os.path.join(parentDir, h, p) - for c in path: - if c == '': - continue - file = os.path.join(file, c) - - try: - self.log.debug('Creating %s' % file) - dir, tail = os.path.split(file) - if not os.path.exists(dir): - os.makedirs(dir) - except: - self.log.debug(get_exception_string()) - - out = open(file, 'w') - - bufSz = 8192 - buf = input.read(bufSz) - while len(buf) > 0: - if isHtml: - # Feed the file into the HTML parser - parser.feed(buf) - if out: - out.write(buf) - buf = input.read(bufSz) - - input.close() - if out: - out.close() - - # Search the retfile here - - # Get the next link in level traversal order - link = parser.getNextLink() - - parser.close() - - def _finalize(self): - try: - # FIXME: get dir from config - dir = 'HOD-log-P%d' % (os.getpid()) - dir = os.path.join('.', dir) - except: - self.log.debug(get_exception_string()) - - self.np.finalize() - - def handleIdleJobTracker(self): - self.log.critical("Detected idle job tracker for %s seconds. The allocation will be cleaned up." \ - % self.cfg['ringmaster']['idleness-limit']) - self.__idlenessDetected = True - - def cd_to_tempdir(self): - dir = self.__get_tempdir() - - if not os.path.exists(dir): - os.makedirs(dir) - os.chdir(dir) - - return dir - - def getWorkload(self): - return self.workload - - def getHostName(self): - return self.__hostname - - def start(self): - """run the thread main loop""" - - self.log.debug("Entered start method.") - hodring = os.path.join(self.cfg['ringmaster']['base-dir'], - 'bin', 'hodring') - largs = [hodring] - targs = self.cfg.get_args(section='hodring') - largs.extend(targs) - - hodringCmd = "" - for item in largs: - hodringCmd = "%s%s " % (hodringCmd, item) - - self.log.debug(hodringCmd) - - if self.np.runWorkers(largs) > 0: - self.log.critical("Failed to start worker.") - - self.log.debug("Returned from runWorkers.") - - self._finalize() - - def __findExitCode(self): - """Determine the exit code based on the status of the cluster or jobs run on them""" - xmlrpcServer = ringMasterServer.instance.logMasterSources - if xmlrpcServer.getServiceAddr('hdfs') == 'not found' or \ - xmlrpcServer.getServiceAddr('hdfs').startswith("Error: "): - self.__exitCode = 7 - elif xmlrpcServer.getServiceAddr('mapred') == 'not found' or \ - xmlrpcServer.getServiceAddr('mapred').startswith("Error: "): - self.__exitCode = 8 - else: - clusterStatus = get_cluster_status(xmlrpcServer.getServiceAddr('hdfs'), - xmlrpcServer.getServiceAddr('mapred')) - if clusterStatus != 0: - self.__exitCode = clusterStatus - else: - self.__exitCode = self.__findHadoopJobsExitCode() - self.log.debug('exit code %s' % self.__exitCode) - - def __findHadoopJobsExitCode(self): - """Determine the consolidate exit code of hadoop jobs run on this cluster, provided - this information is available. Return 0 otherwise""" - ret = 0 - failureStatus = 3 - failureCount = 0 - if self.__jtMonitor: - jobStatusList = self.__jtMonitor.getJobsStatus() - try: - if len(jobStatusList) > 0: - for jobStatus in jobStatusList: - self.log.debug('job status for %s: %s' % (jobStatus.getJobId(), - jobStatus.getStatus())) - if jobStatus.getStatus() == failureStatus: - failureCount = failureCount+1 - if failureCount > 0: - if failureCount == len(jobStatusList): # all jobs failed - ret = 16 - else: - ret = 17 - except: - self.log.debug('exception in finding hadoop jobs exit code' % get_exception_string()) - return ret - - def stop(self): - self.log.debug("RingMaster stop method invoked.") - if self.__stopInProgress or self.__isStopped: - return - self.__stopInProgress = True - if ringMasterServer.instance is not None: - self.log.debug('finding exit code') - self.__findExitCode() - self.log.debug('stopping ringmaster instance') - ringMasterServer.stopService() - else: - self.__exitCode = 6 - if self.__jtMonitor is not None: - self.__jtMonitor.stop() - if self.httpServer: - self.httpServer.stop() - - self.__clean_up() - self.__isStopped = True - - def shouldStop(self): - """Indicates whether the main loop should exit, either due to idleness condition, - or a stop signal was received""" - return self.__idlenessDetected or self.__isStopped - - def getExitCode(self): - """return the exit code of the program""" - return self.__exitCode - -def main(cfg,log): - try: - rm = None - dGen = DescGenerator(cfg) - cfg = dGen.initializeDesc() - rm = RingMaster(cfg, log) - rm.start() - while not rm.shouldStop(): - time.sleep(1) - rm.stop() - log.debug('returning from main') - return rm.getExitCode() - except Exception, e: - if log: - log.critical(get_exception_string()) - raise Exception(e) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Schedulers/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Schedulers/__init__.py deleted file mode 100644 index 12c2f1e1da706ef7b66c7ec27e762b2287340d9f..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Schedulers/__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/Schedulers/torque.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/Schedulers/torque.py deleted file mode 100644 index a4e8f95dfde6055bd112362a1b818876b2f3ee5e..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/Schedulers/torque.py +++ /dev/null @@ -1,175 +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 os, pprint, re, time - -from hodlib.Common.threads import simpleCommand -from hodlib.Common.util import args_to_string -from hodlib.Common.logger import hodDummyLogger - -reQstatLine = re.compile("^\s*(\w+)\s*=\s*(.*)\s*$") - -class torqueInterface: - def __init__(self, torqueDir, environment, log=None): - self.__qsub = os.path.join(torqueDir, 'bin', 'qsub') - self.__qdel = os.path.join(torqueDir, 'bin', 'qdel') - self.__qstat = os.path.join(torqueDir, 'bin', 'qstat') - self.__pbsNodes = os.path.join(torqueDir, 'bin', 'pbsnodes') - self.__pbsdsh = os.path.join(torqueDir, 'bin', 'pbsdsh') - self.__qalter = os.path.join(torqueDir, 'bin', 'qalter') - self.__env = environment - - self.__log = log - if not self.__log: - self.__log = hodDummyLogger() - - def qsub(self, argList, stdinList): - jobID = False - exitCode = 0 - - qsubCommand = "%s %s" % (self.__qsub, args_to_string(argList)) - - self.__log.debug("qsub -> %s" % qsubCommand) - - qsubProcess = simpleCommand('qsub', qsubCommand, env=self.__env) - qsubProcess.start() - - while qsubProcess.stdin == None: - time.sleep(.2) - - try: - for line in stdinList: - self.__log.debug("qsub stdin: %s" % line) - print >>qsubProcess.stdin, line - qsubProcess.stdin.close() - except IOError, i: - # If torque's qsub is given invalid params, it fails & returns immediately - # Check for such errors here - # Wait for command execution to finish - qsubProcess.wait() - qsubProcess.join() - output = qsubProcess.output() - if output!=[]: - self.__log.critical("qsub Failure : %s " % output[0].strip()) - self.__log.critical("qsub Command : %s" % qsubCommand) - return None, qsubProcess.exit_code() - - qsubProcess.wait() - qsubProcess.join() - - exitCode = qsubProcess.exit_code() - if exitCode == 0: - buffer = qsubProcess.output() - jobID = buffer[0].rstrip('\n') - self.__log.debug("qsub jobid: %s" % jobID) - else: - self.__log.critical("qsub error: %s" % qsubProcess.exit_status_string()) - - return jobID, exitCode - - def qstat(self, jobID): - qstatInfo = None - - qstatCommand = "%s -f -1 %s" % (self.__qstat, jobID) - - self.__log.debug(qstatCommand) - - qstatProcess = simpleCommand('qstat', qstatCommand, env=self.__env) - qstatProcess.start() - qstatProcess.wait() - qstatProcess.join() - - exitCode = qstatProcess.exit_code() - if exitCode > 0: - self.__log.warn('qstat error: %s' % qstatProcess.exit_status_string()) - else: - qstatInfo = {} - for line in qstatProcess.output(): - line = line.rstrip() - if line.find('=') != -1: - qstatMatch = reQstatLine.match(line) - if qstatMatch: - key = qstatMatch.group(1) - value = qstatMatch.group(2) - qstatInfo[key] = value - - if 'exec_host' in qstatInfo: - list = qstatInfo['exec_host'].split('+') - addrList = [] - - for item in list: - [head, end] = item.split('/', 1) - addrList.append(head) - - qstatInfo['exec_host'] = addrList - - return qstatInfo, exitCode - - def pbs_nodes(self, argString): - pass - - def qdel(self, jobId, force=False): - exitCode = 0 - qdel = self.__qdel - if force: - qdel = "%s -p %s" % (qdel, jobId) - else: - qdel = "%s %s" % (qdel, jobId) - - self.__log.debug(qdel) - - qdelProcess = simpleCommand('qdel', qdel, env=self.__env) - qdelProcess.start() - qdelProcess.wait() - qdelProcess.join() - - exitCode = qdelProcess.exit_code() - - return exitCode - - def pbsdsh(self, arguments): - status = None - - pbsdshCommand = "%s %s" % (self.__pbsdsh, args_to_string(arguments)) - - self.__log.debug("pbsdsh command: %s" % pbsdshCommand) - - pbsdsh = simpleCommand('pbsdsh', pbsdshCommand, env=self.__env) - pbsdsh.start() - - for i in range(0, 30): - status = pbsdsh.exit_code() - if status: - self.__log.error("pbsdsh failed: %s" % pbsdsh.exit_status_string()) - break - - if not status: status = 0 - - return status - - def qalter(self, fieldName, fieldValue, jobId): - """Update the job field with fieldName with the fieldValue. - The fieldValue must be modifiable after the job is submitted.""" - - # E.g. to alter comment: qalter -W notes='value` jobId - qalterCmd = '%s -W %s=\"%s\" %s' % (self.__qalter, fieldName, fieldValue, jobId) - self.__log.debug("qalter command: %s" % qalterCmd) - qalterProcess = simpleCommand('qalter', qalterCmd, env=self.__env) - qalterProcess.start() - qalterProcess.wait() - qalterProcess.join() - exitCode = qalterProcess.exit_code() - - return exitCode diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceProxy/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceProxy/__init__.py deleted file mode 100644 index 12c2f1e1da706ef7b66c7ec27e762b2287340d9f..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceProxy/__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/ServiceProxy/serviceProxy.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceProxy/serviceProxy.py deleted file mode 100644 index 0e80d20ce642e8b7cbe8ff82bc8338e3efbf876d..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceProxy/serviceProxy.py +++ /dev/null @@ -1,49 +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. -"""HOD Service Proxy Implementation""" -# -*- python -*- - -import sys, time, signal, httplib, socket, threading -import sha, base64, hmac -import xml.dom.minidom - -from hodlib.Common.socketServers import hodHTTPServer -from hodlib.Common.hodsvc import hodBaseService -from hodlib.Common.threads import loop -from hodlib.Common.tcp import tcpSocket -from hodlib.Common.util import get_exception_string -from hodlib.Common.AllocationManagerUtil import * - -class svcpxy(hodBaseService): - def __init__(self, config): - hodBaseService.__init__(self, 'serviceProxy', config['service_proxy'], - xrtype='twisted') - self.amcfg=config['allocation_manager'] - - def _xr_method_isProjectUserValid(self, userid, project, ignoreErrors = False, timeOut = 15): - return self.isProjectUserValid(userid, project, ignoreErrors, timeOut) - - def isProjectUserValid(self, userid, project, ignoreErrors, timeOut): - """Method thats called upon by - the hodshell to verify if the - specified (user, project) combination - is valid""" - self.logs['main'].info("Begin isProjectUserValid()") - am = AllocationManagerUtil.getAllocationManager(self.amcfg['id'], - self.amcfg, - self.logs['main']) - self.logs['main'].info("End isProjectUserValid()") - return am.getQuote(userid, project) diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceRegistry/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceRegistry/__init__.py deleted file mode 100644 index 12c2f1e1da706ef7b66c7ec27e762b2287340d9f..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceRegistry/__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/ServiceRegistry/serviceRegistry.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceRegistry/serviceRegistry.py deleted file mode 100644 index ac18ff77d45383f95cec297c4244f5114ed6c14f..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/ServiceRegistry/serviceRegistry.py +++ /dev/null @@ -1,127 +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 sys, time, socket, threading, copy, pprint - -from hodlib.Common.hodsvc import hodBaseService -from hodlib.Common.threads import loop -from hodlib.Common.tcp import tcpSocket -from hodlib.Common.util import get_exception_string -import logging - -class svcrgy(hodBaseService): - def __init__(self, config, log=None): - hodBaseService.__init__(self, 'serviceRegistry', config) - - self.__serviceDict = {} - self.__failCount = {} - self.__released = {} - self.__locked = {} - - self.__serviceDictLock = threading.Lock() - self.RMErrorMsgs = None # Ringmaster error messages - self.log = log - if self.log is None: - self.log = logging.getLogger() - - def __get_job_key(self, userid, job): - return "%s-%s" % (userid, job) - - def _xr_method_registerService(self, userid, job, host, name, type, dict): - return self.registerService(userid, job, host, name, type, dict) - - def _xr_method_getServiceInfo(self, userid=None, job=None, name=None, - type=None): - return self.getServiceInfo(userid, job, name, type) - - def _xr_method_setRMError(self, args): - self.log.debug("setRMError called with %s" % args) - self.RMErrorMsgs = args - return True - - def _xr_method_getRMError(self): - self.log.debug("getRMError called") - if self.RMErrorMsgs is not None: - return self.RMErrorMsgs - else: - self.log.debug("no Ringmaster error messages") - return False - - def registerService(self, userid, job, host, name, type, dict): - """Method thats called upon by - the ringmaster to register to the - the service registry""" - lock = self.__serviceDictLock - lock.acquire() - try: - self.logs['main'].debug("Registering %s.%s.%s.%s.%s..." % ( - userid, job, host, name, type)) - id = "%s.%s" % (name, type) - - if userid in self.__serviceDict: - if job in self.__serviceDict[userid]: - if host in self.__serviceDict[userid][job]: - self.__serviceDict[userid][job][host].append( - {id : dict,}) - else: - self.__serviceDict[userid][job][host] = [ - {id : dict,},] - else: - self.__serviceDict[userid][job] = {host : [ - { id : dict,},]} - else: - self.__serviceDict[userid] = {job : {host : [ - { id : dict,},]}} - - finally: - lock.release() - - return True - - def getXMLRPCAddr(self): - """return the xml rpc server address""" - return self._xrc.server_address - - def getServiceInfo(self, userid=None, job=None, name=None, type=None): - """This method is called upon by others - to query for a particular service returns - a dictionary of elements""" - - self.logs['main'].debug("inside getServiceInfo: %s.%s.%s" % (userid, job, name)) - retdict = {} - lock = self.__serviceDictLock - lock.acquire() - try: - if userid in self.__serviceDict: - if job in self.__serviceDict[userid]: - if name and type: - retdict = [] - id = "%s.%s" % (name, type) - for host in self.__serviceDict[userid][job]: - for dict in self.__serviceDict[userid][job][host]: - [loopID, ] = dict.keys() - if loopID == id: - retdict.append(dict[id]) - else: - retdict = copy.deepcopy( - self.__serviceDict[userid][job]) - elif not job: - retdict = copy.deepcopy(self.__serviceDict[userid]) - elif not userid: - retdict = copy.deepcopy(self.__serviceDict) - finally: - lock.release() - - return retdict diff --git a/third_party/hadoop-0.20.0/contrib/hod/hodlib/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/hodlib/__init__.py deleted file mode 100644 index 56759d796307be4bb5fc9ff6b4ec461937c483fd..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/hodlib/__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/ivy.xml b/third_party/hadoop-0.20.0/contrib/hod/ivy.xml deleted file mode 100644 index e775663256e61afe59c1b3279fab406e5d3271ba..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/ivy.xml +++ /dev/null @@ -1,22 +0,0 @@ -<?xml version="1.0" ?> -<ivy-module version="1.0"> - <info organisation="org.apache.hadoop" module="${ant.project.name}"> - <license name="Apache 2.0"/> - <ivyauthor name="Apache Hadoop Team" url="http://hadoop.apache.org"/> - <description> - Apache Hadoop - </description> - </info> - <configurations defaultconfmapping="default"> - <!--these match the Maven configurations--> - <conf name="default" extends="master,runtime"/> - <conf name="master" description="contains the artifact but no dependencies"/> - <conf name="runtime" description="runtime but not the artifact" /> - <!--Private configurations. --> - - <conf name="common" visibility="private" - description="artifacts needed to compile/test the application"/> - </configurations> - <dependencies> - </dependencies> -</ivy-module> diff --git a/third_party/hadoop-0.20.0/contrib/hod/ivy/libraries.properties b/third_party/hadoop-0.20.0/contrib/hod/ivy/libraries.properties deleted file mode 100644 index a470b372ad369f765d65e04c057c3ca2630100a9..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/ivy/libraries.properties +++ /dev/null @@ -1,5 +0,0 @@ -#This properties file lists the versions of the various artifacts used by streaming. -#It drives ivy and the generation of a maven POM - -#Please list the dependencies name with version if they are different from the ones -#listed in the global libraries.properties file (in alphabetical order) diff --git a/third_party/hadoop-0.20.0/contrib/hod/support/checklimits.sh b/third_party/hadoop-0.20.0/contrib/hod/support/checklimits.sh deleted file mode 100644 index 61de9cddf997d0a387b24b87eabbbe412ed3f790..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/support/checklimits.sh +++ /dev/null @@ -1,57 +0,0 @@ -#!/bin/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. - -COMMANDS=( "qstat" "qalter" "checkjob" ) -ERROR=0 -for (( i=0; i<${#COMMANDS[@]}; i++ )) -do - cmd=${COMMANDS[$i]} - CMD_PATH=`which $cmd 2>/dev/null` - if [ $? -ne 0 ] - then - echo Could not find $cmd in PATH - ERROR=1 - fi -done -if [ $ERROR -ne 0 ] -then - exit 1 -fi - -jobs=`qstat -i |grep -o -e '^[0-9]*'` -for job in $jobs -do - echo -en "$job\t" - PATTERN="job [^ ]* violates active HARD MAXPROC limit of \([0-9]*\) for user [^ ]*[ ]*(R: \([0-9]*\), U: \([0-9]*\))" - OUT=`checkjob $job 2>&1|grep -o -e "$PATTERN"` - if [ $? -eq 0 ] - then - echo -en "| Exceeds resource limits\t" - COMMENT_FIELD=`echo $OUT|sed -e "s/$PATTERN/User-limits exceeded. Requested:\2 Used:\3 MaxLimit:\1/"` - qstat -f $job|grep '^[ \t]*comment = .*$' >/dev/null - if [ $? -ne 0 ] - then - echo -en "| Comment field updated\t" - qalter $job -W comment="$COMMENT_FIELD" >/dev/null - else - echo -en "| Comment field already set\t" - fi - else - echo -en "| Doesn't exceed limits.\t" - fi - echo -done diff --git a/third_party/hadoop-0.20.0/contrib/hod/support/logcondense.py b/third_party/hadoop-0.20.0/contrib/hod/support/logcondense.py deleted file mode 100644 index c8fd4dbc027cd04a4fe3275a01f5586a154399c2..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/support/logcondense.py +++ /dev/null @@ -1,212 +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 -u $base_name ${1+"$@"} -elif [ -e /usr/bin/python ]; then -exec /usr/bin/python -OO -u $base_name ${1+"$@"} -elif [ -e /usr/local/bin/python ]; then -exec /usr/local/bin/python -OO -u $base_name ${1+"$@"} -else -exec python -OO -u $base_name ${1+"$@"} -fi -":""" - -from os import popen3 -import os, sys -import re -import time -from datetime import datetime -from optparse import OptionParser - -myName = os.path.basename(sys.argv[0]) -myName = re.sub(".*/", "", myName) - -reVersion = re.compile(".*(\d+_\d+).*") - -VERSION = '$HeadURL: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.20/src/contrib/hod/support/logcondense.py $' - -reMatch = reVersion.match(VERSION) -if reMatch: - VERSION = reMatch.group(1) - VERSION = re.sub("_", ".", VERSION) -else: - VERSION = 'DEV' - -options = ( {'short' : "-p", - 'long' : "--package", - 'type' : "string", - 'action' : "store", - 'dest' : "package", - 'metavar' : " ", - 'default' : 'hadoop', - 'help' : "Bin file for hadoop"}, - - {'short' : "-d", - 'long' : "--days", - 'type' : "int", - 'action' : "store", - 'dest' : "days", - 'metavar' : " ", - 'default' : 7, - 'help' : "Number of days before logs are deleted"}, - - {'short' : "-c", - 'long' : "--config", - 'type' : "string", - 'action' : "store", - 'dest' : "config", - 'metavar' : " ", - 'default' : None, - 'help' : "config directory for hadoop"}, - - {'short' : "-l", - 'long' : "--logs", - 'type' : "string", - 'action' : "store", - 'dest' : "log", - 'metavar' : " ", - 'default' : "/user", - 'help' : "directory prefix under which logs are stored per user"}, - - {'short' : "-n", - 'long' : "--dynamicdfs", - 'type' : "string", - 'action' : "store", - 'dest' : "dynamicdfs", - 'metavar' : " ", - 'default' : "false", - 'help' : "'true', if the cluster is used to bring up dynamic dfs clusters, 'false' otherwise"} - ) - -def getDfsCommand(options, args): - if (options.config == None): - cmd = options.package + " " + "dfs " + args - else: - cmd = options.package + " " + "--config " + options.config + " dfs " + args - return cmd - -def runcondense(): - import shutil - - options = process_args() - # if the cluster is used to bring up dynamic dfs, we must leave NameNode and JobTracker logs, - # otherwise only JobTracker logs. Likewise, in case of dynamic dfs, we must also look for - # deleting datanode logs - filteredNames = ['jobtracker'] - deletedNamePrefixes = ['*-tasktracker-*'] - if options.dynamicdfs == 'true': - filteredNames.append('namenode') - deletedNamePrefixes.append('*-datanode-*') - - filepath = '%s/\*/hod-logs/' % (options.log) - cmd = getDfsCommand(options, "-lsr " + filepath) - (stdin, stdout, stderr) = popen3(cmd) - lastjobid = 'none' - toPurge = { } - for line in stdout: - try: - m = re.match("^.*\s(.*)\n$", line) - filename = m.group(1) - # file name format: <prefix>/<user>/hod-logs/<jobid>/[0-9]*-[jobtracker|tasktracker|datanode|namenode|]-hostname-YYYYMMDDtime-random.tar.gz - # first strip prefix: - if filename.startswith(options.log): - filename = filename.lstrip(options.log) - if not filename.startswith('/'): - filename = '/' + filename - else: - continue - - # Now get other details from filename. - k = re.match("/(.*)/hod-logs/(.*)/.*-.*-([0-9][0-9][0-9][0-9])([0-9][0-9])([0-9][0-9]).*$", filename) - if k: - username = k.group(1) - jobid = k.group(2) - datetimefile = datetime(int(k.group(3)), int(k.group(4)), int(k.group(5))) - datetimenow = datetime.utcnow() - diff = datetimenow - datetimefile - filedate = k.group(3) + k.group(4) + k.group(5) - newdate = datetimenow.strftime("%Y%m%d") - print "%s %s %s %d" % (filename, filedate, newdate, diff.days) - - # if the cluster is used to bring up dynamic dfs, we must also leave NameNode logs. - foundFilteredName = False - for name in filteredNames: - if filename.find(name) >= 0: - foundFilteredName = True - break - - if foundFilteredName: - continue - - if (diff.days > options.days): - desttodel = filename - if not toPurge.has_key(jobid): - toPurge[jobid] = options.log.rstrip("/") + "/" + username + "/hod-logs/" + jobid - except Exception, e: - print >> sys.stderr, e - - for job in toPurge.keys(): - try: - for prefix in deletedNamePrefixes: - cmd = getDfsCommand(options, "-rm " + toPurge[job] + '/' + prefix) - print cmd - ret = 0 - ret = os.system(cmd) - if (ret != 0): - print >> sys.stderr, "Command failed to delete file " + cmd - except Exception, e: - print >> sys.stderr, e - - -def process_args(): - global options, myName, VERSION - - usage = "usage: %s <ARGS>" % (myName) - - version = "%s %s" % (myName, VERSION) - - argParser = OptionParser(usage=usage, version=VERSION) - - for option_element in options: - argParser.add_option(option_element['short'], option_element['long'], - type=option_element['type'], action=option_element['action'], - dest=option_element['dest'], default=option_element['default'], - metavar=option_element['metavar'], help=option_element['help']) - - (parsedOptions, args) = argParser.parse_args() - - if not os.path.exists(parsedOptions.package): - argParser.error("Could not find path to hadoop binary: %s" % parsedOptions.package) - if not os.path.exists(parsedOptions.config): - argParser.error("Could not find config: %s" % parsedOptions.config) - if parsedOptions.days <= 0: - argParser.error("Invalid number of days specified, must be > 0: %s" % parsedOptions.config) - if parsedOptions.dynamicdfs!='true' and parsedOptions.dynamicdfs!='false': - argParser.error("Invalid option for dynamicdfs, must be true or false: %s" % parsedOptions.dynamicdfs) - - return parsedOptions - - -if __name__ == '__main__': - runcondense() - diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/__init__.py b/third_party/hadoop-0.20.0/contrib/hod/testing/__init__.py deleted file mode 100644 index 12c2f1e1da706ef7b66c7ec27e762b2287340d9f..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/__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/testing/helper.py b/third_party/hadoop-0.20.0/contrib/hod/testing/helper.py deleted file mode 100644 index 5645d388b772b305509b3856793e93dad0c58148..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/helper.py +++ /dev/null @@ -1,33 +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 sys - -sampleText = "Hello World!" - -if __name__=="__main__": - args = sys.argv[1:] - if args[0] == "1": - # print sample text to stderr - sys.stdout.write(sampleText) - - elif args[0] == "2": - # print sample text to stderr - sys.stderr.write(sampleText) - - # Add any other helper programs here, with different values for args[0] - pass - diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/lib.py b/third_party/hadoop-0.20.0/contrib/hod/testing/lib.py deleted file mode 100644 index 578d812cc0fe801de9fb291cb17a8343ffc6aa16..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/lib.py +++ /dev/null @@ -1,113 +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 unittest, re, sys - -class BaseTestSuite(): - def __init__(self, name, excludes): - self.name = name - self.excludes = excludes - pass - - def runTests(self): - # Create a runner - self.runner = unittest.TextTestRunner() - - # Get all the test-case classes - # From module import * - mod = __import__(self.name, fromlist=['*']) - modItemsList = dir(mod) - - allsuites = [] - - # Create all the test suites - for modItem in modItemsList: - if re.search(r"^test_", modItem): - # Yes this is a test class - if modItem not in self.excludes: - test_class = getattr(mod, modItem) - allsuites.append(unittest.makeSuite(test_class)) - - # Create a master suite to be run. - alltests = unittest.TestSuite(tuple(allsuites)) - - # Run the master test suite. - runner = self.runner.run(alltests) - if(runner.wasSuccessful()): return 0 - printLine( "%s test(s) failed." % runner.failures.__len__()) - printLine( "%s test(s) threw errors." % runner.errors.__len__()) - return runner.failures.__len__() + runner.errors.__len__() - - def cleanUp(self): - # suite tearDown - pass - -def printLine(str): - print >>sys.stderr, str - -def printSeparator(): - str = "" - for i in range(0,79): - str = str + "*" - print >>sys.stderr, "\n", str, "\n" - -# This class captures all log messages logged by hodRunner and other classes. -# It is then used to verify that certain log messages have come. This is one -# way to validate that messages printed to the logger are correctly written. -class MockLogger: - def __init__(self): - self.__logLines = {} - - def info(self, message): - self.__logLines[message] = 'info' - - def critical(self, message): - self.__logLines[message] = 'critical' - - def warn(self, message): - self.__logLines[message] = 'warn' - - def debug(self, message): - # don't track debug lines. - pass - - # verify a certain message has been logged at the defined level of severity. - def hasMessage(self, message, level): - if not self.__logLines.has_key(message): - return False - return self.__logLines[message] == level - -# Stub class to test cluster manipulation operations. -class MockHadoopCluster: - - def __init__(self): - # store the operations received. - self.__operations = {} - - def delete_job(self, jobid): - self.__operations['delete_job'] = [jobid] - - def is_cluster_deallocated(self, dummy): - return False - - def wasOperationPerformed(self, operation, args): - if self.__operations.has_key(operation): - actualArgs = self.__operations[operation] - for arg in actualArgs: - if arg not in args: - break - else: - return True - return False diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/main.py b/third_party/hadoop-0.20.0/contrib/hod/testing/main.py deleted file mode 100644 index ec4d4fdd01d31c187f04bbce42963686f6d83459..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/main.py +++ /dev/null @@ -1,83 +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 unittest, os, sys, re - -myPath = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myPath) -testingDir = os.path.join(rootDirectory, "testing") - -sys.path.append(rootDirectory) - -from testing.lib import printSeparator, printLine - -moduleList = [] -allList = [] -excludes = [ - ] - -# Build a module list by scanning through all files in testingDir -for file in os.listdir(testingDir): - if(re.search(r".py$", file) and re.search(r"^test", file)): - # All .py files with names starting in 'test' - module = re.sub(r"^test","",file) - module = re.sub(r".py$","",module) - allList.append(module) - if module not in excludes: - moduleList.append(module) - -printLine("All testcases - %s" % allList) -printLine("Excluding the testcases - %s" % excludes) -printLine("Executing the testcases - %s" % moduleList) - -testsResult = 0 -# Now import each of these modules and start calling the corresponding -#testSuite methods -for moduleBaseName in moduleList: - try: - module = "testing.test" + moduleBaseName - suiteCaller = "Run" + moduleBaseName + "Tests" - printSeparator() - printLine("Running %s" % suiteCaller) - - # Import the corresponding test cases module - imported_module = __import__(module , fromlist=[suiteCaller] ) - - # Call the corresponding suite method now - testRes = getattr(imported_module, suiteCaller)() - testsResult = testsResult + testRes - printLine("Finished %s. TestSuite Result : %s\n" % \ - (suiteCaller, testRes)) - except ImportError, i: - # Failed to import a test module - printLine(i) - testsResult = testsResult + 1 - pass - except AttributeError, n: - # Failed to get suiteCaller from a test module - printLine(n) - testsResult = testsResult + 1 - pass - except Exception, e: - # Test module suiteCaller threw some exception - printLine("%s failed. \nReason : %s" % (suiteCaller, e)) - printLine("Skipping %s" % suiteCaller) - testsResult = testsResult + 1 - pass - -if testsResult != 0: - printSeparator() - printLine("Total testcases with failure or error : %s" % testsResult) -sys.exit(testsResult) diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testHadoop.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testHadoop.py deleted file mode 100644 index b15f6803dd1f6117eaa65d3cef7b0ab9cdd696c8..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testHadoop.py +++ /dev/null @@ -1,123 +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 unittest, os, sys, re, threading, time - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -from testing.lib import BaseTestSuite - -excludes = [] - -import tempfile, getpass -from xml.dom import minidom - -from hodlib.Hod.hadoop import hadoopConfig - -# All test-case classes should have the naming convention test_.* -class test_hadoopConfig(unittest.TestCase): - def setUp(self): - self.__hadoopConfig = hadoopConfig() - self.rootDir = '/tmp/hod-%s' % getpass.getuser() - if not os.path.exists(self.rootDir): - os.mkdir(self.rootDir) - self.testingDir = tempfile.mkdtemp( dir=self.rootDir, - prefix='HadoopTestSuite.test_hadoopConfig') - self.confDir = tempfile.mkdtemp(dir=self.rootDir, - prefix='HadoopTestSuite.test_hadoopConfig') - self.tempDir = '/tmp/hod-%s/something' % getpass.getuser() - self.hadoopSite = os.path.join(self.confDir,'hadoop-site.xml') - self.numNodes = 4 - self.hdfsAddr = 'nosuchhost1.apache.org:50505' - self.mapredAddr = 'nosuchhost2.apache.org:50506' - self.finalServerParams = { - 'mapred.child.java.opts' : '-Xmx1024m', - 'mapred.compress.map.output' : 'false', - } - self.serverParams = { - 'mapred.userlog.limit' : '200', - 'mapred.userlog.retain.hours' : '10', - 'mapred.reduce.parallel.copies' : '20', - } - self.clientParams = { - 'mapred.tasktracker.tasks.maximum' : '2', - 'io.sort.factor' : '100', - 'io.sort.mb' : '200', - 'mapred.userlog.limit.kb' : '1024', - 'io.file.buffer.size' : '262144', - } - self.clusterFactor = 1.9 - self.mySysDir = '/user/' + getpass.getuser() + '/mapredsystem' - pass - - def testSuccess(self): - self.__hadoopConfig.gen_site_conf( - confDir = self.confDir,\ - tempDir = self.tempDir,\ - numNodes = self.numNodes,\ - hdfsAddr = self.hdfsAddr,\ - mrSysDir = self.mySysDir,\ - mapredAddr = self.mapredAddr,\ - clientParams = self.clientParams,\ - serverParams = self.serverParams,\ - finalServerParams = self.finalServerParams,\ - clusterFactor = self.clusterFactor - - ) - xmldoc = minidom.parse(self.hadoopSite) - xmldoc = xmldoc.childNodes[0] # leave out xml spec - properties = xmldoc.childNodes # children of tag configuration - keyvals = {} - for prop in properties: - if not isinstance(prop,minidom.Comment): - # ---------- tag -------------------- -value elem-- data -- - name = prop.getElementsByTagName('name')[0].childNodes[0].data - value = prop.getElementsByTagName('value')[0].childNodes[0].data - keyvals[name] = value - - # fs.default.name should start with hdfs:// - assert(keyvals['fs.default.name'].startswith('hdfs://')) - assert(keyvals['hadoop.tmp.dir'] == self.tempDir) - - # TODO other tests - pass - - def tearDown(self): - if os.path.exists(self.hadoopSite): os.unlink(self.hadoopSite) - if os.path.exists(self.confDir) : os.rmdir(self.confDir) - if os.path.exists(self.testingDir) : os.rmdir(self.testingDir) - pass - -class HadoopTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - pass - - def cleanUp(self): - # suite tearDown - pass - -def RunHadoopTests(): - suite = HadoopTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunHadoopTests() diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testHod.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testHod.py deleted file mode 100644 index 350cccb6e3b1568a671d448acf2df775e6e9ce6c..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testHod.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. -import unittest, getpass, os, sys, re, threading, time - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -import tempfile -from testing.lib import BaseTestSuite, MockLogger, MockHadoopCluster -from hodlib.Hod.hod import hodRunner, hodState -from hodlib.Common.desc import NodePoolDesc - -excludes = [] - -# Information about all clusters is written to a file called clusters.state. -from hodlib.Hod.hod import CLUSTER_DATA_FILE as TEST_CLUSTER_DATA_FILE, \ - INVALID_STATE_FILE_MSGS - -# Temp directory prefix -TMP_DIR_PREFIX=os.path.join('/tmp', 'hod-%s' % (getpass.getuser())) - -# build a config object with all required keys for initializing hod. -def setupConf(): - cfg = { - 'hod' : { - 'original-dir' : os.getcwd(), - 'stream' : True, - # store all the info about clusters in this directory - 'user_state' : '/tmp/hodtest', - 'debug' : 3, - 'java-home' : os.getenv('JAVA_HOME'), - 'cluster' : 'dummy', - 'cluster-factor' : 1.8, - 'xrs-port-range' : (32768,65536), - 'allocate-wait-time' : 3600, - 'temp-dir' : '/tmp/hod' - }, - # just set everything to dummy. Need something to initialize the - # node pool description object. - 'resource_manager' : { - 'id' : 'dummy', - 'batch-home' : 'dummy', - 'queue' : 'dummy', - } - } - cfg['nodepooldesc'] = NodePoolDesc(cfg['resource_manager']) - return cfg - -# Test class that defines methods to test invalid arguments to hod operations. -class test_InvalidArgsOperations(unittest.TestCase): - def setUp(self): - - self.cfg = setupConf() - # initialize the mock objects - self.log = MockLogger() - self.cluster = MockHadoopCluster() - - # Use the test logger. This will be used for test verification. - self.client = hodRunner(self.cfg, log=self.log, cluster=self.cluster) - # Create the hodState object to set the test state you want. - self.state = hodState(self.cfg['hod']['user_state']) - if not os.path.exists(self.cfg['hod']['user_state']): - os.path.mkdir(self.cfg['hod']['user_state']) - p = os.path.join(self.cfg['hod']['user_state'], '%s.state' % TEST_CLUSTER_DATA_FILE) - # ensure cluster data file exists, so write works in the tests. - f = open(p, 'w') - f.close() - - def tearDown(self): - # clean up cluster data file and directory - p = os.path.join(self.cfg['hod']['user_state'], '%s.state' % TEST_CLUSTER_DATA_FILE) - os.remove(p) - os.rmdir(self.cfg['hod']['user_state']) - - # Test that list works with deleted cluster directories - more than one entries which are invalid. - def testListInvalidDirectory(self): - userState = { os.path.join(TMP_DIR_PREFIX, 'testListInvalidDirectory1') : '123.dummy.id1', - os.path.join(TMP_DIR_PREFIX, 'testListInvalidDirectory2') : '123.dummy.id2' } - self.__setupClusterState(userState) - self.client._op_list(['list']) - # assert that required errors are logged. - for clusterDir in userState.keys(): - self.assertTrue(self.log.hasMessage('cluster state unknown\t%s\t%s' \ - % (userState[clusterDir], clusterDir), 'info')) - - # simulate a test where a directory is deleted, and created again, without deallocation - clusterDir = os.path.join(TMP_DIR_PREFIX, 'testListEmptyDirectory') - os.makedirs(clusterDir) - self.assertTrue(os.path.isdir(clusterDir)) - userState = { clusterDir : '123.dummy.id3' } - self.__setupClusterState(userState, False) - self.client._op_list(['list']) - self.assertTrue(self.log.hasMessage('cluster state unknown\t%s\t%s' \ - % (userState[clusterDir], clusterDir), 'info')) - os.rmdir(clusterDir) - - # Test that info works with a deleted cluster directory - def testInfoInvalidDirectory(self): - clusterDir = os.path.join(TMP_DIR_PREFIX, 'testInfoInvalidDirectory') - userState = { clusterDir : '456.dummy.id' } - self.__setupClusterState(userState) - self.client._op_info(['info', clusterDir]) - self.assertTrue(self.log.hasMessage("Cannot find information for cluster with id '%s' in previously allocated cluster directory '%s'." % (userState[clusterDir], clusterDir), 'critical')) - - # simulate a test where a directory is deleted, and created again, without deallocation - clusterDir = os.path.join(TMP_DIR_PREFIX, 'testInfoEmptyDirectory') - os.makedirs(clusterDir) - self.assertTrue(os.path.isdir(clusterDir)) - userState = { clusterDir : '456.dummy.id1' } - self.__setupClusterState(userState, False) - self.client._op_info(['info', clusterDir]) - self.assertTrue(self.log.hasMessage("Cannot find information for cluster with id '%s' in previously allocated cluster directory '%s'." % (userState[clusterDir], clusterDir), 'critical')) - os.rmdir(clusterDir) - - # Test info works with an invalid cluster directory - def testInfoNonExistentDirectory(self): - clusterDir = '/tmp/hod/testInfoNonExistentDirectory' - self.client._op_info(['info', clusterDir]) - self.assertTrue(self.log.hasMessage("Invalid hod.clusterdir(--hod.clusterdir or -d). %s : No such directory" % (clusterDir), 'critical')) - - # Test that deallocation works on a deleted cluster directory - # by clearing the job, and removing the state - def testDeallocateInvalidDirectory(self): - clusterDir = os.path.join(TMP_DIR_PREFIX,'testDeallocateInvalidDirectory') - jobid = '789.dummy.id' - userState = { clusterDir : jobid } - self.__setupClusterState(userState) - self.client._op_deallocate(['deallocate', clusterDir]) - # verify job was deleted - self.assertTrue(self.cluster.wasOperationPerformed('delete_job', jobid)) - # verify appropriate message was logged. - self.assertTrue(self.log.hasMessage("Cannot find information for cluster with id '%s' in previously allocated cluster directory '%s'." % (userState[clusterDir], clusterDir), 'critical')) - self.assertTrue(self.log.hasMessage("Freeing resources allocated to the cluster.", 'critical')) - # verify that the state information was cleared. - userState = self.state.read(TEST_CLUSTER_DATA_FILE) - self.assertFalse(clusterDir in userState.keys()) - - # simulate a test where a directory is deleted, and created again, without deallocation - clusterDir = os.path.join(TMP_DIR_PREFIX,'testDeallocateEmptyDirectory') - os.makedirs(clusterDir) - self.assertTrue(os.path.isdir(clusterDir)) - jobid = '789.dummy.id1' - userState = { clusterDir : jobid } - self.__setupClusterState(userState, False) - self.client._op_deallocate(['deallocate', clusterDir]) - # verify job was deleted - self.assertTrue(self.cluster.wasOperationPerformed('delete_job', jobid)) - # verify appropriate message was logged. - self.assertTrue(self.log.hasMessage("Cannot find information for cluster with id '%s' in previously allocated cluster directory '%s'." % (userState[clusterDir], clusterDir), 'critical')) - self.assertTrue(self.log.hasMessage("Freeing resources allocated to the cluster.", 'critical')) - # verify that the state information was cleared. - userState = self.state.read(TEST_CLUSTER_DATA_FILE) - self.assertFalse(clusterDir in userState.keys()) - os.rmdir(clusterDir) - - # Test that deallocation works on a nonexistent directory. - def testDeallocateNonExistentDirectory(self): - clusterDir = os.path.join(TMP_DIR_PREFIX,'testDeallocateNonExistentDirectory') - self.client._op_deallocate(['deallocate', clusterDir]) - # there should be no call.. - self.assertFalse(self.cluster.wasOperationPerformed('delete_job', None)) - self.assertTrue(self.log.hasMessage("Invalid hod.clusterdir(--hod.clusterdir or -d). %s : No such directory" % (clusterDir), 'critical')) - - # Test that allocation on an previously deleted directory fails. - def testAllocateOnDeletedDirectory(self): - clusterDir = os.path.join(TMP_DIR_PREFIX, 'testAllocateOnDeletedDirectory') - os.makedirs(clusterDir) - self.assertTrue(os.path.isdir(clusterDir)) - jobid = '1234.abc.com' - userState = { clusterDir : jobid } - self.__setupClusterState(userState, False) - self.client._op_allocate(['allocate', clusterDir, '3']) - self.assertTrue(self.log.hasMessage("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), 'critical')) - os.rmdir(clusterDir) - - def __setupClusterState(self, clusterStateMap, verifyDirIsAbsent=True): - for clusterDir in clusterStateMap.keys(): - # ensure directory doesn't exist, just in case. - if verifyDirIsAbsent: - self.assertFalse(os.path.exists(clusterDir)) - # set up required state. - self.state.write(TEST_CLUSTER_DATA_FILE, clusterStateMap) - # verify everything is stored correctly. - state = self.state.read(TEST_CLUSTER_DATA_FILE) - for clusterDir in clusterStateMap.keys(): - self.assertTrue(clusterDir in state.keys()) - self.assertEquals(clusterStateMap[clusterDir], state[clusterDir]) - -class test_InvalidHodStateFiles(unittest.TestCase): - def setUp(self): - self.rootDir = '/tmp/hod-%s' % getpass.getuser() - self.cfg = setupConf() # creat a conf - # Modify hod.user_state - self.cfg['hod']['user_state'] = tempfile.mkdtemp(dir=self.rootDir, - prefix='HodTestSuite.test_InvalidHodStateFiles_') - self.log = MockLogger() # mock logger - self.cluster = MockHadoopCluster() # mock hadoop cluster - self.client = hodRunner(self.cfg, log=self.log, cluster=self.cluster) - self.state = hodState(self.cfg['hod']['user_state']) - self.statePath = os.path.join(self.cfg['hod']['user_state'], '%s.state' % \ - TEST_CLUSTER_DATA_FILE) - self.clusterDir = tempfile.mkdtemp(dir=self.rootDir, - prefix='HodTestSuite.test_InvalidHodStateFiles_') - - def testOperationWithInvalidStateFile(self): - jobid = '1234.hadoop.apache.org' - # create user state file with invalid permissions - stateFile = open(self.statePath, "w") - os.chmod(self.statePath, 000) # has no read/write permissions - self.client._hodRunner__cfg['hod']['operation'] = \ - "info %s" % self.clusterDir - ret = self.client.operation() - os.chmod(self.statePath, 700) # restore permissions - stateFile.close() - os.remove(self.statePath) - - # print self.log._MockLogger__logLines - self.assertTrue(self.log.hasMessage(INVALID_STATE_FILE_MSGS[0] % \ - os.path.realpath(self.statePath), 'critical')) - self.assertEquals(ret, 1) - - def testAllocateWithInvalidStateFile(self): - jobid = '1234.hadoop.apache.org' - # create user state file with invalid permissions - stateFile = open(self.statePath, "w") - os.chmod(self.statePath, 0400) # has no write permissions - self.client._hodRunner__cfg['hod']['operation'] = \ - "allocate %s %s" % (self.clusterDir, '3') - ret = self.client.operation() - os.chmod(self.statePath, 700) # restore permissions - stateFile.close() - os.remove(self.statePath) - - # print self.log._MockLogger__logLines - self.assertTrue(self.log.hasMessage(INVALID_STATE_FILE_MSGS[2] % \ - os.path.realpath(self.statePath), 'critical')) - self.assertEquals(ret, 1) - - def testAllocateWithInvalidStateStore(self): - jobid = '1234.hadoop.apache.org' - self.client._hodRunner__cfg['hod']['operation'] = \ - "allocate %s %s" % (self.clusterDir, 3) - - ###### check with no executable permissions ###### - stateFile = open(self.statePath, "w") # create user state file - os.chmod(self.cfg['hod']['user_state'], 0600) - ret = self.client.operation() - os.chmod(self.cfg['hod']['user_state'], 0700) # restore permissions - stateFile.close() - os.remove(self.statePath) - # print self.log._MockLogger__logLines - self.assertTrue(self.log.hasMessage(INVALID_STATE_FILE_MSGS[0] % \ - os.path.realpath(self.statePath), 'critical')) - self.assertEquals(ret, 1) - - ###### check with no write permissions ###### - stateFile = open(self.statePath, "w") # create user state file - os.chmod(self.cfg['hod']['user_state'], 0500) - ret = self.client.operation() - os.chmod(self.cfg['hod']['user_state'], 0700) # restore permissions - stateFile.close() - os.remove(self.statePath) - # print self.log._MockLogger__logLines - self.assertTrue(self.log.hasMessage(INVALID_STATE_FILE_MSGS[0] % \ - os.path.realpath(self.statePath), 'critical')) - self.assertEquals(ret, 1) - - def tearDown(self): - if os.path.exists(self.clusterDir): os.rmdir(self.clusterDir) - if os.path.exists(self.cfg['hod']['user_state']): - os.rmdir(self.cfg['hod']['user_state']) - - -class HodTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - pass - - def cleanUp(self): - # suite tearDown - pass - -def RunHodTests(): - # modulename_suite - suite = HodTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunHodTests() diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testHodCleanup.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testHodCleanup.py deleted file mode 100644 index 93e18333fd24f034c096fd0fc1517cdd411709aa..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testHodCleanup.py +++ /dev/null @@ -1,113 +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 unittest, os, sys, re, threading, time - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -from testing.lib import BaseTestSuite -from hodlib.HodRing.hodRing import MRSystemDirectoryManager, createMRSystemDirectoryManager -from hodlib.Common.threads import simpleCommand - -excludes = [] - -# duplicating temporarily until HADOOP-2848 is committed. -class MyMockLogger: - def __init__(self): - self.__logLines = {} - - def info(self, message): - self.__logLines[message] = 'info' - - def critical(self, message): - self.__logLines[message] = 'critical' - - def warn(self, message): - self.__logLines[message] = 'warn' - - def debug(self, message): - # don't track debug lines. - pass - - # verify a certain message has been logged at the defined level of severity. - def hasMessage(self, message, level): - if not self.__logLines.has_key(message): - return False - return self.__logLines[message] == level - -class test_MRSystemDirectoryManager(unittest.TestCase): - - def setUp(self): - self.log = MyMockLogger() - - def testCleanupArgsString(self): - sysDirMgr = MRSystemDirectoryManager(1234, '/user/hod/mapredsystem/hoduser.123.abc.com', \ - 'def.com:5678', '/usr/bin/hadoop', self.log) - str = sysDirMgr.toCleanupArgs() - self.assertTrue(" --jt-pid 1234 --mr-sys-dir /user/hod/mapredsystem/hoduser.123.abc.com --fs-name def.com:5678 --hadoop-path /usr/bin/hadoop ", str) - - def testCreateMRSysDirInvalidParams(self): - # test that no mr system directory manager is created if required keys are not present - # this case will test scenarios of non jobtracker daemons. - keys = [ 'jt-pid', 'mr-sys-dir', 'fs-name', 'hadoop-path' ] - map = { 'jt-pid' : 1234, - 'mr-sys-dir' : '/user/hod/mapredsystem/hoduser.def.com', - 'fs-name' : 'ghi.com:1234', - 'hadoop-path' : '/usr/bin/hadoop' - } - for key in keys: - val = map[key] - map[key] = None - self.assertEquals(createMRSystemDirectoryManager(map, self.log), None) - map[key] = val - - def testUnresponsiveJobTracker(self): - # simulate an unresponsive job tracker, by giving a command that runs longer than the retries - # verify that the program returns with the right error message. - sc = simpleCommand("sleep", "sleep 300") - sc.start() - pid = sc.getPid() - while pid is None: - pid = sc.getPid() - sysDirMgr = MRSystemDirectoryManager(pid, '/user/yhemanth/mapredsystem/hoduser.123.abc.com', \ - 'def.com:5678', '/usr/bin/hadoop', self.log, retries=3) - sysDirMgr.removeMRSystemDirectory() - self.log.hasMessage("Job Tracker did not exit even after a minute. Not going to try and cleanup the system directory", 'warn') - sc.kill() - sc.wait() - sc.join() - -class HodCleanupTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - pass - - def cleanUp(self): - # suite tearDown - pass - -def RunHodCleanupTests(): - # modulename_suite - suite = HodCleanupTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunHodCleanupTests() diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testHodRing.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testHodRing.py deleted file mode 100644 index 609c19908ae116dae7702ac7490da25d2a3d41a0..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testHodRing.py +++ /dev/null @@ -1,117 +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 unittest, os, sys, re, threading, time - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -from testing.lib import BaseTestSuite - -excludes = [] - -import tempfile, getpass, logging -from xml.dom import minidom - -from hodlib.Hod.hadoop import hadoopConfig -from hodlib.HodRing.hodRing import CommandDesc, HadoopCommand - -# All test-case classes should have the naming convention test_.* -class test_HadoopCommand(unittest.TestCase): - def setUp(self): - self.rootDir = '/tmp/hod-%s' % getpass.getuser() - self.id = 0 - self.desc = None - self.tempDir = os.path.join(self.rootDir,'test_HadoopCommand_tempDir') - self.pkgDir = os.path.join(self.rootDir,'test_HadoopCommand_pkgDir') - self.log = logging.getLogger() # TODO Use MockLogger - self.javaHome = '/usr/java/bin/' - self.mrSysDir = '/user/' + getpass.getuser() + '/mapredsystem' - - self.attrs = {} - self.finalAttrs = { - 'fs.default.name': 'nohost.apache.com:56366', - 'mapred.child.java.opts' : '-Xmx1024m', - 'mapred.compress.map.output' : 'false', - } - self.attrs = { - 'mapred.userlog.limit' : '200', - 'mapred.userlog.retain.hours' : '10', - 'mapred.reduce.parallel.copies' : '20', - } - self.desc = CommandDesc( - { - 'name' : 'dummyHadoop', - 'program' : 'bin/hadoop', - 'pkgdirs' : self.pkgDir, - 'final-attrs' : self.finalAttrs, - 'attrs' : self.attrs, - }, self.log - ) - # TODO - # 4th arg to HadoopCommand 'tardir' is not used at all. Instead pkgdir is - # specified through HadoopCommand.run(pkgdir). This could be changed so - # that pkgdir is specified at the time of object creation. - # END OF TODO - self.hadoopCommand = HadoopCommand(self.id, self.desc, self.tempDir, - self.pkgDir, self.log, self.javaHome, - self.mrSysDir, restart=True) - self.hadoopSite = os.path.join( self.hadoopCommand.confdir, - 'hadoop-site.xml') - pass - - def test_createHadoopSiteXml(self): - self.hadoopCommand._createHadoopSiteXml() - xmldoc = minidom.parse(self.hadoopSite) - xmldoc = xmldoc.childNodes[0] # leave out xml spec - properties = xmldoc.childNodes # children of tag configuration - keyvals = {} - for prop in properties: - if not isinstance(prop,minidom.Comment): - # ---------- tag -------------------- -value elem-- data -- - name = prop.getElementsByTagName('name')[0].childNodes[0].data - value = prop.getElementsByTagName('value')[0].childNodes[0].data - keyvals[name] = value - - # fs.default.name should start with hdfs:// - assert(keyvals['fs.default.name'].startswith('hdfs://')) - - # TODO other tests - pass - - def tearDown(self): - pass - -class HodRingTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - pass - - def cleanUp(self): - # suite tearDown - pass - -def RunHodRingTests(): - # modulename_suite - suite = HodRingTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunHodRingTests() diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testModule.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testModule.py deleted file mode 100644 index a09fd04709ee30262aa31535e4495820fd6124b3..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testModule.py +++ /dev/null @@ -1,88 +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 unittest, os, sys, re, threading, time - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -from testing.lib import BaseTestSuite - -excludes = ['test_MINITEST3'] - -# All test-case classes should have the naming convention test_.* -class test_MINITEST1(unittest.TestCase): - def setUp(self): - pass - - # All testMethods have to have their names start with 'test' - def testSuccess(self): - pass - - def testFailure(self): - pass - - def tearDown(self): - pass - -class test_MINITEST2(unittest.TestCase): - def setUp(self): - pass - - # All testMethods have to have their names start with 'test' - def testSuccess(self): - pass - - def testFailure(self): - pass - - def tearDown(self): - pass - -class test_MINITEST3(unittest.TestCase): - def setUp(self): - pass - - # All testMethods have to have their names start with 'test' - def testSuccess(self): - pass - - def testFailure(self): - pass - - def tearDown(self): - pass - -class ModuleTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - pass - - def cleanUp(self): - # suite tearDown - pass - -def RunModuleTests(): - # modulename_suite - suite = ModuleTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunModuleTests() diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testRingmasterRPCs.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testRingmasterRPCs.py deleted file mode 100644 index 5a02e066b0841dfe9dee219238e3836f0ccb898e..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testRingmasterRPCs.py +++ /dev/null @@ -1,171 +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 unittest, os, sys, re, threading, time - -import logging - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -from testing.lib import BaseTestSuite - -excludes = ['test_MINITEST1', 'test_MINITEST2'] - -from hodlib.GridServices import * -from hodlib.Common.desc import ServiceDesc -from hodlib.RingMaster.ringMaster import _LogMasterSources - -configuration = { - 'hod': {}, - 'resource_manager': { - 'id': 'torque', - 'batch-home': '/home/y/' - }, - 'ringmaster': { - 'max-connect' : 2, - 'max-master-failures' : 5 - }, - 'hodring': { - }, - 'gridservice-mapred': { - 'id': 'mapred' - } , - 'gridservice-hdfs': { - 'id': 'hdfs' - }, - 'servicedesc' : {} , - 'nodepooldesc': {} , - } - -# All test-case classes should have the naming convention test_.* -class test_MINITEST1(unittest.TestCase): - def setUp(self): - pass - - # All testMethods have to have their names start with 'test' - def testSuccess(self): - pass - - def testFailure(self): - pass - - def tearDown(self): - pass - -class test_Multiple_Workers(unittest.TestCase): - def setUp(self): - self.config = configuration - self.config['ringmaster']['workers_per_ring'] = 2 - - hdfsDesc = self.config['servicedesc']['hdfs'] = ServiceDesc(self.config['gridservice-hdfs']) - mrDesc = self.config['servicedesc']['mapred'] = ServiceDesc(self.config['gridservice-mapred']) - - self.hdfs = Hdfs(hdfsDesc, [], 0, 19, workers_per_ring = \ - self.config['ringmaster']['workers_per_ring']) - self.mr = MapReduce(mrDesc, [],1, 19, workers_per_ring = \ - self.config['ringmaster']['workers_per_ring']) - - self.log = logging.getLogger() - pass - - # All testMethods have to have their names start with 'test' - def testWorkersCount(self): - self.serviceDict = {} - self.serviceDict[self.hdfs.getName()] = self.hdfs - self.serviceDict[self.mr.getName()] = self.mr - self.rpcSet = _LogMasterSources(self.serviceDict, self.config, None, self.log, None) - - cmdList = self.rpcSet.getCommand('host1') - self.assertEquals(len(cmdList), 2) - self.assertEquals(cmdList[0].dict['argv'][0], 'namenode') - self.assertEquals(cmdList[1].dict['argv'][0], 'namenode') - addParams = ['fs.default.name=host1:51234', 'dfs.http.address=host1:5125' ] - self.rpcSet.addMasterParams('host1', addParams) - # print "NN is launched" - - cmdList = self.rpcSet.getCommand('host2') - self.assertEquals(len(cmdList), 1) - self.assertEquals(cmdList[0].dict['argv'][0], 'jobtracker') - addParams = ['mapred.job.tracker=host2:51236', - 'mapred.job.tracker.http.address=host2:51237'] - self.rpcSet.addMasterParams('host2', addParams) - # print "JT is launched" - - cmdList = self.rpcSet.getCommand('host3') - # Verify the workers count per ring : TTs + DNs - self.assertEquals(len(cmdList), - self.config['ringmaster']['workers_per_ring'] * 2) - pass - - def testFailure(self): - pass - - def tearDown(self): - pass - -class test_GetCommand(unittest.TestCase): - def setUp(self): - self.config = configuration - - hdfsDesc = self.config['servicedesc']['hdfs'] = ServiceDesc(self.config['gridservice-hdfs']) - mrDesc = self.config['servicedesc']['mapred'] = ServiceDesc(self.config['gridservice-mapred']) - - # API : serviceObj = service(desc, workDirs, reqNodes, version) - self.hdfs = Hdfs(hdfsDesc, [], 0, 17) - self.hdfsExternal = HdfsExternal(hdfsDesc, [], 17) - self.mr = MapReduce(mrDesc, [],1, 17) - self.mrExternal = MapReduceExternal(mrDesc, [], 17) - - self.log = logging.getLogger() - pass - - # All testMethods have to have their names start with 'test' - def testBothInternal(self): - self.serviceDict = {} - self.serviceDict[self.hdfs.getName()] = self.hdfs - self.serviceDict[self.mr.getName()] = self.mr - self.rpcSet = _LogMasterSources(self.serviceDict, self.config, None, self.log, None) - - cmdList = self.rpcSet.getCommand('localhost') - self.assertEquals(cmdList.__len__(), 2) - self.assertEquals(cmdList[0].dict['argv'][0], 'namenode') - self.assertEquals(cmdList[1].dict['argv'][0], 'namenode') - pass - - def tearDown(self): - pass - -class RingmasterRPCsTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - pass - - def cleanUp(self): - # suite tearDown - pass - -def RunRingmasterRPCsTests(): - # modulename_suite - suite = RingmasterRPCsTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunRingmasterRPCsTests() diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testThreads.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testThreads.py deleted file mode 100644 index 22753cfe9056c0b2fd60cd788a6c43534d37d837..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testThreads.py +++ /dev/null @@ -1,99 +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 unittest, os, sys, re, threading, time - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -from testing.lib import BaseTestSuite - -# module specific imports -import os, tempfile, random - -excludes = [] - -import getpass -from hodlib.Common.threads import simpleCommand -from testing.helper import sampleText - -# All test-case classes should have the naming convention test_.* -class test_SimpleCommand(unittest.TestCase): - def setUp(self): - self.rootDir = '/tmp/hod-%s' % getpass.getuser() - if not os.path.exists(self.rootDir): - os.mkdir(self.rootDir) - self.prefix= 'ThreadsTestSuite.test_SimpleCommand' - self.testFile = None - pass - - def testRedirectedStdout(self): - self.testFile= tempfile.NamedTemporaryFile(dir=self.rootDir, \ - prefix=self.prefix) - cmd=simpleCommand('helper','%s %s 1 1>%s' % \ - (sys.executable, \ - os.path.join(rootDirectory, "testing", "helper.py"), \ - self.testFile.name)) - - cmd.start() - cmd.join() - - self.testFile.seek(0) - stdout = self.testFile.read() - # print stdout, sampleText - assert(stdout == sampleText) - pass - - def testRedirectedStderr(self): - self.testFile= tempfile.NamedTemporaryFile(dir=self.rootDir, \ - prefix=self.prefix) - cmd=simpleCommand('helper','%s %s 2 2>%s' % \ - (sys.executable, \ - os.path.join(rootDirectory, "testing", "helper.py"), \ - self.testFile.name)) - cmd.start() - cmd.join() - - self.testFile.seek(0) - stderror = self.testFile.read() - # print stderror, sampleText - assert(stderror == sampleText) - pass - - def tearDown(self): - if self.testFile: self.testFile.close() - pass - -class ThreadsTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - pass - - def cleanUp(self): - # suite tearDown - pass - -def RunThreadsTests(): - # modulename_suite - suite = ThreadsTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunThreadsTests() diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testTypes.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testTypes.py deleted file mode 100644 index 7e23dca3bee426e0be5e2b5902ec9566069063ea..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testTypes.py +++ /dev/null @@ -1,180 +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 unittest, os, sys, re, threading, time - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -from testing.lib import BaseTestSuite - -excludes = [''] - -import tempfile, shutil, getpass, random -from hodlib.Common.types import typeValidator - -# All test-case classes should have the naming convention test_.* -class test_typeValidator(unittest.TestCase): - def setUp(self): - self.originalDir = os.getcwd() - self.validator = typeValidator(self.originalDir) - self.tempDir = tempfile.mkdtemp(dir='/tmp/hod-%s' % getpass.getuser(), - prefix='test_Types_typeValidator_tempDir') - self.tempFile = tempfile.NamedTemporaryFile(dir=self.tempDir) - - # verification : error strings - self.errorStringsForVerify = { - 'pos_int' : 0, - 'uri' : '%s is an invalid uri', - 'directory' : 0, - 'file' : 0, - } - - # verification : valid vals - self.verifyValidVals = [ - ('pos_int', 0), - ('pos_int', 1), - ('directory', self.tempDir), - ('directory', '/tmp/hod-%s/../../%s' % \ - (getpass.getuser(), self.tempDir)), - ('file', self.tempFile.name), - ('file', '/tmp/hod-%s/../../%s' % \ - (getpass.getuser(), self.tempFile.name)), - ('uri', 'file://localhost/' + self.tempDir), - ('uri', 'file:///' + self.tempDir), - ('uri', 'file:///tmp/hod-%s/../../%s' % \ - (getpass.getuser(), self.tempDir)), - ('uri', 'file://localhost/tmp/hod-%s/../../%s' % \ - (getpass.getuser(), self.tempDir)), - ('uri', 'http://hadoop.apache.org/core/'), - ('uri', self.tempDir), - ('uri', '/tmp/hod-%s/../../%s' % \ - (getpass.getuser(), self.tempDir)), - ] - - # generate an invalid uri - randomNum = random.random() - while os.path.exists('/%s' % randomNum): - # Just to be sure :) - randomNum = random.random() - invalidUri = 'file://localhost/%s' % randomNum - - # verification : invalid vals - self.verifyInvalidVals = [ - ('pos_int', -1), - ('uri', invalidUri), - ('directory', self.tempFile.name), - ('file', self.tempDir), - ] - - # normalization : vals - self.normalizeVals = [ - ('pos_int', 1, 1), - ('pos_int', '1', 1), - ('directory', self.tempDir, self.tempDir), - ('directory', '/tmp/hod-%s/../../%s' % \ - (getpass.getuser(), self.tempDir), - self.tempDir), - ('file', self.tempFile.name, self.tempFile.name), - ('file', '/tmp/hod-%s/../../%s' % \ - (getpass.getuser(), self.tempFile.name), - self.tempFile.name), - ('uri', 'file://localhost' + self.tempDir, - 'file://' + self.tempDir), - ('uri', 'file://127.0.0.1' + self.tempDir, - 'file://' + self.tempDir), - ('uri', 'http://hadoop.apache.org/core', - 'http://hadoop.apache.org/core'), - ('uri', self.tempDir, self.tempDir), - ('uri', '/tmp/hod-%s/../../%s' % \ - (getpass.getuser(), self.tempDir), - self.tempDir), - ] - pass - - # All testMethods have to have their names start with 'test' - def testnormalize(self): - for (type, originalVal, normalizedVal) in self.normalizeVals: - # print type, originalVal, normalizedVal,\ - # self.validator.normalize(type, originalVal) - assert(self.validator.normalize(type, originalVal) == normalizedVal) - pass - - def test__normalize(self): - # Special test for functionality of private method __normalizedPath - tmpdir = tempfile.mkdtemp(dir=self.originalDir) #create in self.originalDir - oldWd = os.getcwd() - os.chdir('/') - tmpdirName = re.sub(".*/","",tmpdir) - # print re.sub(".*/","",tmpdirName) - # print os.path.join(self.originalDir,tmpdir) - (type, originalVal, normalizedVal) = \ - ('file', tmpdirName, \ - os.path.join(self.originalDir,tmpdirName)) - assert(self.validator.normalize(type, originalVal) == normalizedVal) - os.chdir(oldWd) - os.rmdir(tmpdir) - pass - - def testverify(self): - # test verify method - - # test valid vals - for (type,value) in self.verifyValidVals: - valueInfo = { 'isValid' : 0, 'normalized' : 0, 'errorData' : 0 } - valueInfo = self.validator.verify(type,value) - # print type, value, valueInfo - assert(valueInfo['isValid'] == 1) - - # test invalid vals - for (type,value) in self.verifyInvalidVals: - valueInfo = { 'isValid' : 0, 'normalized' : 0, 'errorData' : 0 } - valueInfo = self.validator.verify(type,value) - # print type, value, valueInfo - assert(valueInfo['isValid'] == 0) - if valueInfo['errorData'] != 0: - # if there is any errorData, check - assert(valueInfo['errorData'] == \ - self.errorStringsForVerify[type] % value) - - pass - - def tearDown(self): - self.tempFile.close() - if os.path.exists(self.tempDir): - shutil.rmtree(self.tempDir) - pass - -class TypesTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - pass - - def cleanUp(self): - # suite tearDown - pass - -def RunTypesTests(): - # modulename_suite - suite = TypesTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunTypesTests() diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testUtil.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testUtil.py deleted file mode 100644 index 62003c99dfe0b674630dc9ea18195f776378b41e..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testUtil.py +++ /dev/null @@ -1,62 +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 unittest, os, sys, re, threading, time - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -from testing.lib import BaseTestSuite -from hodlib.Common.util import * -from hodlib.Common.threads import simpleCommand - -excludes = [] - -class test_Util(unittest.TestCase): - - def testProcessStatus(self): - sc = simpleCommand('testsleep', 'sleep 60') - sc.start() - pid = sc.getPid() - while pid is None: - pid = sc.getPid() - self.assertTrue(isProcessRunning(pid)) - sc.kill() - sc.wait() - sc.join() - self.assertFalse(isProcessRunning(pid)) - - -class UtilTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - pass - - def cleanUp(self): - # suite tearDown - pass - -def RunUtilTests(): - # modulename_suite - suite = UtilTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunUtilTests() diff --git a/third_party/hadoop-0.20.0/contrib/hod/testing/testXmlrpc.py b/third_party/hadoop-0.20.0/contrib/hod/testing/testXmlrpc.py deleted file mode 100644 index f630032dfaa7f627f451d24c6492c0a57f7bf47f..0000000000000000000000000000000000000000 --- a/third_party/hadoop-0.20.0/contrib/hod/testing/testXmlrpc.py +++ /dev/null @@ -1,109 +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 unittest, os, sys, re, threading, time - -myDirectory = os.path.realpath(sys.argv[0]) -rootDirectory = re.sub("/testing/.*", "", myDirectory) - -sys.path.append(rootDirectory) - -from hodlib.Common.xmlrpc import hodXRClient -from hodlib.Common.socketServers import hodXMLRPCServer -from hodlib.GridServices.service import ServiceUtil -from hodlib.Common.util import hodInterrupt, HodInterruptException - -from testing.lib import BaseTestSuite - -excludes = [] - -global serverPort -serverPort = None - -class test_HodXRClient(unittest.TestCase): - def setUp(self): - pass - - # All testMethods have to have their names start with 'test' - def testSuccess(self): - global serverPort - client = hodXRClient('http://localhost:' + str(serverPort), retryRequests=False) - self.assertEqual(client.testing(), True) - pass - - def testFailure(self): - """HOD should raise Exception when unregistered rpc is called""" - global serverPort - client = hodXRClient('http://localhost:' + str(serverPort), retryRequests=False) - self.assertRaises(Exception, client.noMethod) - pass - - def testTimeout(self): - """HOD should raise Exception when rpc call times out""" - # Give client some random nonexistent url - serverPort = ServiceUtil.getUniqRandomPort(h='localhost',low=40000,high=50000) - client = hodXRClient('http://localhost:' + str(serverPort), retryRequests=False) - self.assertRaises(Exception, client.testing) - pass - - def testInterrupt(self): - """ HOD should raise HodInterruptException when interrupted""" - - def interrupt(testClass): - testClass.assertRaises(HodInterruptException, client.testing) - - serverPort = ServiceUtil.getUniqRandomPort(h='localhost',low=40000,high=50000) - client = hodXRClient('http://localhost:' + str(serverPort)) - myThread = threading.Thread(name='testinterrupt', target=interrupt,args=(self,)) - # Set the global interrupt - hodInterrupt.setFlag() - myThread.start() - myThread.join() - pass - - def tearDown(self): - pass - -class XmlrpcTestSuite(BaseTestSuite): - def __init__(self): - # suite setup - BaseTestSuite.__init__(self, __name__, excludes) - - def rpcCall(): - return True - - global serverPort - serverPort = ServiceUtil.getUniqRandomPort(h='localhost',low=40000,high=50000) - self.server = hodXMLRPCServer('localhost', [serverPort]) - self.server.register_function(rpcCall, 'testing') - self.thread = threading.Thread(name="server", - target=self.server._serve_forever) - self.thread.start() - time.sleep(1) # give some time to start server - - def cleanUp(self): - # suite tearDown - self.server.stop() - self.thread.join() - -def RunXmlrpcTests(): - # modulename_suite - suite = XmlrpcTestSuite() - testResult = suite.runTests() - suite.cleanUp() - return testResult - -if __name__ == "__main__": - RunXmlrpcTests() diff --git a/third_party/hadoop-0.20.0/lib/slf4j-api-1.4.3.jar b/third_party/hadoop-0.20.0/lib/slf4j-api-1.4.3.jar deleted file mode 100644 index b34fe8d82de073b1c14316c3228bb221fa623ae2..0000000000000000000000000000000000000000 Binary files a/third_party/hadoop-0.20.0/lib/slf4j-api-1.4.3.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/slf4j-log4j12-1.4.3.jar b/third_party/hadoop-0.20.0/lib/slf4j-log4j12-1.4.3.jar deleted file mode 100644 index 70082fc382bed73c3eb699b988e2ea1de0c95aac..0000000000000000000000000000000000000000 Binary files a/third_party/hadoop-0.20.0/lib/slf4j-log4j12-1.4.3.jar and /dev/null differ diff --git a/third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar b/third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar new file mode 100644 index 0000000000000000000000000000000000000000..d9ef50be6d1c2f767a5cb9491a983e1100667365 Binary files /dev/null and b/third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar differ diff --git a/third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar b/third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar new file mode 100644 index 0000000000000000000000000000000000000000..fb5249346847105c26d30da1048d8e2c364e7d6f Binary files /dev/null and b/third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar differ diff --git a/third_party/mesos.jar b/third_party/mesos.jar index 60d299c8af5464177ae3a0b2c9d2711c33013bda..1852cf8fd090dd910c21934a3fee38b4c97626c0 100644 Binary files a/third_party/mesos.jar and b/third_party/mesos.jar differ diff --git a/third_party/scalacheck_2.8.0.RC3-1.7.jar b/third_party/scalacheck_2.8.0-1.7.jar similarity index 59% rename from third_party/scalacheck_2.8.0.RC3-1.7.jar rename to third_party/scalacheck_2.8.0-1.7.jar index ac9687fc00b6f9c0e11add4eb8b66ddf891acf2f..fb3c0e9e120c10f5053db27d414c73e967576748 100644 Binary files a/third_party/scalacheck_2.8.0.RC3-1.7.jar and b/third_party/scalacheck_2.8.0-1.7.jar differ diff --git a/third_party/scalatest-1.2/LICENSE b/third_party/scalatest-1.2/LICENSE new file mode 100644 index 0000000000000000000000000000000000000000..d645695673349e3947e8e5ae42332d0ac3164cd7 --- /dev/null +++ b/third_party/scalatest-1.2/LICENSE @@ -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/scalatest-1.2/NOTICE b/third_party/scalatest-1.2/NOTICE new file mode 100644 index 0000000000000000000000000000000000000000..a405cbd58a2526695372d905666e04387e97c46c --- /dev/null +++ b/third_party/scalatest-1.2/NOTICE @@ -0,0 +1,7 @@ +================================================================================ +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the ScalaTest distribution. == +================================================================================ + + - This product includes software developed by + Artima, Inc. (http://www.artima.com/). diff --git a/third_party/scalatest-1.2/README.txt b/third_party/scalatest-1.2/README.txt new file mode 100644 index 0000000000000000000000000000000000000000..d505b9c640e38a80c1b98ce77cf0f367ee8d0287 --- /dev/null +++ b/third_party/scalatest-1.2/README.txt @@ -0,0 +1,58 @@ +ScalaTest 1.0 + +ScalaTest is a free, open-source testing toolkit for Scala and +Java programmers. Because different developers take different approaches to creating +software, no single approach to testing is a good fit for everyone. In light of +this reality, ScalaTest is designed to facilitate different styles of testing. ScalaTest +provides several traits that you can mix together into whatever combination makes you feel the most productive. +For some examples of the various styles that ScalaTest supports, see: + +http://www.artima.com/scalatest + +GETTING STARTED + +To learn how to use ScalaTest, please +open in your browser the scaladoc documentation in the +/scalatest-1.0/doc directory. Look first at the documentation for trait +org.scalatest.Suite, which gives a decent intro. All the other types are +documented as well, so you can hop around to learn more. +org.scalatest.tools.Runner explains how to use the application. The +Ignore class is written in Java, and isn't currently shown in the Scaladoc. + +To try it out, you can use ScalaTest to run its own tests, i.e., the tests +used to test ScalaTest itself. This command will run the GUI: + +scala -classpath scalatest-1.0.jar org.scalatest.tools.Runner -p "scalatest-1.0-tests.jar" -g -s org.scalatest.SuiteSuite + +This command will run and just print results to the standard output: + +scala -classpath scalatest-1.0.jar org.scalatest.tools.Runner -p "scalatest-1.0-tests.jar" -o -s org.scalatest.SuiteSuite + +ScalaTest 1.0 was tested with Scala version 2.7.5.final, so it is not +guaranteed to work with earlier Scala versions. + +ABOUT SCALATEST + +ScalaTest was written by Bill Venners, George Berger, Josh Cough, and +other contributors starting in late 2007. ScalaTest, which is almost +exclusively written in Scala, follows and improves upon the Java code +and design of Artima SuiteRunner, a testing tool also written +primarily by Bill Venners, starting in 2001. Over the years a few +other people contributed to SuiteRunner as well, including: + +Mark Brouwer +Chua Chee Seng +Chris Daily +Matt Gerrans +John Mitchel +Frank Sommers + +Several people have helped with ScalaTest, including: + +Corey Haines +Colin Howe +Dianne Marsh +Joel Neely +Jon-Anders Teigen +Daniel Watson + diff --git a/third_party/scalatest-1.2-for-scala-2.8.0.RC3-SNAPSHOT.jar b/third_party/scalatest-1.2/scalatest-1.2.jar similarity index 78% rename from third_party/scalatest-1.2-for-scala-2.8.0.RC3-SNAPSHOT.jar rename to third_party/scalatest-1.2/scalatest-1.2.jar index f5c4af19b039c64193e79ec2ab8929584424f208..cb8db9bdf52a5f2f2c9c8af8b5768299233a23e7 100644 Binary files a/third_party/scalatest-1.2-for-scala-2.8.0.RC3-SNAPSHOT.jar and b/third_party/scalatest-1.2/scalatest-1.2.jar differ diff --git a/third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar b/third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar new file mode 100644 index 0000000000000000000000000000000000000000..42e0ad0de7773da9b94b12f503deda7f5a506015 Binary files /dev/null and b/third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar differ diff --git a/third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar b/third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar new file mode 100644 index 0000000000000000000000000000000000000000..873d11983e18b71aeafbc3d805495d6842339812 Binary files /dev/null and b/third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar differ