You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2014/01/03 20:24:52 UTC

[01/21] git commit: added spark-class and spark-executor to sbin

Updated Branches:
  refs/heads/master 30b9db0ab -> 4ae101ff3


added spark-class and spark-executor to sbin

Signed-off-by: shane-huang <sh...@intel.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/dfbdc9dd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/dfbdc9dd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/dfbdc9dd

Branch: refs/heads/master
Commit: dfbdc9ddb773e2b1149e6a6c661f14b631b692d0
Parents: 834686b
Author: shane-huang <sh...@intel.com>
Authored: Mon Sep 23 11:28:58 2013 +0800
Committer: shane-huang <sh...@intel.com>
Committed: Mon Sep 23 11:28:58 2013 +0800

----------------------------------------------------------------------
 assembly/src/main/assembly/assembly.xml         |  11 +-
 bin/spark-daemon.sh                             |   2 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |   4 +-
 .../scheduler/mesos/MesosSchedulerBackend.scala |   4 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |   2 +-
 .../scala/org/apache/spark/DriverSuite.scala    |   2 +-
 docs/running-on-yarn.md                         |   4 +-
 docs/spark-standalone.md                        |   2 +-
 make-distribution.sh                            |   5 +-
 python/pyspark/java_gateway.py                  |   2 +-
 sbin/spark-class                                | 117 +++++++++++++++++++
 sbin/spark-class.cmd                            |  23 ++++
 sbin/spark-class2.cmd                           |  78 +++++++++++++
 sbin/spark-executor                             |  22 ++++
 spark-class                                     | 117 -------------------
 spark-class.cmd                                 |  23 ----
 spark-class2.cmd                                |  78 -------------
 spark-executor                                  |  22 ----
 18 files changed, 256 insertions(+), 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/assembly/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml
index 47d3fa9..6af383d 100644
--- a/assembly/src/main/assembly/assembly.xml
+++ b/assembly/src/main/assembly/assembly.xml
@@ -39,23 +39,20 @@
     </fileSet>
     <fileSet>
       <directory>
-        ${project.parent.basedir}/bin/
+        ${project.parent.basedir}/sbin/
       </directory>
-      <outputDirectory>/bin</outputDirectory>
+      <outputDirectory>/sbin</outputDirectory>
       <includes>
         <include>**/*</include>
       </includes>
     </fileSet>
     <fileSet>
       <directory>
-        ${project.parent.basedir}
+        ${project.parent.basedir}/bin/
       </directory>
       <outputDirectory>/bin</outputDirectory>
       <includes>
-        <include>run-example*</include>
-        <include>spark-class*</include>
-        <include>spark-shell*</include>
-        <include>spark-executor*</include>
+        <include>**/*</include>
       </includes>
     </fileSet>
   </fileSets>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/bin/spark-daemon.sh
----------------------------------------------------------------------
diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh
index 5bfe967..6baee0c 100755
--- a/bin/spark-daemon.sh
+++ b/bin/spark-daemon.sh
@@ -128,7 +128,7 @@ case $startStop in
     spark_rotate_log "$log"
     echo starting $command, logging to $log
     cd "$SPARK_PREFIX"
-    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
+    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/sbin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
     newpid=$!
     echo $newpid > $pid
     sleep 2

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
index 3dbe61d..9f93491 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
@@ -125,7 +125,7 @@ private[spark] class CoarseMesosSchedulerBackend(
       StandaloneSchedulerBackend.ACTOR_NAME)
     val uri = System.getProperty("spark.executor.uri")
     if (uri == null) {
-      val runScript = new File(sparkHome, "spark-class").getCanonicalPath
+      val runScript = new File(sparkHome, "/sbin/spark-class").getCanonicalPath
       command.setValue(
         "\"%s\" org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
           runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
@@ -134,7 +134,7 @@ private[spark] class CoarseMesosSchedulerBackend(
       // glob the directory "correctly".
       val basename = uri.split('/').last.split('.').head
       command.setValue(
-        "cd %s*; ./spark-class org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
+        "cd %s*; ./sbin/spark-class org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
           basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
       command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala
index 541f86e..c7804a3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala
@@ -101,12 +101,12 @@ private[spark] class MesosSchedulerBackend(
       .setEnvironment(environment)
     val uri = System.getProperty("spark.executor.uri")
     if (uri == null) {
-      command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath)
+      command.setValue(new File(sparkHome, "/sbin/spark-executor").getCanonicalPath)
     } else {
       // Grab everything to the first '.'. We'll use that and '*' to
       // glob the directory "correctly".
       val basename = uri.split('/').last.split('.').head
-      command.setValue("cd %s*; ./spark-executor".format(basename))
+      command.setValue("cd %s*; ./sbin/spark-executor".format(basename))
       command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
     }
     val memory = Resource.newBuilder()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index 3ec9760..b76ad67 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -35,7 +35,7 @@ private[spark] object UIWorkloadGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
-      println("usage: ./spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
+      println("usage: ./sbin/spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
       System.exit(1)
     }
     val master = args(0)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/core/src/test/scala/org/apache/spark/DriverSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
index 01a72d8..5e5ece0 100644
--- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@ -35,7 +35,7 @@ class DriverSuite extends FunSuite with Timeouts {
     val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]"))
     forAll(masters) { (master: String) =>
       failAfter(30 seconds) {
-        Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master),
+        Utils.execute(Seq("./sbin/spark-class", "org.apache.spark.DriverWithoutCleanup", master),
           new File(System.getenv("SPARK_HOME")))
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index c611db0..767eb5c 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -42,7 +42,7 @@ This would be used to connect to the cluster, write to the dfs and submit jobs t
 
 The command to launch the YARN Client is as follows:
 
-    SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./spark-class org.apache.spark.deploy.yarn.Client \
+    SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./sbin/spark-class org.apache.spark.deploy.yarn.Client \
       --jar <YOUR_APP_JAR_FILE> \
       --class <APP_MAIN_CLASS> \
       --args <APP_MAIN_ARGUMENTS> \
@@ -62,7 +62,7 @@ For example:
 
     # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example
     $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
-        ./spark-class org.apache.spark.deploy.yarn.Client \
+        ./sbin/spark-class org.apache.spark.deploy.yarn.Client \
           --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
           --class org.apache.spark.examples.SparkPi \
           --args yarn-standalone \

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 81cdbef..9d4ad1e 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -25,7 +25,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by
 
 Similarly, you can start one or more workers and connect them to the master via:
 
-    ./spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT
+    ./sbin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT
 
 Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default).
 You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS).

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/make-distribution.sh
----------------------------------------------------------------------
diff --git a/make-distribution.sh b/make-distribution.sh
index bffb198..13843bb 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -98,10 +98,7 @@ mkdir "$DISTDIR"/conf
 cp "$FWDIR/conf/*.template" "$DISTDIR"/conf
 cp -r "$FWDIR/bin" "$DISTDIR"
 cp -r "$FWDIR/python" "$DISTDIR"
-cp "$FWDIR/spark-class" "$DISTDIR"
-cp "$FWDIR/spark-shell" "$DISTDIR"
-cp "$FWDIR/spark-executor" "$DISTDIR"
-cp "$FWDIR/pyspark" "$DISTDIR"
+cp -r "$FWDIR/sbin" "$DISTDIR"
 
 
 if [ "$MAKE_TGZ" == "true" ]; then

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/python/pyspark/java_gateway.py
----------------------------------------------------------------------
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index e615c1e..f7834ef 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -31,7 +31,7 @@ def launch_gateway():
     # Launch the Py4j gateway using Spark's run command so that we pick up the
     # proper classpath and SPARK_MEM settings from spark-env.sh
     on_windows = platform.system() == "Windows"
-    script = "spark-class.cmd" if on_windows else "spark-class"
+    script = "/sbin/spark-class.cmd" if on_windows else "/sbin/spark-class"
     command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",
                "--die-on-broken-pipe", "0"]
     if not on_windows:

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/sbin/spark-class
----------------------------------------------------------------------
diff --git a/sbin/spark-class b/sbin/spark-class
new file mode 100755
index 0000000..e111ef6
--- /dev/null
+++ b/sbin/spark-class
@@ -0,0 +1,117 @@
+#!/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.
+#
+
+SCALA_VERSION=2.9.3
+
+# Figure out where the Scala framework is installed
+FWDIR="$(cd `dirname $0`; pwd)"
+
+# Export this as SPARK_HOME
+export SPARK_HOME="$FWDIR"
+
+# 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 [ -z "$1" ]; then
+  echo "Usage: spark-class <class> [<args>]" >&2
+  exit 1
+fi
+
+# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
+# values for that; it doesn't need a lot
+if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then
+  SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
+  SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
+  # Do not overwrite SPARK_JAVA_OPTS environment variable in this script
+  OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS"   # Empty by default
+else
+  OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
+fi
+
+
+# Add java opts for master, worker, executor. The opts maybe null
+case "$1" in
+  'org.apache.spark.deploy.master.Master')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
+    ;;
+  'org.apache.spark.deploy.worker.Worker')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
+    ;;
+  'org.apache.spark.executor.StandaloneExecutorBackend')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+    ;;
+  'org.apache.spark.executor.MesosExecutorBackend')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+    ;;
+  'org.apache.spark.repl.Main')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
+    ;;
+esac
+
+# Find the java binary
+if [ -n "${JAVA_HOME}" ]; then
+  RUNNER="${JAVA_HOME}/bin/java"
+else
+  if [ `command -v java` ]; then
+    RUNNER="java"
+  else
+    echo "JAVA_HOME is not set" >&2
+    exit 1
+  fi
+fi
+
+# Set SPARK_MEM if it isn't already set since we also use it for this process
+SPARK_MEM=${SPARK_MEM:-512m}
+export SPARK_MEM
+
+# Set JAVA_OPTS to be able to load native libraries and to set heap size
+JAVA_OPTS="$OUR_JAVA_OPTS"
+JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
+JAVA_OPTS="$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="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
+fi
+export JAVA_OPTS
+# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
+
+if [ ! -f "$FWDIR/RELEASE" ]; then
+  # Exit if the user hasn't compiled Spark
+  ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null
+  if [[ $? != 0 ]]; then
+    echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2
+    echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
+    exit 1
+  fi
+fi
+
+# Compute classpath using external script
+CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
+export CLASSPATH
+
+if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
+  echo -n "Spark Command: "
+  echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
+  echo "========================================"
+  echo
+fi
+
+exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/sbin/spark-class.cmd
----------------------------------------------------------------------
diff --git a/sbin/spark-class.cmd b/sbin/spark-class.cmd
new file mode 100644
index 0000000..19850db
--- /dev/null
+++ b/sbin/spark-class.cmd
@@ -0,0 +1,23 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+rem This is the entry point for running a Spark class. To avoid polluting
+rem the environment, it just launches a new cmd to do the real work.
+
+cmd /V /E /C %~dp0spark-class2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/sbin/spark-class2.cmd
----------------------------------------------------------------------
diff --git a/sbin/spark-class2.cmd b/sbin/spark-class2.cmd
new file mode 100644
index 0000000..d4d853e
--- /dev/null
+++ b/sbin/spark-class2.cmd
@@ -0,0 +1,78 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+set SCALA_VERSION=2.9.3
+
+rem Figure out where the Spark framework is installed
+set FWDIR=%~dp0
+
+rem Export this as SPARK_HOME
+set SPARK_HOME=%FWDIR%
+
+rem Load environment variables from conf\spark-env.cmd, if it exists
+if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+
+rem Test that an argument was given
+if not "x%1"=="x" goto arg_given
+  echo Usage: spark-class ^<class^> [^<args^>]
+  goto exit
+:arg_given
+
+set RUNNING_DAEMON=0
+if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1
+if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1
+if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m
+set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true
+if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY%
+rem Do not overwrite SPARK_JAVA_OPTS environment variable in this script
+if "%RUNNING_DAEMON%"=="0" set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS%
+if "%RUNNING_DAEMON%"=="1" set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS%
+
+rem Figure out how much memory to use per executor and set it as an environment
+rem variable so that our process sees it and can report it to Mesos
+if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m
+
+rem Set JAVA_OPTS to be able to load native libraries and to set heap size
+set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM%
+rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
+
+rem Test whether the user has built Spark
+if exist "%FWDIR%RELEASE" goto skip_build_test
+set FOUND_JAR=0
+for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
+  set FOUND_JAR=1
+)
+if "%FOUND_JAR%"=="0" (
+  echo Failed to find Spark assembly JAR.
+  echo You need to build Spark with sbt\sbt assembly before running this program.
+  goto exit
+)
+:skip_build_test
+
+rem Compute classpath using external script
+set DONT_PRINT_CLASSPATH=1
+call "%FWDIR%bin\compute-classpath.cmd"
+set DONT_PRINT_CLASSPATH=0
+
+rem Figure out where java is.
+set RUNNER=java
+if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java
+
+"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %*
+:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/sbin/spark-executor
----------------------------------------------------------------------
diff --git a/sbin/spark-executor b/sbin/spark-executor
new file mode 100755
index 0000000..2c07c54
--- /dev/null
+++ b/sbin/spark-executor
@@ -0,0 +1,22 @@
+#!/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.
+#
+
+FWDIR="`dirname $0`"
+echo "Running spark-executor with framework dir = $FWDIR"
+exec $FWDIR/spark-class org.apache.spark.executor.MesosExecutorBackend

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/spark-class
----------------------------------------------------------------------
diff --git a/spark-class b/spark-class
deleted file mode 100755
index e111ef6..0000000
--- a/spark-class
+++ /dev/null
@@ -1,117 +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.
-#
-
-SCALA_VERSION=2.9.3
-
-# Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
-
-# Export this as SPARK_HOME
-export SPARK_HOME="$FWDIR"
-
-# 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 [ -z "$1" ]; then
-  echo "Usage: spark-class <class> [<args>]" >&2
-  exit 1
-fi
-
-# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
-# values for that; it doesn't need a lot
-if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then
-  SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
-  SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
-  # Do not overwrite SPARK_JAVA_OPTS environment variable in this script
-  OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS"   # Empty by default
-else
-  OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
-fi
-
-
-# Add java opts for master, worker, executor. The opts maybe null
-case "$1" in
-  'org.apache.spark.deploy.master.Master')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
-    ;;
-  'org.apache.spark.deploy.worker.Worker')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
-    ;;
-  'org.apache.spark.executor.StandaloneExecutorBackend')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
-    ;;
-  'org.apache.spark.executor.MesosExecutorBackend')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
-    ;;
-  'org.apache.spark.repl.Main')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
-    ;;
-esac
-
-# Find the java binary
-if [ -n "${JAVA_HOME}" ]; then
-  RUNNER="${JAVA_HOME}/bin/java"
-else
-  if [ `command -v java` ]; then
-    RUNNER="java"
-  else
-    echo "JAVA_HOME is not set" >&2
-    exit 1
-  fi
-fi
-
-# Set SPARK_MEM if it isn't already set since we also use it for this process
-SPARK_MEM=${SPARK_MEM:-512m}
-export SPARK_MEM
-
-# Set JAVA_OPTS to be able to load native libraries and to set heap size
-JAVA_OPTS="$OUR_JAVA_OPTS"
-JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
-JAVA_OPTS="$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="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
-fi
-export JAVA_OPTS
-# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
-
-if [ ! -f "$FWDIR/RELEASE" ]; then
-  # Exit if the user hasn't compiled Spark
-  ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null
-  if [[ $? != 0 ]]; then
-    echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2
-    echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
-    exit 1
-  fi
-fi
-
-# Compute classpath using external script
-CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
-export CLASSPATH
-
-if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
-  echo -n "Spark Command: "
-  echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
-  echo "========================================"
-  echo
-fi
-
-exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/spark-class.cmd
----------------------------------------------------------------------
diff --git a/spark-class.cmd b/spark-class.cmd
deleted file mode 100644
index 19850db..0000000
--- a/spark-class.cmd
+++ /dev/null
@@ -1,23 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-rem This is the entry point for running a Spark class. To avoid polluting
-rem the environment, it just launches a new cmd to do the real work.
-
-cmd /V /E /C %~dp0spark-class2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/spark-class2.cmd
----------------------------------------------------------------------
diff --git a/spark-class2.cmd b/spark-class2.cmd
deleted file mode 100644
index d4d853e..0000000
--- a/spark-class2.cmd
+++ /dev/null
@@ -1,78 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-set SCALA_VERSION=2.9.3
-
-rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
-
-rem Export this as SPARK_HOME
-set SPARK_HOME=%FWDIR%
-
-rem Load environment variables from conf\spark-env.cmd, if it exists
-if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
-
-rem Test that an argument was given
-if not "x%1"=="x" goto arg_given
-  echo Usage: spark-class ^<class^> [^<args^>]
-  goto exit
-:arg_given
-
-set RUNNING_DAEMON=0
-if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1
-if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1
-if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m
-set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true
-if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY%
-rem Do not overwrite SPARK_JAVA_OPTS environment variable in this script
-if "%RUNNING_DAEMON%"=="0" set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS%
-if "%RUNNING_DAEMON%"=="1" set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS%
-
-rem Figure out how much memory to use per executor and set it as an environment
-rem variable so that our process sees it and can report it to Mesos
-if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m
-
-rem Set JAVA_OPTS to be able to load native libraries and to set heap size
-set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM%
-rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
-
-rem Test whether the user has built Spark
-if exist "%FWDIR%RELEASE" goto skip_build_test
-set FOUND_JAR=0
-for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
-  set FOUND_JAR=1
-)
-if "%FOUND_JAR%"=="0" (
-  echo Failed to find Spark assembly JAR.
-  echo You need to build Spark with sbt\sbt assembly before running this program.
-  goto exit
-)
-:skip_build_test
-
-rem Compute classpath using external script
-set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
-set DONT_PRINT_CLASSPATH=0
-
-rem Figure out where java is.
-set RUNNER=java
-if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java
-
-"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %*
-:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/dfbdc9dd/spark-executor
----------------------------------------------------------------------
diff --git a/spark-executor b/spark-executor
deleted file mode 100755
index 2c07c54..0000000
--- a/spark-executor
+++ /dev/null
@@ -1,22 +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.
-#
-
-FWDIR="`dirname $0`"
-echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/spark-class org.apache.spark.executor.MesosExecutorBackend


[05/21] moved user scripts to bin folder

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/data/lr_data.txt
----------------------------------------------------------------------
diff --git a/data/lr_data.txt b/data/lr_data.txt
new file mode 100644
index 0000000..d4df063
--- /dev/null
+++ b/data/lr_data.txt
@@ -0,0 +1,1000 @@
+1 2.1419053154730548 1.919407948982788 0.0501333631091041 -0.10699028639933772 1.2809776380727795 1.6846227956326554 0.18277859260127316 -0.39664340267804343 0.8090554869291249 2.48621339239065
+1 1.8023071496873626 0.8784870753345065 2.4105062239438624 0.3597672177864262 -0.20964445925329134 1.3537576978720287 0.5096503508009924 1.5507215382743629 -0.20355100196508347 1.3210160806416416
+1 2.5511476388671834 1.438530286247105 1.481598060824539 2.519631078968068 0.7231682708126751 0.9160610215051366 2.255833005788796 0.6747272061334229 0.8267096669389163 -0.8585851445864527
+1 2.4238069456328435 -0.3637260240750231 -0.964666098753878 0.08140515606581078 -1.5488873933848062 -0.6309606578419305 0.8779952253801084 2.289159071801577 0.7308611443440066 1.257491408509089
+1 0.6800856239954673 -0.7684998592513064 0.5165496871407542 0.4900095346106301 2.116673376966199 0.9590527984827171 -0.10767151692007948 2.8623214176471947 2.1457411377091526 -0.05867720489309214
+1 2.0725991339400673 -0.9317441520296659 1.30102521611535 1.2475231582804265 2.4061568492490872 -0.5202207203569256 1.2709294126920896 1.5612492848137771 0.4701704219631393 1.5390221914988276
+1 3.2123402141787243 0.36706643122715576 -0.8831759122084633 1.3865659853763344 1.3258292709064945 0.09869568049999977 0.9973196910923824 0.5260407450146751 0.4520218452340974 0.9808998515280365
+1 2.6468163882596327 -0.10706259221579106 1.5938103926672538 0.8443353789148835 1.6632872929286855 2.2267933606886228 1.8839698437730905 1.2217245467021294 1.9197020859698617 0.2606241814111323
+1 1.803517749531419 0.7460582552369641 0.23616113949394446 -0.8645567427274516 -0.861306200027518 0.423400118883695 0.5910061937877524 1.2484609376165419 0.5190870450972256 1.4462120573539101
+1 0.5534111111196087 1.0456386878650537 1.704566327313564 0.7281759816328417 1.0807487791523882 2.2590964696340183 1.7635098382407333 2.7220810801509723 1.1459500540537249 0.005336987537813309
+1 1.2007496259633872 1.8962364439355677 2.5117192131332224 -0.40347372807487814 -0.9069696484274985 2.3685654487373133 0.44032696763461554 1.7446081536741977 2.5736655956810672 2.128043441818191
+1 0.8079184133027463 -1.2544936618345086 1.439851862908128 1.6568003265998676 0.2550498385706287 2.1994753269490133 2.7797467521986703 1.0674041520757056 2.2950640220107115 0.4173234715497547
+1 1.7688682382458407 1.4176645501737688 0.5309077640093247 1.4141481732625842 1.663022727536151 1.8671946375362718 1.2967008778056806 1.3215230565153893 3.2242953580982188 1.8358482078498959
+1 -0.1933022979733765 1.1188051459900596 1.5580410346433533 -0.9527104650970353 2.4960553383489517 0.2374178113187807 1.8951776489120973 0.817329097076558 1.9297634639960395 0.5625196401726915
+1 0.8950890609697704 0.3885617561119906 1.3527646644845603 -0.14451661079866773 0.34616820106951784 3.677097108514281 1.1513217164424643 2.8470372001182738 1.440743314981174 1.8773090852445982
+1 1.946980694388772 0.3002263539854614 -1.315207227451069 1.0948002011749645 1.1920371028231238 -0.008130832288609113 -1.150717205632501 2.6170416083849215 1.5473509656354905 2.6230096333098776
+1 1.369669298870147 2.2240526315272633 1.8751209163514155 0.7099955723660032 1.4333345396190893 2.0069743967645715 2.783008145523796 2.356870316505785 1.4459302415658664 2.3915127940536753
+1 1.0329554152547427 0.19817512014940342 0.9828173667832262 -0.3164854365297216 0.9721814447840595 2.9719833390831583 2.3758681039407463 -0.2706898498985282 1.2920337802284907 2.533319271731563
+1 1.1046204258897305 -0.31316036717589113 2.779996494431689 1.3952547694086233 0.49953716767570155 -1.0407393926238933 2.0869289165797924 -0.04084913117769684 2.9616582572418197 1.9258632212977318
+1 2.361656934659277 3.8896525506477344 0.5089863292545287 0.28980141682319804 2.570466720662197 0.15759150270048905 0.6680692313979322 -0.698847669879108 0.4688584882078929 -1.5875629832762232
+1 1.301564524776174 -0.15280528962364026 -0.7133285086762593 1.081319758035075 -0.3278612176303164 1.6965862080356764 -0.28767133135763223 2.2509059068665724 1.0125522002674598 1.6566974914450203
+1 -0.3213530059013969 1.8149172295041944 1.6110409277400992 1.1234808948785417 1.3884025750196511 0.41787276194289835 1.4334356888417783 0.20395689549800888 1.0639952991231423 0.25788892433087685
+1 2.1806635961066307 1.9198186083780135 2.238005178835123 0.9291144984960873 0.4341039397491093 2.050821228244721 1.9441165305261188 0.30883909322226666 1.8859638093504212 -1.533371339542391
+1 1.4163203752064484 1.4062903984061705 1.8418616457792907 0.6519263935739821 2.0703545150299583 0.7652230912847241 1.1557263986072353 1.6683095785190067 1.3685121432402299 1.0970993371965074
+1 -0.23885375176985146 0.7346703244086044 0.39686127458413645 0.8536167113915564 2.8821103658250253 2.843586967989016 0.2256284103968883 0.8466499260789964 1.1372088070346282 0.0880674005359322
+1 1.190682102191321 1.7232172113039872 0.5636637342794258 0.8190845829178903 1.803778929309528 2.386253140767585 0.651507090146642 2.053713849719438 1.049889279545437 2.367448527229836
+1 1.2667391586127408 1.0272601665986936 0.1694838905810353 1.3980698432838456 1.2347363543406824 1.519978239538835 0.7755635065536938 1.9518789476720877 0.8463891970929239 -0.1594658182609312
+1 1.9177143967118988 0.1062210539075672 1.0776111251281053 1.969732837479783 0.5806581670596382 0.9622645870604398 0.5267699759271061 0.14462924425226986 3.205183137564584 0.3349768610796714
+1 2.8022977941941876 1.7233623251887376 1.8343656581164236 2.5078868235362135 2.8732773429688496 1.175657348763883 1.8230498418068863 -0.06420099579179217 -0.31850161026000223 1.3953402446037735
+1 1.293815946466546 1.9082454404595959 1.0390424276302468 1.4123446397119441 0.14272371474828127 0.5954644427489499 1.9311182993772318 1.4425836945233532 0.23593915711070867 -0.0046799615367818514
+1 2.1489058966224226 1.5823735498702165 0.47984538863958215 0.05725411130294378 -0.19205537448285037 2.578016006340281 2.635623602110286 1.9829002135878433 0.19799288106884738 1.7028918814014005
+1 1.5672862680104924 -0.0987393491518127 0.7244061201774454 -0.41182579172916434 1.1979110917942835 -0.12481753033835274 0.5630131395041615 1.385537735117697 -0.8919101455344216 2.7424648070251116
+1 0.6879772771184975 1.582111812261079 0.3665634721723976 0.850798208790375 0.9426300131823666 1.983603842699607 0.8130990941989288 -1.0826899070777283 0.7979163057567745 -0.12841040130621417
+1 0.49726755658797983 1.1012109678729847 0.27184530927569217 0.09590187123183869 2.7114680848906723 1.0712539490680686 0.4661357697833658 1.1666136730805596 1.0060435328852553 1.3752864302671253
+1 1.5705074035386362 2.5388314004618415 3.705325086899449 1.7253747699098896 0.2905920924621258 2.2062201954483274 1.7686772759307146 -0.14389818761776474 1.317117811881067 1.960659458484061
+1 -0.6097266693243066 1.5050792404611277 1.5597531261282835 1.801921952517151 1.021637610172004 1.0147308245966982 0.496200008835183 1.2470065877402576 1.09033470655824 2.154244343371553
+1 1.7311626690342417 -0.7981106861881657 1.576306673263288 2.0139307462486293 0.9669340713114077 2.6079849454993758 2.4417756902619443 0.97773788498047 -0.02280274021786477 1.9625031913007136
+1 0.034608060780454086 0.43324370378601906 0.6464567365972307 0.16942820411876358 2.773634414356671 0.950387120399953 0.20399015246948005 2.45383876915324 1.4728192154140967 0.27665303590986445
+1 0.669423341908155 2.753528514524716 -0.3114457433066151 0.42623362468295967 0.17585723777040074 0.3896466198418058 3.382230016050147 0.5628980580934769 0.1855399231085304 -1.0368812374682252
+1 1.1578929223859837 -0.9772673038070927 1.628472811304047 0.1706064825334408 -0.4368078914563116 1.3238749660151412 -0.6328206376503045 -0.1268798336415804 1.4614917163766068 0.05098215234403425
+1 1.9810025566400666 1.076214892921874 -1.1668914854936587 1.6219892570599912 0.5991126181156119 1.0668387700181805 -0.38561466584746307 -0.3346008538706646 -0.13693208851002447 1.082271823637847
+1 1.6753996221697711 -0.2204800911406224 1.3643600908733924 1.3667965239511641 1.4202494777278367 0.1990171616310349 1.3814657607888683 1.0156848718344853 1.1547747341458854 1.919747223811457
+1 2.306325804101286 2.013331566156439 1.1223877708770225 -0.06481662603037197 1.7942868367810174 0.7587370182842376 0.8698939230717255 0.37170451929485726 1.353135265304875 -0.013085996169272862
+1 0.20271462066175472 1.8670116701629946 0.1618067461065149 -0.2974653145373134 2.0274885311314446 1.7489571027636028 2.991328245656333 2.3823300780216257 2.078511519846326 1.97782037580114
+1 2.2596721244733233 1.006588878797566 2.2453074888557705 0.4245510909203909 1.557587461354759 1.7728855159117356 1.0648265192392103 1.1365923061997036 0.5379050122382909 0.9997617294083609
+1 2.414464891572643 0.30469754105126257 2.1935238570960616 2.587308021245376 1.5756963983924648 1.9319407933274975 0.8074477639415376 1.7357619185236388 0.23815230672958865 -0.4761137753554259
+1 1.3855245092290591 1.955100157523304 1.4341819377958671 0.28696565179644584 1.7291061523286055 1.714048489489178 1.164672495926134 1.6545959369641716 1.9496841789853843 2.5374349926535062
+1 1.1158271727931894 2.213425162173939 1.36638012222097 -0.023757883337165886 2.406876786398608 1.1126742159637397 0.12318438504039564 2.8153485847571273 0.15506376286728374 0.33355971489136393
+1 1.7297171728443748 0.6719390218027237 1.3753247894650051 -0.10182607341800742 1.7453755134851177 1.0960805604241037 0.40205225932790567 1.6103118877057256 -1.03955805358224 -0.3213966754338211
+1 1.316257046547979 1.2853238426515166 2.0480481778475728 0.6602539720919305 0.7379613133231193 2.0626091656565495 1.4509651703701687 1.864003948893211 2.2982171285406796 0.9359019132591221
+1 1.6046620370312947 2.321499271109006 2.2161407602345786 0.5862066390480085 -1.06591519642831 0.4488708706540525 0.9764088582932869 -0.17539686817265143 1.0261570987217379 1.8924236336247766
+1 -0.013917852015644883 0.4901030850643481 0.574360829130456 0.08844371614484736 1.3233068279136773 0.7589759244353294 1.7201737182853447 0.517426440952053 2.7274693051068777 0.036397493927961544
+1 1.2232096749473036 1.4768480172452538 1.5300887552091489 1.8810354040615782 -0.6436862913845212 1.5878631039716906 0.09394891272528805 1.7766036014727926 -0.08618397395873112 1.5926757324414604
+1 -0.006190798924250895 -1.1803586949394225 2.237721401521945 0.7324966516613158 1.4038442669165114 -0.06019103023815764 -0.7655029652453154 -0.3991986433215591 2.3296187529650685 0.38065062537135896
+1 1.0869918851572522 -0.37412852726006984 0.27965894114884915 -0.0733849426330444 0.7458288899809582 0.38504406064556884 1.3823407462352355 1.0530056181901168 -0.10908828320629294 -0.3163748213825457
+1 2.0800232080218937 0.6793681518120379 1.0126904247021766 0.5099365686965533 1.4765728601491988 -0.90922098444035 0.01578092821031385 2.531202299543557 1.3694116442965245 0.03526109196146243
+1 2.52004533036052 -0.11716335755537322 2.043801269881338 -0.4889959907470973 1.3717334116816158 -0.5907796618760839 2.9080140714861864 2.3969176626246114 0.9445325920064912 0.9620736405334235
+1 0.8261430232725533 0.9003472941846893 1.2648199316806048 1.3110765897825498 0.9484044458467761 1.5971370020069537 1.89838012162931 0.5844972943740565 2.1114035373528974 2.8066708339226407
+1 1.7131825192258492 0.5164803724034563 1.3400031460569826 1.159025272879641 -0.6475319792487726 0.7895415906096561 0.3591049378091684 0.3507368152114154 0.46463582975963413 1.2784917703092404
+1 0.9196047831077019 0.6917912743533342 1.7505158395265692 2.275307243506136 2.9871554281485713 0.584299496238456 1.2741949422522685 0.42838234246585094 2.613957509033075 1.479280190769243
+1 0.6865489083893408 1.6888181847006614 1.5612615114298305 0.28075030293939784 0.7611637101018122 0.17543992215891036 0.8532136322118986 1.6171101997247541 2.487562859731773 2.1695780390240165
+1 3.746488178488735 0.5902211931946351 1.4116785188193897 -0.302213259977852 1.3900348431280398 1.8058092139513118 1.9063920023065686 -0.6748417828946516 1.2856680423450677 1.4181322176013937
+1 1.3957855809267268 0.6788775338735233 1.2694449274462256 0.7739220722195589 1.6662774494836934 0.2263815064326532 0.3746198256735065 0.6981525121209534 0.6659194682736781 2.34383566814983
+1 0.3820962920141968 -0.11474969137094182 1.4456430767826618 1.7541264342573286 0.5841263905944027 0.3310478153678522 0.1361074962599954 2.1517668203954323 2.1312973802189523 0.08816171787088545
+1 0.44857483955792765 -1.3332507048491813 0.5685902212376108 1.1213432607484823 2.634120632788485 0.7837711869120604 1.0078687896423884 1.8982652887205418 1.1818816137394528 1.2876714951624808
+1 1.1951146419526084 0.9947742549449248 0.19840725400812698 2.48569644222758 1.7391898607628944 2.40036741337463 2.0600530189294144 -0.5340832975220873 2.0467391216154094 1.1908285513553203
+1 0.9918935330929904 -0.3542942677260328 1.3105513869382395 1.1904643448960697 -0.3602658438636872 0.6816024636806379 1.9768303812038046 0.4000132856795251 0.09352911692893684 1.9754791705404877
+1 1.0081698742896188 0.8916746417259931 1.496601632133103 1.8174757593692714 0.49297596177715564 1.828839820849067 1.662627028300793 1.2253219256823615 -1.6200329115107013 1.051770724619957
+1 0.9867026242209636 2.0915066394830326 0.2608828095090572 1.5275154403994393 0.3157310747415396 -0.7181525036523673 1.281115387917441 2.286539214837881 0.5653973688805878 3.0047565660570132
+1 0.9224469399191068 1.2533868053906783 -0.10077556308999824 0.06127395021274762 -0.18013801007271568 0.8043572428627129 -0.3236336059948026 1.6130489732175104 3.313472221318618 -0.15122165909659913
+1 0.7882345197971014 1.141304212890955 0.9030550623054504 2.543084656196279 0.7468302223968317 1.6832418500477586 0.10324287869065907 0.8952909318554702 1.7968146536867757 1.8337447891715968
+1 1.5801885793428398 2.438564562880532 1.346652611597816 2.013682644266395 0.5423884037920474 1.5509096942566918 -0.09721979565291483 0.7802050454421068 -0.07405588910002847 1.1020403166091144
+1 0.03083257777543913 0.09561020933135189 2.783828684436811 0.6702011711663662 1.1177709598763554 1.507733845629784 0.7190681946142053 0.4421675532332505 2.0062047937031338 1.3078544626787887
+1 0.029946310071738202 2.9974008035637247 1.2712685297793174 1.564287715942167 0.9318120646963208 1.9611220391387494 0.6955370789941844 2.8474941997466665 1.7216550057775473 1.033229285227095
+1 1.7919476706914224 2.674070943673579 1.0707436458201804 -1.2652465769212773 0.13786669485292458 -0.9521873641153344 -0.5112273884476357 1.8041566655420045 2.0489287678822823 1.4526766050251194
+1 2.1567394248692624 0.2787475011337476 1.2693515582998967 2.141920061908346 -0.311063434715769 2.7871358520284515 0.4011362416354143 1.2240722802790835 2.0224267357566696 0.6055884380482317
+1 1.2810578825169523 -0.06149076783837382 -0.3631214532063931 1.8242040060835376 0.936708636871513 0.9599645524867305 -0.2864664075189678 1.4575636141356014 -0.6521604857506678 1.4782024605158144
+1 1.922007864215502 0.41092515579085087 1.3614694131826193 1.2516141141035275 1.1032104604396404 1.5618738178080496 0.22277705609915832 -0.10552941002887595 0.8187789394182741 1.1899147160759034
+1 -1.101159111435701 2.0868811582857676 2.061754901850132 0.831389858205579 1.1022205058106118 -0.15327367461990105 3.263172683870654 -0.13185404063281925 0.4215198415563227 0.5983645772645423
+1 0.9017414538285525 1.5815719854072032 -0.33621575096987555 0.7353127316624433 2.000881249246564 1.752079037914068 2.188342812418916 2.464770657128536 1.9873120348231552 2.5280681270799197
+1 0.36229490936502484 0.9764447193507352 0.5513927408959507 1.2450834166369436 1.0347591040069144 0.23319917869834939 2.9368656872660264 1.3867291773435497 2.0279815142744324 1.3025138236731233
+1 0.12338005279277287 -0.11881556712737162 1.0293241194113785 2.053803566510112 1.694932390223226 1.2851644900727108 -0.09123042470171838 1.4542526750729492 0.9314422039244139 1.484525799738803
+1 2.2791038050359416 0.13652686573061323 0.34425341235820794 0.5134789845294401 1.199131994695721 1.285766903846671 1.6396476063943415 0.37354865288496775 -0.9325874103952065 1.9432993173271385
+1 0.3187247126988978 -0.23565755255952947 1.4653008405179144 1.4073930754043715 1.86867235923796 -0.8601040662125556 0.17314198154775828 1.359209951341465 1.8780560671833557 1.0497896254122507
+1 -0.35095212337482606 2.1382594819736456 0.21582557882234288 1.563987660659988 0.8742557302587846 2.7376537243676307 1.1089682445267717 0.3906567030119056 0.90272045105723 0.3199475930277361
+1 -1.0755666969659972 2.587500753780116 0.43523091172933415 1.9715380667335656 -1.206591074948113 2.3082117218149953 2.9003512906773183 1.8894617822889117 0.2612428397679113 2.3034517860165904
+1 1.2752641746970284 -0.8368104009920136 0.03573979915049008 0.9337645939367554 1.8180936927791564 0.35607066313035163 0.9553794086170463 2.3774664468818862 0.27151841486690464 0.5861688049602704
+1 1.3242463950740633 1.5079874960068127 2.2093340505083026 1.2611978264745287 1.7161846809846164 -0.49880331209390905 2.2386520558115137 1.259321190419847 1.3434715137362212 2.044909528652566
+1 0.8795598947051465 1.8282710612070696 0.8010144751459073 0.6664561865521288 0.4104626238753195 0.23255356821870798 0.33916496869925716 -0.2708146821069548 0.9241466333878707 -0.450452229744047
+1 1.9192448235188513 0.4969214523219533 2.4011260745046066 1.1346909629811026 -0.6596351603517379 -0.5351409933958904 0.02441943738258512 2.288141877404522 1.2367780341721122 1.584102117316426
+1 0.9682490849657925 -1.8650300168768377 0.8811925017526988 1.1594483122156354 1.121203677520715 0.9099984493527551 0.08826662255652562 -0.7539889420899628 0.4595729579317809 -0.7165782835963082
+1 1.5995281560764565 0.20521558652985616 -1.1164794717138746 1.5074668507140967 0.7877952768927691 0.902667397635835 1.6081861816054732 1.3133186016363785 1.5296162271430345 1.0712740040810271
+1 0.42211731340992986 0.502442828209289 0.3565737103297629 0.4478456815580649 1.617182070323055 0.9823042873485613 1.0704168281976632 -0.26776498356102985 1.8711459938723063 0.791693835933734
+1 0.23896637909254625 0.6184009702378752 1.484473242669571 -2.0960256478350034 1.007509277044258 1.4880525091303394 0.14825818901395527 2.918617492389175 2.7162682081607343 1.2852769131414254
+1 0.09951845043296148 0.10778080557671554 1.6153805572528395 0.21496629935184874 0.5695206599630613 0.5995686906470605 1.6226444344121718 1.400956890784598 2.5804792645155237 1.8818183326984712
+1 1.5660653841435699 1.9424448683907583 -0.5018032946330131 0.38813943551967744 0.21678795998247846 0.4592981799067166 0.3853775631077989 0.782922855791653 2.9697907962454226 2.0478747128589188
+1 0.5992085726320009 0.8326763829762222 1.0404230260991942 1.3571653199047529 0.05351664648320875 -1.8860610207228041 -0.5191719995314692 1.4226132032544871 1.6669779033604124 0.3253081253110943
+1 1.5903828533545434 1.894569333674546 1.5910544740636994 -1.6611392075582438 0.23842067636563624 -0.5406681576023691 1.7385589161163928 0.08969602776306584 1.4276561463432735 2.1566164427616634
+1 1.1913811808857528 0.32434695668325997 1.323498708189486 1.3596937187302878 3.4642496063989223 1.2876491657559253 -0.6543683402478666 1.4762502189363769 1.7353590098925795 2.8134629202660317
+1 3.123286693375267 1.877368736310955 0.9503145430714942 0.5342686470311402 0.3451961663217381 0.23995547380392213 0.5196925578399603 1.3087329089934692 0.5609549451755507 2.0018380155694433
+1 -0.70471754448335 0.396960196596961 2.8076920787881408 1.0486680479609312 0.1272088037522776 0.46477225522402743 1.0400518017377827 1.724354900707523 0.5172234824476354 0.70073364273413
+1 -0.04890176228714482 1.183623201015611 0.31679837772569197 2.442803942979677 2.475613952046278 1.316874640917748 2.1326668609632957 -1.1984022921949467 1.6326265827096553 0.13549684503148585
+1 1.532730344901386 1.8862673099243719 0.8433953501998975 0.9617349215859397 0.9632178266458564 1.7656392455188015 0.6166388141868028 0.36673723822668447 1.6148100615636092 1.9120508667715108
+1 1.8531415713908175 1.9856258806463458 0.8742545608077308 0.01891740612207793 0.754430421572012 1.2629533382356322 2.5668913595968625 0.7074626529557771 1.471180058040478 0.14210105766798764
+1 0.2946588114247314 1.7385325023150382 2.05805803890677 1.1285587768294627 0.30443899971020716 0.17710198470084348 -0.5876955744308521 1.6684452883987464 0.7429316176330647 0.24223269345723197
+1 0.12828383509135766 2.8251621371579123 -0.8683350630211126 1.3881503321455106 -0.9269673097143274 1.1340435175521124 1.1482061370168226 0.9886836766952749 1.3639211879675324 2.221424872356976
+1 1.6230819590031813 2.1140726634236273 0.8803195980146348 0.6957671564440406 1.3391648515238626 3.3118192086623672 1.206763244141946 0.5724427229085818 2.3692467877986934 1.2731917884083277
+1 0.6095837137279339 2.0886462170941087 1.5293277948541921 0.875698342933093 0.9739071638488416 -0.6284005601740021 0.7080909588024915 1.2483475820206364 0.39878604428574227 0.45167768471833614
+1 0.6622065044914254 0.7302732598978321 1.5839711558395906 0.33559568645900273 1.3094508963156517 1.5256964735790022 -0.2606881050391294 -0.13646086393521872 0.858395568393544 0.7983659548572369
+1 1.6030491170288057 0.8411660994073609 2.2968025114870225 0.7039288437264786 2.8125132767337133 0.23511452019598467 1.1415093151481583 -0.5416578453683565 2.121640334408583 -0.29666850192733474
+1 2.0779652161151883 1.0668503227493862 -0.3461938034511103 -1.9467096604673708 -0.4997902436835773 0.3419044702794434 0.8098524987621489 0.8131208951963917 1.3237950963836287 1.0429693266336961
+1 0.37001171609371697 0.29180348786692334 -0.2507809978364861 1.152821888667346 3.0890087304413267 1.215489406549123 1.199447470435283 0.789305354976556 0.8365245923088752 0.9787024262828808
+1 0.9296046114728362 2.19739063739452 1.533572358281578 0.7759925327491899 1.557482584766074 1.7151021392829757 0.9544359521103486 0.20077841759520276 1.59524901629763 2.175430873131662
+1 0.8112131582336873 0.2864940430793351 0.5833958780431041 1.7741485867050852 0.7779977372833543 1.8236769123328878 1.9278891617195901 -1.0188957672300982 0.9197794797358201 0.045052296436480455
+1 1.3702354298117274 0.5815346064645623 -0.04109583670633299 2.5064872968829004 1.206757887015013 0.2506549572813025 0.655306538898329 -0.3438030831151808 0.36458112520078056 0.8710435445702591
+1 1.4561762683494108 0.9681359328856552 3.136045420267423 0.7520560598452287 1.6528697058481434 0.9607920473099414 0.7156379077840067 1.857016542269911 -0.16277187766324142 0.4874157744630184
+1 1.2664980583047298 0.4023544599875911 0.9080313985150303 0.6549364577494126 2.738329489381062 2.3768996789882744 1.3393128915299277 -1.0430311123744418 0.8323494096430804 -0.12738742588819885
+1 0.8365391310807251 2.2822870725882503 2.6266615690102215 0.004265515881109128 2.4879345431323623 0.4875299849317022 1.351118317094851 1.245328886439785 0.8575534087593427 0.669435902035294
+1 0.8058511262644885 0.7473099050414014 2.303189816277799 1.2225351585963724 1.8247316651754097 -0.30810342366775534 0.2821704820687452 -1.6099991877186302 0.8406234201201898 2.0583805330826985
+1 2.250164789914201 1.7436544269774978 2.947667398091067 1.4771471077132423 -1.586188610201127 2.320910876555482 1.636258094383067 1.2987326716659215 -1.311058489828028 -0.011700890501986194
+1 0.8080250762510234 1.6440873832130936 0.8879459460961949 1.2082440017762488 -0.3984868670511643 -1.6750959916314896 0.9349087046999264 0.7232463907082566 2.2386173679423806 -0.017579999213251485
+1 1.0323998857804233 -0.7718677431568479 1.776325436331275 0.5932669960371175 1.7054720461060777 1.709001306281528 2.088236771173788 -0.13891858312535765 2.4540464522669634 2.581504187930639
+1 -0.36589663467243794 0.9800989499410697 1.512657907848574 2.481982348891716 1.879063921040467 1.6783314697156686 2.519822194339233 1.5139378983098026 1.4765499639533166 -0.4586543768759259
+1 1.031519656541507 0.37677631561513636 1.215439603971527 -0.8333793025092529 1.2297449965589116 0.7309661122339723 0.2233308234176088 1.8978096741161727 1.0017178523256016 1.540799199113878
+1 0.37535440891823324 1.05838458440246 1.7478919610180488 1.4358567778260587 2.634621031491021 2.6733943020176536 1.4038023921761382 2.09456237109269 0.18751380927669214 0.9030253353081665
+1 0.6050644162204089 0.42475868702885367 0.67729642342563 0.9159762799821485 0.9966211703282338 1.0325406378266162 -0.31600956837305927 1.1275195620810772 0.7550807758634188 2.0556587502944152
+1 0.9639628237078233 1.6612996949785008 0.15018611313458818 3.079012778712338 1.6765505664424296 -0.3164200745592767 1.180094372490766 0.16048718182365862 2.6754833932699764 0.2861554471536204
+1 -0.4733123063374025 2.215557819873761 1.4809169546161616 0.5331014736871407 0.509471219211528 -0.5366908461365221 2.5757870803346328 1.3082491695854135 1.3064213366309576 0.9305958816930349
+1 3.0207863567912003 0.23781737522480972 0.07878478120317567 1.6302281378682424 0.5980775385393649 1.5928976343724883 0.3212142395168056 1.7151012207401586 1.593816382695755 0.7481118256003316
+1 -0.5298380895168147 -0.34947847130115894 1.259810473989246 1.907798036285846 0.35944121815361163 0.6444888816334708 0.34377708875002244 0.6836686767703974 1.2932110945792579 -0.458790316071632
+1 1.8401629428690227 2.259471445176863 -0.3223229794980764 0.7728238347557039 1.5724556976510322 1.3274646917002721 1.6717333483877963 0.03745904530831912 2.6550649930379056 0.9705596819145808
+1 0.12431297464461755 1.7563279244667416 0.7774986621540451 0.5111136337905993 0.6433978537639469 1.8971862751406254 0.45959793718271824 1.781102107071228 1.4062626338777793 0.6234780410061468
+1 0.8407772366817298 0.35964705320370294 -0.9623019831100632 0.44149536693473657 2.074342161562674 0.9904199365414913 3.2137011456900098 1.0337076328449122 2.0693337269664083 1.8277506449533987
+1 1.0113056814830639 0.9851992899356764 0.873659978134487 1.0421853488103219 2.299837087915077 0.8071982744117732 -0.1096427502124051 2.5599638730556995 2.3458120257795656 1.9104294240298325
+1 -0.2652413955956079 0.2771478177147122 -1.7578972328231406 0.5091791920398325 1.3694768197526315 0.5806835043255031 -0.0948278795711135 3.822899721567823 0.5484905756054144 -0.25075975842777454
+1 0.6859095316452635 0.791069272223955 1.2193553385123195 0.7291514560030636 1.3876944292574216 0.8892463484292987 3.4273502454413576 0.6580296103521155 0.3238972925695067 -0.6496800158558074
+1 -1.5436851049150522 1.956099227374563 0.2779057405377705 0.7339456639197723 0.014024861431684466 2.6630936618511405 0.7161890905680435 0.5077767425517368 1.3259571967911001 0.9137278907925384
+1 -0.292961767713223 1.3071340106236198 -0.7017668375142168 1.2860358231830809 -0.8122076288210658 1.7211614223707081 1.8304680327555625 0.16021436599026517 0.19612682942548998 1.2082198804992264
+1 1.5187520786413158 0.1828654866775874 0.7328431724966722 1.7953629646772824 0.8216669452081463 -0.4014319711127199 0.23334012012093153 1.534537449937785 1.3889014942993092 -0.8511049828025341
+1 0.8451858363611996 1.3418063089585763 -0.8238999092902703 -1.575942571644518 2.0750484405729095 2.033997248128906 1.4449221159961598 2.0253497341487448 2.2283973766958023 2.404323890979427
+1 1.6107433076928133 0.5404780687423208 0.7937155331805563 -0.6077722620726684 0.21332376555661758 -0.9993545668337882 0.31523750335957845 0.5473005319402997 0.960730821903916 -0.28012631768751084
+1 1.9389616507358387 1.9532576203532324 1.2153193637879869 -1.4069714611803268 0.4662801445447652 -0.6193751496277011 -0.028999422131398056 1.3038353983411688 1.4946684162238129 -0.7409848880778342
+1 0.9021404373434705 1.5851981284549943 0.6057610277009148 1.1112421784262574 1.413214054275196 1.9417673251914613 1.634690668060366 -0.08301380649683576 2.1711500689414116 2.99282324374365
+1 0.1637260233089869 0.49637480750763263 -0.5285944959659445 1.5681001289396956 1.6803958442936107 1.2246294425310562 2.5669221884551776 0.7567621149423418 1.5037234063128802 0.3463214960951032
+1 1.5723472760593176 0.6432239887651015 1.804758599642208 1.2176050861917662 1.8717138471483157 4.077916319312581 1.5133550052844793 1.3823856879297753 2.6113216067389695 -1.1093237177115047
+1 0.8602744779765249 2.178619602525301 2.453544172271271 1.0510379811276036 1.8409684994496875 0.11803069280172118 0.3230760986621918 2.259943083391159 0.6024489055423363 1.1990484290135006
+1 1.649184578143986 1.616265278882509 2.2742015008761607 2.626169250389406 -1.1492939072912116 1.0408825980561895 0.4369989721349081 0.9034290059197084 -0.11385932074779648 1.0982078408810698
+1 0.6341310783502718 -0.9708605273806881 -0.017201345919524602 0.8926037502408949 0.22822364223265212 0.9096851395074563 2.0473818885200648 -0.7848615761262032 1.4441059896043467 -0.24922705201528594
+1 1.4520344107406407 1.2639986753730716 -0.8513007095320302 1.6293092619132934 0.7394579998929112 1.3445648999777857 1.5178679268046242 0.9933053628903701 -0.9336323582033459 -1.6920287783811307
+1 -0.584837407411567 0.9604177163540187 -0.003828672372695019 0.1731711935522725 3.512170380159825 0.4926659491064572 1.1587769448255618 0.6600987191801231 0.9926496119226857 1.9870269736899853
+1 0.40697221517240734 0.7915676379059069 1.4331616842644888 1.6198603975182355 1.6417243704332136 1.6270560025018783 1.6799759614717393 1.700588227134973 1.8464436799312134 -0.9250687955521861
+1 0.04736288349237683 1.5587027295355322 0.12163352594242882 1.124943757807633 0.2850023846865297 -0.07621319541134719 0.6373292813835088 2.5571634870370934 1.905346123931221 0.30969838202705213
+1 0.23757107697869606 0.7009274223790678 -0.6005151170274707 0.46131870148693055 0.694253134444586 1.8704279215134783 1.9559864883094595 1.5475302665627626 0.902775266852526 2.253986651760284
+1 0.0931484209802732 -1.0536269817119295 0.7832662454709735 1.3370869763110287 1.8021230335269156 1.0422523333084228 0.5539002500282262 1.1402739247006104 1.3778884263982012 0.9839666885480669
+1 1.4022006973888672 0.3301442305911556 1.4159864215392552 1.0753881627418582 -0.2194812627814522 1.576874528728394 0.351144790840509 2.9042579131410218 0.33439079197692423 -0.21115533384764373
+1 0.9200624394093888 1.9601307267236312 1.3048792499777433 1.044019487533702 1.295476599028682 1.06479650163913 -0.8347875409017176 0.8767774440123639 0.1631761919249426 0.962325538273012
+1 0.4606387639284839 1.93128591538725 3.2494332751166293 0.4217241090513292 0.5940126704202255 0.12271071800591238 0.009005952876745105 0.0631236875750606 1.2229161931162333 2.3879030147755866
+1 3.2172098250997503 -0.021922357496697797 1.1859662862492402 1.2154601324678136 -0.3071029158823224 2.1738376762747613 2.2872633132290443 0.954809047991948 1.901337785669559 1.3011976479019711
+1 1.1885608047442375 2.721310638802292 0.9617587859607313 0.12651320336878014 0.12567757686210834 1.887061564570169 0.8860616196551063 0.6430168020234137 -0.030733700547949327 1.0564998980605065
+1 1.352748382066948 0.5202126729710697 0.14331687879826782 0.40785023484169414 1.9641960196192663 2.7910712640458297 0.7740423932819342 1.52559135640059 0.3239548613578228 2.31826432040899
+1 0.5203741956670356 0.884417958844451 1.3777220780800918 -0.4643847508675174 -0.37572084642581793 0.1262513952897556 1.5518202424896383 3.3877379158242378 -1.403581970685686 0.1009940122529609
+1 0.9894392616099077 -0.0034178714976433877 0.689046476206714 1.4208906847616534 1.5473446325066496 0.44218920279820595 0.24101228948954234 1.1801070630847152 0.8039116009276253 -0.46102470089902536
+1 0.6361572167176843 1.5563186537784683 0.8983823810124998 1.0798802186419254 -0.038600239378366874 1.6649842223710727 1.6378836320811345 0.3059309271799856 0.8901320418030211 0.10914549884068314
+1 -0.18003932381317478 1.5693004310535423 1.8013396839368538 1.7544292528839476 2.460230078664536 0.8072540575395855 0.8326108318826944 1.5006349728524033 0.7460792678168342 2.6820859579435474
+1 1.8960169042497794 2.1576293718618 2.424978645426269 0.6268556772800932 4.221588312115547 1.1780884004744951 1.5616604868899797 1.8886529082537074 1.6168854045075025 2.7308325759110224
+1 0.12878554700508837 2.1150328351027246 0.5356772045785253 0.8698163232516893 2.3406750293658183 0.6627125907242539 2.4239833684636736 -0.17649747406412253 0.34655417092691454 0.37167266730649473
+1 0.7700976682797439 1.2052165149892542 2.0323449543315446 1.8093079753157488 2.677682507242789 1.2230772168351174 0.10002304289163721 0.38829774391404126 0.7382541961293962 1.4604650485834432
+1 1.2304476527122155 1.5911723818857464 -0.6663405193368004 1.9423332506900772 1.4218831147452045 0.7172255125851585 -0.12990659585261488 0.9108053409327858 0.11424096453618027 1.1083558363715305
+1 0.5195105474968298 0.5710613703505523 2.2928613438234455 0.021245928903329103 2.1269497746764197 0.8932419976165424 0.9360795887134954 0.4206153958722527 -0.013928240567511851 1.9267860815714657
+1 -0.27500090463981786 1.163598213361118 2.396756337306596 0.7166497755216299 0.5087064238485857 1.2644991273445112 2.207063036182604 1.511076159763578 0.7514616147389759 -0.386653321343986
+1 1.275981257794266 0.28386450023604437 2.0468065778588445 0.3368819014778913 0.7803798072812063 -0.11268418399709335 1.0692622536985994 0.7450466892913328 0.6521234033954817 0.3533878920228143
+1 -0.26632749480506046 0.09964814030131464 -0.14774546592772242 -0.44102911713759774 -0.8175624623446118 0.5982737657645009 1.8018589102471618 1.0206495963947055 2.1703414097910376 2.509625756793014
+1 -1.084176873793715 0.003374206020577475 1.0490056163609893 0.7413062315194299 0.5457392593753987 0.47876209776833123 2.7997789450020427 0.8473717379952329 0.07511100942298876 2.342980564354181
+1 -0.6060249411337237 0.3100831921729499 2.5027389254157533 0.4950992021162349 -0.7743243396300394 2.254986439984994 1.524435417647438 1.5581584085809914 0.7613263552054441 0.7313335506205685
+1 1.252570109684499 -0.2259101116089468 2.02870927406763 -0.1982100935627482 -1.0747860634656639 0.5696675160105826 2.0536113238469964 2.436984468208358 1.087350912351074 1.6355207346806782
+1 0.08793454138157841 -0.7701820062667433 1.6526323582054276 2.648211639393969 1.5418579075681154 0.9489571984728947 0.05918410476639424 -0.9099915058439798 1.4346179896632103 -0.7890540352574975
+1 0.3047705090908783 -0.041817851700766795 1.864590556312606 2.2126512576725283 0.850687528022706 1.1516079924281961 0.7160824885255048 0.23428914563411007 1.5892718454214458 2.0304685172157515
+1 1.8541494516233115 0.4996871983195521 0.9048408243621995 0.7096255802229431 0.33910504796127783 1.3134581495613444 -0.2753494959695286 2.3289922141730686 0.7323942203055318 -0.274626661821493
+1 -1.338544772611924 1.2944523849511644 1.821257734737301 1.6793492696385324 1.5967736493283293 1.712864874826922 1.5745612820947925 0.4891550646810052 0.47846091208172825 -0.1743221254069207
+1 2.131766719148957 0.7608227099296399 1.0630568268599263 -1.1476984731054647 2.3867190880037636 1.130561984384332 0.9131559753959471 0.2973457770910879 1.3007036631285942 0.4372322143839449
+1 0.7708567792295566 0.580257476003238 1.5887140302216574 1.0413330688401965 0.7733129718389264 -0.5163740146933058 0.07497254374425988 0.28623086041167667 1.5489309172205683 0.8551008347224718
+1 3.4595137256272586 1.1532560360380666 1.588361571148596 1.3802224477267615 -0.7001860654912402 1.8740796848274577 0.14520299815591176 2.5193824279795254 0.03909705046483791 0.7357475729770275
+1 -0.6544136676184351 2.8745518291193553 2.1515280898247315 2.757731240766754 2.429606589051394 2.330014751072225 0.9115033589433934 2.6873787753182583 1.2992135444029829 2.3920287356459284
+1 1.885270281917602 1.858016821901751 -0.06157363620807099 0.308401967243883 -0.31307820201782555 1.461038889339163 1.6128329392090914 1.5772000116247265 2.710615509497419 0.8050419240018178
+1 1.405879563380197 0.659914831493603 1.912269260893395 0.529404740699135 1.4277377811246783 1.2913475473601614 1.7339294107927208 0.5215235778431477 1.7550541630505698 1.4400196124978555
+1 0.3245588747842635 0.42197424404348816 3.6539265313256526 1.2857918279043645 -0.03655209163203632 1.2407043968389915 0.4433829786888507 -0.07023065483472712 -0.6733771504197963 1.4798448078129154
+1 0.9085359200450331 -0.009624824747410887 1.0280527195285618 2.14148134591638 1.0562537066073983 0.8809817771790907 1.4071063563557673 -0.6597423723027149 1.5583011903165707 2.3154204049509683
+1 1.8050769097358077 1.7786869407899135 2.6495184641125515 1.158177494691216 1.1671375960394383 -0.45722370125523115 0.9835693406300088 1.6357021360875077 -0.16826461081967703 1.1932740024664812
+1 0.576688853348233 2.151495453088904 0.8572555252181385 3.405728819429614 2.101231270195057 1.6771308649271772 1.2637521672030567 3.1154229758040874 2.485850964748577 1.7694224707976827
+1 -0.22806118428106337 -0.9061154967479863 0.8964938904788088 0.6816585601664856 2.013761003670729 1.0313228363661557 0.9260597798962866 -0.18946147062989205 0.28527619220858247 0.8963510651947846
+1 0.3148947081465582 2.161975824817249 2.609645991041186 0.959492387316128 2.397824851151471 0.6697921252418206 2.313069590047294 0.8776639563036727 1.0599994333376752 2.8237989480782524
+1 2.652125755323301 1.8602107889115338 0.7683127593190835 2.2682293581606165 -0.6222001971107851 1.7327348607601576 1.7973442155328485 2.3026732779864645 1.6376913865909977 1.4336254291699817
+1 -0.033946588281949186 2.300669560977641 1.160077113314741 -1.035089589522486 -0.3088401922649133 2.2246952213732962 1.5263288862385613 1.2041606436782568 0.6360015906365958 -0.46568448099058934
+1 -0.8340563619947565 1.4168203411347104 -0.5724699864440952 -0.5633561206742383 1.454288263940742 2.091140792301254 -0.9346927324544323 0.0969827614306541 0.9901527415253794 2.0293060494871034
+1 2.1766440722293696 2.1765927443625097 -0.9288701141928257 -0.4887885438886057 1.415145042839749 0.7869820800801398 1.3531410283773004 0.38467574204818133 1.265876278197796 -0.2027790078386682
+1 0.8270879503594885 2.371236015912422 1.8437897438725939 1.7890683065643116 0.7718878947557098 0.1132854516378462 2.6937038226634122 1.34827091113804 1.8024405913978527 0.9733403683960185
+1 2.4175771508586754 0.8851307536623965 0.965109486208773 2.4006169759083864 1.1967556814639715 1.2950307543358157 1.9415648218013744 0.35864528885541735 0.40940436545238557 0.7868294504129988
+1 2.2098184536505663 0.889100413360103 2.1851586347238285 0.13494389682652308 -1.1445348600024268 0.8595807349607005 0.46845661480480505 0.07882338616350792 0.222858479263641 1.6187566311742603
+1 1.5395105587908753 1.5090442727804423 0.8644957394514675 1.2222062988283733 -0.657302278508328 -0.8584774737648058 0.7847354502810749 1.066321874171543 0.6763302367935397 -0.3056807220148554
+1 1.3241371059217268 1.1998033042587848 1.6413385242724854 1.2616652980595755 0.8214439629174916 0.7323804916810981 1.446327599557899 2.1344373550969333 0.5323048652541784 1.325312471981157
+1 0.44793596733276986 3.5291804831601397 2.304481907075438 1.7159536021092872 0.49378464200637107 0.529685187245525 -0.19498379135409039 0.6257392880667672 -0.5922944256976155 0.9677085580549932
+1 1.6001908684230077 0.8441053959985582 2.191005295444758 1.8601204690315698 1.4231646338661619 0.7172326899436327 1.3685291716454426 1.7459708463423858 -0.20021564447567597 0.7886037237104406
+1 -0.832715908403886 0.9821249159854097 1.9340136298649147 2.0863867471576207 0.8588263222826337 0.3940359686539505 0.5667076617327207 0.6813674534100007 1.0601080933156564 0.9940095449693623
+1 0.5362749326926859 1.3784556073957994 0.7830926551836939 0.7926130115032175 -0.45867401264881047 0.7649235836439627 1.9252198419840811 -0.5932278037833087 -0.20495235948345436 0.8228620061430476
+1 -0.5026862346261936 0.32379950915933053 0.4877018370232078 1.848487603750593 2.5612814512394575 2.6996258863788105 0.15501963775759875 1.779188209155349 -1.1587607119995043 0.5286988956500273
+1 0.03890979688369878 2.5700833608321876 -0.41167989902736224 0.4405078623025871 0.11339883057634925 1.2618969624421223 0.5661859841701755 0.4450152294875418 0.06553355298472463 2.9653045304903003
+1 1.2066695218108954 -1.135846422758188 1.3472000646449644 1.995247004371493 0.4067019132360835 0.6014718489518214 1.1945804244235247 2.563237911092928 -0.30000446942459824 0.6782859264246553
+1 0.43145271645135497 -0.15638436316804127 1.806542814206817 2.509982504123812 0.2908319784765735 1.093034072836503 1.8310934308417324 -0.428111571478186 1.0227258944948991 1.3181088073443865
+1 0.6593145377977876 0.5513227059953492 0.08971356052593105 0.6997087344297779 0.3547337578286779 2.044316172416025 1.7054002807979272 1.177077903869836 1.6118683425448608 1.3817764734854732
+1 3.26027582916473 1.922453791560931 1.5445220345277253 -0.3361563876793128 -0.20451311346146506 -0.02755370253733158 0.2523835913052155 1.8457060509750052 0.7729749699076125 1.2691512131543639
+1 0.7853510230572176 1.92550267228468 1.3840760296517856 1.019170128522936 1.257277800158144 0.2954835667658987 -0.02339082355482236 2.344976472145047 0.8650491281625572 1.6705466337391612
+1 1.0256022223771357 1.2521800754728607 2.5454645690960165 1.519642791108941 0.8120657189050374 1.395012570155324 1.0067859707833062 1.6154722360698295 -0.1911479039843622 0.3192273565677406
+1 0.9212215747887599 1.614097542109768 2.153211482594465 0.25851295883461667 0.015421396864703008 2.910093225363264 1.180736322866857 -0.024920942327103957 2.669708944799861 -0.4455433802815518
+1 1.5936186055028179 2.948335176521773 -0.9304959929630894 -0.25674218734698395 0.856450569458336 2.2464434469263295 2.2695814273033834 0.9023024874886443 0.1998192758289271 0.9614747140727596
+1 0.4171564598259989 1.2341430652292795 0.7613883447910024 1.4327906124857261 0.8248656963940865 -0.09370178940656282 0.5302446693348143 0.5977304498921516 1.9672679105851836 1.8549778581991436
+1 1.9988876732611685 1.7067688718725715 0.709840257121064 1.8195818549115197 -0.196218309209645 2.158975719537872 -0.387052375493828 0.2684905146219133 1.1751943798566946 -0.08233263071043195
+1 -0.004588558850024516 1.280146957738293 2.2274500380613915 2.068436441505224 2.4406629422607455 -0.020552259353522784 -1.9306504989533266 1.606929445859563 0.12204039563080737 1.554314194847439
+1 0.04312231827054913 2.293183585915505 0.5515907062418919 2.0319631309075303 0.2043494544647857 2.163212294566986 0.24687989300151647 2.1776229267798914 1.1368594510956058 1.1067868768921156
+1 0.8380882562583268 2.7318988397710573 1.4749062376973399 2.3244811915569885 1.498055997999189 1.4901966783173328 0.9547300656875682 1.2938212544822327 0.920830744648933 0.7960603079946061
+1 1.1730459404168871 2.4157763285361744 2.2769114804572554 1.781254882347914 1.8939310535271043 1.8204037399884672 1.2330253630970833 0.24898375343327694 1.4526754173493885 1.2327670337378527
+1 0.7828957363283248 1.961806185656672 1.0945811949626496 0.6471160715303457 1.2988151512993327 0.9231258952067597 1.7059995140840485 1.582221842249981 0.5731086038064922 2.929881320548402
+1 0.4240209410200867 2.0612687767691504 1.4013347045251126 1.0775762488985852 -0.5648359238473468 1.5394818276041304 0.5250719203859092 0.3867254288273827 1.836032841951298 -0.02644684457005053
+1 0.12838309666764036 -0.2524433635395231 0.14063539701460914 -0.8169781441139783 2.638413098813798 1.5872934688325704 1.343252734685199 1.1584200404773857 0.6163819194666804 0.6654328763469552
+1 -0.26416941528334714 0.32620704315453675 -0.7502936599619701 0.8401389782535786 0.09753988131424873 1.796236698582462 1.5877879186693455 0.9856032545638709 1.2072784259771 2.4653229099496707
+1 -0.6337999979940661 0.8076685452502981 1.2207084350653477 0.9123689527781019 1.838283774286254 2.2836210170990996 1.7394640050289512 0.6351189156017663 0.9629884451362287 1.7680252591425618
+1 1.8654459163757884 0.06089772776268909 0.9679374944456427 0.8889470807355174 -0.08754935246071827 -0.12680613988340284 -1.0637769092192588 1.512338996915241 1.9515416090320272 0.5015769881603198
+1 1.7247706923845918 0.360222898716523 0.18071931378959916 2.0371848423820293 1.5266006033053001 1.353704597154892 -0.2696414308039541 1.343721201156886 0.46275842064535144 2.3294944321291413
+1 2.1105081742950267 0.5116093610246693 2.2446634834462875 0.658957834299546 0.34134432630789047 0.4247161540652681 0.3292829996171407 -0.19362053618697583 2.62788746256027 1.3966627696966927
+1 1.8475295891856125 1.3887694988244523 0.6817244598020126 2.5809988844215908 0.32696789850689245 1.081015261872673 0.2386938164664013 1.0118382786145506 2.209217716205016 0.7574090447478952
+1 1.082260517720307 -0.6266070913930977 0.6832252128874979 1.2966340694320664 2.324615742379285 2.5627557774177543 1.72092865539378 0.15590225454118978 -0.2816198860581334 -0.5099568334403046
+1 1.6725629461607472 1.0353690658867798 -0.8225360006266837 2.1324720159286894 1.9885924374595836 2.537256632003289 0.9677496818620155 1.454681559021501 1.3029797950165192 0.26385709812366753
+1 0.31156560050102955 2.1652814753810112 2.0058163682540036 -0.04562872657851469 2.724179402266973 0.6222125728521903 0.42811650448637917 1.0387953213300416 1.8914700820960233 -0.5893540202775569
+1 0.2578251741975023 0.11378011266272059 2.797638612913183 0.13983902653928637 -0.03255261699221346 1.2576586825716858 -0.6642415184742925 1.2799765368331657 2.3385679931813983 1.8159437052025178
+1 0.33578001261352897 2.0063591095825952 1.0807987120174516 0.3543665780473314 -0.4202063816731054 2.113462588586846 2.306817160855979 0.9446592793327631 -0.6774687350899611 1.6189786930902486
+1 0.8614448755152566 0.27807051666810034 1.490952308696544 0.42812809570277155 -0.6130395196516234 0.23931476380563366 1.3454272824526288 1.8553493467683078 0.7262585485463864 0.8060386596767135
+1 1.509477780297391 3.879562737499862 0.5886532526077162 1.2655619776606024 1.3990929522583664 -0.34170560649024506 1.7418923966881366 1.629417743427085 1.7445593580979215 0.5930685838392928
+1 -0.17633273947080386 1.8278089865738787 1.6079874279761104 2.0641657251872525 0.0013949787963080107 0.9779219807727019 -0.9229761793545943 -1.0291570090345807 1.3628786284816425 0.5752391889181461
+1 -1.0143862085431188 1.1194733654329676 0.372026303777525 0.4779765819717211 0.873963169712578 0.8031044909741862 1.438202993892749 1.483386025663741 0.39707846786644874 -0.5347159094832814
+1 0.11016676987687668 1.44535659616203 0.47296285732106014 0.9569700223555272 0.22754986353621043 1.1107842631735818 -0.20365888995072612 1.7095423750241086 -0.848293390426655 0.857847169492578
+1 0.7508129008937717 2.8747883333024182 0.8289112296791319 1.5951701814113632 0.7420525998761323 1.9537834679324622 0.5603407250007024 0.6017647337718439 0.6431621236261322 1.7673108381156395
+1 -0.1852593368859976 2.2089214215364246 0.17988209448256942 1.720553251777205 1.2120857158218548 1.296273725719677 -0.25129199617788966 2.0013217992492613 0.5065314908683332 0.4536706566267381
+1 0.3257759973178981 0.17932720424930182 1.2245897173975124 1.4392674655132107 -0.19990974032801478 1.616015721370362 1.0976249377861196 2.286751487136163 0.5998423893372578 -0.10744364268832474
+1 -0.18860318421456523 0.6481395082246904 0.8471055242008172 0.8364035710726628 0.5027181893375049 -0.04737632027053729 0.6081198234429218 1.8117061812925739 0.7882062608326725 0.501707612022315
+1 1.4843082385614745 1.1158750459458913 -1.4894665738544455 0.25826376510509763 0.8737547870296022 0.6842381688703825 1.5781821909490459 -0.8859809290045597 2.6448010296898516 1.0451355125183155
+1 1.7920903749688475 2.181377042700981 -0.2580670741698272 0.835878310743556 0.8282113555574907 1.2918481880236576 1.2845735763240005 -0.6226879211726246 1.7452863581983848 0.35415213876681106
+1 1.6059906951044978 0.5477408796911678 2.033456301629621 -0.6056116844976043 2.3157299435817342 1.0282347361444912 -0.37895653151562936 0.9752299146785057 -0.41816188526715736 0.9125445080555991
+1 0.36434340752558814 0.6902917518300258 0.9253611225661063 -0.42114130346772227 2.0970094095591443 2.7085188507498557 1.4289293922116237 0.9542757519821615 1.0546374187652479 1.3258156303811686
+1 1.4902539943349453 1.6573630488454014 -0.3809764834643814 0.9358657723296077 2.7348124001551435 0.9897672456356681 2.560439397267852 2.494870519932018 1.6580041060544213 0.276867359286432
+1 1.1191344811462158 -0.6181668923123884 1.5490411146166472 1.8183809809806493 1.3028570357467482 1.486951380254144 1.1831247980434945 1.780974941037947 -1.827510680099897 2.305550677513012
+1 0.849190160180726 0.927714888220189 0.4152982301284849 1.7201547897444616 1.0010482110516308 0.47888318535920815 1.7303425098316922 1.5212540746719077 1.2164640343110604 0.8672666819224022
+1 1.1818789164071632 2.3299574339825355 -0.2238086965126307 1.0866668603828966 1.777789469252217 -0.2473412361708398 2.4917056426594892 1.0985567817486692 0.8205900594343175 -0.4507497282180284
+1 0.4806312370873962 0.768849921524061 2.2816919830317324 1.8888027374056304 1.3666588628364746 0.313010983641146 -0.9582374160527103 1.7350822166838902 -1.0292285073997203 0.6398099597089605
+1 2.387963695369674 -0.5899448356258876 0.21621305588176487 0.9380272998222627 0.6981388782356867 -0.4629800914467903 0.7722932223610299 1.5585013561079406 0.39398387576565874 1.605900840338324
+1 1.2715952476157897 1.439635629557708 1.0983640636833376 0.9812043919910073 1.5353214720014243 1.0984936772644822 1.1502708274998623 -1.295397653899192 0.2861064908535764 -0.9932837563816654
+1 1.3012696782417956 0.7849306120035814 0.5043907367704977 1.317902271109904 1.2355512152607722 1.7921035283313613 1.3780045579049331 -1.1334086181295735 0.7594490553748667 1.2920327236325173
+1 0.7390703584602525 2.457743695195635 0.3128347254263576 3.2777913748283356 -0.3729594628152144 2.2165912805252592 -0.3208945778133039 0.25945266028499947 0.12129953303222862 0.9577961880424101
+1 0.8445123778336028 1.4240300974070288 0.1873583546229668 0.4955218063785525 0.9094332296150236 1.3540661068354631 0.9171697258910753 0.41888437045897486 2.9462218414395487 0.6502477720645555
+1 1.3877586550503413 0.987611562870769 1.2584972385417663 -0.31990526604547664 1.8690834901315843 1.7043650395994414 -0.9964092334530854 1.1408598689320075 1.4213381391949258 1.3073798077919028
+1 0.06076427697113995 0.42120236957849067 0.592901981159774 1.3720471193027384 0.9036775292098581 0.8953372123185973 1.5452404312257344 2.0708178196722606 -0.8979750106430204 1.6853058787444881
+1 1.1694470503331111 -0.7289698765725721 -0.3241777565346444 -0.02733490335945188 1.8863228847530946 0.8073024667207529 -0.9818689747023401 -0.4283553318571569 0.9994871828689351 0.07075638531545037
+1 1.1047596078086386 1.7708874592017232 -0.1612806069289101 0.08556210685307786 1.8572899576629136 0.7200423074285855 1.2170692625583286 2.0347880443589847 2.7432017121214005 1.3957939162622077
+1 1.197861378414133 1.556444574585297 0.629813576730021 2.4550574210435823 1.9226732616821978 1.9859797173418605 2.186728551603152 2.221928254196631 0.8555508774400884 1.723787004755138
+1 1.161571044817612 0.07979292393847359 0.473025751301427 1.205676831999432 -0.5466232243147817 0.8191419439472176 1.0060075056738604 0.785322530707329 0.22058837011880694 2.6154680787761726
+1 0.17077134170060482 1.1137337091671946 2.318497500926356 0.3973424625226393 1.461779582118195 1.9295571893710908 0.7785519323891255 1.0672230065462434 2.1223852587473258 1.5460766694219767
+1 1.1564652200933274 2.510183232201066 1.6891434345580443 0.13174662119947889 0.8871123877951895 1.4958243544578553 2.9794729912305575 0.901901296036228 1.3871706497633103 2.8969924652525334
+1 -1.0521680406383696 -0.0031861766791221324 -0.10915897400357322 -0.1303567225640898 -0.09337344840645234 0.7148597244723245 1.2180327568998717 3.4184983500514545 1.697740318234704 2.002711960184084
+1 2.376709016910577 0.958001009693663 -0.1081121213002203 1.327468223880286 -0.41205779656829145 1.4289978911250902 0.9819807423748184 2.3188491121493113 0.8657078618437748 0.9391669120890416
+1 0.9776980417955967 -0.6674206197457981 -1.5563935251898675 1.5446269906729104 3.047754956305709 0.3970621484971374 2.7173431471851766 1.7243005353672034 1.9755492634674017 -0.7077753665556163
+1 1.1671355902086602 -0.8193057764678835 1.410567460875851 1.7497653081783076 0.6901637048786208 1.2119799048759736 1.3226344341934888 2.2695811100443404 0.9907324730003678 0.5558635315480431
+1 2.4336171222847973 -0.73180099697987 0.110963544711143 0.2466617891220264 -0.8154643837784403 1.7051343160057892 0.4485983625979719 2.319215306602568 -0.5223921322733727 -0.05099278306658839
+1 1.901698041087508 0.8988295187852892 0.6511477798135669 3.0420349436695076 1.3810269156306683 -0.24628147854970273 0.5188524250377791 1.4141097609090438 0.24777660167964255 1.535797527794107
+1 1.7629403294957187 -0.13022007315691875 1.1647647804960592 0.5890754693324485 2.06533631915097 2.21452694737647 0.673652898562904 2.2005666335367784 1.5261645592168471 0.9017580067794544
+1 1.7376137405520378 1.227528622148764 2.1537333953075093 -0.7244714994487282 0.9737436380972475 1.1956909226237713 2.612848244020281 0.30122025453481716 2.973720741303093 1.8186667174448368
+1 -0.2742361456988558 2.1098716503801613 2.953664212753427 1.574905508426148 1.8552665501344494 1.321110382365208 1.7445198966258182 2.471288236145563 -0.11919705782427648 1.8624551969544791
+1 1.5436386497853212 1.8153339598609863 1.363613793156124 3.0510249899073756 0.5489376037189108 0.007578350689908864 -1.1820947864458877 1.3011272158310803 0.07518458687451968 1.5312667541972245
+1 0.3224512020283108 -0.2209974586026877 2.042104637824572 -0.37728305633852743 -0.5498729693279798 0.7193283373851307 1.2590924907118073 -0.3944236589332939 1.1250230341812884 1.4070211742408931
+1 1.1444341603579156 1.3629504333367566 1.6939924628296188 1.9479380654467797 0.7894876586788064 1.049604859005768 0.3408015558912614 0.6014994900100508 1.4716224256141708 1.185118554114717
+1 1.5859690594959832 0.30570898129196966 0.7464020043785254 2.2285474871009723 2.412881908798376 0.6904305558007539 1.6192643153889568 0.5920043651364744 0.7807197394828229 -0.20297994754139137
+1 1.2950387623080977 1.0916188301034222 0.6600573067651259 1.862615598644322 0.6876153259228353 1.1481594206078056 0.8784422750187779 0.24715809175194348 0.7857238169348668 2.1619479520100247
+1 3.0828763562487733 1.7362496731683166 -0.20896157853930264 1.5332869652046193 -0.21794910668079526 0.9202735211245334 2.574049390833994 1.5268503392385662 -0.38999953644207186 0.22479935308805854
+1 1.7627009184421887 2.2255381870678437 -1.016295091642716 0.6254801643275638 0.6618861479958897 0.9047308122786223 0.852721929456685 -0.7505113940627413 1.7250343985280407 1.8166918481323084
+1 -0.5022420621997736 2.733043970376204 1.5120949360070959 1.9428063677250476 1.3780749670748853 2.2350181236519657 0.8716131236741619 0.2782380235553522 -0.297799811324456 0.16653587974789763
+1 -0.2981918597327633 2.860715416679886 2.1275708273598566 -0.29508534819399324 0.846188811185981 1.8713251354650118 1.0723090993878512 0.4374636574396571 2.210140762205574 0.6809712558014431
+1 1.5619715587750584 1.2704149431309402 1.9712386149819312 0.026280766936758293 0.8206955786918028 1.6318403698412411 -0.5566358146889887 1.7571793612461013 -0.5366638533754291 -0.040269040641153
+1 1.2643496455778207 2.038185139306229 0.6395741359412223 0.27135915089505125 1.4201127961240902 1.5041067668659303 -0.09091064494863543 1.109133071144227 -0.4794905621068224 1.3208155875591663
+1 -0.02895244930542762 -0.49403509214487396 0.712435362084801 2.5460059356446374 0.9396714328426592 -0.7949960754019478 1.6183020075071732 -0.38577084963397135 1.6991710568290967 2.786233832662353
+1 1.261753017958196 1.0918709535770748 1.1265646053317926 0.9867326079450506 0.8288572122803143 2.4418772115091816 1.0454798487585901 -0.19993011811143235 0.14523995518141886 0.866687319252661
+1 1.6985511320556277 0.795437122527888 1.556653786587669 2.1174479278276426 0.3999172845317358 -0.5010796653100276 -0.08438438589923591 1.1138001295987414 -0.30602571964029956 1.4972214829613484
+1 0.41786595805108906 0.6459011706826348 3.657046684462284 0.8222874793996409 0.050062147599186035 0.23963259661744873 3.98442324525362 0.28119552752146837 0.8964441562070578 -0.253526879649719
+1 1.4488020919552733 0.8929138056330631 0.3161270487767218 0.7331766954467245 2.3366307109566495 0.6815405492334983 1.5281435010244593 1.6431760386153362 0.5321346633571438 0.34130859830303917
+1 1.2748486181912866 0.33303368481427886 1.2151848478627916 1.0756517104783787 1.2083219051593854 0.8277625946461055 1.9666455377419778 0.6651325140447175 0.16327294989918317 0.8603717402697098
+1 1.5090300715612457 1.5180463731650495 0.6972598598076571 1.3556192196865902 0.9126434148820246 0.8127664907242128 1.3311309435526322 1.279157714746425 1.7829837559894246 2.988071791570289
+1 0.2727158735259818 1.2998080669104182 1.5121347623238246 -1.5679984907159152 1.515508708019623 -0.15391403969184858 3.1311081089984323 1.847318459389865 1.3425374198002933 1.296082544224974
+1 2.408189206457478 1.2760154921881726 2.1197548437178906 0.05936234352435599 0.19907763560203529 1.5479638808770004 2.471816233765586 2.4680208521093805 1.4113824572688618 0.383801428379995
+1 -0.17965112079351564 -0.3404976625536871 2.7837262771738205 2.6881515223765398 -0.30847324983815394 0.9993265400000024 1.1374605736665502 2.2049953998249694 -0.2513007616550551 0.448830380725894
+1 1.3443693966742452 -0.025711889743784466 2.2443775230207503 0.14834884628873723 0.7271367845373308 2.4714407353590957 2.562158361402452 1.7047011572226343 1.6769293581505482 -7.308081317807247E-4
+1 -0.41870353312467423 1.2877545442386 -0.3164789161896502 1.803839696410392 1.008076378658354 0.10616668976164723 0.4098865481816575 1.146539676959654 1.1538344544688937 0.05907242504921317
+1 1.7936911543812046 1.485342520804878 0.31800311694795325 1.9199555201066274 1.9312631279902837 1.362366670774782 2.6306006265218365 0.133055817623004 2.5078649689837027 1.2068433004457952
+1 -0.1411582634165307 -1.0426813196108524 1.434523926692467 -0.25113509019608093 0.507539296016366 0.23168671363927917 1.1893212121098466 0.8304584451378183 1.4556473134325054 0.6534542423873613
+1 0.6079927716629916 0.09194609771904183 1.6120179701101955 -0.5022953903177365 1.2170945269028797 2.100831302657739 0.8386155807612904 1.5684558466558434 0.27605209581418555 1.5594274213225667
+1 0.07428493649230228 2.293483112741116 0.9708779280979398 -0.45177079067335923 -0.057110219872378076 0.015433876379835065 1.0794154562045615 2.105620271870406 0.9395998613200235 1.2851835351116119
+1 1.578883010870155 1.5609283984502076 1.8223960032380064 2.2142614021520837 0.7130462722633009 0.9252426132551667 2.868560600039225 1.6968141988566166 1.9976720397763048 1.6813323051682774
+1 0.5016495406992045 1.04908195692884 -0.07722896372502253 1.330713406245241 1.1267715047602667 1.6360574586472572 1.2420706446269942 1.9672850660325922 1.054929403781838 1.6077148722801038
+1 2.0538334867970534 1.9213949071716163 1.8934373144800345 1.2381794078176593 0.9175279056098742 0.8206265873347616 -0.8312726444851357 -0.5131966390183769 2.567300850622103 1.6719008505918898
+1 1.2689208746241893 1.4402293624087208 2.7176532271741003 0.01336457957384174 0.1702333910599565 2.3778902914738547 1.7217780353501682 0.7054536312666535 0.3361164972231122 1.1589949811743772
+1 -0.5767062059491888 1.7138887496399136 -1.1154021033816348 0.7168636442060621 2.217046440509127 -0.8161420769580656 1.6271150941587713 -0.09702287214964955 0.22946937882986906 2.7922011937600097
+1 0.9710624979613078 1.5610147329117985 -1.5053608758479413 0.9711728502628203 -0.5150150692664308 0.49562546380947603 1.7163450863443273 1.306018285087743 0.5473958850146698 1.8540315462762198
+1 0.6425941154359618 -0.31480994520520533 -0.056642174933536404 2.2269443093694914 0.6505566385114631 -0.3709635056159635 1.8873810442041976 0.5119563367121428 1.291713540770698 -0.6943082761794022
+1 0.5927308007246384 0.8464951673655936 0.18447571041818456 -0.006190250203252257 -0.012631850494107644 0.81828806055344 0.03231106794400085 2.0927752513240994 -0.12600012916564518 1.9639580630933335
+1 -0.34831756463523855 1.623268907572022 2.1594197097470325 1.0562200902265129 0.9414684460546705 1.4340305236290405 0.7654931413466368 0.01719894816346723 1.5959585538584955 0.2885792827923064
+1 2.2697657120238466 3.1420889453091094 -0.8210208940698709 0.2035264954846796 0.34878833066083437 1.3187569677046596 1.0219701238612262 -0.1213159939916395 1.0802611304225862 1.3078831016284853
+1 1.2480724077104584 1.9077146304274128 0.702946174596962 2.3286147355852034 1.0071749708265634 2.5149204905160154 1.349779745606328 1.044016863507004 0.365723895391459 0.6519926945711725
+1 -0.8985903846454402 -0.5021240182148043 -0.01073065243449256 2.290069714856683 1.9819036535789476 0.03105672582226615 1.339000036426309 0.3323749578280565 0.8021635756060409 1.195220952578341
+1 3.008655872898343 1.0129636641232918 -1.5088469891308582 -0.6947292093040875 1.2487527838514174 0.9032973743393249 1.9979774814850564 0.0435076158833696 0.8478193472405138 0.5026222405279126
+1 -1.0608662183020523 1.511703517053053 0.4555272804535656 2.076056547724862 1.754307244984986 1.3854010129660659 1.8247443481696117 -0.0246162652477655 0.24988078939072067 0.9872960257572898
+1 0.8740725946015646 1.7804072513374016 1.9060935705517543 1.8265003967793456 0.91953745409342 1.3629234354248754 -0.2803757506365385 -1.0129022749852892 2.5019279152710756 1.5245757538298341
+1 0.32688805354617134 1.6000098575767967 -0.1786618864414944 2.3806085458526325 2.3338676324290164 0.7609884113833272 0.1498428862635196 -0.25090796239660373 2.3770456932981814 1.6131488558961797
+1 2.290620763512112 1.3541047134925366 1.2421787622602398 0.8804930591189608 0.6595899728536196 1.6277353547734075 0.18759874372088237 -1.1351531086694964 0.18251082831485133 -0.5713204010530248
+1 -0.22047844715313447 0.8310592465340738 1.7892315227363613 1.1470591393757708 1.0726224455927464 -0.10592031044447459 1.9817888345656018 2.432077040490821 2.2450973493606203 1.3210707817547482
+1 2.070368262568201 2.3671178117141207 0.8627035047548697 1.366475314693422 -0.8331190909005985 0.7551440285820138 2.178737629795865 1.0323167492638525 -0.3148106607913368 0.50662477745953
+1 0.8604853943488086 -0.09592589897715587 2.600032474430587 0.9839706092809413 1.519739305696014 2.1260793286184008 0.03744939964524108 1.2611070446598698 -0.511324151442442 0.5454482162340912
+1 1.8946369523511708 3.362602104881858 1.8838436706953976 1.2491758602363099 0.0054680988441749845 2.651799339501261 0.6411444300353089 1.1035969889037076 0.8324869555591509 1.3031776807447846
+1 2.5154071822014554 1.6803408091264473 0.37434333648729623 2.496324926040323 -0.16401882096773224 -0.5744479735763091 0.9352239350517153 2.442683227544391 -0.5264039462194898 3.015307788051603
+1 1.5111987262832436 0.6410066045062515 1.0002585904405568 -0.8894537972030532 2.8014684904508944 -0.5393437655384221 1.1524079090931012 0.021728095470450404 2.1130698813482622 0.9468113077109184
+1 2.246571391447209 1.2010599601897547 1.234941576895316 -1.7706644509786722 1.471058855485551 0.8939500026890757 3.0844244960496563 0.3937694347012187 2.4529138646148967 1.1858907139355346
+1 2.4615314217465514 2.138799653615231 0.6155097299332213 -0.26863064780465895 1.4804373561575783 1.9409343558847068 0.44935568187190045 1.4016783544796323 0.5844124030092861 3.560614430022461
+1 2.170074376135311 -0.044012090187616204 0.4876588954783079 2.3603606696538524 2.125197091710744 2.4134190214591262 0.41472234938098607 1.9434029103795312 0.10273955644383004 1.235145974467383
+1 1.2969727061242051 3.098685038424812 0.9785969987985332 0.5224703037252412 2.5948178849934393 1.9056896554251344 2.1303162130115787 1.6936027246350522 1.591959269634407 1.3287905654720076
+1 -0.015989877059035873 1.5072072218307366 0.08389293810681375 0.9234581285114085 0.4320229724446347 -0.17718855392460764 0.7238001450159828 1.8397437251675461 0.9523656518925097 2.513817935317845
+1 3.7089889925376345 1.6027646547595036 0.30439608816889874 1.325556017740845 1.5649758448214102 2.0480467830712694 1.4268815678658604 -0.08232989657136769 2.0319641149268852 0.4859663282113227
+1 2.9299411753408178 0.6939333819644463 0.5980477746930858 1.1544643358350055 0.5988463132053894 0.8004691945155193 -0.7969681294710653 -1.246477065340748 0.7551153563842066 2.2320600943025157
+1 1.5618544649786017 -1.2039729275512823 1.9863936078958404 -0.7698679015907834 0.6433908271785455 1.7173978058694828 0.8771509209324759 2.664740793299653 -0.6994627263844606 0.6322436483068374
+1 1.187061394437512 -0.6451485516060627 2.476357446033039 1.7693108617562059 1.3697550089364834 0.40908284287939223 -0.5656163253633264 3.468763307766636 1.617455962016709 0.4894706139195705
+1 -0.4273229723387111 -0.26809867009452515 1.3843160982545846 0.8212240154930317 1.1784396971750364 1.872828424638627 1.3779623371802083 1.1888620042820783 -0.10589695125965615 1.4199981576509952
+1 0.12193951392066005 2.616540426567961 -1.337357835943099 -0.10743949585791679 0.3939788495591735 -0.02266440276523496 2.766246408329433 1.779318925725903 1.1626163281228863 1.1568240129972165
+1 1.4669291522156196 -0.8005956562590923 -0.6879775244399986 3.461310058748968 1.1339641121124138 3.0998254868058384 0.245952923446367 0.7214863675143265 1.0108020940282363 1.8538791497646767
+1 0.37376581529952313 0.3065031814805871 1.3343221577395563 -0.36245405167755473 -0.7157134718616156 0.9091314241626773 0.6213443407765016 -0.3159031135243049 1.0607486905684709 -0.2566933833287508
+1 2.0069622762472235 1.3555276909717138 1.3738458420384927 1.3307981771643953 1.1352058939547374 1.1872314739705727 2.0206074946330155 2.6193996043859977 0.9754506254457527 2.4788773949517737
+1 1.6559576152851871 1.5613387714537157 0.9820632656447196 0.24990370738791912 0.6790482468297928 0.7177001456270966 1.2177661518329543 -0.010128389509312274 0.9949778601566439 0.2730735896651332
+1 3.3541347870312084 1.8903267206950842 1.6609607533550115 0.6313086218186583 1.0174443932043256 2.1002778641752133 -0.7433879263515524 3.6635365130163358 -0.12072379016630852 1.2613991803119946
+1 0.741882011562536 -0.33389745909875646 0.49850980476986007 0.6209294892871532 -0.9345674636388526 1.0706987501267613 0.17174378573602178 1.4966350235504806 1.7786390376763213 1.6231643119303771
+1 0.737851271176944 3.1107332677301804 0.5595554860713969 0.03240910648046724 0.7418890189368929 2.5744268937009354 0.08490736311553437 0.9454019320976027 2.3004255005209213 2.673423266074501
+1 0.9964678056269282 -0.4050367214023043 0.7634512054670727 0.6104047048598984 -0.18420038230329872 2.8225484519075694 -0.17480506682904684 1.188578222519793 2.3609744942610704 2.0104954250932927
+1 0.8561825142599002 1.4715100244558175 1.1551932439330008 -0.866432954658839 0.06672467583391328 0.6567191940892094 2.1238239921343776 1.9236498444842514 1.774783717232303 2.1705643226440356
+1 2.1686685144492652 -0.46548035607855187 1.7905868508290022 1.7291739618095732 1.8420059988367683 1.2812869543894454 0.7094922226284579 4.578093325453002 2.159649972834322 -0.703298751877151
+1 0.01038121312435214 2.041036231629956 1.406313867978486 1.3944476209150578 -0.7450794741024422 0.36098991012411563 -0.8145936978526842 1.0085439903773337 0.6693692426324003 0.6121851518794861
+1 1.8571542967953807 1.4070713551879899 0.5321067816124654 0.6429601839486434 0.9165980917544774 1.071305634192637 -0.06040670535870918 2.5384035240078604 -0.21377477606093764 0.3369977088082866
+1 2.405103563655566 -0.4546855764355364 -0.24489042907792635 1.3318409806777944 1.2523408877207844 0.9313587923017596 1.2089956458520745 3.0921428523894092 1.956850142357836 0.7702767453893322
+1 0.9086347130699683 1.2100828227228213 0.5327052367165771 -0.6550532780225489 2.5505664076947587 1.4300751019325881 -0.9806442677198526 1.9110672232516768 1.956204319904626 -0.6406447989012172
+1 1.750246620105648 1.3081292130126525 1.4716986993259968 -0.3042704857661218 0.2354470475646966 -0.6074481355981227 0.9333801721029178 1.3220227127047701 2.0998355566318203 3.340047345554312
+1 0.8132766080998793 0.345182592805539 -0.08434230880799043 0.371975995128044 1.030128701009812 -0.0838490306566615 1.891400724652641 2.133657072232741 2.4719821498192935 0.9603084853474415
+1 1.426463569977554 2.123479869287884 1.8449734404123337 0.8841571967965259 1.3206820715765568 2.414835584218742 1.129163483268984 -0.8781190476518506 1.5162895167347454 -0.6528866908043633
+1 1.2017423534681941 1.9686754970835203 1.3014044708959847 -1.0240935923675734 0.7502387139905979 0.8253575777839712 1.224646644221756 1.480689489076607 1.7640815996729344 0.2056821278829375
+1 2.7250146939462083 2.227656483011149 2.84947399343455 2.451014425645574 -0.3739053762247364 1.1582450151950303 1.741290414111453 1.376435447217923 0.35033655530431784 0.4806336989868223
+1 1.3542581369916695 0.415546436380271 0.6688613033041042 0.9102881456111578 0.2547986420844246 1.378444594707075 3.43963729226003 1.3067301378198568 1.5647303411064155 2.043293980780698
+1 1.0913358352352922 2.1175733214306947 0.929020839478381 3.090469607746358 0.09151751891798587 1.5634842729294367 1.8016069710014775 1.4861336762215835 1.6076296539436097 -0.26097034661822094
+1 -0.709300017934053 -0.14570511438959777 0.8487791028889955 -0.3957122997819824 0.23663565146376286 2.66035473479832 2.1479897842790923 1.2106691413007877 -0.45712691497148206 2.4225765811823203
+1 0.14756832470608838 2.3704041393692425 0.6496201584931938 -0.11807063222136005 -0.20506086896030706 1.5881151061076393 3.797132222832481 0.943542745977901 0.8565267747881888 1.1864294682583807
+1 -0.3889342935852145 -0.17743324011571104 1.3604682904339318 0.6593714174698198 -0.3584830057001256 3.514136269889732 0.595913513718282 0.1683068614180695 2.0746193584112143 0.6903921573893614
+1 0.2920446897752229 2.9937346155977957 2.251247553131803 0.6975169699248711 0.4494567463916379 1.319277335273955 0.5367328026447278 2.5267557692090836 0.350600102811225 0.5606888320387985
+1 1.228653481176321 1.0182555282617969 -0.5982787788962058 2.6333900117968314 2.0366003161170663 0.5499289981699178 2.542904251265296 2.2146577311919637 0.3954898163391639 0.6205263945903541
+1 -0.0520426119593238 1.590564747318753 1.6958053948956031 1.3511042599706389 -0.047969026912866974 0.55701288765553 0.9263968623271992 0.590838546777129 2.3308650721102633 0.5135257132439688
+1 1.016635594241282 1.8948650280358326 1.440434304566253 1.4592759362683134 1.6827383192498666 -1.0918246492897437 0.43238661798429845 1.5624487435653098 2.220285861909854 1.271128145985624
+1 -0.7222589043422267 0.5115698429182437 1.3516909750379982 1.6184323538658458 0.3138663124851314 -0.02913500500520727 0.8551827087816364 1.6317432725857857 0.6646228309777373 1.886929067576903
+1 1.4628654761642204 1.8652907041028732 0.6622303129185922 0.7509202647315306 -0.036376585463356426 0.7850159634599014 2.2985430427240017 1.0460715145011406 0.8526933674534585 1.1533090709516742
+1 1.0669747034293164 -0.1510400394042828 -0.34893623474816793 1.7754617342041603 1.3436972220233374 3.022419531056307 1.9684180926734447 1.4858550357170357 2.9588700999527395 -0.02437800790558642
+1 0.5379644371164043 -0.27906681292084 0.3380177280655655 0.33722013060203193 0.6571438211538795 1.2052933591547657 1.7731403611930516 0.5077273284789499 1.5626883295465674 -0.050171508356717576
+1 1.2224363031291428 2.179387632259403 1.729844754655598 1.7261086434406607 1.6565721133198088 1.889839925928689 1.8345686999088797 1.051447084834809 0.9359370646456183 0.7645291821631122
+1 2.60292814182841 0.8804157611166004 -0.955075955060207 1.2946117062161222 2.107044588585438 0.2497683006856819 1.6038124754155476 -0.7214552551237594 0.452098771396898 0.6986965061465407
+1 1.0412661702670807 -1.3958762787534025 3.074541266637782 1.76411325380808 -0.39903368929064653 1.3136620541582826 1.1746725568355456 -0.6576469095064521 0.15286303171879478 2.117286307501297
+1 0.31859147805604837 1.2450573919933268 -0.5933863589583486 1.616822450960686 2.3307511175574707 1.4675892671924506 -0.6797208500497198 -0.6357164936808151 2.6616070340209608 0.12503414768311838
+1 0.015640995722970286 0.9521770024879528 -0.021136921124242036 1.5781474391889052 0.7227013060272598 0.7987343733885311 -0.6768705185766593 1.2194260902982417 0.6115575336879959 1.776636860101025
+1 1.7473265876837165 -1.3416662707254097 -0.3178957317552682 -0.7952748363966 -0.0012367493892466719 1.5102140866553868 1.3893554303705593 1.253090374551591 0.37849714433826975 3.8427708908843417
+1 0.1249935088342321 0.9175321556781342 1.2521433252052363 0.10448935908110157 1.748729859258747 1.9013556247400216 2.348145639899152 0.4626753070549736 3.7821319980165344 0.47822934584228827
+1 1.5461491524524733 1.0442419265941036 -0.016418025211677234 -0.6189521317249826 0.9719604409404735 1.1409654487054224 0.5144932080563054 1.677400744669605 1.60852217407324 0.9996875540653996
+1 1.1571589981163284 2.815325710919601 0.20772173229184132 -0.27577989741307296 0.14104944330527658 0.2590225341905401 -0.33859238160667027 2.803757221911037 1.035764969030257 0.16925873998127916
+1 1.8759906736161591 -0.7858122581388844 1.0848147823038492 1.346569014348389 -0.7811951242276918 -0.28091748058441146 0.10734544787850497 1.1946024654289003 1.6406107469177638 1.418186454569726
+1 -0.2974414971504451 -0.7263225506198576 1.667022614186794 1.1033345452667596 -0.2451904831865781 -0.011381119202380274 -0.2081120315941396 0.19505925177058225 1.083883779309256 0.2476147974455678
+1 1.9875844064011776 -1.0551408447589177 0.9235522752742322 -0.1465157757078015 -0.24048981040870454 -0.3751333753617203 1.6243406244366847 -0.38149309424785227 -0.2845380129435624 -0.4586888921471284
+1 -0.43391027275254457 1.3012041634540212 0.34931152784647057 0.2724840573311986 1.895997027401461 0.7955372939424181 2.717841382622603 0.9983211958138658 3.297958269369362 0.28612843397709364
+1 0.09388869926828014 0.7292780962393748 -0.48425219833973965 1.2122506447105803 0.7074049606666732 1.0448613427298579 1.4758560188256675 -0.32361188073438485 2.040268428137505 1.685468904484563
+1 1.0792167846288987 -0.2826348408764243 1.3133025554220168 -0.29264376303967365 0.12334584816456384 1.7916405818476433 2.4401329350478367 1.373668417749465 1.1438238823893943 2.9513159396946955
+1 0.6272602458353195 0.012788348875383604 3.339583303835828 -0.5656471248096915 1.7436358009297308 -0.0849133378284781 1.8766630914593128 0.3286471991737121 0.8557785757636693 1.204343384424849
+1 0.9053623358277365 2.851790381485327 1.0805997920016692 -0.5635383000263379 0.9576644151670836 1.9289302434370748 -0.13805339731578536 3.4861795141210807 0.2005081416731367 1.6544819624039082
+1 0.4910096613955415 1.6681822364133903 0.8202936721704033 2.148200954440342 2.558162860929867 0.6606047330906034 0.7989603259919102 1.0689702044523541 0.7184320065316048 2.023034231513219
+1 1.1256411487276385 0.19900785835501755 1.2085575135898547 -1.356418780267496 0.785218957218392 2.70677848091574 1.9987708656840728 0.6868097252341125 -1.241646154239319 2.9393145029129917
+1 1.9337642982267669 -0.7156557544578908 0.16408179712477566 1.9408268646309592 1.0190820244131475 1.1951052545533123 0.4481509783235238 1.2668590723499928 0.8102310436768919 0.7718152165895394
+1 1.614923882092461 0.19469602471151815 3.766869874799438 -1.3377164159484254 -0.878559530240216 0.3364262245077355 1.8010436667360947 1.777688731609198 2.311140988026292 1.1771602185088652
+1 0.6784758917678138 -0.18464751605809093 1.6835398190359525 0.9616873095363908 1.8625881930711616 1.9970275330538905 1.0465679673330561 1.7874857759504277 1.7797672480031759 0.9806567017840313
+1 1.9543101838028707 -0.44413349405470304 0.3787949477054693 0.09081285199753486 2.460919892284841 0.29445632839265967 0.9120233970904723 1.120046161146032 0.3979415181383884 1.6677498018942478
+1 2.7931886788791984 0.05569901049144255 1.2190718219058607 1.3326923562520578 1.7863786156200971 1.8057619970370333 0.9782497583237075 1.1631245252370526 -0.10647683276082942 0.8291413719741013
+1 0.6746786109931104 0.693150020176567 0.8806942321642721 1.3171663922040504 -0.18964506284133353 1.752816912385852 0.0197418639082243 0.04087366490530042 -0.31356701603876047 1.1688888267402135
+1 -0.8047119894089716 -0.19086822099982692 0.7230280053386025 0.47661575325565886 2.783553868954165 0.39034536568120837 2.4620798409550657 0.3460544872000194 1.6811241975213127 -0.5755589941181993
+1 -0.43736971419082993 0.9731234165917454 0.044303702104787734 1.3285736602137515 1.8134256070231687 4.003995098206477 -0.5823423595861437 1.1000778881670024 2.275332508162996 1.7059404281570498
+1 2.7870499907770374 1.5359115092914868 0.4415840592158585 3.0819184178594012 1.0142235114013434 1.4175457438753696 0.7830675289154578 0.718110803107776 1.752603937821668 0.8681755199560836
+1 1.6629646464798866 1.5720752857585811 1.866918319229821 2.011503983207959 -0.08953127029042407 3.250764941529524 0.8681970712263898 1.8122090555675 0.30361209115382115 1.6190898270526166
+1 0.8689387257925889 1.088532128821611 -0.9638248404112064 -0.03629852962978575 1.5819544244821397 0.533196869581712 1.1629368405935705 0.5952984584910554 0.5901966383762997 0.8680425050414964
+1 0.5657393409043414 0.1269546832382663 -4.0341609669503065E-4 1.1489057321179976 0.25156572912668473 0.48265829258343707 1.051802672080171 -0.797907065268961 0.40336920791124586 0.34951103336108913
+1 2.842259431863403 0.4523061399118463 1.1073417696817962 0.820613792637092 1.2347466769629105 2.445490993196761 -0.1542908283123816 0.8816264920520589 1.7423151819076375 1.6594291913667136
+1 1.5860855260228202 2.8392671863491734 0.5188572450043611 1.047507505252711 3.054126605012979 -0.6006852937930467 0.34982369626834076 0.11607093207054109 1.829510982388106 0.001994427476862848
+1 0.17902283956677512 0.41558050427565774 1.5871923905064695 1.5996558530208187 0.07353003075760078 1.0705630115074813 2.675599132354674 0.7650850730679759 0.8607570887706816 0.9903122299033713
+1 0.7379554955291575 2.072325148209555 0.4462636170973716 0.6880836555742617 0.3535374515580053 0.19240929522338934 2.2791306741261153 1.7199300904991563 2.3790655960655718 -0.4294392660855837
+1 0.5642895627754023 0.9044762545519158 1.4797756442552041 0.6976030137900451 2.5013240752661825 0.8121543920897196 1.864316073466811 1.3213558088397361 2.17814424984865 1.8979547805463015
+1 1.103147738753372 1.616958446219673 2.8479619253624797 3.368348617090012 2.5438833831666434 1.6704650810547208 0.8562521160479526 0.7542938264829215 0.5266574196400498 -0.2890730154742367
+1 1.9142555817765898 0.8049202262783679 2.5019528805928912 0.5238106873271193 1.5359406981988988 2.8356323728714847 3.239716573932437 1.2510518752596296 1.715571851101242 1.222780980267732
+1 0.6041885893884307 0.5707299204297884 1.2540953158421435 1.5510759633503302 -0.4667440237195346 0.26676051631424014 -0.565572799800238 1.4387028778945943 0.9644694652315191 2.1255685675532967
+1 1.7491189390587218 1.2227275279214738 -0.8505836769821726 -0.903216529384467 0.29076052330579005 0.2892222629138922 2.3647508720986217 1.2652921314867005 1.0348376197540348 -0.2562195481430878
+1 2.3800831934663433 -0.010431805594117938 0.8430880161541389 1.278733772829872 1.585905177486663 0.28093811664192425 1.5849634563502026 1.078413585522204 0.4426572711340797 0.6530352928058241
+1 1.7049361022681717 -0.27653366462628215 0.9445796767766628 0.041969783781791725 0.3467762982688263 -0.4874473134901387 0.7531152429497019 0.30167927793354254 2.765258841783637 -0.23618185513880707
+1 0.8097421163995955 0.17729598233902988 2.5214858992792863 1.5180096630697852 1.9899028361613595 0.57436615658855 0.5307905908280097 0.9190155285250498 0.6466076660416842 -0.10626054479014013
+1 2.395022852849255 2.3321432458593208 1.6804528385827555 2.2258435456318937 1.4611936535655663 1.058998523699314 0.31838562794784586 0.39659928716273496 1.4494935872166117 1.391374864616476
+1 1.735291612472487 -0.3191446365558481 0.6607372043463824 1.541446196262466 0.4947578059034823 -0.8293819909066149 0.76596276473359 -0.0851263113957168 1.9200627040331277 1.5173271962047457
+1 0.48007434755469713 0.7936351950677151 1.365699852551887 1.1109515050883414 -0.12031241802004855 -0.18610833660205306 0.2974034656359261 1.3687489920730513 2.1059823724132523 0.941953020877809
+1 2.4520203316077964 1.11003521338105 0.4722773485870979 2.737384705503226 0.7192036221774767 0.6242245483941781 1.2609692406366446 2.0575095746651133 1.3495884659991346 2.0764197346896935
+1 -0.7842236897873944 1.492890069052242 1.765349236922137 1.300042277956386 1.5799338298744416 1.060819121020154 1.1674652333797013 -0.4149263766035056 0.09348961754442264 3.5461008823168543
+1 0.8620605536733185 0.08406312778559633 1.5415557685694021 0.2051913612441839 0.19504752604759068 1.534576255114414 3.107649420779101 1.020214612080108 0.3221723632541289 1.4874661690065234
+1 1.489728417116672 0.06558708406688907 -1.8670045751011424 1.7828483838262912 -0.683023788962926 1.79761793764676 1.5085129455490893 1.2434470961660735 0.5774571270514824 1.4932340982697638
+1 -1.5669127739356443 0.34356934741624334 3.0594253296534424 0.774762761699532 1.0055392162451373 1.3241023069988664 1.1749986092813367 2.19297533155391 1.0435550797072737 2.095514184709966
+1 -0.3634276403952408 1.4409978371532932 0.3823184763192483 0.6254885387609036 -0.35123251562864244 1.819196851350437 2.14116717870738 0.46320929513337494 0.5695755038115515 2.501714843566015
+1 0.013632028077828595 1.8215490521966027 1.7653867346915684 1.4163095749484134 0.25841398470159227 2.2048024054278192 0.9286824219992222 1.133706943250312 1.7330998187732773 1.3552028632095436
+1 1.012536342646575 1.4202805284853588 1.1660963924281333 2.7434608590955594 2.405339566810934 0.35678139532687714 0.7007075773809261 -0.1461824532706133 -0.1116775801341563 2.455669156783493
+1 1.7224210079670872 0.25824562782106875 1.896388948392676 1.5490245294926566 0.566495628127113 1.4439902246901806 -1.1659487820432086 1.2648317293133733 -0.8687762383751962 2.055108054071261
+1 3.5125527162365486 -0.022436189584495336 1.1332983732450903 -0.07450694962415794 0.09001591132041731 0.5853417525905302 3.337681467433381 -0.32222401787392774 2.539181628048545 1.0754745872100386
+1 0.2455099848454918 1.2693508037734986 1.6546347888138584 -2.148792530729241 0.46441142559185566 1.1734134286137057 1.0258039884088828 -0.5586646913499485 -0.3258731206571115 -0.821219883870792
+1 1.827217125452903 1.731864545109457 0.928872208086588 1.2056977735867256 1.818214291632629 0.6585878488136441 1.8002230735809155 0.8708150904043206 -1.5838120389612023 0.8585857536471672
+1 2.2021363682137154 0.4761145331093257 -0.025920931323458296 1.7449566792074553 0.8629966232032906 1.4723084204343524 1.6159540778305606 2.029453834185225 2.26325946376582 1.376244768900244
+1 0.010342658978543584 1.515273076994554 0.19611635441491626 1.654784841440513 -0.033943991780339244 0.6714632219862774 0.2641936457650498 -0.700825233754335 0.23452605282080619 1.621398184902529
+1 1.0480165819981573 0.8797819263901776 -0.641443663240362 0.12817609127433438 1.3647120235220283 -0.48615470921060977 1.0720144074421256 -0.38026314794700733 0.8069083073855456 1.3433152284915995
+1 0.3761857330260455 0.23219703324626284 1.921560210024654 0.38896862067672255 1.1468761246542036 0.8203362705962437 -0.23996402764305458 1.5950906570841252 3.639574852127676 -0.2443366415192889
+1 0.8759552320204246 0.33529291747248857 -0.2551391418074267 0.29090645845832075 -1.1529071816719476 0.7412858224772877 1.2719555749592364 1.3289131183268248 1.3228711885726534 1.5021325652417783
+1 0.439646111605676 0.8753273571625453 -0.5195310985749739 2.656469182704334 0.8907416242841371 1.4150606950578886 3.175298549230411 0.44910268745784754 0.8447367653706002 1.668648718911232
+1 1.1404102468865998 1.4857266483300324 -0.31291554366933605 1.3205568580259288 2.4092775306975023 1.6397731783027976 1.1251407071414252 2.3565497583137436 1.8353622317028138 -1.1683108743275552
+1 2.08122023149769 1.1571239260956436 -0.08056173908716335 0.768249986206349 1.3171573148662759 -0.18023949555734187 -0.25107977208536614 0.3528408329964078 0.7749381509220793 -0.7113421449812265
+1 0.1473845257811165 -1.0521567114122852 -0.47637816156748225 1.4949699096476212 2.271087115324705 1.3826153478446757 2.7436405167916025 -0.02075677223859529 1.1765040243159015 -0.025438785956181542
+1 2.7027482205114826 1.577562959861571 -0.5669337503778331 1.5215534981321372 1.2652067920381662 2.7463387790797444 -0.10995208915345178 -0.9887358827125716 0.7108329384066776 1.3629285100379036
+1 2.9573936017540556 0.1614860515756119 -0.3278573695860796 1.0550562822356224 1.4787913549079965 1.6928275048278305 1.0586362008998798 1.1651361732301 2.361382321862904 2.524722697822938
+1 -0.918683252112166 1.1912188403555544 -0.6386682219001243 0.12852707081177273 1.0186959070915036 -0.7396656648881279 1.390222924345315 -0.6776334559974988 1.6871484268646286 0.9835794195231572
+1 -0.9501651670329723 1.6369415588995389 0.6124916702658543 2.055786019572368 0.20091594691375603 0.27955238962400497 1.8462485957757835 0.766850497882725 0.6439523544318226 -0.45529021581249385
+1 0.08294835997064665 -0.27721496031157833 -0.35456350243850276 0.11228054309930591 3.4737188479123104 0.8438116500646802 1.2682583387249549 2.2187948258289913 1.6181904099869335 2.2762749025565983
+1 1.83339856452743 2.673091344347915 0.7389331991568107 2.067911927048983 1.3782410940205578 2.030974790626103 0.6888073746059981 -0.518101069445974 0.6230936256620102 1.633224100697245
+1 1.7398691778151973 1.1247533360425708 0.2807774763651275 -0.6955611341182046 1.592036824083598 -0.04050352181158767 1.3865010706574772 1.4019929481612587 -0.2642443959402707 0.5934301817863643
+1 -2.019173847473457 2.1681048424611418 1.3422907243645614 0.6467676712250852 0.49642291457381404 1.289806437146178 0.5287383514431835 2.8692305624115457 0.37484482468477054 2.4484351720405875
+1 0.024288362749408376 1.0351720632502537 1.6837605528916666 1.3872579136738206 1.2679651380538202 1.4021182744167016 -0.7041852642469104 1.6806756125489901 0.1307750250742319 2.3317291973580314
+1 -0.06080175616636896 1.0543357215752764 2.099562273809995 0.6174473985800795 0.5458218639483579 -0.1330076265446425 1.782807067124061 3.835868752952487 1.0749746574622228 2.2318191600680155
+1 2.7819388327740797 1.1294517177544148 1.4625685601160094 0.8160359631571115 1.5866067958993928 3.0076062737914184 1.5740992429858394 1.3901837375360109 2.7120095549614893 -0.5329184800190412
+1 -0.08342899095133993 3.2552165445304735 -0.6127389181137219 0.20728621073827602 1.1715077138725913 0.496873621214974 0.7991470651533773 0.5625481785655475 0.7904628851956959 0.485293468158445
+1 0.5879363673253968 0.5480289705171163 0.26878358296170424 0.9493365691333653 0.34421794272116246 1.4045876345319372 0.8323003475233924 1.3822841645472739 1.9408510354113169 2.3160979297534636
+1 2.049725023995715 1.138714228201635 2.228635558152831 1.4833354495511806 0.5549789742701208 1.3850264438047617 1.4418684507619366 3.131909530291612 3.2277156524053705 0.5657214292376471
+1 0.7278339716721132 0.8342775647290255 -0.7804056350094557 1.8999099617115354 1.5129989349558883 1.6238396258236993 -0.13761070763179828 0.6429461405182848 -0.2642956636249272 0.8065034962137944
+1 2.5931023834096854 0.9018261137939111 1.5584456516926881 -0.5802390356360938 1.941618818488975 0.9214260344294213 0.556884632504891 0.26832249168681577 2.4966263079255677 1.1243846486761992
+1 0.14419967158797142 0.9874339005630041 0.8076366869263152 0.515723994659785 -0.9385248237540935 -0.17924876736882722 1.1150091706474443 1.5543894995228547 1.615026336442979 1.1708620595483625
+1 2.1530687310737866 -1.8203657185808888 0.6380519600335401 2.02809789647314 0.30946138948160296 1.7692953099290327 1.0369557864170398 0.3326256746163322 -0.275581422683832 0.21583516634100164
+1 0.896534730391731 2.1309314580821708 0.9688774738233893 0.7810503130534793 1.3417441924762596 0.10748935054015485 0.8725839981470569 2.68470748226214 0.5000051011542708 1.6309858671990054
+1 0.2798388059875424 0.46301766350582063 -0.21330838748068315 1.516256000433057 -0.9521989902404524 1.8668922242244914 -1.429783656173199 0.24500379527846305 1.0717746705573634 2.929223328366103
+1 1.5580038958637812 1.4690967454818293 3.5043865357520065 0.8077006250670602 1.70873452721819 1.725133865080763 -0.17803725982825802 1.2072416111273427 0.7258484330322263 0.9666451576387228
+1 -0.2937927716783808 2.209449837105502 2.471323239279583 1.9931843786987273 0.4670001618859797 1.2200671907651737 1.3884758303330187 1.1014939571310298 1.2017172341718294 2.657179062084367
+1 0.9402246743347112 0.40154461288043775 3.407916599846658 0.732993794216273 0.7120872061718131 0.7443371156456304 0.261691914047522 -1.7816254435328527 1.1872515149455043 1.2859514985608926
+1 1.5116064491281778 2.2468889028407437 0.45828491922709613 1.2192147082911882 0.6354365593721796 -0.2656322662271462 0.22961524227015095 0.6580482520092654 0.8557895993898526 1.1404110974520998
+1 2.738506436693102 1.129940083852354 -0.2531479159181209 -0.3313565595449408 2.157889045868747 0.7757459702743189 2.5165730696859523 -0.504719944568053 0.19221810745654677 0.4962627597149971
+1 3.141323496200573 1.4040718012832414 0.663

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/data/pagerank_data.txt
----------------------------------------------------------------------
diff --git a/data/pagerank_data.txt b/data/pagerank_data.txt
new file mode 100644
index 0000000..95755ab
--- /dev/null
+++ b/data/pagerank_data.txt
@@ -0,0 +1,6 @@
+1 2
+1 3
+1 4
+2 1
+3 1
+4 1


[17/21] git commit: pyspark -> bin/pyspark

Posted by pw...@apache.org.
pyspark -> bin/pyspark


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/a3f90a2e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/a3f90a2e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/a3f90a2e

Branch: refs/heads/master
Commit: a3f90a2ecf14a01aa27fc95c133b1ff375583adb
Parents: 94b7a7f
Author: Prashant Sharma <pr...@imaginea.com>
Authored: Thu Jan 2 18:50:12 2014 +0530
Committer: Prashant Sharma <pr...@imaginea.com>
Committed: Thu Jan 2 18:50:12 2014 +0530

----------------------------------------------------------------------
 README.md                        |  2 +-
 docs/index.md                    |  2 +-
 docs/python-programming-guide.md | 28 ++++++++++++++--------------
 docs/quick-start.md              |  4 ++--
 python/pyspark/shell.py          |  2 +-
 5 files changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a3f90a2e/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 7154165..f7ad26d 100644
--- a/README.md
+++ b/README.md
@@ -21,7 +21,7 @@ Once you've built Spark, the easiest way to start using it is the shell:
 
     ./bin/spark-shell
 
-Or, for the Python API, the Python shell (`./pyspark`).
+Or, for the Python API, the Python shell (`./bin/pyspark`).
 
 Spark also comes with several sample programs in the `examples` directory.
 To run one of them, use `./bin/run-example <class> <params>`. For example:

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a3f90a2e/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index f0990ff..86d574d 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -35,7 +35,7 @@ or `local` to run locally with one thread, or `local[N]` to run locally with N t
 `local` for testing.
 
 Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or
-Python interpreter (`./pyspark`). These are a great way to learn the framework.
+Python interpreter (`./bin/pyspark`). These are a great way to learn the framework.
 
 # Launching on a Cluster
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a3f90a2e/docs/python-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 96f93e2..dc187b3 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -47,7 +47,7 @@ PySpark will automatically ship these functions to workers, along with any objec
 Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers.
 The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers.
 
-In addition, PySpark fully supports interactive use---simply run `./pyspark` to launch an interactive shell.
+In addition, PySpark fully supports interactive use---simply run `./bin/pyspark` to launch an interactive shell.
 
 
 # Installing and Configuring PySpark
@@ -60,17 +60,17 @@ By default, PySpark requires `python` to be available on the system `PATH` and u
 
 All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported.
 
-Standalone PySpark applications should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`.
-The script automatically adds the `pyspark` package to the `PYTHONPATH`.
+Standalone PySpark applications should be run using the `bin/pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`.
+The script automatically adds the `bin/pyspark` package to the `PYTHONPATH`.
 
 
 # Interactive Use
 
-The `pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
+The `bin/pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
 
 {% highlight bash %}
 $ sbt/sbt assembly
-$ ./pyspark
+$ ./bin/pyspark
 {% endhighlight %}
 
 The Python shell can be used explore data interactively and is a simple way to learn the API:
@@ -82,35 +82,35 @@ The Python shell can be used explore data interactively and is a simple way to l
 >>> help(pyspark) # Show all pyspark functions
 {% endhighlight %}
 
-By default, the `pyspark` shell creates SparkContext that runs applications locally on a single core.
+By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on a single core.
 To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable.
-For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
+For example, to use the `bin/pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
 
 {% highlight bash %}
-$ MASTER=spark://IP:PORT ./pyspark
+$ MASTER=spark://IP:PORT ./bin/pyspark
 {% endhighlight %}
 
 Or, to use four cores on the local machine:
 
 {% highlight bash %}
-$ MASTER=local[4] ./pyspark
+$ MASTER=local[4] ./bin/pyspark
 {% endhighlight %}
 
 
 ## IPython
 
 It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter.
-To do this, set the `IPYTHON` variable to `1` when running `pyspark`:
+To do this, set the `IPYTHON` variable to `1` when running `bin/pyspark`:
 
 {% highlight bash %}
-$ IPYTHON=1 ./pyspark
+$ IPYTHON=1 ./bin/pyspark
 {% endhighlight %}
 
 Alternatively, you can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch
 the [IPython Notebook](http://ipython.org/notebook.html) with PyLab graphing support:
 
 {% highlight bash %}
-$ IPYTHON_OPTS="notebook --pylab inline" ./pyspark
+$ IPYTHON_OPTS="notebook --pylab inline" ./bin/pyspark
 {% endhighlight %}
 
 IPython also works on a cluster or on multiple cores if you set the `MASTER` environment variable.
@@ -118,7 +118,7 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env
 
 # Standalone Programs
 
-PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `pyspark`.
+PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/pyspark`.
 The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application.
 
 Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor:
@@ -153,6 +153,6 @@ Many of the methods also contain [doctests](http://docs.python.org/2/library/doc
 PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/incubator-spark/tree/master/python/examples).
 You can run them by passing the files to `pyspark`; e.g.:
 
-    ./pyspark python/examples/wordcount.py
+    ./bin/pyspark python/examples/wordcount.py
 
 Each program prints usage help when run without arguments.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a3f90a2e/docs/quick-start.md
----------------------------------------------------------------------
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 912f332..2fa2bd7 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -277,11 +277,11 @@ We can pass Python functions to Spark, which are automatically serialized along
 For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html).
 `SimpleApp` is simple enough that we do not need to specify any code dependencies.
 
-We can run this application using the `pyspark` script:
+We can run this application using the `bin/pyspark` script:
 
 {% highlight python %}
 $ cd $SPARK_HOME
-$ ./pyspark SimpleApp.py
+$ ./bin/pyspark SimpleApp.py
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight python %}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a3f90a2e/python/pyspark/shell.py
----------------------------------------------------------------------
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index ef07eb4..1602227 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -47,7 +47,7 @@ print "Spark context available as sc."
 if add_files != None:
     print "Adding files: [%s]" % ", ".join(add_files)
 
-# The ./pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
+# The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
 # which allows us to execute the user's PYTHONSTARTUP file:
 _pythonstartup = os.environ.get('OLD_PYTHONSTARTUP')
 if _pythonstartup and os.path.isfile(_pythonstartup):


[07/21] git commit: add scripts in bin

Posted by pw...@apache.org.
add scripts in bin

Signed-off-by: shane-huang <sh...@intel.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/1d53792a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/1d53792a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/1d53792a

Branch: refs/heads/master
Commit: 1d53792a0a48695824c29274be84b74d8d6a2e6a
Parents: 1d1a625
Author: shane-huang <sh...@intel.com>
Authored: Mon Sep 23 16:13:46 2013 +0800
Committer: shane-huang <sh...@intel.com>
Committed: Mon Sep 23 16:13:46 2013 +0800

----------------------------------------------------------------------
 bin/pyspark           |   2 +-
 bin/pyspark2.cmd      |   2 +-
 bin/run-example       |   4 +-
 bin/run-example2.cmd  |   4 +-
 bin/spark             | 117 +++++++++++++++++++++++++++++++++++++++++++++
 bin/spark-shell       |   4 +-
 bin/spark-shell.cmd   |   5 +-
 bin/spark.cmd         |  27 +++++++++++
 python/run-tests      |   2 +-
 sbin/spark-class      |   4 +-
 sbin/spark-class2.cmd |   4 +-
 sbin/spark-executor   |   5 +-
 12 files changed, 163 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/bin/pyspark
----------------------------------------------------------------------
diff --git a/bin/pyspark b/bin/pyspark
index 4941a36..45a2308 100755
--- a/bin/pyspark
+++ b/bin/pyspark
@@ -18,7 +18,7 @@
 #
 
 # Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Export this as SPARK_HOME
 export SPARK_HOME="$FWDIR"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/bin/pyspark2.cmd
----------------------------------------------------------------------
diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd
index f58e349..bb8e624 100644
--- a/bin/pyspark2.cmd
+++ b/bin/pyspark2.cmd
@@ -20,7 +20,7 @@ rem
 set SCALA_VERSION=2.9.3
 
 rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
 
 rem Export this as SPARK_HOME
 set SPARK_HOME=%FWDIR%

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/bin/run-example
----------------------------------------------------------------------
diff --git a/bin/run-example b/bin/run-example
index 08ec717..f29bb2c 100755
--- a/bin/run-example
+++ b/bin/run-example
@@ -20,7 +20,7 @@
 SCALA_VERSION=2.9.3
 
 # Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Export this as SPARK_HOME
 export SPARK_HOME="$FWDIR"
@@ -56,7 +56,7 @@ fi
 
 # Since the examples JAR ideally shouldn't include spark-core (that dependency should be
 # "provided"), also add our standard Spark classpath, built using compute-classpath.sh.
-CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
+CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
 CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH"
 
 # Find java binary

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/bin/run-example2.cmd
----------------------------------------------------------------------
diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd
index dbb371e..e250a92 100644
--- a/bin/run-example2.cmd
+++ b/bin/run-example2.cmd
@@ -20,7 +20,7 @@ rem
 set SCALA_VERSION=2.9.3
 
 rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
 
 rem Export this as SPARK_HOME
 set SPARK_HOME=%FWDIR%
@@ -49,7 +49,7 @@ if "x%SPARK_EXAMPLES_JAR%"=="x" (
 
 rem Compute Spark classpath using external script
 set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
+call "%FWDIR%sbin\compute-classpath.cmd"
 set DONT_PRINT_CLASSPATH=0
 set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH%
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/bin/spark
----------------------------------------------------------------------
diff --git a/bin/spark b/bin/spark
new file mode 100755
index 0000000..7f25fe1
--- /dev/null
+++ b/bin/spark
@@ -0,0 +1,117 @@
+#!/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.
+#
+
+SCALA_VERSION=2.9.3
+
+# Figure out where the Scala framework is installed
+FWDIR="$(cd `dirname $0`/..; pwd)"
+
+# Export this as SPARK_HOME
+export SPARK_HOME="$FWDIR"
+
+# 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 [ -z "$1" ]; then
+  echo "Usage: spark-class <class> [<args>]" >&2
+  exit 1
+fi
+
+# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
+# values for that; it doesn't need a lot
+if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then
+  SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
+  SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
+  # Do not overwrite SPARK_JAVA_OPTS environment variable in this script
+  OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS"   # Empty by default
+else
+  OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
+fi
+
+
+# Add java opts for master, worker, executor. The opts maybe null
+case "$1" in
+  'org.apache.spark.deploy.master.Master')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
+    ;;
+  'org.apache.spark.deploy.worker.Worker')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
+    ;;
+  'org.apache.spark.executor.StandaloneExecutorBackend')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+    ;;
+  'org.apache.spark.executor.MesosExecutorBackend')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+    ;;
+  'org.apache.spark.repl.Main')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
+    ;;
+esac
+
+# Find the java binary
+if [ -n "${JAVA_HOME}" ]; then
+  RUNNER="${JAVA_HOME}/bin/java"
+else
+  if [ `command -v java` ]; then
+    RUNNER="java"
+  else
+    echo "JAVA_HOME is not set" >&2
+    exit 1
+  fi
+fi
+
+# Set SPARK_MEM if it isn't already set since we also use it for this process
+SPARK_MEM=${SPARK_MEM:-512m}
+export SPARK_MEM
+
+# Set JAVA_OPTS to be able to load native libraries and to set heap size
+JAVA_OPTS="$OUR_JAVA_OPTS"
+JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
+JAVA_OPTS="$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="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
+fi
+export JAVA_OPTS
+# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
+
+if [ ! -f "$FWDIR/RELEASE" ]; then
+  # Exit if the user hasn't compiled Spark
+  ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null
+  if [[ $? != 0 ]]; then
+    echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2
+    echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
+    exit 1
+  fi
+fi
+
+# Compute classpath using external script
+CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
+export CLASSPATH
+
+if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
+  echo -n "Spark Command: "
+  echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
+  echo "========================================"
+  echo
+fi
+
+exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/bin/spark-shell
----------------------------------------------------------------------
diff --git a/bin/spark-shell b/bin/spark-shell
index 9608bd3..6717fe7 100755
--- a/bin/spark-shell
+++ b/bin/spark-shell
@@ -28,7 +28,7 @@
 # Enter posix mode for bash
 set -o posix
 
-FWDIR="`dirname $0`"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 for o in "$@"; do
   if [ "$1" = "-c" -o "$1" = "--cores" ]; then
@@ -79,7 +79,7 @@ if [[ ! $? ]]; then
   saved_stty=""
 fi
 
-$FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
+$FWDIR/sbin/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
 
 # record the exit status lest it be overwritten:
 # then reenable echo and propagate the code.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/bin/spark-shell.cmd
----------------------------------------------------------------------
diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd
index 3e52bf8..23973e3 100644
--- a/bin/spark-shell.cmd
+++ b/bin/spark-shell.cmd
@@ -17,6 +17,7 @@ rem See the License for the specific language governing permissions and
 rem limitations under the License.
 rem
 
-set FWDIR=%~dp0
+rem Find the path of sbin
+set SBIN=%~dp0..\sbin\
 
-cmd /V /E /C %FWDIR%spark-class2.cmd org.apache.spark.repl.Main %*
+cmd /V /E /C %SBIN%spark-class2.cmd org.apache.spark.repl.Main %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/bin/spark.cmd
----------------------------------------------------------------------
diff --git a/bin/spark.cmd b/bin/spark.cmd
new file mode 100644
index 0000000..a162339
--- /dev/null
+++ b/bin/spark.cmd
@@ -0,0 +1,27 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+rem This is the entry point for running a Spark class. To avoid polluting
+rem the environment, it just launches a new cmd to do the real work.
+
+
+rem Find the path of sbin
+set SBIN=%~dp0..\sbin\
+
+cmd /V /E /C %SBIN%spark-class2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/python/run-tests
----------------------------------------------------------------------
diff --git a/python/run-tests b/python/run-tests
index cbc554e..8a08ae3 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -29,7 +29,7 @@ FAILED=0
 rm -f unit-tests.log
 
 function run_test() {
-    $FWDIR/pyspark $1 2>&1 | tee -a unit-tests.log
+    $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log
     FAILED=$((PIPESTATUS[0]||$FAILED))
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/sbin/spark-class
----------------------------------------------------------------------
diff --git a/sbin/spark-class b/sbin/spark-class
index e111ef6..7f25fe1 100755
--- a/sbin/spark-class
+++ b/sbin/spark-class
@@ -20,7 +20,7 @@
 SCALA_VERSION=2.9.3
 
 # Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Export this as SPARK_HOME
 export SPARK_HOME="$FWDIR"
@@ -104,7 +104,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then
 fi
 
 # Compute classpath using external script
-CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
+CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
 export CLASSPATH
 
 if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/sbin/spark-class2.cmd
----------------------------------------------------------------------
diff --git a/sbin/spark-class2.cmd b/sbin/spark-class2.cmd
index d4d853e..5e00bd3 100644
--- a/sbin/spark-class2.cmd
+++ b/sbin/spark-class2.cmd
@@ -20,7 +20,7 @@ rem
 set SCALA_VERSION=2.9.3
 
 rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
 
 rem Export this as SPARK_HOME
 set SPARK_HOME=%FWDIR%
@@ -67,7 +67,7 @@ if "%FOUND_JAR%"=="0" (
 
 rem Compute classpath using external script
 set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
+call "%FWDIR%sbin\compute-classpath.cmd"
 set DONT_PRINT_CLASSPATH=0
 
 rem Figure out where java is.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d53792a/sbin/spark-executor
----------------------------------------------------------------------
diff --git a/sbin/spark-executor b/sbin/spark-executor
index 2c07c54..214e00f 100755
--- a/sbin/spark-executor
+++ b/sbin/spark-executor
@@ -17,6 +17,7 @@
 # limitations under the License.
 #
 
-FWDIR="`dirname $0`"
+FWDIR="$(cd `dirname $0`/..; pwd)"
+
 echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/spark-class org.apache.spark.executor.MesosExecutorBackend
+exec $FWDIR/sbin/spark-class org.apache.spark.executor.MesosExecutorBackend


[10/21] git commit: rm bin/spark.cmd as we don't have windows test environment. Will added it later if needed

Posted by pw...@apache.org.
rm bin/spark.cmd as we don't have windows test environment. Will added it later if needed

Signed-off-by: shane-huang <sh...@intel.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/3a5aa920
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/3a5aa920
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/3a5aa920

Branch: refs/heads/master
Commit: 3a5aa920fc9839aa99ea1befc467cc1f60230f3d
Parents: e8b1ee0
Author: shane-huang <sh...@intel.com>
Authored: Thu Sep 26 17:10:08 2013 +0800
Committer: shane-huang <sh...@intel.com>
Committed: Thu Sep 26 17:10:08 2013 +0800

----------------------------------------------------------------------
 bin/spark.cmd | 27 ---------------------------
 1 file changed, 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3a5aa920/bin/spark.cmd
----------------------------------------------------------------------
diff --git a/bin/spark.cmd b/bin/spark.cmd
deleted file mode 100644
index a162339..0000000
--- a/bin/spark.cmd
+++ /dev/null
@@ -1,27 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-rem This is the entry point for running a Spark class. To avoid polluting
-rem the environment, it just launches a new cmd to do the real work.
-
-
-rem Find the path of sbin
-set SBIN=%~dp0..\sbin\
-
-cmd /V /E /C %SBIN%spark-class2.cmd %*


[21/21] git commit: Merge pull request #317 from ScrapCodes/spark-915-segregate-scripts

Posted by pw...@apache.org.
Merge pull request #317 from ScrapCodes/spark-915-segregate-scripts

Spark-915 segregate scripts


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/4ae101ff
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/4ae101ff
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/4ae101ff

Branch: refs/heads/master
Commit: 4ae101ff38f4cd107e54d97d9493ecd4a0c3a633
Parents: 30b9db0 9ae382c
Author: Patrick Wendell <pw...@gmail.com>
Authored: Fri Jan 3 11:24:35 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Fri Jan 3 11:24:35 2014 -0800

----------------------------------------------------------------------
 README.md                                       |    8 +-
 assembly/src/main/assembly/assembly.xml         |   11 +-
 bin/compute-classpath.cmd                       |    2 +-
 bin/compute-classpath.sh                        |    2 +-
 bin/pyspark                                     |   70 ++
 bin/pyspark.cmd                                 |   23 +
 bin/pyspark2.cmd                                |   55 +
 bin/run-example                                 |   91 ++
 bin/run-example.cmd                             |   23 +
 bin/run-example2.cmd                            |   61 ++
 bin/slaves.sh                                   |   91 --
 bin/spark-class                                 |  154 +++
 bin/spark-class.cmd                             |   23 +
 bin/spark-class2.cmd                            |   85 ++
 bin/spark-config.sh                             |   36 -
 bin/spark-daemon.sh                             |  183 ----
 bin/spark-daemons.sh                            |   35 -
 bin/spark-shell                                 |  102 ++
 bin/spark-shell.cmd                             |   23 +
 bin/start-all.sh                                |   34 -
 bin/start-master.sh                             |   52 -
 bin/start-slave.sh                              |   35 -
 bin/start-slaves.sh                             |   48 -
 bin/stop-all.sh                                 |   32 -
 bin/stop-master.sh                              |   27 -
 bin/stop-slaves.sh                              |   35 -
 .../mesos/CoarseMesosSchedulerBackend.scala     |    4 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |    4 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |    4 +-
 .../scala/org/apache/spark/DriverSuite.scala    |    2 +-
 data/kmeans_data.txt                            |    6 +
 data/lr_data.txt                                | 1000 ++++++++++++++++++
 data/pagerank_data.txt                          |    6 +
 docs/bagel-programming-guide.md                 |    4 +-
 docs/index.md                                   |   10 +-
 docs/java-programming-guide.md                  |    4 +-
 docs/mllib-guide.md                             |    2 +-
 docs/python-programming-guide.md                |   28 +-
 docs/quick-start.md                             |    8 +-
 docs/running-on-yarn.md                         |    8 +-
 docs/scala-programming-guide.md                 |   14 +-
 docs/spark-debugger.md                          |    2 +-
 docs/spark-standalone.md                        |   20 +-
 docs/streaming-programming-guide.md             |    4 +-
 ec2/spark_ec2.py                                |    2 +-
 .../streaming/examples/JavaKafkaWordCount.java  |    2 +-
 .../streaming/examples/ActorWordCount.scala     |    4 +-
 .../streaming/examples/HdfsWordCount.scala      |    2 +-
 .../streaming/examples/KafkaWordCount.scala     |    2 +-
 .../streaming/examples/MQTTWordCount.scala      |    4 +-
 .../streaming/examples/NetworkWordCount.scala   |    2 +-
 .../examples/StatefulNetworkWordCount.scala     |    2 +-
 .../streaming/examples/ZeroMQWordCount.scala    |    4 +-
 .../clickstream/PageViewGenerator.scala         |    4 +-
 .../examples/clickstream/PageViewStream.scala   |    4 +-
 kmeans_data.txt                                 |    6 -
 lr_data.txt                                     | 1000 ------------------
 make-distribution.sh                            |   11 +-
 pagerank_data.txt                               |    6 -
 pyspark                                         |   70 --
 pyspark.cmd                                     |   23 -
 pyspark2.cmd                                    |   55 -
 python/pyspark/java_gateway.py                  |    2 +-
 python/pyspark/shell.py                         |    2 +-
 python/run-tests                                |    2 +-
 repl-bin/src/deb/bin/run                        |    3 +-
 repl/pom.xml                                    |    1 -
 run-example                                     |   91 --
 run-example.cmd                                 |   23 -
 run-example2.cmd                                |   61 --
 sbin/slaves.sh                                  |   91 ++
 sbin/spark-config.sh                            |   36 +
 sbin/spark-daemon.sh                            |  183 ++++
 sbin/spark-daemons.sh                           |   35 +
 sbin/spark-executor                             |   23 +
 sbin/start-all.sh                               |   34 +
 sbin/start-master.sh                            |   52 +
 sbin/start-slave.sh                             |   35 +
 sbin/start-slaves.sh                            |   48 +
 sbin/stop-all.sh                                |   32 +
 sbin/stop-master.sh                             |   27 +
 sbin/stop-slaves.sh                             |   35 +
 spark-class                                     |  154 ---
 spark-class.cmd                                 |   23 -
 spark-class2.cmd                                |   85 --
 spark-executor                                  |   22 -
 spark-shell                                     |  102 --
 spark-shell.cmd                                 |   22 -
 88 files changed, 2446 insertions(+), 2452 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4ae101ff/docs/running-on-yarn.md
----------------------------------------------------------------------


[08/21] git commit: fix path

Posted by pw...@apache.org.
fix path

Signed-off-by: shane-huang <sh...@intel.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/14098037
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/14098037
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/14098037

Branch: refs/heads/master
Commit: 14098037630196d2672431539503f27be67be480
Parents: 1d53792
Author: shane-huang <sh...@intel.com>
Authored: Thu Sep 26 12:50:10 2013 +0800
Committer: shane-huang <sh...@intel.com>
Committed: Thu Sep 26 12:50:10 2013 +0800

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/14098037/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index e3dc30e..d48545e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -120,7 +120,7 @@ private[spark] class ExecutorRunner(
     // Figure out our classpath with the external compute-classpath script
     val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh"
     val classPath = Utils.executeAndGetOutput(
-        Seq(sparkHome + "/bin/compute-classpath" + ext),
+        Seq(sparkHome + "/sbin/compute-classpath" + ext),
         extraEnvironment=appDesc.command.environment)
 
     Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts


[16/21] git commit: run-example -> bin/run-example

Posted by pw...@apache.org.
run-example -> bin/run-example


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/94b7a7fe
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/94b7a7fe
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/94b7a7fe

Branch: refs/heads/master
Commit: 94b7a7fe37a4b1459bfdbece2a4162451d6a8ac2
Parents: b810a85
Author: Prashant Sharma <pr...@imaginea.com>
Authored: Thu Jan 2 18:41:21 2014 +0530
Committer: Prashant Sharma <pr...@imaginea.com>
Committed: Thu Jan 2 18:41:21 2014 +0530

----------------------------------------------------------------------
 README.md                                                      | 4 ++--
 bin/run-example                                                | 2 +-
 bin/run-example2.cmd                                           | 2 +-
 docs/bagel-programming-guide.md                                | 4 ++--
 docs/index.md                                                  | 6 +++---
 docs/java-programming-guide.md                                 | 4 ++--
 docs/running-on-yarn.md                                        | 2 +-
 docs/scala-programming-guide.md                                | 4 ++--
 docs/streaming-programming-guide.md                            | 4 ++--
 .../apache/spark/streaming/examples/JavaKafkaWordCount.java    | 2 +-
 .../org/apache/spark/streaming/examples/ActorWordCount.scala   | 4 ++--
 .../org/apache/spark/streaming/examples/HdfsWordCount.scala    | 2 +-
 .../org/apache/spark/streaming/examples/KafkaWordCount.scala   | 2 +-
 .../org/apache/spark/streaming/examples/MQTTWordCount.scala    | 4 ++--
 .../org/apache/spark/streaming/examples/NetworkWordCount.scala | 2 +-
 .../spark/streaming/examples/StatefulNetworkWordCount.scala    | 2 +-
 .../org/apache/spark/streaming/examples/ZeroMQWordCount.scala  | 4 ++--
 .../streaming/examples/clickstream/PageViewGenerator.scala     | 4 ++--
 .../spark/streaming/examples/clickstream/PageViewStream.scala  | 4 ++--
 19 files changed, 31 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 170e964..7154165 100644
--- a/README.md
+++ b/README.md
@@ -24,9 +24,9 @@ Once you've built Spark, the easiest way to start using it is the shell:
 Or, for the Python API, the Python shell (`./pyspark`).
 
 Spark also comes with several sample programs in the `examples` directory.
-To run one of them, use `./run-example <class> <params>`. For example:
+To run one of them, use `./bin/run-example <class> <params>`. For example:
 
-    ./run-example org.apache.spark.examples.SparkLR local[2]
+    ./bin/run-example org.apache.spark.examples.SparkLR local[2]
 
 will run the Logistic Regression example locally on 2 CPUs.
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/bin/run-example
----------------------------------------------------------------------
diff --git a/bin/run-example b/bin/run-example
index f2699c3..a3976be 100755
--- a/bin/run-example
+++ b/bin/run-example
@@ -36,7 +36,7 @@ if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
 fi
 
 if [ -z "$1" ]; then
-  echo "Usage: run-example <example-class> [<args>]" >&2
+  echo "Usage: bin/run-example <example-class> [<args>]" >&2
   exit 1
 fi
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/bin/run-example2.cmd
----------------------------------------------------------------------
diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd
index 6861334..d9c1054 100644
--- a/bin/run-example2.cmd
+++ b/bin/run-example2.cmd
@@ -30,7 +30,7 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
 
 rem Test that an argument was given
 if not "x%1"=="x" goto arg_given
-  echo Usage: run-example ^<example-class^> [^<args^>]
+  echo Usage: bin/run-example ^<example-class^> [^<args^>]
   goto exit
 :arg_given
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/docs/bagel-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md
index de001e6..c4f1f6d 100644
--- a/docs/bagel-programming-guide.md
+++ b/docs/bagel-programming-guide.md
@@ -157,8 +157,8 @@ trait Message[K] {
 
 # Where to Go from Here
 
-Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark; e.g.:
+Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `bin/run-example` script included in Spark; e.g.:
 
-    ./run-example org.apache.spark.examples.bagel.WikipediaPageRank
+    ./bin/run-example org.apache.spark.examples.bagel.WikipediaPageRank
 
 Each example program prints usage help when run without any arguments.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index a5c179c..f0990ff 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -24,9 +24,9 @@ For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VE
 # Running the Examples and Shell
 
 Spark comes with several sample programs in the `examples` directory.
-To run one of the samples, use `./run-example <class> <params>` in the top-level Spark directory
-(the `run-example` script sets up the appropriate paths and launches that program).
-For example, try `./run-example org.apache.spark.examples.SparkPi local`.
+To run one of the samples, use `./bin/run-example <class> <params>` in the top-level Spark directory
+(the `bin/run-example` script sets up the appropriate paths and launches that program).
+For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`.
 Each example prints usage help when run with no parameters.
 
 Note that all of the sample programs take a `<master>` parameter specifying the cluster URL

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/docs/java-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md
index 53085cc..07732fa 100644
--- a/docs/java-programming-guide.md
+++ b/docs/java-programming-guide.md
@@ -190,9 +190,9 @@ We hope to generate documentation with Java-style syntax in the future.
 
 Spark includes several sample programs using the Java API in
 [`examples/src/main/java`](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/examples).  You can run them by passing the class name to the
-`run-example` script included in Spark; for example:
+`bin/run-example` script included in Spark; for example:
 
-    ./run-example org.apache.spark.examples.JavaWordCount
+    ./bin/run-example org.apache.spark.examples.JavaWordCount
 
 Each example program prints usage help when run
 without any arguments.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index be32034..e2b21f9 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -107,7 +107,7 @@ For example:
 
     SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
     SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
-    ./run-example org.apache.spark.examples.SparkPi yarn-client
+    ./bin/run-example org.apache.spark.examples.SparkPi yarn-client
 
 
     SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/docs/scala-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 16ae150..c1ef46a 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -366,9 +366,9 @@ res2: Int = 10
 # Where to Go from Here
 
 You can see some [example Spark programs](http://spark.incubator.apache.org/examples.html) on the Spark website.
-In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark; for example:
+In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `bin/run-example` script included in Spark; for example:
 
-    ./run-example org.apache.spark.examples.SparkPi
+    ./bin/run-example org.apache.spark.examples.SparkPi
 
 Each example program prints usage help when run without any arguments.
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/docs/streaming-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 6f97db6..1c9ece6 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -245,7 +245,7 @@ $ nc -lk 9999
 Then, in a different terminal, you can start NetworkWordCount by using
 
 {% highlight bash %}
-$ ./run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
 {% endhighlight %}
 
 This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen.
@@ -283,7 +283,7 @@ Time: 1357008430000 ms
 </td>
 </table>
 
-You can find more examples in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
+You can find more examples in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./bin/run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
 
 # DStream Persistence
 Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
index 22994fb..75b588e 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
@@ -41,7 +41,7 @@ import scala.Tuple2;
  *   <numThreads> is the number of threads the kafka consumer should use
  *
  * Example:
- *    `./run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
+ *    `./bin/run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
  *    zoo03 my-consumer-group topic1,topic2 1`
  */
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
index 2402409..3641517 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
@@ -134,9 +134,9 @@ object FeederActor {
  *   <hostname> and <port> describe the AkkaSystem that Spark Sample feeder is running on.
  *
  * To run this example locally, you may run Feeder Actor as
- *    `$ ./run-example spark.streaming.examples.FeederActor 127.0.1.1 9999`
+ *    `$ ./bin/run-example spark.streaming.examples.FeederActor 127.0.1.1 9999`
  * and then run the example
- *    `$ ./run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
+ *    `$ ./bin/run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
  */
 object ActorWordCount {
   def main(args: Array[String]) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
index bc8564b..61be1ce 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
@@ -28,7 +28,7 @@ import org.apache.spark.streaming.StreamingContext._
  *   <directory> is the directory that Spark Streaming will use to find and read new text files.
  *
  * To run this on your local machine on directory `localdir`, run this example
- *    `$ ./run-example spark.streaming.examples.HdfsWordCount local[2] localdir`
+ *    `$ ./bin/run-example spark.streaming.examples.HdfsWordCount local[2] localdir`
  * Then create a text file in `localdir` and the words in the file will get counted.
  */
 object HdfsWordCount {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
index 570ba4c..8dc8a35 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
@@ -35,7 +35,7 @@ import org.apache.spark.streaming.util.RawTextHelper._
  *   <numThreads> is the number of threads the kafka consumer should use
  *
  * Example:
- *    `./run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
+ *    `./bin/run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
  */
 object KafkaWordCount {
   def main(args: Array[String]) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
index ff332a0..ea138f5 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
@@ -79,9 +79,9 @@ object MQTTPublisher {
  *   <MqttbrokerUrl> and <topic> describe where Mqtt publisher is running.
  *
  * To run this example locally, you may run publisher as
- *    `$ ./run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
  * and run the example as
- *    `$ ./run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
  */
 object MQTTWordCount {
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
index e2487dc..ce8df8c 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
@@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._
  * To run this on your local machine, you need to first run a Netcat server
  *    `$ nc -lk 9999`
  * and then run the example
- *    `$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
+ *    `$ ./bin/run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
  */
 object NetworkWordCount {
   def main(args: Array[String]) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
index cb30c4e..e55d71e 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
@@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._
  * To run this on your local machine, you need to first run a Netcat server
  *    `$ nc -lk 9999`
  * and then run the example
- *    `$ ./run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999`
+ *    `$ ./bin/run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999`
  */
 object StatefulNetworkWordCount {
   def main(args: Array[String]) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
index e83ce78..2948aa7 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
@@ -62,9 +62,9 @@ object SimpleZeroMQPublisher {
  *   <zeroMQurl> and <topic> describe where zeroMq publisher is running.
  *
  * To run this example locally, you may run publisher as
- *    `$ ./run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
+ *    `$ ./bin/run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
  * and run the example as
- *    `$ ./run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
+ *    `$ ./bin/run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
  */
 object ZeroMQWordCount {
   def main(args: Array[String]) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
index de70c50..1a40fdb 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -39,8 +39,8 @@ object PageView extends Serializable {
 /** Generates streaming events to simulate page views on a website.
   *
   * This should be used in tandem with PageViewStream.scala. Example:
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+  * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+  * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
   *
   * When running this, you may want to set the root logging level to ERROR in
   * conf/log4j.properties to reduce the verbosity of the output.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/94b7a7fe/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
index 8282cc9..968b578 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -25,8 +25,8 @@ import org.apache.spark.SparkContext._
   * operators available in Spark streaming.
   *
   * This should be used in tandem with PageViewStream.scala. Example:
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+  * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+  * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
   */
 object PageViewStream {
   def main(args: Array[String]) {


[06/21] git commit: moved user scripts to bin folder

Posted by pw...@apache.org.
moved user scripts to bin folder

Signed-off-by: shane-huang <sh...@intel.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/1d1a6258
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/1d1a6258
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/1d1a6258

Branch: refs/heads/master
Commit: 1d1a6258008fa6d0e3aeeee35169315e441ec2c5
Parents: fcfe4f9
Author: shane-huang <sh...@intel.com>
Authored: Mon Sep 23 12:46:48 2013 +0800
Committer: shane-huang <sh...@intel.com>
Committed: Mon Sep 23 12:46:48 2013 +0800

----------------------------------------------------------------------
 bin/pyspark            |   66 +++
 bin/pyspark.cmd        |   23 +
 bin/pyspark2.cmd       |   55 +++
 bin/run-example        |   81 ++++
 bin/run-example.cmd    |   23 +
 bin/run-example2.cmd   |   61 +++
 bin/spark-shell        |   87 ++++
 bin/spark-shell.cmd    |   22 +
 data/kmeans_data.txt   |    6 +
 data/lr_data.txt       | 1000 +++++++++++++++++++++++++++++++++++++++++++
 data/pagerank_data.txt |    6 +
 kmeans_data.txt        |    6 -
 lr_data.txt            | 1000 -------------------------------------------
 pagerank_data.txt      |    6 -
 pyspark                |   66 ---
 pyspark.cmd            |   23 -
 pyspark2.cmd           |   55 ---
 run-example            |   81 ----
 run-example.cmd        |   23 -
 run-example2.cmd       |   61 ---
 spark-shell            |   87 ----
 spark-shell.cmd        |   22 -
 22 files changed, 1430 insertions(+), 1430 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/bin/pyspark
----------------------------------------------------------------------
diff --git a/bin/pyspark b/bin/pyspark
new file mode 100755
index 0000000..4941a36
--- /dev/null
+++ b/bin/pyspark
@@ -0,0 +1,66 @@
+#!/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.
+#
+
+# Figure out where the Scala framework is installed
+FWDIR="$(cd `dirname $0`; pwd)"
+
+# Export this as SPARK_HOME
+export SPARK_HOME="$FWDIR"
+
+SCALA_VERSION=2.9.3
+
+# Exit if the user hasn't compiled Spark
+if [ ! -f "$FWDIR/RELEASE" ]; then
+  # Exit if the user hasn't compiled Spark
+  ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null
+  if [[ $? != 0 ]]; then
+    echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2
+    echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
+    exit 1
+  fi
+fi
+
+# 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
+
+# Figure out which Python executable to use
+if [ -z "$PYSPARK_PYTHON" ] ; then
+  PYSPARK_PYTHON="python"
+fi
+export PYSPARK_PYTHON
+
+# Add the PySpark classes to the Python path:
+export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH
+
+# Load the PySpark shell.py script when ./pyspark is used interactively:
+export OLD_PYTHONSTARTUP=$PYTHONSTARTUP
+export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py
+
+if [ -n "$IPYTHON_OPTS" ]; then
+  IPYTHON=1
+fi
+
+if [[ "$IPYTHON" = "1" ]] ; then
+  IPYTHON_OPTS=${IPYTHON_OPTS:--i}
+  exec ipython "$IPYTHON_OPTS" -c "%run $PYTHONSTARTUP"
+else
+  exec "$PYSPARK_PYTHON" "$@"
+fi

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/bin/pyspark.cmd
----------------------------------------------------------------------
diff --git a/bin/pyspark.cmd b/bin/pyspark.cmd
new file mode 100644
index 0000000..7c26fbb
--- /dev/null
+++ b/bin/pyspark.cmd
@@ -0,0 +1,23 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+rem This is the entry point for running PySpark. To avoid polluting the
+rem environment, it just launches a new cmd to do the real work.
+
+cmd /V /E /C %~dp0pyspark2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/bin/pyspark2.cmd
----------------------------------------------------------------------
diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd
new file mode 100644
index 0000000..f58e349
--- /dev/null
+++ b/bin/pyspark2.cmd
@@ -0,0 +1,55 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+set SCALA_VERSION=2.9.3
+
+rem Figure out where the Spark framework is installed
+set FWDIR=%~dp0
+
+rem Export this as SPARK_HOME
+set SPARK_HOME=%FWDIR%
+
+rem Test whether the user has built Spark
+if exist "%FWDIR%RELEASE" goto skip_build_test
+set FOUND_JAR=0
+for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
+  set FOUND_JAR=1
+)
+if "%FOUND_JAR%"=="0" (
+  echo Failed to find Spark assembly JAR.
+  echo You need to build Spark with sbt\sbt assembly before running this program.
+  goto exit
+)
+:skip_build_test
+
+rem Load environment variables from conf\spark-env.cmd, if it exists
+if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+
+rem Figure out which Python to use.
+if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python
+
+set PYTHONPATH=%FWDIR%python;%PYTHONPATH%
+
+set OLD_PYTHONSTARTUP=%PYTHONSTARTUP%
+set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py
+
+echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH%
+
+"%PYSPARK_PYTHON%" %*
+:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/bin/run-example
----------------------------------------------------------------------
diff --git a/bin/run-example b/bin/run-example
new file mode 100755
index 0000000..08ec717
--- /dev/null
+++ b/bin/run-example
@@ -0,0 +1,81 @@
+#!/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.
+#
+
+SCALA_VERSION=2.9.3
+
+# Figure out where the Scala framework is installed
+FWDIR="$(cd `dirname $0`; pwd)"
+
+# Export this as SPARK_HOME
+export SPARK_HOME="$FWDIR"
+
+# 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 [ -z "$1" ]; then
+  echo "Usage: run-example <example-class> [<args>]" >&2
+  exit 1
+fi
+
+# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
+# to avoid the -sources and -doc packages that are built by publish-local.
+EXAMPLES_DIR="$FWDIR"/examples
+SPARK_EXAMPLES_JAR=""
+if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
+  # Use the JAR from the SBT build
+  export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
+fi
+if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then
+  # Use the JAR from the Maven build
+  # TODO: this also needs to become an assembly!
+  export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar`
+fi
+if [[ -z $SPARK_EXAMPLES_JAR ]]; then
+  echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2
+  echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
+  exit 1
+fi
+
+# Since the examples JAR ideally shouldn't include spark-core (that dependency should be
+# "provided"), also add our standard Spark classpath, built using compute-classpath.sh.
+CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
+CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH"
+
+# Find java binary
+if [ -n "${JAVA_HOME}" ]; then
+  RUNNER="${JAVA_HOME}/bin/java"
+else
+  if [ `command -v java` ]; then
+    RUNNER="java"
+  else
+    echo "JAVA_HOME is not set" >&2
+    exit 1
+  fi
+fi
+
+if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
+  echo -n "Spark Command: "
+  echo "$RUNNER" -cp "$CLASSPATH" "$@"
+  echo "========================================"
+  echo
+fi
+
+exec "$RUNNER" -cp "$CLASSPATH" "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/bin/run-example.cmd
----------------------------------------------------------------------
diff --git a/bin/run-example.cmd b/bin/run-example.cmd
new file mode 100644
index 0000000..5b2d048
--- /dev/null
+++ b/bin/run-example.cmd
@@ -0,0 +1,23 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+rem This is the entry point for running a Spark example. To avoid polluting
+rem the environment, it just launches a new cmd to do the real work.
+
+cmd /V /E /C %~dp0run-example2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/bin/run-example2.cmd
----------------------------------------------------------------------
diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd
new file mode 100644
index 0000000..dbb371e
--- /dev/null
+++ b/bin/run-example2.cmd
@@ -0,0 +1,61 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+set SCALA_VERSION=2.9.3
+
+rem Figure out where the Spark framework is installed
+set FWDIR=%~dp0
+
+rem Export this as SPARK_HOME
+set SPARK_HOME=%FWDIR%
+
+rem Load environment variables from conf\spark-env.cmd, if it exists
+if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+
+rem Test that an argument was given
+if not "x%1"=="x" goto arg_given
+  echo Usage: run-example ^<example-class^> [^<args^>]
+  goto exit
+:arg_given
+
+set EXAMPLES_DIR=%FWDIR%examples
+
+rem Figure out the JAR file that our examples were packaged into.
+set SPARK_EXAMPLES_JAR=
+for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*assembly*.jar") do (
+  set SPARK_EXAMPLES_JAR=%%d
+)
+if "x%SPARK_EXAMPLES_JAR%"=="x" (
+  echo Failed to find Spark examples assembly JAR.
+  echo You need to build Spark with sbt\sbt assembly before running this program.
+  goto exit
+)
+
+rem Compute Spark classpath using external script
+set DONT_PRINT_CLASSPATH=1
+call "%FWDIR%bin\compute-classpath.cmd"
+set DONT_PRINT_CLASSPATH=0
+set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH%
+
+rem Figure out where java is.
+set RUNNER=java
+if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java
+
+"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %*
+:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/bin/spark-shell
----------------------------------------------------------------------
diff --git a/bin/spark-shell b/bin/spark-shell
new file mode 100755
index 0000000..9608bd3
--- /dev/null
+++ b/bin/spark-shell
@@ -0,0 +1,87 @@
+#!/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.
+#
+
+#
+# Shell script for starting the Spark Shell REPL
+# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}
+# if those two env vars are set in spark-env.sh but MASTER is not.
+# Options:
+#    -c <cores>    Set the number of cores for REPL to use
+#
+
+# Enter posix mode for bash
+set -o posix
+
+FWDIR="`dirname $0`"
+
+for o in "$@"; do
+  if [ "$1" = "-c" -o "$1" = "--cores" ]; then
+    shift
+    if [ -n "$1" ]; then
+      OPTIONS="-Dspark.cores.max=$1"
+      shift
+    fi
+  fi
+done
+
+# Set MASTER from spark-env if possible
+if [ -z "$MASTER" ]; then
+  if [ -e "$FWDIR/conf/spark-env.sh" ]; then
+    . "$FWDIR/conf/spark-env.sh"
+  fi
+  if [[ "x" != "x$SPARK_MASTER_IP" && "y" != "y$SPARK_MASTER_PORT" ]]; then
+    MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}"
+    export MASTER
+  fi
+fi
+
+# Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in
+# binary distribution of Spark where Scala is not installed
+exit_status=127
+saved_stty=""
+
+# restore stty settings (echo in particular)
+function restoreSttySettings() {
+  stty $saved_stty
+  saved_stty=""
+}
+
+function onExit() {
+  if [[ "$saved_stty" != "" ]]; then
+    restoreSttySettings
+  fi
+  exit $exit_status
+}
+
+# to reenable echo if we are interrupted before completing.
+trap onExit INT
+
+# save terminal settings
+saved_stty=$(stty -g 2>/dev/null)
+# clear on error so we don't later try to restore them
+if [[ ! $? ]]; then
+  saved_stty=""
+fi
+
+$FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
+
+# record the exit status lest it be overwritten:
+# then reenable echo and propagate the code.
+exit_status=$?
+onExit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/bin/spark-shell.cmd
----------------------------------------------------------------------
diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd
new file mode 100644
index 0000000..3e52bf8
--- /dev/null
+++ b/bin/spark-shell.cmd
@@ -0,0 +1,22 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+set FWDIR=%~dp0
+
+cmd /V /E /C %FWDIR%spark-class2.cmd org.apache.spark.repl.Main %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/data/kmeans_data.txt
----------------------------------------------------------------------
diff --git a/data/kmeans_data.txt b/data/kmeans_data.txt
new file mode 100644
index 0000000..338664f
--- /dev/null
+++ b/data/kmeans_data.txt
@@ -0,0 +1,6 @@
+0.0 0.0 0.0
+0.1 0.1 0.1
+0.2 0.2 0.2
+9.0 9.0 9.0
+9.1 9.1 9.1
+9.2 9.2 9.2


[20/21] git commit: sbin/compute-classpath* bin/compute-classpath*

Posted by pw...@apache.org.
sbin/compute-classpath* bin/compute-classpath*


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/9ae382c3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/9ae382c3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/9ae382c3

Branch: refs/heads/master
Commit: 9ae382c363202eac8ef0b8e0fe1a7eab3de545b1
Parents: 74ba97f
Author: Prashant Sharma <pr...@imaginea.com>
Authored: Fri Jan 3 15:12:09 2014 +0530
Committer: Prashant Sharma <pr...@imaginea.com>
Committed: Fri Jan 3 15:12:29 2014 +0530

----------------------------------------------------------------------
 bin/compute-classpath.cmd                       | 69 ++++++++++++++++++
 bin/compute-classpath.sh                        | 75 ++++++++++++++++++++
 bin/run-example                                 |  2 +-
 bin/spark-class                                 |  2 +-
 .../spark/deploy/worker/ExecutorRunner.scala    |  2 +-
 sbin/compute-classpath.cmd                      | 69 ------------------
 sbin/compute-classpath.sh                       | 75 --------------------
 7 files changed, 147 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9ae382c3/bin/compute-classpath.cmd
----------------------------------------------------------------------
diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd
new file mode 100644
index 0000000..4f60bff
--- /dev/null
+++ b/bin/compute-classpath.cmd
@@ -0,0 +1,69 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
+rem script and the ExecutorRunner in standalone cluster mode.
+
+set SCALA_VERSION=2.10
+
+rem Figure out where the Spark framework is installed
+set FWDIR=%~dp0..\
+
+rem Load environment variables from conf\spark-env.cmd, if it exists
+if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+
+rem Build up classpath
+set CLASSPATH=%FWDIR%conf
+if exist "%FWDIR%RELEASE" (
+  for %%d in ("%FWDIR%jars\spark-assembly*.jar") do (
+    set ASSEMBLY_JAR=%%d
+  )
+) else (
+  for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
+    set ASSEMBLY_JAR=%%d
+  )
+)
+set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR%
+
+if "x%SPARK_TESTING%"=="x1" (
+  rem Add test clases to path
+  set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes
+  set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes
+  set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes
+  set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes
+  set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes
+)
+
+rem Add hadoop conf dir - else FileSystem.*, etc fail
+rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
+rem the configurtion files.
+if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir
+  set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR%
+:no_hadoop_conf_dir
+
+if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir
+  set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%
+:no_yarn_conf_dir
+
+rem A bit of a hack to allow calling this script within run2.cmd without seeing output
+if "%DONT_PRINT_CLASSPATH%"=="1" goto exit
+
+echo %CLASSPATH%
+
+:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9ae382c3/bin/compute-classpath.sh
----------------------------------------------------------------------
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
new file mode 100755
index 0000000..0c82310
--- /dev/null
+++ b/bin/compute-classpath.sh
@@ -0,0 +1,75 @@
+#!/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.
+#
+
+# This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
+# script and the ExecutorRunner in standalone cluster mode.
+
+SCALA_VERSION=2.10
+
+# Figure out where Spark is installed
+FWDIR="$(cd `dirname $0`/..; pwd)"
+
+# 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
+
+# Build up classpath
+CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
+
+# First check if we have a dependencies jar. If so, include binary classes with the deps jar
+if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar ]; then
+  CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes"
+
+  DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar`
+  CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR"
+else
+  # Else use spark-assembly jar from either RELEASE or assembly directory
+  if [ -f "$FWDIR/RELEASE" ]; then
+    ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
+  else
+    ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
+  fi
+  CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
+fi
+
+# Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1
+if [[ $SPARK_TESTING == 1 ]]; then
+  CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes"
+fi
+
+# Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail !
+# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
+# the configurtion files.
+if [ "x" != "x$HADOOP_CONF_DIR" ]; then
+  CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR"
+fi
+if [ "x" != "x$YARN_CONF_DIR" ]; then
+  CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
+fi
+
+echo "$CLASSPATH"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9ae382c3/bin/run-example
----------------------------------------------------------------------
diff --git a/bin/run-example b/bin/run-example
index f2699c3..6c5d4a6 100755
--- a/bin/run-example
+++ b/bin/run-example
@@ -61,7 +61,7 @@ fi
 
 # Since the examples JAR ideally shouldn't include spark-core (that dependency should be
 # "provided"), also add our standard Spark classpath, built using compute-classpath.sh.
-CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
+CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
 CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH"
 
 if $cygwin; then

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9ae382c3/bin/spark-class
----------------------------------------------------------------------
diff --git a/bin/spark-class b/bin/spark-class
index 4e440d8..c4225a3 100755
--- a/bin/spark-class
+++ b/bin/spark-class
@@ -128,7 +128,7 @@ if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then
 fi
 
 # Compute classpath using external script
-CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
+CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
 
 if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
   CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9ae382c3/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index d408482..fff9cb6 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -122,7 +122,7 @@ private[spark] class ExecutorRunner(
     // Figure out our classpath with the external compute-classpath script
     val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh"
     val classPath = Utils.executeAndGetOutput(
-        Seq(sparkHome + "/sbin/compute-classpath" + ext),
+        Seq(sparkHome + "/bin/compute-classpath" + ext),
         extraEnvironment=appDesc.command.environment)
 
     Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9ae382c3/sbin/compute-classpath.cmd
----------------------------------------------------------------------
diff --git a/sbin/compute-classpath.cmd b/sbin/compute-classpath.cmd
deleted file mode 100644
index 4f60bff..0000000
--- a/sbin/compute-classpath.cmd
+++ /dev/null
@@ -1,69 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
-rem script and the ExecutorRunner in standalone cluster mode.
-
-set SCALA_VERSION=2.10
-
-rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0..\
-
-rem Load environment variables from conf\spark-env.cmd, if it exists
-if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
-
-rem Build up classpath
-set CLASSPATH=%FWDIR%conf
-if exist "%FWDIR%RELEASE" (
-  for %%d in ("%FWDIR%jars\spark-assembly*.jar") do (
-    set ASSEMBLY_JAR=%%d
-  )
-) else (
-  for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
-    set ASSEMBLY_JAR=%%d
-  )
-)
-set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR%
-
-if "x%SPARK_TESTING%"=="x1" (
-  rem Add test clases to path
-  set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes
-  set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes
-  set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes
-  set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes
-  set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes
-)
-
-rem Add hadoop conf dir - else FileSystem.*, etc fail
-rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
-rem the configurtion files.
-if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir
-  set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR%
-:no_hadoop_conf_dir
-
-if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir
-  set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%
-:no_yarn_conf_dir
-
-rem A bit of a hack to allow calling this script within run2.cmd without seeing output
-if "%DONT_PRINT_CLASSPATH%"=="1" goto exit
-
-echo %CLASSPATH%
-
-:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9ae382c3/sbin/compute-classpath.sh
----------------------------------------------------------------------
diff --git a/sbin/compute-classpath.sh b/sbin/compute-classpath.sh
deleted file mode 100755
index 0c82310..0000000
--- a/sbin/compute-classpath.sh
+++ /dev/null
@@ -1,75 +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.
-#
-
-# This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
-# script and the ExecutorRunner in standalone cluster mode.
-
-SCALA_VERSION=2.10
-
-# Figure out where Spark is installed
-FWDIR="$(cd `dirname $0`/..; pwd)"
-
-# 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
-
-# Build up classpath
-CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
-
-# First check if we have a dependencies jar. If so, include binary classes with the deps jar
-if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar ]; then
-  CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes"
-
-  DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar`
-  CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR"
-else
-  # Else use spark-assembly jar from either RELEASE or assembly directory
-  if [ -f "$FWDIR/RELEASE" ]; then
-    ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
-  else
-    ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
-  fi
-  CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
-fi
-
-# Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1
-if [[ $SPARK_TESTING == 1 ]]; then
-  CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes"
-fi
-
-# Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail !
-# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
-# the configurtion files.
-if [ "x" != "x$HADOOP_CONF_DIR" ]; then
-  CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR"
-fi
-if [ "x" != "x$YARN_CONF_DIR" ]; then
-  CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
-fi
-
-echo "$CLASSPATH"


[14/21] git commit: Merge branch 'scripts-reorg' of github.com:shane-huang/incubator-spark into spark-915-segregate-scripts

Posted by pw...@apache.org.
Merge branch 'scripts-reorg' of github.com:shane-huang/incubator-spark into spark-915-segregate-scripts

Conflicts:
	bin/spark-shell
	core/pom.xml
	core/src/main/scala/org/apache/spark/SparkContext.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
	core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
	core/src/test/scala/org/apache/spark/DriverSuite.scala
	python/run-tests
	sbin/compute-classpath.sh
	sbin/spark-class
	sbin/stop-slaves.sh


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/980afd28
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/980afd28
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/980afd28

Branch: refs/heads/master
Commit: 980afd280a331103ce7391adaf484dd497218741
Parents: 3713f81 52ccf4f
Author: Prashant Sharma <pr...@imaginea.com>
Authored: Thu Jan 2 17:55:21 2014 +0530
Committer: Prashant Sharma <pr...@imaginea.com>
Committed: Thu Jan 2 17:55:21 2014 +0530

----------------------------------------------------------------------
 assembly/src/main/assembly/assembly.xml         |   11 +-
 bin/compute-classpath.cmd                       |   69 --
 bin/compute-classpath.sh                        |   75 --
 bin/pyspark                                     |   70 ++
 bin/pyspark.cmd                                 |   23 +
 bin/pyspark2.cmd                                |   55 +
 bin/run-example                                 |   91 ++
 bin/run-example.cmd                             |   23 +
 bin/run-example2.cmd                            |   61 ++
 bin/slaves.sh                                   |   91 --
 bin/spark-config.sh                             |   36 -
 bin/spark-daemon.sh                             |  183 ----
 bin/spark-daemons.sh                            |   35 -
 bin/spark-shell                                 |  102 ++
 bin/spark-shell.cmd                             |   23 +
 bin/start-all.sh                                |   34 -
 bin/start-master.sh                             |   52 -
 bin/start-slave.sh                              |   35 -
 bin/start-slaves.sh                             |   48 -
 bin/stop-all.sh                                 |   32 -
 bin/stop-master.sh                              |   27 -
 bin/stop-slaves.sh                              |   35 -
 .../spark/deploy/worker/ExecutorRunner.scala    |    2 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |    4 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |    4 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |    2 +-
 .../scala/org/apache/spark/DriverSuite.scala    |    2 +-
 data/kmeans_data.txt                            |    6 +
 data/lr_data.txt                                | 1000 ++++++++++++++++++
 data/pagerank_data.txt                          |    6 +
 docs/running-on-yarn.md                         |    4 +-
 docs/spark-standalone.md                        |   14 +-
 kmeans_data.txt                                 |    6 -
 lr_data.txt                                     | 1000 ------------------
 make-distribution.sh                            |    5 +-
 pagerank_data.txt                               |    6 -
 pyspark                                         |   70 --
 pyspark.cmd                                     |   23 -
 pyspark2.cmd                                    |   55 -
 python/pyspark/java_gateway.py                  |    2 +-
 python/run-tests                                |    2 +-
 repl-bin/src/deb/bin/run                        |    3 +-
 repl/pom.xml                                    |    1 -
 run-example                                     |   91 --
 run-example.cmd                                 |   23 -
 run-example2.cmd                                |   61 --
 sbin/compute-classpath.cmd                      |   69 ++
 sbin/compute-classpath.sh                       |   75 ++
 sbin/slaves.sh                                  |   91 ++
 sbin/spark-class                                |  154 +++
 sbin/spark-class.cmd                            |   23 +
 sbin/spark-class2.cmd                           |   85 ++
 sbin/spark-config.sh                            |   36 +
 sbin/spark-daemon.sh                            |  183 ++++
 sbin/spark-daemons.sh                           |   35 +
 sbin/spark-executor                             |   23 +
 sbin/start-all.sh                               |   34 +
 sbin/start-master.sh                            |   52 +
 sbin/start-slave.sh                             |   35 +
 sbin/start-slaves.sh                            |   48 +
 sbin/stop-all.sh                                |   32 +
 sbin/stop-master.sh                             |   27 +
 sbin/stop-slaves.sh                             |   35 +
 spark-class                                     |  154 ---
 spark-class.cmd                                 |   23 -
 spark-class2.cmd                                |   85 --
 spark-executor                                  |   22 -
 spark-shell                                     |  102 --
 spark-shell.cmd                                 |   22 -
 69 files changed, 2521 insertions(+), 2527 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/bin/pyspark
----------------------------------------------------------------------
diff --cc bin/pyspark
index 0000000,392a92b..d6810f4
mode 000000,100755..100755
--- a/bin/pyspark
+++ b/bin/pyspark
@@@ -1,0 -1,66 +1,70 @@@
+ #!/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.
+ #
+ 
+ # Figure out where the Scala framework is installed
+ FWDIR="$(cd `dirname $0`/..; pwd)"
+ 
+ # Export this as SPARK_HOME
+ export SPARK_HOME="$FWDIR"
+ 
 -SCALA_VERSION=2.9.3
++SCALA_VERSION=2.10
+ 
+ # Exit if the user hasn't compiled Spark
+ if [ ! -f "$FWDIR/RELEASE" ]; then
+   # Exit if the user hasn't compiled Spark
+   ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null
+   if [[ $? != 0 ]]; then
+     echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2
+     echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
+     exit 1
+   fi
+ fi
+ 
+ # 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
+ 
+ # Figure out which Python executable to use
+ if [ -z "$PYSPARK_PYTHON" ] ; then
+   PYSPARK_PYTHON="python"
+ fi
+ export PYSPARK_PYTHON
+ 
+ # Add the PySpark classes to the Python path:
+ export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH
+ 
+ # Load the PySpark shell.py script when ./pyspark is used interactively:
+ export OLD_PYTHONSTARTUP=$PYTHONSTARTUP
+ export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py
+ 
+ if [ -n "$IPYTHON_OPTS" ]; then
+   IPYTHON=1
+ fi
+ 
+ if [[ "$IPYTHON" = "1" ]] ; then
 -  IPYTHON_OPTS=${IPYTHON_OPTS:--i}
 -  exec ipython "$IPYTHON_OPTS" -c "%run $PYTHONSTARTUP"
++  # IPython <1.0.0 doesn't honor PYTHONSTARTUP, while 1.0.0+ does. 
++  # Hence we clear PYTHONSTARTUP and use the -c "%run $IPYTHONSTARTUP" command which works on all versions
++  # We also force interactive mode with "-i"
++  IPYTHONSTARTUP=$PYTHONSTARTUP
++  PYTHONSTARTUP=
++  exec ipython "$IPYTHON_OPTS" -i -c "%run $IPYTHONSTARTUP"
+ else
+   exec "$PYSPARK_PYTHON" "$@"
+ fi

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/bin/pyspark2.cmd
----------------------------------------------------------------------
diff --cc bin/pyspark2.cmd
index 0000000,bb8e624..9579109
mode 000000,100644..100644
--- a/bin/pyspark2.cmd
+++ b/bin/pyspark2.cmd
@@@ -1,0 -1,55 +1,55 @@@
+ @echo off
+ 
+ rem
+ rem Licensed to the Apache Software Foundation (ASF) under one or more
+ rem contributor license agreements.  See the NOTICE file distributed with
+ rem this work for additional information regarding copyright ownership.
+ rem The ASF licenses this file to You under the Apache License, Version 2.0
+ rem (the "License"); you may not use this file except in compliance with
+ rem the License.  You may obtain a copy of the License at
+ rem
+ rem    http://www.apache.org/licenses/LICENSE-2.0
+ rem
+ rem Unless required by applicable law or agreed to in writing, software
+ rem distributed under the License is distributed on an "AS IS" BASIS,
+ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ rem See the License for the specific language governing permissions and
+ rem limitations under the License.
+ rem
+ 
 -set SCALA_VERSION=2.9.3
++set SCALA_VERSION=2.10
+ 
+ rem Figure out where the Spark framework is installed
+ set FWDIR=%~dp0..\
+ 
+ rem Export this as SPARK_HOME
+ set SPARK_HOME=%FWDIR%
+ 
+ rem Test whether the user has built Spark
+ if exist "%FWDIR%RELEASE" goto skip_build_test
+ set FOUND_JAR=0
+ for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
+   set FOUND_JAR=1
+ )
+ if "%FOUND_JAR%"=="0" (
+   echo Failed to find Spark assembly JAR.
+   echo You need to build Spark with sbt\sbt assembly before running this program.
+   goto exit
+ )
+ :skip_build_test
+ 
+ rem Load environment variables from conf\spark-env.cmd, if it exists
+ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+ 
+ rem Figure out which Python to use.
+ if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python
+ 
+ set PYTHONPATH=%FWDIR%python;%PYTHONPATH%
+ 
+ set OLD_PYTHONSTARTUP=%PYTHONSTARTUP%
+ set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py
+ 
+ echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH%
+ 
+ "%PYSPARK_PYTHON%" %*
+ :exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/bin/run-example
----------------------------------------------------------------------
diff --cc bin/run-example
index 0000000,ded08a8..f2699c3
mode 000000,100755..100755
--- a/bin/run-example
+++ b/bin/run-example
@@@ -1,0 -1,81 +1,91 @@@
+ #!/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.
+ #
+ 
 -SCALA_VERSION=2.9.3
++cygwin=false
++case "`uname`" in
++    CYGWIN*) cygwin=true;;
++esac
++
++SCALA_VERSION=2.10
+ 
+ # Figure out where the Scala framework is installed
+ FWDIR="$(cd `dirname $0`/..; pwd)"
+ 
+ # Export this as SPARK_HOME
+ export SPARK_HOME="$FWDIR"
+ 
+ # 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 [ -z "$1" ]; then
+   echo "Usage: run-example <example-class> [<args>]" >&2
+   exit 1
+ fi
+ 
+ # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
+ # to avoid the -sources and -doc packages that are built by publish-local.
+ EXAMPLES_DIR="$FWDIR"/examples
+ SPARK_EXAMPLES_JAR=""
+ if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
+   # Use the JAR from the SBT build
+   export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
+ fi
+ if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then
+   # Use the JAR from the Maven build
+   # TODO: this also needs to become an assembly!
+   export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar`
+ fi
+ if [[ -z $SPARK_EXAMPLES_JAR ]]; then
+   echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2
+   echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
+   exit 1
+ fi
+ 
+ # Since the examples JAR ideally shouldn't include spark-core (that dependency should be
+ # "provided"), also add our standard Spark classpath, built using compute-classpath.sh.
+ CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
+ CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH"
+ 
++if $cygwin; then
++    CLASSPATH=`cygpath -wp $CLASSPATH`
++    export SPARK_EXAMPLES_JAR=`cygpath -w $SPARK_EXAMPLES_JAR`
++fi
++
+ # Find java binary
+ if [ -n "${JAVA_HOME}" ]; then
+   RUNNER="${JAVA_HOME}/bin/java"
+ else
+   if [ `command -v java` ]; then
+     RUNNER="java"
+   else
+     echo "JAVA_HOME is not set" >&2
+     exit 1
+   fi
+ fi
+ 
+ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
+   echo -n "Spark Command: "
+   echo "$RUNNER" -cp "$CLASSPATH" "$@"
+   echo "========================================"
+   echo
+ fi
+ 
+ exec "$RUNNER" -cp "$CLASSPATH" "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/bin/run-example2.cmd
----------------------------------------------------------------------
diff --cc bin/run-example2.cmd
index 0000000,e250a92..6861334
mode 000000,100644..100644
--- a/bin/run-example2.cmd
+++ b/bin/run-example2.cmd
@@@ -1,0 -1,61 +1,61 @@@
+ @echo off
+ 
+ rem
+ rem Licensed to the Apache Software Foundation (ASF) under one or more
+ rem contributor license agreements.  See the NOTICE file distributed with
+ rem this work for additional information regarding copyright ownership.
+ rem The ASF licenses this file to You under the Apache License, Version 2.0
+ rem (the "License"); you may not use this file except in compliance with
+ rem the License.  You may obtain a copy of the License at
+ rem
+ rem    http://www.apache.org/licenses/LICENSE-2.0
+ rem
+ rem Unless required by applicable law or agreed to in writing, software
+ rem distributed under the License is distributed on an "AS IS" BASIS,
+ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ rem See the License for the specific language governing permissions and
+ rem limitations under the License.
+ rem
+ 
 -set SCALA_VERSION=2.9.3
++set SCALA_VERSION=2.10
+ 
+ rem Figure out where the Spark framework is installed
+ set FWDIR=%~dp0..\
+ 
+ rem Export this as SPARK_HOME
+ set SPARK_HOME=%FWDIR%
+ 
+ rem Load environment variables from conf\spark-env.cmd, if it exists
+ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+ 
+ rem Test that an argument was given
+ if not "x%1"=="x" goto arg_given
+   echo Usage: run-example ^<example-class^> [^<args^>]
+   goto exit
+ :arg_given
+ 
+ set EXAMPLES_DIR=%FWDIR%examples
+ 
+ rem Figure out the JAR file that our examples were packaged into.
+ set SPARK_EXAMPLES_JAR=
+ for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*assembly*.jar") do (
+   set SPARK_EXAMPLES_JAR=%%d
+ )
+ if "x%SPARK_EXAMPLES_JAR%"=="x" (
+   echo Failed to find Spark examples assembly JAR.
+   echo You need to build Spark with sbt\sbt assembly before running this program.
+   goto exit
+ )
+ 
+ rem Compute Spark classpath using external script
+ set DONT_PRINT_CLASSPATH=1
+ call "%FWDIR%sbin\compute-classpath.cmd"
+ set DONT_PRINT_CLASSPATH=0
+ set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH%
+ 
+ rem Figure out where java is.
+ set RUNNER=java
+ if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java
+ 
+ "%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %*
+ :exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/bin/spark-shell
----------------------------------------------------------------------
diff --cc bin/spark-shell
index 0000000,6717fe7..bc7386d
mode 000000,100755..100755
--- a/bin/spark-shell
+++ b/bin/spark-shell
@@@ -1,0 -1,87 +1,102 @@@
+ #!/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.
+ #
+ 
+ #
+ # Shell script for starting the Spark Shell REPL
+ # Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}
+ # if those two env vars are set in spark-env.sh but MASTER is not.
+ # Options:
+ #    -c <cores>    Set the number of cores for REPL to use
 -#
++
++cygwin=false
++case "`uname`" in
++    CYGWIN*) cygwin=true;;
++esac
+ 
+ # Enter posix mode for bash
+ set -o posix
+ 
+ FWDIR="$(cd `dirname $0`/..; pwd)"
+ 
+ for o in "$@"; do
+   if [ "$1" = "-c" -o "$1" = "--cores" ]; then
+     shift
+     if [ -n "$1" ]; then
+       OPTIONS="-Dspark.cores.max=$1"
+       shift
+     fi
+   fi
+ done
+ 
+ # Set MASTER from spark-env if possible
+ if [ -z "$MASTER" ]; then
+   if [ -e "$FWDIR/conf/spark-env.sh" ]; then
+     . "$FWDIR/conf/spark-env.sh"
+   fi
+   if [[ "x" != "x$SPARK_MASTER_IP" && "y" != "y$SPARK_MASTER_PORT" ]]; then
+     MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}"
+     export MASTER
+   fi
+ fi
+ 
+ # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in
+ # binary distribution of Spark where Scala is not installed
+ exit_status=127
+ saved_stty=""
+ 
+ # restore stty settings (echo in particular)
+ function restoreSttySettings() {
+   stty $saved_stty
+   saved_stty=""
+ }
+ 
+ function onExit() {
+   if [[ "$saved_stty" != "" ]]; then
+     restoreSttySettings
+   fi
+   exit $exit_status
+ }
+ 
+ # to reenable echo if we are interrupted before completing.
+ trap onExit INT
+ 
+ # save terminal settings
+ saved_stty=$(stty -g 2>/dev/null)
+ # clear on error so we don't later try to restore them
+ if [[ ! $? ]]; then
+   saved_stty=""
+ fi
+ 
 -$FWDIR/sbin/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
++if $cygwin; then
++    # Workaround for issue involving JLine and Cygwin
++    # (see http://sourceforge.net/p/jline/bugs/40/).
++    # If you're using the Mintty terminal emulator in Cygwin, may need to set the
++    # "Backspace sends ^H" setting in "Keys" section of the Mintty options
++    # (see https://github.com/sbt/sbt/issues/562).
++    stty -icanon min 1 -echo > /dev/null 2>&1
++    $FWDIR/sbin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@"
++    stty icanon echo > /dev/null 2>&1
++else
++    $FWDIR/sbin/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
++fi
+ 
+ # record the exit status lest it be overwritten:
+ # then reenable echo and propagate the code.
+ exit_status=$?
+ onExit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 0881152,15b3397..0494ca8
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@@ -121,23 -119,23 +121,23 @@@ private[spark] class CoarseMesosSchedul
      }
      val command = CommandInfo.newBuilder()
        .setEnvironment(environment)
 -    val driverUrl = "akka://spark@%s:%s/user/%s".format(
 -      System.getProperty("spark.driver.host"),
 -      System.getProperty("spark.driver.port"),
 -      StandaloneSchedulerBackend.ACTOR_NAME)
 -    val uri = System.getProperty("spark.executor.uri")
 +    val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
 +      conf.get("spark.driver.host"),
 +      conf.get("spark.driver.port"),
 +      CoarseGrainedSchedulerBackend.ACTOR_NAME)
 +    val uri = conf.get("spark.executor.uri", null)
      if (uri == null) {
-       val runScript = new File(sparkHome, "spark-class").getCanonicalPath
+       val runScript = new File(sparkHome, "./sbin/spark-class").getCanonicalPath
        command.setValue(
 -        "\"%s\" org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
 +        "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format(
            runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
      } else {
        // Grab everything to the first '.'. We'll use that and '*' to
        // glob the directory "correctly".
        val basename = uri.split('/').last.split('.').head
        command.setValue(
-         "cd %s*; ./spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
 -        "cd %s*; ./sbin/spark-class org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
 -          basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
++        "cd %s*; ./sbin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
 +          .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
        command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
      }
      return command.build()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index bb278fb,7e9c05c..ae8d527
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@@ -100,9 -101,9 +100,9 @@@ private[spark] class MesosSchedulerBack
      }
      val command = CommandInfo.newBuilder()
        .setEnvironment(environment)
 -    val uri = System.getProperty("spark.executor.uri")
 +    val uri = sc.conf.get("spark.executor.uri", null)
      if (uri == null) {
-       command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath)
+       command.setValue(new File(sparkHome, "/sbin/spark-executor").getCanonicalPath)
      } else {
        // Grab everything to the first '.'. We'll use that and '*' to
        // glob the directory "correctly".

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index 58d47a2,b4128b7..f207627
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@@ -36,17 -35,17 +36,17 @@@ private[spark] object UIWorkloadGenerat
  
    def main(args: Array[String]) {
      if (args.length < 2) {
-       println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
 -      println("usage: ./sbin/spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
++      println("usage: ./sbin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
        System.exit(1)
      }
 -    val master = args(0)
 -    val schedulingMode = SchedulingMode.withName(args(1))
 -    val appName = "Spark UI Tester"
  
 +    val conf = new SparkConf().setMaster(args(0)).setAppName("Spark UI tester")
 +
 +    val schedulingMode = SchedulingMode.withName(args(1))
      if (schedulingMode == SchedulingMode.FAIR) {
 -      System.setProperty("spark.scheduler.mode", "FAIR")
 +      conf.set("spark.scheduler.mode", "FAIR")
      }
 -    val sc = new SparkContext(master, appName)
 +    val sc = new SparkContext(conf)
  
      def setProperties(s: String) = {
        if(schedulingMode == SchedulingMode.FAIR) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/core/src/test/scala/org/apache/spark/DriverSuite.scala
----------------------------------------------------------------------
diff --cc core/src/test/scala/org/apache/spark/DriverSuite.scala
index 6d1695e,5e5ece0..c37fd9a
--- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@@ -34,8 -34,8 +34,8 @@@ class DriverSuite extends FunSuite wit
      // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing"
      val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]"))
      forAll(masters) { (master: String) =>
 -      failAfter(30 seconds) {
 +      failAfter(60 seconds) {
-         Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master),
+         Utils.execute(Seq("./sbin/spark-class", "org.apache.spark.DriverWithoutCleanup", master),
            new File(System.getenv("SPARK_HOME")))
        }
      }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --cc docs/running-on-yarn.md
index aa75ca4,767eb5c..00cad99
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@@ -48,13 -40,9 +48,13 @@@ System Properties
  Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the hadoop cluster.
  This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager.
  
 +There are two scheduler mode that can be used to launch spark application on YARN.
 +
 +## Launch spark application by YARN Client with yarn-standalone mode.
 +
  The command to launch the YARN Client is as follows:
  
-     SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./spark-class org.apache.spark.deploy.yarn.Client \
+     SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./sbin/spark-class org.apache.spark.deploy.yarn.Client \
        --jar <YOUR_APP_JAR_FILE> \
        --class <APP_MAIN_CLASS> \
        --args <APP_MAIN_ARGUMENTS> \

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/docs/spark-standalone.md
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/make-distribution.sh
----------------------------------------------------------------------
diff --cc make-distribution.sh
index 32bbdb9,13843bb..82638a2
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@@ -95,13 -95,10 +95,10 @@@ cp $FWDIR/assembly/target/scala*/*assem
  
  # Copy other things
  mkdir "$DISTDIR"/conf
 -cp "$FWDIR/conf/*.template" "$DISTDIR"/conf
 +cp "$FWDIR"/conf/*.template "$DISTDIR"/conf
  cp -r "$FWDIR/bin" "$DISTDIR"
  cp -r "$FWDIR/python" "$DISTDIR"
- cp "$FWDIR/spark-class" "$DISTDIR"
- cp "$FWDIR/spark-shell" "$DISTDIR"
- cp "$FWDIR/spark-executor" "$DISTDIR"
- cp "$FWDIR/pyspark" "$DISTDIR"
+ cp -r "$FWDIR/sbin" "$DISTDIR"
  
  
  if [ "$MAKE_TGZ" == "true" ]; then

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/python/pyspark/java_gateway.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/python/run-tests
----------------------------------------------------------------------
diff --cc python/run-tests
index 4b71fff,8a08ae3..feba97c
--- a/python/run-tests
+++ b/python/run-tests
@@@ -29,7 -29,7 +29,7 @@@ FAILED=
  rm -f unit-tests.log
  
  function run_test() {
-     SPARK_TESTING=0 $FWDIR/pyspark $1 2>&1 | tee -a unit-tests.log
 -    $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log
++    SPARK_TESTING=0 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log
      FAILED=$((PIPESTATUS[0]||$FAILED))
  }
  

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/repl-bin/src/deb/bin/run
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/repl/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/sbin/compute-classpath.cmd
----------------------------------------------------------------------
diff --cc sbin/compute-classpath.cmd
index 0000000,e0b8a8e..4f60bff
mode 000000,100644..100644
--- a/sbin/compute-classpath.cmd
+++ b/sbin/compute-classpath.cmd
@@@ -1,0 -1,69 +1,69 @@@
+ @echo off
+ 
+ rem
+ rem Licensed to the Apache Software Foundation (ASF) under one or more
+ rem contributor license agreements.  See the NOTICE file distributed with
+ rem this work for additional information regarding copyright ownership.
+ rem The ASF licenses this file to You under the Apache License, Version 2.0
+ rem (the "License"); you may not use this file except in compliance with
+ rem the License.  You may obtain a copy of the License at
+ rem
+ rem    http://www.apache.org/licenses/LICENSE-2.0
+ rem
+ rem Unless required by applicable law or agreed to in writing, software
+ rem distributed under the License is distributed on an "AS IS" BASIS,
+ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ rem See the License for the specific language governing permissions and
+ rem limitations under the License.
+ rem
+ 
+ rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
+ rem script and the ExecutorRunner in standalone cluster mode.
+ 
 -set SCALA_VERSION=2.9.3
++set SCALA_VERSION=2.10
+ 
+ rem Figure out where the Spark framework is installed
+ set FWDIR=%~dp0..\
+ 
+ rem Load environment variables from conf\spark-env.cmd, if it exists
+ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+ 
+ rem Build up classpath
+ set CLASSPATH=%FWDIR%conf
+ if exist "%FWDIR%RELEASE" (
+   for %%d in ("%FWDIR%jars\spark-assembly*.jar") do (
+     set ASSEMBLY_JAR=%%d
+   )
+ ) else (
+   for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
+     set ASSEMBLY_JAR=%%d
+   )
+ )
+ set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR%
+ 
+ if "x%SPARK_TESTING%"=="x1" (
+   rem Add test clases to path
+   set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes
+   set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes
+   set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes
+   set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes
+   set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes
+ )
+ 
+ rem Add hadoop conf dir - else FileSystem.*, etc fail
+ rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
+ rem the configurtion files.
+ if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir
+   set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR%
+ :no_hadoop_conf_dir
+ 
+ if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir
+   set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%
+ :no_yarn_conf_dir
+ 
+ rem A bit of a hack to allow calling this script within run2.cmd without seeing output
+ if "%DONT_PRINT_CLASSPATH%"=="1" goto exit
+ 
+ echo %CLASSPATH%
+ 
+ :exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/sbin/compute-classpath.sh
----------------------------------------------------------------------
diff --cc sbin/compute-classpath.sh
index 0000000,cfe5fe7..0c82310
mode 000000,100755..100755
--- a/sbin/compute-classpath.sh
+++ b/sbin/compute-classpath.sh
@@@ -1,0 -1,61 +1,75 @@@
+ #!/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.
+ #
+ 
+ # This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
+ # script and the ExecutorRunner in standalone cluster mode.
+ 
 -SCALA_VERSION=2.9.3
++SCALA_VERSION=2.10
+ 
+ # Figure out where Spark is installed
+ FWDIR="$(cd `dirname $0`/..; pwd)"
+ 
+ # 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
+ 
+ # Build up classpath
 -CLASSPATH="$FWDIR/conf"
 -if [ -f "$FWDIR/RELEASE" ]; then
 -  ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
++CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
++
++# First check if we have a dependencies jar. If so, include binary classes with the deps jar
++if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar ]; then
++  CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes"
++  CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes"
++  CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes"
++  CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes"
++  CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes"
++
++  DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar`
++  CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR"
+ else
 -  ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
++  # Else use spark-assembly jar from either RELEASE or assembly directory
++  if [ -f "$FWDIR/RELEASE" ]; then
++    ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
++  else
++    ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
++  fi
++  CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
+ fi
 -CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
+ 
+ # Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1
+ if [[ $SPARK_TESTING == 1 ]]; then
+   CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes"
+   CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes"
+   CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes"
+   CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes"
+   CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes"
+ fi
+ 
+ # Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail !
+ # Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
+ # the configurtion files.
+ if [ "x" != "x$HADOOP_CONF_DIR" ]; then
+   CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR"
+ fi
+ if [ "x" != "x$YARN_CONF_DIR" ]; then
+   CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
+ fi
+ 
+ echo "$CLASSPATH"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/sbin/slaves.sh
----------------------------------------------------------------------
diff --cc sbin/slaves.sh
index 0000000,68408bc..a5bc218
mode 000000,100755..100755
--- a/sbin/slaves.sh
+++ b/sbin/slaves.sh
@@@ -1,0 -1,74 +1,91 @@@
+ #!/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.
+ #
+ 
+ # Run a shell command on all slave hosts.
+ #
+ # Environment Variables
+ #
+ #   SPARK_SLAVES    File naming remote hosts.
+ #     Default is ${SPARK_CONF_DIR}/slaves.
+ #   SPARK_CONF_DIR  Alternate conf dir. Default is ${SPARK_HOME}/conf.
+ #   SPARK_SLAVE_SLEEP Seconds to sleep between spawning remote commands.
+ #   SPARK_SSH_OPTS Options passed to ssh when running remote commands.
+ ##
+ 
 -usage="Usage: slaves.sh [--config confdir] command..."
++usage="Usage: slaves.sh [--config <conf-dir>] command..."
+ 
+ # if no args specified, show usage
+ if [ $# -le 0 ]; then
+   echo $usage
+   exit 1
+ fi
+ 
+ sbin=`dirname "$0"`
+ sbin=`cd "$sbin"; pwd`
+ 
+ . "$sbin/spark-config.sh"
+ 
+ # If the slaves file is specified in the command line,
+ # then it takes precedence over the definition in
+ # spark-env.sh. Save it here.
+ HOSTLIST=$SPARK_SLAVES
+ 
++# Check if --config is passed as an argument. It is an optional parameter.
++# Exit if the argument is not a directory.
++if [ "$1" == "--config" ]
++then
++  shift
++  conf_dir=$1
++  if [ ! -d "$conf_dir" ]
++  then
++    echo "ERROR : $conf_dir is not a directory"
++    echo $usage
++    exit 1
++  else
++    export SPARK_CONF_DIR=$conf_dir
++  fi
++  shift
++fi
++
+ if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
+   . "${SPARK_CONF_DIR}/spark-env.sh"
+ fi
+ 
+ if [ "$HOSTLIST" = "" ]; then
+   if [ "$SPARK_SLAVES" = "" ]; then
+     export HOSTLIST="${SPARK_CONF_DIR}/slaves"
+   else
+     export HOSTLIST="${SPARK_SLAVES}"
+   fi
+ fi
+ 
+ # By default disable strict host key checking
+ if [ "$SPARK_SSH_OPTS" = "" ]; then
+   SPARK_SSH_OPTS="-o StrictHostKeyChecking=no"
+ fi
+ 
+ for slave in `cat "$HOSTLIST"|sed  "s/#.*$//;/^$/d"`; do
+  ssh $SPARK_SSH_OPTS $slave $"${@// /\\ }" \
+    2>&1 | sed "s/^/$slave: /" &
+  if [ "$SPARK_SLAVE_SLEEP" != "" ]; then
+    sleep $SPARK_SLAVE_SLEEP
+  fi
+ done
+ 
+ wait

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/sbin/spark-class
----------------------------------------------------------------------
diff --cc sbin/spark-class
index 0000000,3bdc29e..4e440d8
mode 000000,100755..100755
--- a/sbin/spark-class
+++ b/sbin/spark-class
@@@ -1,0 -1,117 +1,154 @@@
+ #!/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.
+ #
+ 
 -SCALA_VERSION=2.9.3
++cygwin=false
++case "`uname`" in
++    CYGWIN*) cygwin=true;;
++esac
++
++SCALA_VERSION=2.10
+ 
+ # Figure out where the Scala framework is installed
+ FWDIR="$(cd `dirname $0`/..; pwd)"
+ 
+ # Export this as SPARK_HOME
+ export SPARK_HOME="$FWDIR"
+ 
+ # 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 [ -z "$1" ]; then
+   echo "Usage: spark-class <class> [<args>]" >&2
+   exit 1
+ fi
+ 
+ # If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
+ # values for that; it doesn't need a lot
+ if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then
+   SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
+   SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
+   # Do not overwrite SPARK_JAVA_OPTS environment variable in this script
+   OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS"   # Empty by default
+ else
+   OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
+ fi
+ 
+ 
+ # Add java opts for master, worker, executor. The opts maybe null
+ case "$1" in
+   'org.apache.spark.deploy.master.Master')
+     OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
+     ;;
+   'org.apache.spark.deploy.worker.Worker')
+     OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
+     ;;
 -  'org.apache.spark.executor.StandaloneExecutorBackend')
++  'org.apache.spark.executor.CoarseGrainedExecutorBackend')
+     OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+     ;;
+   'org.apache.spark.executor.MesosExecutorBackend')
+     OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+     ;;
+   'org.apache.spark.repl.Main')
+     OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
+     ;;
+ esac
+ 
+ # Find the java binary
+ if [ -n "${JAVA_HOME}" ]; then
+   RUNNER="${JAVA_HOME}/bin/java"
+ else
+   if [ `command -v java` ]; then
+     RUNNER="java"
+   else
+     echo "JAVA_HOME is not set" >&2
+     exit 1
+   fi
+ fi
+ 
+ # Set SPARK_MEM if it isn't already set since we also use it for this process
+ SPARK_MEM=${SPARK_MEM:-512m}
+ export SPARK_MEM
+ 
+ # Set JAVA_OPTS to be able to load native libraries and to set heap size
+ JAVA_OPTS="$OUR_JAVA_OPTS"
+ JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
+ JAVA_OPTS="$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="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
+ fi
+ export JAVA_OPTS
+ # Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
+ 
+ if [ ! -f "$FWDIR/RELEASE" ]; then
+   # Exit if the user hasn't compiled Spark
 -  ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null
 -  if [[ $? != 0 ]]; then
 -    echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2
 -    echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
++  num_jars=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar" | wc -l)
++  jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar")
++  if [ "$num_jars" -eq "0" ]; then
++    echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2
++    echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2
++    exit 1
++  fi
++  if [ "$num_jars" -gt "1" ]; then
++    echo "Found multiple Spark assembly jars in $FWDIR/assembly/target/scala-$SCALA_VERSION:" >&2
++    echo "$jars_list"
++    echo "Please remove all but one jar."
+     exit 1
+   fi
+ fi
+ 
++TOOLS_DIR="$FWDIR"/tools
++SPARK_TOOLS_JAR=""
++if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
++  # Use the JAR from the SBT build
++  export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
++fi
++if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then
++  # Use the JAR from the Maven build
++  # TODO: this also needs to become an assembly!
++  export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar`
++fi
++
+ # Compute classpath using external script
+ CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
++
++if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
++  CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR"
++fi
++
++if $cygwin; then
++  CLASSPATH=`cygpath -wp $CLASSPATH`
++  if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
++    export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR`
++  fi
++fi
+ export CLASSPATH
+ 
+ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
+   echo -n "Spark Command: "
+   echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
+   echo "========================================"
+   echo
+ fi
+ 
+ exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
++
++

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/sbin/spark-class2.cmd
----------------------------------------------------------------------
diff --cc sbin/spark-class2.cmd
index 0000000,5e00bd3..460e661
mode 000000,100644..100644
--- a/sbin/spark-class2.cmd
+++ b/sbin/spark-class2.cmd
@@@ -1,0 -1,78 +1,85 @@@
+ @echo off
+ 
+ rem
+ rem Licensed to the Apache Software Foundation (ASF) under one or more
+ rem contributor license agreements.  See the NOTICE file distributed with
+ rem this work for additional information regarding copyright ownership.
+ rem The ASF licenses this file to You under the Apache License, Version 2.0
+ rem (the "License"); you may not use this file except in compliance with
+ rem the License.  You may obtain a copy of the License at
+ rem
+ rem    http://www.apache.org/licenses/LICENSE-2.0
+ rem
+ rem Unless required by applicable law or agreed to in writing, software
+ rem distributed under the License is distributed on an "AS IS" BASIS,
+ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ rem See the License for the specific language governing permissions and
+ rem limitations under the License.
+ rem
+ 
 -set SCALA_VERSION=2.9.3
++set SCALA_VERSION=2.10
+ 
+ rem Figure out where the Spark framework is installed
+ set FWDIR=%~dp0..\
+ 
+ rem Export this as SPARK_HOME
+ set SPARK_HOME=%FWDIR%
+ 
+ rem Load environment variables from conf\spark-env.cmd, if it exists
+ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+ 
+ rem Test that an argument was given
+ if not "x%1"=="x" goto arg_given
+   echo Usage: spark-class ^<class^> [^<args^>]
+   goto exit
+ :arg_given
+ 
+ set RUNNING_DAEMON=0
+ if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1
+ if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1
+ if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m
+ set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true
+ if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY%
+ rem Do not overwrite SPARK_JAVA_OPTS environment variable in this script
+ if "%RUNNING_DAEMON%"=="0" set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS%
+ if "%RUNNING_DAEMON%"=="1" set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS%
+ 
+ rem Figure out how much memory to use per executor and set it as an environment
+ rem variable so that our process sees it and can report it to Mesos
+ if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m
+ 
+ rem Set JAVA_OPTS to be able to load native libraries and to set heap size
+ set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM%
+ rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
+ 
+ rem Test whether the user has built Spark
+ if exist "%FWDIR%RELEASE" goto skip_build_test
+ set FOUND_JAR=0
+ for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
+   set FOUND_JAR=1
+ )
+ if "%FOUND_JAR%"=="0" (
+   echo Failed to find Spark assembly JAR.
+   echo You need to build Spark with sbt\sbt assembly before running this program.
+   goto exit
+ )
+ :skip_build_test
+ 
++set TOOLS_DIR=%FWDIR%tools
++set SPARK_TOOLS_JAR=
++for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar") do (
++  set SPARK_TOOLS_JAR=%%d
++)
++
+ rem Compute classpath using external script
+ set DONT_PRINT_CLASSPATH=1
+ call "%FWDIR%sbin\compute-classpath.cmd"
+ set DONT_PRINT_CLASSPATH=0
++set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR%
+ 
+ rem Figure out where java is.
+ set RUNNER=java
+ if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java
+ 
+ "%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %*
+ :exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/sbin/spark-daemon.sh
----------------------------------------------------------------------
diff --cc sbin/spark-daemon.sh
index 0000000,ae82349..ca6b893
mode 000000,100755..100755
--- a/sbin/spark-daemon.sh
+++ b/sbin/spark-daemon.sh
@@@ -1,0 -1,164 +1,183 @@@
+ #!/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.
+ #
+ 
+ # Runs a Spark command as a daemon.
+ #
+ # Environment Variables
+ #
+ #   SPARK_CONF_DIR  Alternate conf dir. Default is ${SPARK_PREFIX}/conf.
+ #   SPARK_LOG_DIR   Where log files are stored.  PWD by default.
+ #   SPARK_MASTER    host:path where spark code should be rsync'd from
+ #   SPARK_PID_DIR   The pid files are stored. /tmp by default.
+ #   SPARK_IDENT_STRING   A string representing this instance of spark. $USER by default
+ #   SPARK_NICENESS The scheduling priority for daemons. Defaults to 0.
+ ##
+ 
 -usage="Usage: spark-daemon.sh [--config <conf-dir>] [--hosts hostlistfile] (start|stop) <spark-command> <spark-instance-number> <args...>"
++usage="Usage: spark-daemon.sh [--config <conf-dir>] (start|stop) <spark-command> <spark-instance-number> <args...>"
+ 
+ # if no args specified, show usage
+ if [ $# -le 1 ]; then
+   echo $usage
+   exit 1
+ fi
+ 
+ sbin=`dirname "$0"`
+ sbin=`cd "$sbin"; pwd`
+ 
+ . "$sbin/spark-config.sh"
+ 
+ # get arguments
++
++# Check if --config is passed as an argument. It is an optional parameter.
++# Exit if the argument is not a directory.
++
++if [ "$1" == "--config" ]
++then
++  shift
++  conf_dir=$1
++  if [ ! -d "$conf_dir" ]
++  then
++    echo "ERROR : $conf_dir is not a directory"
++    echo $usage
++    exit 1
++  else
++    export SPARK_CONF_DIR=$conf_dir
++  fi
++  shift
++fi
++
+ startStop=$1
+ shift
+ command=$1
+ shift
+ instance=$1
+ shift
+ 
+ spark_rotate_log ()
+ {
+     log=$1;
+     num=5;
+     if [ -n "$2" ]; then
+ 	num=$2
+     fi
+     if [ -f "$log" ]; then # rotate logs
+ 	while [ $num -gt 1 ]; do
+ 	    prev=`expr $num - 1`
+ 	    [ -f "$log.$prev" ] && mv "$log.$prev" "$log.$num"
+ 	    num=$prev
+ 	done
+ 	mv "$log" "$log.$num";
+     fi
+ }
+ 
+ if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
+   . "${SPARK_CONF_DIR}/spark-env.sh"
+ fi
+ 
+ if [ "$SPARK_IDENT_STRING" = "" ]; then
+   export SPARK_IDENT_STRING="$USER"
+ fi
+ 
+ 
+ export SPARK_PRINT_LAUNCH_COMMAND="1"
+ 
+ # get log directory
+ if [ "$SPARK_LOG_DIR" = "" ]; then
+   export SPARK_LOG_DIR="$SPARK_HOME/logs"
+ fi
+ mkdir -p "$SPARK_LOG_DIR"
+ touch $SPARK_LOG_DIR/.spark_test > /dev/null 2>&1
+ TEST_LOG_DIR=$?
+ if [ "${TEST_LOG_DIR}" = "0" ]; then
+   rm -f $SPARK_LOG_DIR/.spark_test
+ else
+   chown $SPARK_IDENT_STRING $SPARK_LOG_DIR
+ fi
+ 
+ if [ "$SPARK_PID_DIR" = "" ]; then
+   SPARK_PID_DIR=/tmp
+ fi
+ 
+ # some variables
+ export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.log
+ export SPARK_ROOT_LOGGER="INFO,DRFA"
+ log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out
+ pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid
+ 
+ # Set default scheduling priority
+ if [ "$SPARK_NICENESS" = "" ]; then
+     export SPARK_NICENESS=0
+ fi
+ 
+ 
+ case $startStop in
+ 
+   (start)
+ 
+     mkdir -p "$SPARK_PID_DIR"
+ 
+     if [ -f $pid ]; then
+       if kill -0 `cat $pid` > /dev/null 2>&1; then
+         echo $command running as process `cat $pid`.  Stop it first.
+         exit 1
+       fi
+     fi
+ 
+     if [ "$SPARK_MASTER" != "" ]; then
+       echo rsync from $SPARK_MASTER
+       rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME"
+     fi
+ 
+     spark_rotate_log "$log"
+     echo starting $command, logging to $log
+     cd "$SPARK_PREFIX"
+     nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/sbin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
+     newpid=$!
+     echo $newpid > $pid
+     sleep 2
+     # Check if the process has died; in that case we'll tail the log so the user can see
+     if ! kill -0 $newpid >/dev/null 2>&1; then
+       echo "failed to launch $command:"
+       tail -2 "$log" | sed 's/^/  /'
+       echo "full log in $log"
+     fi
+     ;;
+ 
+   (stop)
+ 
+     if [ -f $pid ]; then
+       if kill -0 `cat $pid` > /dev/null 2>&1; then
+         echo stopping $command
+         kill `cat $pid`
+       else
+         echo no $command to stop
+       fi
+     else
+       echo no $command to stop
+     fi
+     ;;
+ 
+   (*)
+     echo $usage
+     exit 1
+     ;;
+ 
+ esac
+ 
+ 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/sbin/spark-daemons.sh
----------------------------------------------------------------------
diff --cc sbin/spark-daemons.sh
index 0000000,d91254b..5d9f2bb
mode 000000,100755..100755
--- a/sbin/spark-daemons.sh
+++ b/sbin/spark-daemons.sh
@@@ -1,0 -1,35 +1,35 @@@
+ #!/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.
+ #
+ 
+ # Run a Spark command on all slave hosts.
+ 
 -usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..."
++usage="Usage: spark-daemons.sh [--config <conf-dir>] [start|stop] command instance-number args..."
+ 
+ # if no args specified, show usage
+ if [ $# -le 1 ]; then
+   echo $usage
+   exit 1
+ fi
+ 
+ sbin=`dirname "$0"`
+ sbin=`cd "$sbin"; pwd`
+ 
+ . "$sbin/spark-config.sh"
+ 
+ exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/spark-daemon.sh" "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/980afd28/sbin/stop-slaves.sh
----------------------------------------------------------------------
diff --cc sbin/stop-slaves.sh
index 0000000,63802e6..c6b0b6a
mode 000000,100755..100755
--- a/sbin/stop-slaves.sh
+++ b/sbin/stop-slaves.sh
@@@ -1,0 -1,37 +1,35 @@@
+ #!/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.
+ #
+ 
 -# Starts the master on the machine this script is executed on.
 -
 -sbin=`dirname "$0"`
 -sbin=`cd "$sbin"; pwd`
++bin=`dirname "$0"`
++bin=`cd "$sbin"; pwd`
+ 
+ . "$sbin/spark-config.sh"
+ 
+ if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
+   . "${SPARK_CONF_DIR}/spark-env.sh"
+ fi
+ 
+ if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
+   "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
+ else
+   for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
+     "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
+   done
+ fi


[09/21] git commit: fix paths and change spark to use APP_MEM as application driver memory instead of SPARK_MEM, user should add application jars to SPARK_CLASSPATH

Posted by pw...@apache.org.
fix paths and change spark to use APP_MEM as application driver memory instead of SPARK_MEM, user should add application jars to SPARK_CLASSPATH

Signed-off-by: shane-huang <sh...@intel.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/e8b1ee04
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/e8b1ee04
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/e8b1ee04

Branch: refs/heads/master
Commit: e8b1ee04fcb4cd7b666d3148d6d5ff148551ce72
Parents: 1409803
Author: shane-huang <sh...@intel.com>
Authored: Thu Sep 26 17:08:47 2013 +0800
Committer: shane-huang <sh...@intel.com>
Committed: Thu Sep 26 17:08:47 2013 +0800

----------------------------------------------------------------------
 bin/spark                                       | 41 ++++----------------
 .../mesos/CoarseMesosSchedulerBackend.scala     |  2 +-
 python/pyspark/java_gateway.py                  |  2 +-
 3 files changed, 10 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e8b1ee04/bin/spark
----------------------------------------------------------------------
diff --git a/bin/spark b/bin/spark
index 7f25fe1..aa005a5 100755
--- a/bin/spark
+++ b/bin/spark
@@ -31,40 +31,11 @@ if [ -e $FWDIR/conf/spark-env.sh ] ; then
 fi
 
 if [ -z "$1" ]; then
-  echo "Usage: spark-class <class> [<args>]" >&2
+  echo "Usage: spark <class> [<args>]" >&2
+  echo "Usage: export SPARK_CLASSPATH before running the command" >&2
   exit 1
 fi
 
-# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
-# values for that; it doesn't need a lot
-if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then
-  SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
-  SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
-  # Do not overwrite SPARK_JAVA_OPTS environment variable in this script
-  OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS"   # Empty by default
-else
-  OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
-fi
-
-
-# Add java opts for master, worker, executor. The opts maybe null
-case "$1" in
-  'org.apache.spark.deploy.master.Master')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
-    ;;
-  'org.apache.spark.deploy.worker.Worker')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
-    ;;
-  'org.apache.spark.executor.StandaloneExecutorBackend')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
-    ;;
-  'org.apache.spark.executor.MesosExecutorBackend')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
-    ;;
-  'org.apache.spark.repl.Main')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
-    ;;
-esac
 
 # Find the java binary
 if [ -n "${JAVA_HOME}" ]; then
@@ -78,14 +49,18 @@ else
   fi
 fi
 
-# Set SPARK_MEM if it isn't already set since we also use it for this process
+# Set SPARK_MEM if it isn't already set
 SPARK_MEM=${SPARK_MEM:-512m}
 export SPARK_MEM
 
+# Set APP_MEM if it isn't already set, we use this for this process as the app driver process may need 
+# as much memory as specified in SPARK_MEM
+APP_MEM=${APP_MEM:-512m}
+
 # Set JAVA_OPTS to be able to load native libraries and to set heap size
 JAVA_OPTS="$OUR_JAVA_OPTS"
 JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
-JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM"
+JAVA_OPTS="$JAVA_OPTS -Xms$APP_MEM -Xmx$APP_MEM"
 # Load extra JAVA_OPTS from conf/java-opts, if it exists
 if [ -e $FWDIR/conf/java-opts ] ; then
   JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e8b1ee04/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
index 9f93491..544b205 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
@@ -125,7 +125,7 @@ private[spark] class CoarseMesosSchedulerBackend(
       StandaloneSchedulerBackend.ACTOR_NAME)
     val uri = System.getProperty("spark.executor.uri")
     if (uri == null) {
-      val runScript = new File(sparkHome, "/sbin/spark-class").getCanonicalPath
+      val runScript = new File(sparkHome, "./sbin/spark-class").getCanonicalPath
       command.setValue(
         "\"%s\" org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
           runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e8b1ee04/python/pyspark/java_gateway.py
----------------------------------------------------------------------
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index f7834ef..b872ae6 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -31,7 +31,7 @@ def launch_gateway():
     # Launch the Py4j gateway using Spark's run command so that we pick up the
     # proper classpath and SPARK_MEM settings from spark-env.sh
     on_windows = platform.system() == "Windows"
-    script = "/sbin/spark-class.cmd" if on_windows else "/sbin/spark-class"
+    script = "./sbin/spark-class.cmd" if on_windows else "./sbin/spark-class"
     command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",
                "--die-on-broken-pipe", "0"]
     if not on_windows:


[19/21] git commit: sbin/spark-class* -> bin/spark-class*

Posted by pw...@apache.org.
sbin/spark-class* -> bin/spark-class*


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/74ba97fc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/74ba97fc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/74ba97fc

Branch: refs/heads/master
Commit: 74ba97fcf75ae61833295a896dcdf775036ae1d8
Parents: 59e8009
Author: Prashant Sharma <pr...@imaginea.com>
Authored: Fri Jan 3 15:08:01 2014 +0530
Committer: Prashant Sharma <pr...@imaginea.com>
Committed: Fri Jan 3 15:08:01 2014 +0530

----------------------------------------------------------------------
 bin/run-example                                 |   2 +-
 bin/run-example2.cmd                            |   2 +-
 bin/spark-class                                 | 154 +++++++++++++++++++
 bin/spark-class.cmd                             |  23 +++
 bin/spark-class2.cmd                            |  85 ++++++++++
 bin/spark-shell                                 |   4 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |   4 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |   4 +-
 .../scala/org/apache/spark/DriverSuite.scala    |   2 +-
 docs/running-on-yarn.md                         |   4 +-
 docs/spark-standalone.md                        |   2 +-
 python/pyspark/java_gateway.py                  |   2 +-
 sbin/spark-class                                | 154 -------------------
 sbin/spark-class.cmd                            |  23 ---
 sbin/spark-class2.cmd                           |  85 ----------
 sbin/spark-daemon.sh                            |   2 +-
 sbin/spark-executor                             |   2 +-
 17 files changed, 277 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/bin/run-example
----------------------------------------------------------------------
diff --git a/bin/run-example b/bin/run-example
index a3976be..f2699c3 100755
--- a/bin/run-example
+++ b/bin/run-example
@@ -36,7 +36,7 @@ if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
 fi
 
 if [ -z "$1" ]; then
-  echo "Usage: bin/run-example <example-class> [<args>]" >&2
+  echo "Usage: run-example <example-class> [<args>]" >&2
   exit 1
 fi
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/bin/run-example2.cmd
----------------------------------------------------------------------
diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd
index d9c1054..6861334 100644
--- a/bin/run-example2.cmd
+++ b/bin/run-example2.cmd
@@ -30,7 +30,7 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
 
 rem Test that an argument was given
 if not "x%1"=="x" goto arg_given
-  echo Usage: bin/run-example ^<example-class^> [^<args^>]
+  echo Usage: run-example ^<example-class^> [^<args^>]
   goto exit
 :arg_given
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/bin/spark-class
----------------------------------------------------------------------
diff --git a/bin/spark-class b/bin/spark-class
new file mode 100755
index 0000000..4e440d8
--- /dev/null
+++ b/bin/spark-class
@@ -0,0 +1,154 @@
+#!/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.
+#
+
+cygwin=false
+case "`uname`" in
+    CYGWIN*) cygwin=true;;
+esac
+
+SCALA_VERSION=2.10
+
+# Figure out where the Scala framework is installed
+FWDIR="$(cd `dirname $0`/..; pwd)"
+
+# Export this as SPARK_HOME
+export SPARK_HOME="$FWDIR"
+
+# 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 [ -z "$1" ]; then
+  echo "Usage: spark-class <class> [<args>]" >&2
+  exit 1
+fi
+
+# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
+# values for that; it doesn't need a lot
+if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then
+  SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
+  SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
+  # Do not overwrite SPARK_JAVA_OPTS environment variable in this script
+  OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS"   # Empty by default
+else
+  OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
+fi
+
+
+# Add java opts for master, worker, executor. The opts maybe null
+case "$1" in
+  'org.apache.spark.deploy.master.Master')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
+    ;;
+  'org.apache.spark.deploy.worker.Worker')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
+    ;;
+  'org.apache.spark.executor.CoarseGrainedExecutorBackend')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+    ;;
+  'org.apache.spark.executor.MesosExecutorBackend')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+    ;;
+  'org.apache.spark.repl.Main')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
+    ;;
+esac
+
+# Find the java binary
+if [ -n "${JAVA_HOME}" ]; then
+  RUNNER="${JAVA_HOME}/bin/java"
+else
+  if [ `command -v java` ]; then
+    RUNNER="java"
+  else
+    echo "JAVA_HOME is not set" >&2
+    exit 1
+  fi
+fi
+
+# Set SPARK_MEM if it isn't already set since we also use it for this process
+SPARK_MEM=${SPARK_MEM:-512m}
+export SPARK_MEM
+
+# Set JAVA_OPTS to be able to load native libraries and to set heap size
+JAVA_OPTS="$OUR_JAVA_OPTS"
+JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
+JAVA_OPTS="$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="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
+fi
+export JAVA_OPTS
+# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
+
+if [ ! -f "$FWDIR/RELEASE" ]; then
+  # Exit if the user hasn't compiled Spark
+  num_jars=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar" | wc -l)
+  jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar")
+  if [ "$num_jars" -eq "0" ]; then
+    echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2
+    echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2
+    exit 1
+  fi
+  if [ "$num_jars" -gt "1" ]; then
+    echo "Found multiple Spark assembly jars in $FWDIR/assembly/target/scala-$SCALA_VERSION:" >&2
+    echo "$jars_list"
+    echo "Please remove all but one jar."
+    exit 1
+  fi
+fi
+
+TOOLS_DIR="$FWDIR"/tools
+SPARK_TOOLS_JAR=""
+if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
+  # Use the JAR from the SBT build
+  export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
+fi
+if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then
+  # Use the JAR from the Maven build
+  # TODO: this also needs to become an assembly!
+  export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar`
+fi
+
+# Compute classpath using external script
+CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
+
+if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
+  CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR"
+fi
+
+if $cygwin; then
+  CLASSPATH=`cygpath -wp $CLASSPATH`
+  if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
+    export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR`
+  fi
+fi
+export CLASSPATH
+
+if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
+  echo -n "Spark Command: "
+  echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
+  echo "========================================"
+  echo
+fi
+
+exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
+
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/bin/spark-class.cmd
----------------------------------------------------------------------
diff --git a/bin/spark-class.cmd b/bin/spark-class.cmd
new file mode 100644
index 0000000..19850db
--- /dev/null
+++ b/bin/spark-class.cmd
@@ -0,0 +1,23 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+rem This is the entry point for running a Spark class. To avoid polluting
+rem the environment, it just launches a new cmd to do the real work.
+
+cmd /V /E /C %~dp0spark-class2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/bin/spark-class2.cmd
----------------------------------------------------------------------
diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd
new file mode 100644
index 0000000..460e661
--- /dev/null
+++ b/bin/spark-class2.cmd
@@ -0,0 +1,85 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+set SCALA_VERSION=2.10
+
+rem Figure out where the Spark framework is installed
+set FWDIR=%~dp0..\
+
+rem Export this as SPARK_HOME
+set SPARK_HOME=%FWDIR%
+
+rem Load environment variables from conf\spark-env.cmd, if it exists
+if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+
+rem Test that an argument was given
+if not "x%1"=="x" goto arg_given
+  echo Usage: spark-class ^<class^> [^<args^>]
+  goto exit
+:arg_given
+
+set RUNNING_DAEMON=0
+if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1
+if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1
+if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m
+set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true
+if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY%
+rem Do not overwrite SPARK_JAVA_OPTS environment variable in this script
+if "%RUNNING_DAEMON%"=="0" set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS%
+if "%RUNNING_DAEMON%"=="1" set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS%
+
+rem Figure out how much memory to use per executor and set it as an environment
+rem variable so that our process sees it and can report it to Mesos
+if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m
+
+rem Set JAVA_OPTS to be able to load native libraries and to set heap size
+set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM%
+rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
+
+rem Test whether the user has built Spark
+if exist "%FWDIR%RELEASE" goto skip_build_test
+set FOUND_JAR=0
+for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
+  set FOUND_JAR=1
+)
+if "%FOUND_JAR%"=="0" (
+  echo Failed to find Spark assembly JAR.
+  echo You need to build Spark with sbt\sbt assembly before running this program.
+  goto exit
+)
+:skip_build_test
+
+set TOOLS_DIR=%FWDIR%tools
+set SPARK_TOOLS_JAR=
+for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar") do (
+  set SPARK_TOOLS_JAR=%%d
+)
+
+rem Compute classpath using external script
+set DONT_PRINT_CLASSPATH=1
+call "%FWDIR%sbin\compute-classpath.cmd"
+set DONT_PRINT_CLASSPATH=0
+set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR%
+
+rem Figure out where java is.
+set RUNNER=java
+if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java
+
+"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %*
+:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/bin/spark-shell
----------------------------------------------------------------------
diff --git a/bin/spark-shell b/bin/spark-shell
index bc7386d..e6885b5 100755
--- a/bin/spark-shell
+++ b/bin/spark-shell
@@ -90,10 +90,10 @@ if $cygwin; then
     # "Backspace sends ^H" setting in "Keys" section of the Mintty options
     # (see https://github.com/sbt/sbt/issues/562).
     stty -icanon min 1 -echo > /dev/null 2>&1
-    $FWDIR/sbin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@"
+    $FWDIR/bin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@"
     stty icanon echo > /dev/null 2>&1
 else
-    $FWDIR/sbin/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
+    $FWDIR/bin/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
 fi
 
 # record the exit status lest it be overwritten:

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 0494ca8..d46fceb 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -127,7 +127,7 @@ private[spark] class CoarseMesosSchedulerBackend(
       CoarseGrainedSchedulerBackend.ACTOR_NAME)
     val uri = conf.get("spark.executor.uri", null)
     if (uri == null) {
-      val runScript = new File(sparkHome, "./sbin/spark-class").getCanonicalPath
+      val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath
       command.setValue(
         "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format(
           runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
@@ -136,7 +136,7 @@ private[spark] class CoarseMesosSchedulerBackend(
       // glob the directory "correctly".
       val basename = uri.split('/').last.split('.').head
       command.setValue(
-        "cd %s*; ./sbin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
+        "cd %s*; ./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
           .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
       command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index f207627..6ba1518 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -27,7 +27,7 @@ import org.apache.spark.scheduler.SchedulingMode
 /**
  * Continuously generates jobs that expose various features of the WebUI (internal testing tool).
  *
- * Usage: ./run spark.ui.UIWorkloadGenerator [master]
+ * Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]
  */
 private[spark] object UIWorkloadGenerator {
 
@@ -36,7 +36,7 @@ private[spark] object UIWorkloadGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
-      println("usage: ./sbin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
+      println("usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
       System.exit(1)
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/core/src/test/scala/org/apache/spark/DriverSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
index c37fd9a..605588f 100644
--- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@ -35,7 +35,7 @@ class DriverSuite extends FunSuite with Timeouts {
     val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]"))
     forAll(masters) { (master: String) =>
       failAfter(60 seconds) {
-        Utils.execute(Seq("./sbin/spark-class", "org.apache.spark.DriverWithoutCleanup", master),
+        Utils.execute(Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master),
           new File(System.getenv("SPARK_HOME")))
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index e2b21f9..e218123 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -54,7 +54,7 @@ There are two scheduler mode that can be used to launch spark application on YAR
 
 The command to launch the YARN Client is as follows:
 
-    SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./sbin/spark-class org.apache.spark.deploy.yarn.Client \
+    SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./bin/spark-class org.apache.spark.deploy.yarn.Client \
       --jar <YOUR_APP_JAR_FILE> \
       --class <APP_MAIN_CLASS> \
       --args <APP_MAIN_ARGUMENTS> \
@@ -79,7 +79,7 @@ For example:
 
     # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example
     $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
-        ./sbin/spark-class org.apache.spark.deploy.yarn.Client \
+        ./bin/spark-class org.apache.spark.deploy.yarn.Client \
           --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
           --class org.apache.spark.examples.SparkPi \
           --args yarn-standalone \

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index baa0a06..c851833 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -28,7 +28,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by
 
 Similarly, you can start one or more workers and connect them to the master via:
 
-    ./sbin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT
+    ./bin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT
 
 Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default).
 You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS).

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/python/pyspark/java_gateway.py
----------------------------------------------------------------------
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 7243ee6..c15add5 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -31,7 +31,7 @@ def launch_gateway():
     # Launch the Py4j gateway using Spark's run command so that we pick up the
     # proper classpath and SPARK_MEM settings from spark-env.sh
     on_windows = platform.system() == "Windows"
-    script = "./sbin/spark-class.cmd" if on_windows else "./sbin/spark-class"
+    script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class"
     command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",
                "--die-on-broken-pipe", "0"]
     if not on_windows:

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/sbin/spark-class
----------------------------------------------------------------------
diff --git a/sbin/spark-class b/sbin/spark-class
deleted file mode 100755
index 4e440d8..0000000
--- a/sbin/spark-class
+++ /dev/null
@@ -1,154 +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.
-#
-
-cygwin=false
-case "`uname`" in
-    CYGWIN*) cygwin=true;;
-esac
-
-SCALA_VERSION=2.10
-
-# Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`/..; pwd)"
-
-# Export this as SPARK_HOME
-export SPARK_HOME="$FWDIR"
-
-# 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 [ -z "$1" ]; then
-  echo "Usage: spark-class <class> [<args>]" >&2
-  exit 1
-fi
-
-# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
-# values for that; it doesn't need a lot
-if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then
-  SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
-  SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
-  # Do not overwrite SPARK_JAVA_OPTS environment variable in this script
-  OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS"   # Empty by default
-else
-  OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
-fi
-
-
-# Add java opts for master, worker, executor. The opts maybe null
-case "$1" in
-  'org.apache.spark.deploy.master.Master')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
-    ;;
-  'org.apache.spark.deploy.worker.Worker')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
-    ;;
-  'org.apache.spark.executor.CoarseGrainedExecutorBackend')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
-    ;;
-  'org.apache.spark.executor.MesosExecutorBackend')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
-    ;;
-  'org.apache.spark.repl.Main')
-    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
-    ;;
-esac
-
-# Find the java binary
-if [ -n "${JAVA_HOME}" ]; then
-  RUNNER="${JAVA_HOME}/bin/java"
-else
-  if [ `command -v java` ]; then
-    RUNNER="java"
-  else
-    echo "JAVA_HOME is not set" >&2
-    exit 1
-  fi
-fi
-
-# Set SPARK_MEM if it isn't already set since we also use it for this process
-SPARK_MEM=${SPARK_MEM:-512m}
-export SPARK_MEM
-
-# Set JAVA_OPTS to be able to load native libraries and to set heap size
-JAVA_OPTS="$OUR_JAVA_OPTS"
-JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
-JAVA_OPTS="$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="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
-fi
-export JAVA_OPTS
-# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
-
-if [ ! -f "$FWDIR/RELEASE" ]; then
-  # Exit if the user hasn't compiled Spark
-  num_jars=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar" | wc -l)
-  jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar")
-  if [ "$num_jars" -eq "0" ]; then
-    echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2
-    echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2
-    exit 1
-  fi
-  if [ "$num_jars" -gt "1" ]; then
-    echo "Found multiple Spark assembly jars in $FWDIR/assembly/target/scala-$SCALA_VERSION:" >&2
-    echo "$jars_list"
-    echo "Please remove all but one jar."
-    exit 1
-  fi
-fi
-
-TOOLS_DIR="$FWDIR"/tools
-SPARK_TOOLS_JAR=""
-if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
-  # Use the JAR from the SBT build
-  export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
-fi
-if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then
-  # Use the JAR from the Maven build
-  # TODO: this also needs to become an assembly!
-  export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar`
-fi
-
-# Compute classpath using external script
-CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
-
-if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
-  CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR"
-fi
-
-if $cygwin; then
-  CLASSPATH=`cygpath -wp $CLASSPATH`
-  if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
-    export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR`
-  fi
-fi
-export CLASSPATH
-
-if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
-  echo -n "Spark Command: "
-  echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
-  echo "========================================"
-  echo
-fi
-
-exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
-
-

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/sbin/spark-class.cmd
----------------------------------------------------------------------
diff --git a/sbin/spark-class.cmd b/sbin/spark-class.cmd
deleted file mode 100644
index 19850db..0000000
--- a/sbin/spark-class.cmd
+++ /dev/null
@@ -1,23 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-rem This is the entry point for running a Spark class. To avoid polluting
-rem the environment, it just launches a new cmd to do the real work.
-
-cmd /V /E /C %~dp0spark-class2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/sbin/spark-class2.cmd
----------------------------------------------------------------------
diff --git a/sbin/spark-class2.cmd b/sbin/spark-class2.cmd
deleted file mode 100644
index 460e661..0000000
--- a/sbin/spark-class2.cmd
+++ /dev/null
@@ -1,85 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-set SCALA_VERSION=2.10
-
-rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0..\
-
-rem Export this as SPARK_HOME
-set SPARK_HOME=%FWDIR%
-
-rem Load environment variables from conf\spark-env.cmd, if it exists
-if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
-
-rem Test that an argument was given
-if not "x%1"=="x" goto arg_given
-  echo Usage: spark-class ^<class^> [^<args^>]
-  goto exit
-:arg_given
-
-set RUNNING_DAEMON=0
-if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1
-if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1
-if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m
-set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true
-if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY%
-rem Do not overwrite SPARK_JAVA_OPTS environment variable in this script
-if "%RUNNING_DAEMON%"=="0" set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS%
-if "%RUNNING_DAEMON%"=="1" set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS%
-
-rem Figure out how much memory to use per executor and set it as an environment
-rem variable so that our process sees it and can report it to Mesos
-if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m
-
-rem Set JAVA_OPTS to be able to load native libraries and to set heap size
-set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM%
-rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
-
-rem Test whether the user has built Spark
-if exist "%FWDIR%RELEASE" goto skip_build_test
-set FOUND_JAR=0
-for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
-  set FOUND_JAR=1
-)
-if "%FOUND_JAR%"=="0" (
-  echo Failed to find Spark assembly JAR.
-  echo You need to build Spark with sbt\sbt assembly before running this program.
-  goto exit
-)
-:skip_build_test
-
-set TOOLS_DIR=%FWDIR%tools
-set SPARK_TOOLS_JAR=
-for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar") do (
-  set SPARK_TOOLS_JAR=%%d
-)
-
-rem Compute classpath using external script
-set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%sbin\compute-classpath.cmd"
-set DONT_PRINT_CLASSPATH=0
-set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR%
-
-rem Figure out where java is.
-set RUNNER=java
-if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java
-
-"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %*
-:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/sbin/spark-daemon.sh
----------------------------------------------------------------------
diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh
index ca6b893..2be2b3d 100755
--- a/sbin/spark-daemon.sh
+++ b/sbin/spark-daemon.sh
@@ -147,7 +147,7 @@ case $startStop in
     spark_rotate_log "$log"
     echo starting $command, logging to $log
     cd "$SPARK_PREFIX"
-    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/sbin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
+    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
     newpid=$!
     echo $newpid > $pid
     sleep 2

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/74ba97fc/sbin/spark-executor
----------------------------------------------------------------------
diff --git a/sbin/spark-executor b/sbin/spark-executor
index 214e00f..de5bfab 100755
--- a/sbin/spark-executor
+++ b/sbin/spark-executor
@@ -20,4 +20,4 @@
 FWDIR="$(cd `dirname $0`/..; pwd)"
 
 echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/sbin/spark-class org.apache.spark.executor.MesosExecutorBackend
+exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend


[18/21] git commit: a few left over document change

Posted by pw...@apache.org.
a few left over document change


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/59e8009b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/59e8009b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/59e8009b

Branch: refs/heads/master
Commit: 59e8009b8d5e51b6f776720de8c9ecb09e1072dc
Parents: a3f90a2
Author: Prashant Sharma <pr...@imaginea.com>
Authored: Thu Jan 2 18:54:08 2014 +0530
Committer: Prashant Sharma <sc...@gmail.com>
Committed: Thu Jan 2 21:48:44 2014 +0530

----------------------------------------------------------------------
 docs/spark-standalone.md | 2 +-
 ec2/spark_ec2.py         | 2 +-
 make-distribution.sh     | 4 ++--
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/59e8009b/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index ec5ae5b..baa0a06 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -20,7 +20,7 @@ then modify `conf/spark-env.sh` in the `dist/` directory before deploying to all
 
 You can start a standalone master server by executing:
 
-    ./bin/start-master.sh
+    ./sbin/start-master.sh
 
 Once started, the master will print out a `spark://HOST:PORT` URL for itself, which you can use to connect workers to it,
 or pass as the "master" argument to `SparkContext`. You can also find this URL on

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/59e8009b/ec2/spark_ec2.py
----------------------------------------------------------------------
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index a2b0e7e..d82a1e1 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -436,7 +436,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
 def setup_standalone_cluster(master, slave_nodes, opts):
   slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes])
   ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips))
-  ssh(master, opts, "/root/spark/bin/start-all.sh")
+  ssh(master, opts, "/root/spark/sbin/start-all.sh")
 
 def setup_spark_cluster(master, opts):
   ssh(master, opts, "chmod u+x spark-ec2/setup.sh")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/59e8009b/make-distribution.sh
----------------------------------------------------------------------
diff --git a/make-distribution.sh b/make-distribution.sh
index 8765c7e..0463d14 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -31,9 +31,9 @@
 #
 # Recommended deploy/testing procedure (standalone mode):
 # 1) Rsync / deploy the dist/ dir to one host
-# 2) cd to deploy dir; ./bin/start-master.sh
+# 2) cd to deploy dir; ./sbin/start-master.sh
 # 3) Verify master is up by visiting web page, ie http://master-ip:8080.  Note the spark:// URL.
-# 4) ./bin/start-slave.sh 1 <<spark:// URL>>
+# 4) ./sbin/start-slave.sh 1 <<spark:// URL>>
 # 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell
 #
 


[04/21] moved user scripts to bin folder

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/kmeans_data.txt
----------------------------------------------------------------------
diff --git a/kmeans_data.txt b/kmeans_data.txt
deleted file mode 100644
index 338664f..0000000
--- a/kmeans_data.txt
+++ /dev/null
@@ -1,6 +0,0 @@
-0.0 0.0 0.0
-0.1 0.1 0.1
-0.2 0.2 0.2
-9.0 9.0 9.0
-9.1 9.1 9.1
-9.2 9.2 9.2

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/lr_data.txt
----------------------------------------------------------------------
diff --git a/lr_data.txt b/lr_data.txt
deleted file mode 100644
index d4df063..0000000
--- a/lr_data.txt
+++ /dev/null
@@ -1,1000 +0,0 @@
-1 2.1419053154730548 1.919407948982788 0.0501333631091041 -0.10699028639933772 1.2809776380727795 1.6846227956326554 0.18277859260127316 -0.39664340267804343 0.8090554869291249 2.48621339239065
-1 1.8023071496873626 0.8784870753345065 2.4105062239438624 0.3597672177864262 -0.20964445925329134 1.3537576978720287 0.5096503508009924 1.5507215382743629 -0.20355100196508347 1.3210160806416416
-1 2.5511476388671834 1.438530286247105 1.481598060824539 2.519631078968068 0.7231682708126751 0.9160610215051366 2.255833005788796 0.6747272061334229 0.8267096669389163 -0.8585851445864527
-1 2.4238069456328435 -0.3637260240750231 -0.964666098753878 0.08140515606581078 -1.5488873933848062 -0.6309606578419305 0.8779952253801084 2.289159071801577 0.7308611443440066 1.257491408509089
-1 0.6800856239954673 -0.7684998592513064 0.5165496871407542 0.4900095346106301 2.116673376966199 0.9590527984827171 -0.10767151692007948 2.8623214176471947 2.1457411377091526 -0.05867720489309214
-1 2.0725991339400673 -0.9317441520296659 1.30102521611535 1.2475231582804265 2.4061568492490872 -0.5202207203569256 1.2709294126920896 1.5612492848137771 0.4701704219631393 1.5390221914988276
-1 3.2123402141787243 0.36706643122715576 -0.8831759122084633 1.3865659853763344 1.3258292709064945 0.09869568049999977 0.9973196910923824 0.5260407450146751 0.4520218452340974 0.9808998515280365
-1 2.6468163882596327 -0.10706259221579106 1.5938103926672538 0.8443353789148835 1.6632872929286855 2.2267933606886228 1.8839698437730905 1.2217245467021294 1.9197020859698617 0.2606241814111323
-1 1.803517749531419 0.7460582552369641 0.23616113949394446 -0.8645567427274516 -0.861306200027518 0.423400118883695 0.5910061937877524 1.2484609376165419 0.5190870450972256 1.4462120573539101
-1 0.5534111111196087 1.0456386878650537 1.704566327313564 0.7281759816328417 1.0807487791523882 2.2590964696340183 1.7635098382407333 2.7220810801509723 1.1459500540537249 0.005336987537813309
-1 1.2007496259633872 1.8962364439355677 2.5117192131332224 -0.40347372807487814 -0.9069696484274985 2.3685654487373133 0.44032696763461554 1.7446081536741977 2.5736655956810672 2.128043441818191
-1 0.8079184133027463 -1.2544936618345086 1.439851862908128 1.6568003265998676 0.2550498385706287 2.1994753269490133 2.7797467521986703 1.0674041520757056 2.2950640220107115 0.4173234715497547
-1 1.7688682382458407 1.4176645501737688 0.5309077640093247 1.4141481732625842 1.663022727536151 1.8671946375362718 1.2967008778056806 1.3215230565153893 3.2242953580982188 1.8358482078498959
-1 -0.1933022979733765 1.1188051459900596 1.5580410346433533 -0.9527104650970353 2.4960553383489517 0.2374178113187807 1.8951776489120973 0.817329097076558 1.9297634639960395 0.5625196401726915
-1 0.8950890609697704 0.3885617561119906 1.3527646644845603 -0.14451661079866773 0.34616820106951784 3.677097108514281 1.1513217164424643 2.8470372001182738 1.440743314981174 1.8773090852445982
-1 1.946980694388772 0.3002263539854614 -1.315207227451069 1.0948002011749645 1.1920371028231238 -0.008130832288609113 -1.150717205632501 2.6170416083849215 1.5473509656354905 2.6230096333098776
-1 1.369669298870147 2.2240526315272633 1.8751209163514155 0.7099955723660032 1.4333345396190893 2.0069743967645715 2.783008145523796 2.356870316505785 1.4459302415658664 2.3915127940536753
-1 1.0329554152547427 0.19817512014940342 0.9828173667832262 -0.3164854365297216 0.9721814447840595 2.9719833390831583 2.3758681039407463 -0.2706898498985282 1.2920337802284907 2.533319271731563
-1 1.1046204258897305 -0.31316036717589113 2.779996494431689 1.3952547694086233 0.49953716767570155 -1.0407393926238933 2.0869289165797924 -0.04084913117769684 2.9616582572418197 1.9258632212977318
-1 2.361656934659277 3.8896525506477344 0.5089863292545287 0.28980141682319804 2.570466720662197 0.15759150270048905 0.6680692313979322 -0.698847669879108 0.4688584882078929 -1.5875629832762232
-1 1.301564524776174 -0.15280528962364026 -0.7133285086762593 1.081319758035075 -0.3278612176303164 1.6965862080356764 -0.28767133135763223 2.2509059068665724 1.0125522002674598 1.6566974914450203
-1 -0.3213530059013969 1.8149172295041944 1.6110409277400992 1.1234808948785417 1.3884025750196511 0.41787276194289835 1.4334356888417783 0.20395689549800888 1.0639952991231423 0.25788892433087685
-1 2.1806635961066307 1.9198186083780135 2.238005178835123 0.9291144984960873 0.4341039397491093 2.050821228244721 1.9441165305261188 0.30883909322226666 1.8859638093504212 -1.533371339542391
-1 1.4163203752064484 1.4062903984061705 1.8418616457792907 0.6519263935739821 2.0703545150299583 0.7652230912847241 1.1557263986072353 1.6683095785190067 1.3685121432402299 1.0970993371965074
-1 -0.23885375176985146 0.7346703244086044 0.39686127458413645 0.8536167113915564 2.8821103658250253 2.843586967989016 0.2256284103968883 0.8466499260789964 1.1372088070346282 0.0880674005359322
-1 1.190682102191321 1.7232172113039872 0.5636637342794258 0.8190845829178903 1.803778929309528 2.386253140767585 0.651507090146642 2.053713849719438 1.049889279545437 2.367448527229836
-1 1.2667391586127408 1.0272601665986936 0.1694838905810353 1.3980698432838456 1.2347363543406824 1.519978239538835 0.7755635065536938 1.9518789476720877 0.8463891970929239 -0.1594658182609312
-1 1.9177143967118988 0.1062210539075672 1.0776111251281053 1.969732837479783 0.5806581670596382 0.9622645870604398 0.5267699759271061 0.14462924425226986 3.205183137564584 0.3349768610796714
-1 2.8022977941941876 1.7233623251887376 1.8343656581164236 2.5078868235362135 2.8732773429688496 1.175657348763883 1.8230498418068863 -0.06420099579179217 -0.31850161026000223 1.3953402446037735
-1 1.293815946466546 1.9082454404595959 1.0390424276302468 1.4123446397119441 0.14272371474828127 0.5954644427489499 1.9311182993772318 1.4425836945233532 0.23593915711070867 -0.0046799615367818514
-1 2.1489058966224226 1.5823735498702165 0.47984538863958215 0.05725411130294378 -0.19205537448285037 2.578016006340281 2.635623602110286 1.9829002135878433 0.19799288106884738 1.7028918814014005
-1 1.5672862680104924 -0.0987393491518127 0.7244061201774454 -0.41182579172916434 1.1979110917942835 -0.12481753033835274 0.5630131395041615 1.385537735117697 -0.8919101455344216 2.7424648070251116
-1 0.6879772771184975 1.582111812261079 0.3665634721723976 0.850798208790375 0.9426300131823666 1.983603842699607 0.8130990941989288 -1.0826899070777283 0.7979163057567745 -0.12841040130621417
-1 0.49726755658797983 1.1012109678729847 0.27184530927569217 0.09590187123183869 2.7114680848906723 1.0712539490680686 0.4661357697833658 1.1666136730805596 1.0060435328852553 1.3752864302671253
-1 1.5705074035386362 2.5388314004618415 3.705325086899449 1.7253747699098896 0.2905920924621258 2.2062201954483274 1.7686772759307146 -0.14389818761776474 1.317117811881067 1.960659458484061
-1 -0.6097266693243066 1.5050792404611277 1.5597531261282835 1.801921952517151 1.021637610172004 1.0147308245966982 0.496200008835183 1.2470065877402576 1.09033470655824 2.154244343371553
-1 1.7311626690342417 -0.7981106861881657 1.576306673263288 2.0139307462486293 0.9669340713114077 2.6079849454993758 2.4417756902619443 0.97773788498047 -0.02280274021786477 1.9625031913007136
-1 0.034608060780454086 0.43324370378601906 0.6464567365972307 0.16942820411876358 2.773634414356671 0.950387120399953 0.20399015246948005 2.45383876915324 1.4728192154140967 0.27665303590986445
-1 0.669423341908155 2.753528514524716 -0.3114457433066151 0.42623362468295967 0.17585723777040074 0.3896466198418058 3.382230016050147 0.5628980580934769 0.1855399231085304 -1.0368812374682252
-1 1.1578929223859837 -0.9772673038070927 1.628472811304047 0.1706064825334408 -0.4368078914563116 1.3238749660151412 -0.6328206376503045 -0.1268798336415804 1.4614917163766068 0.05098215234403425
-1 1.9810025566400666 1.076214892921874 -1.1668914854936587 1.6219892570599912 0.5991126181156119 1.0668387700181805 -0.38561466584746307 -0.3346008538706646 -0.13693208851002447 1.082271823637847
-1 1.6753996221697711 -0.2204800911406224 1.3643600908733924 1.3667965239511641 1.4202494777278367 0.1990171616310349 1.3814657607888683 1.0156848718344853 1.1547747341458854 1.919747223811457
-1 2.306325804101286 2.013331566156439 1.1223877708770225 -0.06481662603037197 1.7942868367810174 0.7587370182842376 0.8698939230717255 0.37170451929485726 1.353135265304875 -0.013085996169272862
-1 0.20271462066175472 1.8670116701629946 0.1618067461065149 -0.2974653145373134 2.0274885311314446 1.7489571027636028 2.991328245656333 2.3823300780216257 2.078511519846326 1.97782037580114
-1 2.2596721244733233 1.006588878797566 2.2453074888557705 0.4245510909203909 1.557587461354759 1.7728855159117356 1.0648265192392103 1.1365923061997036 0.5379050122382909 0.9997617294083609
-1 2.414464891572643 0.30469754105126257 2.1935238570960616 2.587308021245376 1.5756963983924648 1.9319407933274975 0.8074477639415376 1.7357619185236388 0.23815230672958865 -0.4761137753554259
-1 1.3855245092290591 1.955100157523304 1.4341819377958671 0.28696565179644584 1.7291061523286055 1.714048489489178 1.164672495926134 1.6545959369641716 1.9496841789853843 2.5374349926535062
-1 1.1158271727931894 2.213425162173939 1.36638012222097 -0.023757883337165886 2.406876786398608 1.1126742159637397 0.12318438504039564 2.8153485847571273 0.15506376286728374 0.33355971489136393
-1 1.7297171728443748 0.6719390218027237 1.3753247894650051 -0.10182607341800742 1.7453755134851177 1.0960805604241037 0.40205225932790567 1.6103118877057256 -1.03955805358224 -0.3213966754338211
-1 1.316257046547979 1.2853238426515166 2.0480481778475728 0.6602539720919305 0.7379613133231193 2.0626091656565495 1.4509651703701687 1.864003948893211 2.2982171285406796 0.9359019132591221
-1 1.6046620370312947 2.321499271109006 2.2161407602345786 0.5862066390480085 -1.06591519642831 0.4488708706540525 0.9764088582932869 -0.17539686817265143 1.0261570987217379 1.8924236336247766
-1 -0.013917852015644883 0.4901030850643481 0.574360829130456 0.08844371614484736 1.3233068279136773 0.7589759244353294 1.7201737182853447 0.517426440952053 2.7274693051068777 0.036397493927961544
-1 1.2232096749473036 1.4768480172452538 1.5300887552091489 1.8810354040615782 -0.6436862913845212 1.5878631039716906 0.09394891272528805 1.7766036014727926 -0.08618397395873112 1.5926757324414604
-1 -0.006190798924250895 -1.1803586949394225 2.237721401521945 0.7324966516613158 1.4038442669165114 -0.06019103023815764 -0.7655029652453154 -0.3991986433215591 2.3296187529650685 0.38065062537135896
-1 1.0869918851572522 -0.37412852726006984 0.27965894114884915 -0.0733849426330444 0.7458288899809582 0.38504406064556884 1.3823407462352355 1.0530056181901168 -0.10908828320629294 -0.3163748213825457
-1 2.0800232080218937 0.6793681518120379 1.0126904247021766 0.5099365686965533 1.4765728601491988 -0.90922098444035 0.01578092821031385 2.531202299543557 1.3694116442965245 0.03526109196146243
-1 2.52004533036052 -0.11716335755537322 2.043801269881338 -0.4889959907470973 1.3717334116816158 -0.5907796618760839 2.9080140714861864 2.3969176626246114 0.9445325920064912 0.9620736405334235
-1 0.8261430232725533 0.9003472941846893 1.2648199316806048 1.3110765897825498 0.9484044458467761 1.5971370020069537 1.89838012162931 0.5844972943740565 2.1114035373528974 2.8066708339226407
-1 1.7131825192258492 0.5164803724034563 1.3400031460569826 1.159025272879641 -0.6475319792487726 0.7895415906096561 0.3591049378091684 0.3507368152114154 0.46463582975963413 1.2784917703092404
-1 0.9196047831077019 0.6917912743533342 1.7505158395265692 2.275307243506136 2.9871554281485713 0.584299496238456 1.2741949422522685 0.42838234246585094 2.613957509033075 1.479280190769243
-1 0.6865489083893408 1.6888181847006614 1.5612615114298305 0.28075030293939784 0.7611637101018122 0.17543992215891036 0.8532136322118986 1.6171101997247541 2.487562859731773 2.1695780390240165
-1 3.746488178488735 0.5902211931946351 1.4116785188193897 -0.302213259977852 1.3900348431280398 1.8058092139513118 1.9063920023065686 -0.6748417828946516 1.2856680423450677 1.4181322176013937
-1 1.3957855809267268 0.6788775338735233 1.2694449274462256 0.7739220722195589 1.6662774494836934 0.2263815064326532 0.3746198256735065 0.6981525121209534 0.6659194682736781 2.34383566814983
-1 0.3820962920141968 -0.11474969137094182 1.4456430767826618 1.7541264342573286 0.5841263905944027 0.3310478153678522 0.1361074962599954 2.1517668203954323 2.1312973802189523 0.08816171787088545
-1 0.44857483955792765 -1.3332507048491813 0.5685902212376108 1.1213432607484823 2.634120632788485 0.7837711869120604 1.0078687896423884 1.8982652887205418 1.1818816137394528 1.2876714951624808
-1 1.1951146419526084 0.9947742549449248 0.19840725400812698 2.48569644222758 1.7391898607628944 2.40036741337463 2.0600530189294144 -0.5340832975220873 2.0467391216154094 1.1908285513553203
-1 0.9918935330929904 -0.3542942677260328 1.3105513869382395 1.1904643448960697 -0.3602658438636872 0.6816024636806379 1.9768303812038046 0.4000132856795251 0.09352911692893684 1.9754791705404877
-1 1.0081698742896188 0.8916746417259931 1.496601632133103 1.8174757593692714 0.49297596177715564 1.828839820849067 1.662627028300793 1.2253219256823615 -1.6200329115107013 1.051770724619957
-1 0.9867026242209636 2.0915066394830326 0.2608828095090572 1.5275154403994393 0.3157310747415396 -0.7181525036523673 1.281115387917441 2.286539214837881 0.5653973688805878 3.0047565660570132
-1 0.9224469399191068 1.2533868053906783 -0.10077556308999824 0.06127395021274762 -0.18013801007271568 0.8043572428627129 -0.3236336059948026 1.6130489732175104 3.313472221318618 -0.15122165909659913
-1 0.7882345197971014 1.141304212890955 0.9030550623054504 2.543084656196279 0.7468302223968317 1.6832418500477586 0.10324287869065907 0.8952909318554702 1.7968146536867757 1.8337447891715968
-1 1.5801885793428398 2.438564562880532 1.346652611597816 2.013682644266395 0.5423884037920474 1.5509096942566918 -0.09721979565291483 0.7802050454421068 -0.07405588910002847 1.1020403166091144
-1 0.03083257777543913 0.09561020933135189 2.783828684436811 0.6702011711663662 1.1177709598763554 1.507733845629784 0.7190681946142053 0.4421675532332505 2.0062047937031338 1.3078544626787887
-1 0.029946310071738202 2.9974008035637247 1.2712685297793174 1.564287715942167 0.9318120646963208 1.9611220391387494 0.6955370789941844 2.8474941997466665 1.7216550057775473 1.033229285227095
-1 1.7919476706914224 2.674070943673579 1.0707436458201804 -1.2652465769212773 0.13786669485292458 -0.9521873641153344 -0.5112273884476357 1.8041566655420045 2.0489287678822823 1.4526766050251194
-1 2.1567394248692624 0.2787475011337476 1.2693515582998967 2.141920061908346 -0.311063434715769 2.7871358520284515 0.4011362416354143 1.2240722802790835 2.0224267357566696 0.6055884380482317
-1 1.2810578825169523 -0.06149076783837382 -0.3631214532063931 1.8242040060835376 0.936708636871513 0.9599645524867305 -0.2864664075189678 1.4575636141356014 -0.6521604857506678 1.4782024605158144
-1 1.922007864215502 0.41092515579085087 1.3614694131826193 1.2516141141035275 1.1032104604396404 1.5618738178080496 0.22277705609915832 -0.10552941002887595 0.8187789394182741 1.1899147160759034
-1 -1.101159111435701 2.0868811582857676 2.061754901850132 0.831389858205579 1.1022205058106118 -0.15327367461990105 3.263172683870654 -0.13185404063281925 0.4215198415563227 0.5983645772645423
-1 0.9017414538285525 1.5815719854072032 -0.33621575096987555 0.7353127316624433 2.000881249246564 1.752079037914068 2.188342812418916 2.464770657128536 1.9873120348231552 2.5280681270799197
-1 0.36229490936502484 0.9764447193507352 0.5513927408959507 1.2450834166369436 1.0347591040069144 0.23319917869834939 2.9368656872660264 1.3867291773435497 2.0279815142744324 1.3025138236731233
-1 0.12338005279277287 -0.11881556712737162 1.0293241194113785 2.053803566510112 1.694932390223226 1.2851644900727108 -0.09123042470171838 1.4542526750729492 0.9314422039244139 1.484525799738803
-1 2.2791038050359416 0.13652686573061323 0.34425341235820794 0.5134789845294401 1.199131994695721 1.285766903846671 1.6396476063943415 0.37354865288496775 -0.9325874103952065 1.9432993173271385
-1 0.3187247126988978 -0.23565755255952947 1.4653008405179144 1.4073930754043715 1.86867235923796 -0.8601040662125556 0.17314198154775828 1.359209951341465 1.8780560671833557 1.0497896254122507
-1 -0.35095212337482606 2.1382594819736456 0.21582557882234288 1.563987660659988 0.8742557302587846 2.7376537243676307 1.1089682445267717 0.3906567030119056 0.90272045105723 0.3199475930277361
-1 -1.0755666969659972 2.587500753780116 0.43523091172933415 1.9715380667335656 -1.206591074948113 2.3082117218149953 2.9003512906773183 1.8894617822889117 0.2612428397679113 2.3034517860165904
-1 1.2752641746970284 -0.8368104009920136 0.03573979915049008 0.9337645939367554 1.8180936927791564 0.35607066313035163 0.9553794086170463 2.3774664468818862 0.27151841486690464 0.5861688049602704
-1 1.3242463950740633 1.5079874960068127 2.2093340505083026 1.2611978264745287 1.7161846809846164 -0.49880331209390905 2.2386520558115137 1.259321190419847 1.3434715137362212 2.044909528652566
-1 0.8795598947051465 1.8282710612070696 0.8010144751459073 0.6664561865521288 0.4104626238753195 0.23255356821870798 0.33916496869925716 -0.2708146821069548 0.9241466333878707 -0.450452229744047
-1 1.9192448235188513 0.4969214523219533 2.4011260745046066 1.1346909629811026 -0.6596351603517379 -0.5351409933958904 0.02441943738258512 2.288141877404522 1.2367780341721122 1.584102117316426
-1 0.9682490849657925 -1.8650300168768377 0.8811925017526988 1.1594483122156354 1.121203677520715 0.9099984493527551 0.08826662255652562 -0.7539889420899628 0.4595729579317809 -0.7165782835963082
-1 1.5995281560764565 0.20521558652985616 -1.1164794717138746 1.5074668507140967 0.7877952768927691 0.902667397635835 1.6081861816054732 1.3133186016363785 1.5296162271430345 1.0712740040810271
-1 0.42211731340992986 0.502442828209289 0.3565737103297629 0.4478456815580649 1.617182070323055 0.9823042873485613 1.0704168281976632 -0.26776498356102985 1.8711459938723063 0.791693835933734
-1 0.23896637909254625 0.6184009702378752 1.484473242669571 -2.0960256478350034 1.007509277044258 1.4880525091303394 0.14825818901395527 2.918617492389175 2.7162682081607343 1.2852769131414254
-1 0.09951845043296148 0.10778080557671554 1.6153805572528395 0.21496629935184874 0.5695206599630613 0.5995686906470605 1.6226444344121718 1.400956890784598 2.5804792645155237 1.8818183326984712
-1 1.5660653841435699 1.9424448683907583 -0.5018032946330131 0.38813943551967744 0.21678795998247846 0.4592981799067166 0.3853775631077989 0.782922855791653 2.9697907962454226 2.0478747128589188
-1 0.5992085726320009 0.8326763829762222 1.0404230260991942 1.3571653199047529 0.05351664648320875 -1.8860610207228041 -0.5191719995314692 1.4226132032544871 1.6669779033604124 0.3253081253110943
-1 1.5903828533545434 1.894569333674546 1.5910544740636994 -1.6611392075582438 0.23842067636563624 -0.5406681576023691 1.7385589161163928 0.08969602776306584 1.4276561463432735 2.1566164427616634
-1 1.1913811808857528 0.32434695668325997 1.323498708189486 1.3596937187302878 3.4642496063989223 1.2876491657559253 -0.6543683402478666 1.4762502189363769 1.7353590098925795 2.8134629202660317
-1 3.123286693375267 1.877368736310955 0.9503145430714942 0.5342686470311402 0.3451961663217381 0.23995547380392213 0.5196925578399603 1.3087329089934692 0.5609549451755507 2.0018380155694433
-1 -0.70471754448335 0.396960196596961 2.8076920787881408 1.0486680479609312 0.1272088037522776 0.46477225522402743 1.0400518017377827 1.724354900707523 0.5172234824476354 0.70073364273413
-1 -0.04890176228714482 1.183623201015611 0.31679837772569197 2.442803942979677 2.475613952046278 1.316874640917748 2.1326668609632957 -1.1984022921949467 1.6326265827096553 0.13549684503148585
-1 1.532730344901386 1.8862673099243719 0.8433953501998975 0.9617349215859397 0.9632178266458564 1.7656392455188015 0.6166388141868028 0.36673723822668447 1.6148100615636092 1.9120508667715108
-1 1.8531415713908175 1.9856258806463458 0.8742545608077308 0.01891740612207793 0.754430421572012 1.2629533382356322 2.5668913595968625 0.7074626529557771 1.471180058040478 0.14210105766798764
-1 0.2946588114247314 1.7385325023150382 2.05805803890677 1.1285587768294627 0.30443899971020716 0.17710198470084348 -0.5876955744308521 1.6684452883987464 0.7429316176330647 0.24223269345723197
-1 0.12828383509135766 2.8251621371579123 -0.8683350630211126 1.3881503321455106 -0.9269673097143274 1.1340435175521124 1.1482061370168226 0.9886836766952749 1.3639211879675324 2.221424872356976
-1 1.6230819590031813 2.1140726634236273 0.8803195980146348 0.6957671564440406 1.3391648515238626 3.3118192086623672 1.206763244141946 0.5724427229085818 2.3692467877986934 1.2731917884083277
-1 0.6095837137279339 2.0886462170941087 1.5293277948541921 0.875698342933093 0.9739071638488416 -0.6284005601740021 0.7080909588024915 1.2483475820206364 0.39878604428574227 0.45167768471833614
-1 0.6622065044914254 0.7302732598978321 1.5839711558395906 0.33559568645900273 1.3094508963156517 1.5256964735790022 -0.2606881050391294 -0.13646086393521872 0.858395568393544 0.7983659548572369
-1 1.6030491170288057 0.8411660994073609 2.2968025114870225 0.7039288437264786 2.8125132767337133 0.23511452019598467 1.1415093151481583 -0.5416578453683565 2.121640334408583 -0.29666850192733474
-1 2.0779652161151883 1.0668503227493862 -0.3461938034511103 -1.9467096604673708 -0.4997902436835773 0.3419044702794434 0.8098524987621489 0.8131208951963917 1.3237950963836287 1.0429693266336961
-1 0.37001171609371697 0.29180348786692334 -0.2507809978364861 1.152821888667346 3.0890087304413267 1.215489406549123 1.199447470435283 0.789305354976556 0.8365245923088752 0.9787024262828808
-1 0.9296046114728362 2.19739063739452 1.533572358281578 0.7759925327491899 1.557482584766074 1.7151021392829757 0.9544359521103486 0.20077841759520276 1.59524901629763 2.175430873131662
-1 0.8112131582336873 0.2864940430793351 0.5833958780431041 1.7741485867050852 0.7779977372833543 1.8236769123328878 1.9278891617195901 -1.0188957672300982 0.9197794797358201 0.045052296436480455
-1 1.3702354298117274 0.5815346064645623 -0.04109583670633299 2.5064872968829004 1.206757887015013 0.2506549572813025 0.655306538898329 -0.3438030831151808 0.36458112520078056 0.8710435445702591
-1 1.4561762683494108 0.9681359328856552 3.136045420267423 0.7520560598452287 1.6528697058481434 0.9607920473099414 0.7156379077840067 1.857016542269911 -0.16277187766324142 0.4874157744630184
-1 1.2664980583047298 0.4023544599875911 0.9080313985150303 0.6549364577494126 2.738329489381062 2.3768996789882744 1.3393128915299277 -1.0430311123744418 0.8323494096430804 -0.12738742588819885
-1 0.8365391310807251 2.2822870725882503 2.6266615690102215 0.004265515881109128 2.4879345431323623 0.4875299849317022 1.351118317094851 1.245328886439785 0.8575534087593427 0.669435902035294
-1 0.8058511262644885 0.7473099050414014 2.303189816277799 1.2225351585963724 1.8247316651754097 -0.30810342366775534 0.2821704820687452 -1.6099991877186302 0.8406234201201898 2.0583805330826985
-1 2.250164789914201 1.7436544269774978 2.947667398091067 1.4771471077132423 -1.586188610201127 2.320910876555482 1.636258094383067 1.2987326716659215 -1.311058489828028 -0.011700890501986194
-1 0.8080250762510234 1.6440873832130936 0.8879459460961949 1.2082440017762488 -0.3984868670511643 -1.6750959916314896 0.9349087046999264 0.7232463907082566 2.2386173679423806 -0.017579999213251485
-1 1.0323998857804233 -0.7718677431568479 1.776325436331275 0.5932669960371175 1.7054720461060777 1.709001306281528 2.088236771173788 -0.13891858312535765 2.4540464522669634 2.581504187930639
-1 -0.36589663467243794 0.9800989499410697 1.512657907848574 2.481982348891716 1.879063921040467 1.6783314697156686 2.519822194339233 1.5139378983098026 1.4765499639533166 -0.4586543768759259
-1 1.031519656541507 0.37677631561513636 1.215439603971527 -0.8333793025092529 1.2297449965589116 0.7309661122339723 0.2233308234176088 1.8978096741161727 1.0017178523256016 1.540799199113878
-1 0.37535440891823324 1.05838458440246 1.7478919610180488 1.4358567778260587 2.634621031491021 2.6733943020176536 1.4038023921761382 2.09456237109269 0.18751380927669214 0.9030253353081665
-1 0.6050644162204089 0.42475868702885367 0.67729642342563 0.9159762799821485 0.9966211703282338 1.0325406378266162 -0.31600956837305927 1.1275195620810772 0.7550807758634188 2.0556587502944152
-1 0.9639628237078233 1.6612996949785008 0.15018611313458818 3.079012778712338 1.6765505664424296 -0.3164200745592767 1.180094372490766 0.16048718182365862 2.6754833932699764 0.2861554471536204
-1 -0.4733123063374025 2.215557819873761 1.4809169546161616 0.5331014736871407 0.509471219211528 -0.5366908461365221 2.5757870803346328 1.3082491695854135 1.3064213366309576 0.9305958816930349
-1 3.0207863567912003 0.23781737522480972 0.07878478120317567 1.6302281378682424 0.5980775385393649 1.5928976343724883 0.3212142395168056 1.7151012207401586 1.593816382695755 0.7481118256003316
-1 -0.5298380895168147 -0.34947847130115894 1.259810473989246 1.907798036285846 0.35944121815361163 0.6444888816334708 0.34377708875002244 0.6836686767703974 1.2932110945792579 -0.458790316071632
-1 1.8401629428690227 2.259471445176863 -0.3223229794980764 0.7728238347557039 1.5724556976510322 1.3274646917002721 1.6717333483877963 0.03745904530831912 2.6550649930379056 0.9705596819145808
-1 0.12431297464461755 1.7563279244667416 0.7774986621540451 0.5111136337905993 0.6433978537639469 1.8971862751406254 0.45959793718271824 1.781102107071228 1.4062626338777793 0.6234780410061468
-1 0.8407772366817298 0.35964705320370294 -0.9623019831100632 0.44149536693473657 2.074342161562674 0.9904199365414913 3.2137011456900098 1.0337076328449122 2.0693337269664083 1.8277506449533987
-1 1.0113056814830639 0.9851992899356764 0.873659978134487 1.0421853488103219 2.299837087915077 0.8071982744117732 -0.1096427502124051 2.5599638730556995 2.3458120257795656 1.9104294240298325
-1 -0.2652413955956079 0.2771478177147122 -1.7578972328231406 0.5091791920398325 1.3694768197526315 0.5806835043255031 -0.0948278795711135 3.822899721567823 0.5484905756054144 -0.25075975842777454
-1 0.6859095316452635 0.791069272223955 1.2193553385123195 0.7291514560030636 1.3876944292574216 0.8892463484292987 3.4273502454413576 0.6580296103521155 0.3238972925695067 -0.6496800158558074
-1 -1.5436851049150522 1.956099227374563 0.2779057405377705 0.7339456639197723 0.014024861431684466 2.6630936618511405 0.7161890905680435 0.5077767425517368 1.3259571967911001 0.9137278907925384
-1 -0.292961767713223 1.3071340106236198 -0.7017668375142168 1.2860358231830809 -0.8122076288210658 1.7211614223707081 1.8304680327555625 0.16021436599026517 0.19612682942548998 1.2082198804992264
-1 1.5187520786413158 0.1828654866775874 0.7328431724966722 1.7953629646772824 0.8216669452081463 -0.4014319711127199 0.23334012012093153 1.534537449937785 1.3889014942993092 -0.8511049828025341
-1 0.8451858363611996 1.3418063089585763 -0.8238999092902703 -1.575942571644518 2.0750484405729095 2.033997248128906 1.4449221159961598 2.0253497341487448 2.2283973766958023 2.404323890979427
-1 1.6107433076928133 0.5404780687423208 0.7937155331805563 -0.6077722620726684 0.21332376555661758 -0.9993545668337882 0.31523750335957845 0.5473005319402997 0.960730821903916 -0.28012631768751084
-1 1.9389616507358387 1.9532576203532324 1.2153193637879869 -1.4069714611803268 0.4662801445447652 -0.6193751496277011 -0.028999422131398056 1.3038353983411688 1.4946684162238129 -0.7409848880778342
-1 0.9021404373434705 1.5851981284549943 0.6057610277009148 1.1112421784262574 1.413214054275196 1.9417673251914613 1.634690668060366 -0.08301380649683576 2.1711500689414116 2.99282324374365
-1 0.1637260233089869 0.49637480750763263 -0.5285944959659445 1.5681001289396956 1.6803958442936107 1.2246294425310562 2.5669221884551776 0.7567621149423418 1.5037234063128802 0.3463214960951032
-1 1.5723472760593176 0.6432239887651015 1.804758599642208 1.2176050861917662 1.8717138471483157 4.077916319312581 1.5133550052844793 1.3823856879297753 2.6113216067389695 -1.1093237177115047
-1 0.8602744779765249 2.178619602525301 2.453544172271271 1.0510379811276036 1.8409684994496875 0.11803069280172118 0.3230760986621918 2.259943083391159 0.6024489055423363 1.1990484290135006
-1 1.649184578143986 1.616265278882509 2.2742015008761607 2.626169250389406 -1.1492939072912116 1.0408825980561895 0.4369989721349081 0.9034290059197084 -0.11385932074779648 1.0982078408810698
-1 0.6341310783502718 -0.9708605273806881 -0.017201345919524602 0.8926037502408949 0.22822364223265212 0.9096851395074563 2.0473818885200648 -0.7848615761262032 1.4441059896043467 -0.24922705201528594
-1 1.4520344107406407 1.2639986753730716 -0.8513007095320302 1.6293092619132934 0.7394579998929112 1.3445648999777857 1.5178679268046242 0.9933053628903701 -0.9336323582033459 -1.6920287783811307
-1 -0.584837407411567 0.9604177163540187 -0.003828672372695019 0.1731711935522725 3.512170380159825 0.4926659491064572 1.1587769448255618 0.6600987191801231 0.9926496119226857 1.9870269736899853
-1 0.40697221517240734 0.7915676379059069 1.4331616842644888 1.6198603975182355 1.6417243704332136 1.6270560025018783 1.6799759614717393 1.700588227134973 1.8464436799312134 -0.9250687955521861
-1 0.04736288349237683 1.5587027295355322 0.12163352594242882 1.124943757807633 0.2850023846865297 -0.07621319541134719 0.6373292813835088 2.5571634870370934 1.905346123931221 0.30969838202705213
-1 0.23757107697869606 0.7009274223790678 -0.6005151170274707 0.46131870148693055 0.694253134444586 1.8704279215134783 1.9559864883094595 1.5475302665627626 0.902775266852526 2.253986651760284
-1 0.0931484209802732 -1.0536269817119295 0.7832662454709735 1.3370869763110287 1.8021230335269156 1.0422523333084228 0.5539002500282262 1.1402739247006104 1.3778884263982012 0.9839666885480669
-1 1.4022006973888672 0.3301442305911556 1.4159864215392552 1.0753881627418582 -0.2194812627814522 1.576874528728394 0.351144790840509 2.9042579131410218 0.33439079197692423 -0.21115533384764373
-1 0.9200624394093888 1.9601307267236312 1.3048792499777433 1.044019487533702 1.295476599028682 1.06479650163913 -0.8347875409017176 0.8767774440123639 0.1631761919249426 0.962325538273012
-1 0.4606387639284839 1.93128591538725 3.2494332751166293 0.4217241090513292 0.5940126704202255 0.12271071800591238 0.009005952876745105 0.0631236875750606 1.2229161931162333 2.3879030147755866
-1 3.2172098250997503 -0.021922357496697797 1.1859662862492402 1.2154601324678136 -0.3071029158823224 2.1738376762747613 2.2872633132290443 0.954809047991948 1.901337785669559 1.3011976479019711
-1 1.1885608047442375 2.721310638802292 0.9617587859607313 0.12651320336878014 0.12567757686210834 1.887061564570169 0.8860616196551063 0.6430168020234137 -0.030733700547949327 1.0564998980605065
-1 1.352748382066948 0.5202126729710697 0.14331687879826782 0.40785023484169414 1.9641960196192663 2.7910712640458297 0.7740423932819342 1.52559135640059 0.3239548613578228 2.31826432040899
-1 0.5203741956670356 0.884417958844451 1.3777220780800918 -0.4643847508675174 -0.37572084642581793 0.1262513952897556 1.5518202424896383 3.3877379158242378 -1.403581970685686 0.1009940122529609
-1 0.9894392616099077 -0.0034178714976433877 0.689046476206714 1.4208906847616534 1.5473446325066496 0.44218920279820595 0.24101228948954234 1.1801070630847152 0.8039116009276253 -0.46102470089902536
-1 0.6361572167176843 1.5563186537784683 0.8983823810124998 1.0798802186419254 -0.038600239378366874 1.6649842223710727 1.6378836320811345 0.3059309271799856 0.8901320418030211 0.10914549884068314
-1 -0.18003932381317478 1.5693004310535423 1.8013396839368538 1.7544292528839476 2.460230078664536 0.8072540575395855 0.8326108318826944 1.5006349728524033 0.7460792678168342 2.6820859579435474
-1 1.8960169042497794 2.1576293718618 2.424978645426269 0.6268556772800932 4.221588312115547 1.1780884004744951 1.5616604868899797 1.8886529082537074 1.6168854045075025 2.7308325759110224
-1 0.12878554700508837 2.1150328351027246 0.5356772045785253 0.8698163232516893 2.3406750293658183 0.6627125907242539 2.4239833684636736 -0.17649747406412253 0.34655417092691454 0.37167266730649473
-1 0.7700976682797439 1.2052165149892542 2.0323449543315446 1.8093079753157488 2.677682507242789 1.2230772168351174 0.10002304289163721 0.38829774391404126 0.7382541961293962 1.4604650485834432
-1 1.2304476527122155 1.5911723818857464 -0.6663405193368004 1.9423332506900772 1.4218831147452045 0.7172255125851585 -0.12990659585261488 0.9108053409327858 0.11424096453618027 1.1083558363715305
-1 0.5195105474968298 0.5710613703505523 2.2928613438234455 0.021245928903329103 2.1269497746764197 0.8932419976165424 0.9360795887134954 0.4206153958722527 -0.013928240567511851 1.9267860815714657
-1 -0.27500090463981786 1.163598213361118 2.396756337306596 0.7166497755216299 0.5087064238485857 1.2644991273445112 2.207063036182604 1.511076159763578 0.7514616147389759 -0.386653321343986
-1 1.275981257794266 0.28386450023604437 2.0468065778588445 0.3368819014778913 0.7803798072812063 -0.11268418399709335 1.0692622536985994 0.7450466892913328 0.6521234033954817 0.3533878920228143
-1 -0.26632749480506046 0.09964814030131464 -0.14774546592772242 -0.44102911713759774 -0.8175624623446118 0.5982737657645009 1.8018589102471618 1.0206495963947055 2.1703414097910376 2.509625756793014
-1 -1.084176873793715 0.003374206020577475 1.0490056163609893 0.7413062315194299 0.5457392593753987 0.47876209776833123 2.7997789450020427 0.8473717379952329 0.07511100942298876 2.342980564354181
-1 -0.6060249411337237 0.3100831921729499 2.5027389254157533 0.4950992021162349 -0.7743243396300394 2.254986439984994 1.524435417647438 1.5581584085809914 0.7613263552054441 0.7313335506205685
-1 1.252570109684499 -0.2259101116089468 2.02870927406763 -0.1982100935627482 -1.0747860634656639 0.5696675160105826 2.0536113238469964 2.436984468208358 1.087350912351074 1.6355207346806782
-1 0.08793454138157841 -0.7701820062667433 1.6526323582054276 2.648211639393969 1.5418579075681154 0.9489571984728947 0.05918410476639424 -0.9099915058439798 1.4346179896632103 -0.7890540352574975
-1 0.3047705090908783 -0.041817851700766795 1.864590556312606 2.2126512576725283 0.850687528022706 1.1516079924281961 0.7160824885255048 0.23428914563411007 1.5892718454214458 2.0304685172157515
-1 1.8541494516233115 0.4996871983195521 0.9048408243621995 0.7096255802229431 0.33910504796127783 1.3134581495613444 -0.2753494959695286 2.3289922141730686 0.7323942203055318 -0.274626661821493
-1 -1.338544772611924 1.2944523849511644 1.821257734737301 1.6793492696385324 1.5967736493283293 1.712864874826922 1.5745612820947925 0.4891550646810052 0.47846091208172825 -0.1743221254069207
-1 2.131766719148957 0.7608227099296399 1.0630568268599263 -1.1476984731054647 2.3867190880037636 1.130561984384332 0.9131559753959471 0.2973457770910879 1.3007036631285942 0.4372322143839449
-1 0.7708567792295566 0.580257476003238 1.5887140302216574 1.0413330688401965 0.7733129718389264 -0.5163740146933058 0.07497254374425988 0.28623086041167667 1.5489309172205683 0.8551008347224718
-1 3.4595137256272586 1.1532560360380666 1.588361571148596 1.3802224477267615 -0.7001860654912402 1.8740796848274577 0.14520299815591176 2.5193824279795254 0.03909705046483791 0.7357475729770275
-1 -0.6544136676184351 2.8745518291193553 2.1515280898247315 2.757731240766754 2.429606589051394 2.330014751072225 0.9115033589433934 2.6873787753182583 1.2992135444029829 2.3920287356459284
-1 1.885270281917602 1.858016821901751 -0.06157363620807099 0.308401967243883 -0.31307820201782555 1.461038889339163 1.6128329392090914 1.5772000116247265 2.710615509497419 0.8050419240018178
-1 1.405879563380197 0.659914831493603 1.912269260893395 0.529404740699135 1.4277377811246783 1.2913475473601614 1.7339294107927208 0.5215235778431477 1.7550541630505698 1.4400196124978555
-1 0.3245588747842635 0.42197424404348816 3.6539265313256526 1.2857918279043645 -0.03655209163203632 1.2407043968389915 0.4433829786888507 -0.07023065483472712 -0.6733771504197963 1.4798448078129154
-1 0.9085359200450331 -0.009624824747410887 1.0280527195285618 2.14148134591638 1.0562537066073983 0.8809817771790907 1.4071063563557673 -0.6597423723027149 1.5583011903165707 2.3154204049509683
-1 1.8050769097358077 1.7786869407899135 2.6495184641125515 1.158177494691216 1.1671375960394383 -0.45722370125523115 0.9835693406300088 1.6357021360875077 -0.16826461081967703 1.1932740024664812
-1 0.576688853348233 2.151495453088904 0.8572555252181385 3.405728819429614 2.101231270195057 1.6771308649271772 1.2637521672030567 3.1154229758040874 2.485850964748577 1.7694224707976827
-1 -0.22806118428106337 -0.9061154967479863 0.8964938904788088 0.6816585601664856 2.013761003670729 1.0313228363661557 0.9260597798962866 -0.18946147062989205 0.28527619220858247 0.8963510651947846
-1 0.3148947081465582 2.161975824817249 2.609645991041186 0.959492387316128 2.397824851151471 0.6697921252418206 2.313069590047294 0.8776639563036727 1.0599994333376752 2.8237989480782524
-1 2.652125755323301 1.8602107889115338 0.7683127593190835 2.2682293581606165 -0.6222001971107851 1.7327348607601576 1.7973442155328485 2.3026732779864645 1.6376913865909977 1.4336254291699817
-1 -0.033946588281949186 2.300669560977641 1.160077113314741 -1.035089589522486 -0.3088401922649133 2.2246952213732962 1.5263288862385613 1.2041606436782568 0.6360015906365958 -0.46568448099058934
-1 -0.8340563619947565 1.4168203411347104 -0.5724699864440952 -0.5633561206742383 1.454288263940742 2.091140792301254 -0.9346927324544323 0.0969827614306541 0.9901527415253794 2.0293060494871034
-1 2.1766440722293696 2.1765927443625097 -0.9288701141928257 -0.4887885438886057 1.415145042839749 0.7869820800801398 1.3531410283773004 0.38467574204818133 1.265876278197796 -0.2027790078386682
-1 0.8270879503594885 2.371236015912422 1.8437897438725939 1.7890683065643116 0.7718878947557098 0.1132854516378462 2.6937038226634122 1.34827091113804 1.8024405913978527 0.9733403683960185
-1 2.4175771508586754 0.8851307536623965 0.965109486208773 2.4006169759083864 1.1967556814639715 1.2950307543358157 1.9415648218013744 0.35864528885541735 0.40940436545238557 0.7868294504129988
-1 2.2098184536505663 0.889100413360103 2.1851586347238285 0.13494389682652308 -1.1445348600024268 0.8595807349607005 0.46845661480480505 0.07882338616350792 0.222858479263641 1.6187566311742603
-1 1.5395105587908753 1.5090442727804423 0.8644957394514675 1.2222062988283733 -0.657302278508328 -0.8584774737648058 0.7847354502810749 1.066321874171543 0.6763302367935397 -0.3056807220148554
-1 1.3241371059217268 1.1998033042587848 1.6413385242724854 1.2616652980595755 0.8214439629174916 0.7323804916810981 1.446327599557899 2.1344373550969333 0.5323048652541784 1.325312471981157
-1 0.44793596733276986 3.5291804831601397 2.304481907075438 1.7159536021092872 0.49378464200637107 0.529685187245525 -0.19498379135409039 0.6257392880667672 -0.5922944256976155 0.9677085580549932
-1 1.6001908684230077 0.8441053959985582 2.191005295444758 1.8601204690315698 1.4231646338661619 0.7172326899436327 1.3685291716454426 1.7459708463423858 -0.20021564447567597 0.7886037237104406
-1 -0.832715908403886 0.9821249159854097 1.9340136298649147 2.0863867471576207 0.8588263222826337 0.3940359686539505 0.5667076617327207 0.6813674534100007 1.0601080933156564 0.9940095449693623
-1 0.5362749326926859 1.3784556073957994 0.7830926551836939 0.7926130115032175 -0.45867401264881047 0.7649235836439627 1.9252198419840811 -0.5932278037833087 -0.20495235948345436 0.8228620061430476
-1 -0.5026862346261936 0.32379950915933053 0.4877018370232078 1.848487603750593 2.5612814512394575 2.6996258863788105 0.15501963775759875 1.779188209155349 -1.1587607119995043 0.5286988956500273
-1 0.03890979688369878 2.5700833608321876 -0.41167989902736224 0.4405078623025871 0.11339883057634925 1.2618969624421223 0.5661859841701755 0.4450152294875418 0.06553355298472463 2.9653045304903003
-1 1.2066695218108954 -1.135846422758188 1.3472000646449644 1.995247004371493 0.4067019132360835 0.6014718489518214 1.1945804244235247 2.563237911092928 -0.30000446942459824 0.6782859264246553
-1 0.43145271645135497 -0.15638436316804127 1.806542814206817 2.509982504123812 0.2908319784765735 1.093034072836503 1.8310934308417324 -0.428111571478186 1.0227258944948991 1.3181088073443865
-1 0.6593145377977876 0.5513227059953492 0.08971356052593105 0.6997087344297779 0.3547337578286779 2.044316172416025 1.7054002807979272 1.177077903869836 1.6118683425448608 1.3817764734854732
-1 3.26027582916473 1.922453791560931 1.5445220345277253 -0.3361563876793128 -0.20451311346146506 -0.02755370253733158 0.2523835913052155 1.8457060509750052 0.7729749699076125 1.2691512131543639
-1 0.7853510230572176 1.92550267228468 1.3840760296517856 1.019170128522936 1.257277800158144 0.2954835667658987 -0.02339082355482236 2.344976472145047 0.8650491281625572 1.6705466337391612
-1 1.0256022223771357 1.2521800754728607 2.5454645690960165 1.519642791108941 0.8120657189050374 1.395012570155324 1.0067859707833062 1.6154722360698295 -0.1911479039843622 0.3192273565677406
-1 0.9212215747887599 1.614097542109768 2.153211482594465 0.25851295883461667 0.015421396864703008 2.910093225363264 1.180736322866857 -0.024920942327103957 2.669708944799861 -0.4455433802815518
-1 1.5936186055028179 2.948335176521773 -0.9304959929630894 -0.25674218734698395 0.856450569458336 2.2464434469263295 2.2695814273033834 0.9023024874886443 0.1998192758289271 0.9614747140727596
-1 0.4171564598259989 1.2341430652292795 0.7613883447910024 1.4327906124857261 0.8248656963940865 -0.09370178940656282 0.5302446693348143 0.5977304498921516 1.9672679105851836 1.8549778581991436
-1 1.9988876732611685 1.7067688718725715 0.709840257121064 1.8195818549115197 -0.196218309209645 2.158975719537872 -0.387052375493828 0.2684905146219133 1.1751943798566946 -0.08233263071043195
-1 -0.004588558850024516 1.280146957738293 2.2274500380613915 2.068436441505224 2.4406629422607455 -0.020552259353522784 -1.9306504989533266 1.606929445859563 0.12204039563080737 1.554314194847439
-1 0.04312231827054913 2.293183585915505 0.5515907062418919 2.0319631309075303 0.2043494544647857 2.163212294566986 0.24687989300151647 2.1776229267798914 1.1368594510956058 1.1067868768921156
-1 0.8380882562583268 2.7318988397710573 1.4749062376973399 2.3244811915569885 1.498055997999189 1.4901966783173328 0.9547300656875682 1.2938212544822327 0.920830744648933 0.7960603079946061
-1 1.1730459404168871 2.4157763285361744 2.2769114804572554 1.781254882347914 1.8939310535271043 1.8204037399884672 1.2330253630970833 0.24898375343327694 1.4526754173493885 1.2327670337378527
-1 0.7828957363283248 1.961806185656672 1.0945811949626496 0.6471160715303457 1.2988151512993327 0.9231258952067597 1.7059995140840485 1.582221842249981 0.5731086038064922 2.929881320548402
-1 0.4240209410200867 2.0612687767691504 1.4013347045251126 1.0775762488985852 -0.5648359238473468 1.5394818276041304 0.5250719203859092 0.3867254288273827 1.836032841951298 -0.02644684457005053
-1 0.12838309666764036 -0.2524433635395231 0.14063539701460914 -0.8169781441139783 2.638413098813798 1.5872934688325704 1.343252734685199 1.1584200404773857 0.6163819194666804 0.6654328763469552
-1 -0.26416941528334714 0.32620704315453675 -0.7502936599619701 0.8401389782535786 0.09753988131424873 1.796236698582462 1.5877879186693455 0.9856032545638709 1.2072784259771 2.4653229099496707
-1 -0.6337999979940661 0.8076685452502981 1.2207084350653477 0.9123689527781019 1.838283774286254 2.2836210170990996 1.7394640050289512 0.6351189156017663 0.9629884451362287 1.7680252591425618
-1 1.8654459163757884 0.06089772776268909 0.9679374944456427 0.8889470807355174 -0.08754935246071827 -0.12680613988340284 -1.0637769092192588 1.512338996915241 1.9515416090320272 0.5015769881603198
-1 1.7247706923845918 0.360222898716523 0.18071931378959916 2.0371848423820293 1.5266006033053001 1.353704597154892 -0.2696414308039541 1.343721201156886 0.46275842064535144 2.3294944321291413
-1 2.1105081742950267 0.5116093610246693 2.2446634834462875 0.658957834299546 0.34134432630789047 0.4247161540652681 0.3292829996171407 -0.19362053618697583 2.62788746256027 1.3966627696966927
-1 1.8475295891856125 1.3887694988244523 0.6817244598020126 2.5809988844215908 0.32696789850689245 1.081015261872673 0.2386938164664013 1.0118382786145506 2.209217716205016 0.7574090447478952
-1 1.082260517720307 -0.6266070913930977 0.6832252128874979 1.2966340694320664 2.324615742379285 2.5627557774177543 1.72092865539378 0.15590225454118978 -0.2816198860581334 -0.5099568334403046
-1 1.6725629461607472 1.0353690658867798 -0.8225360006266837 2.1324720159286894 1.9885924374595836 2.537256632003289 0.9677496818620155 1.454681559021501 1.3029797950165192 0.26385709812366753
-1 0.31156560050102955 2.1652814753810112 2.0058163682540036 -0.04562872657851469 2.724179402266973 0.6222125728521903 0.42811650448637917 1.0387953213300416 1.8914700820960233 -0.5893540202775569
-1 0.2578251741975023 0.11378011266272059 2.797638612913183 0.13983902653928637 -0.03255261699221346 1.2576586825716858 -0.6642415184742925 1.2799765368331657 2.3385679931813983 1.8159437052025178
-1 0.33578001261352897 2.0063591095825952 1.0807987120174516 0.3543665780473314 -0.4202063816731054 2.113462588586846 2.306817160855979 0.9446592793327631 -0.6774687350899611 1.6189786930902486
-1 0.8614448755152566 0.27807051666810034 1.490952308696544 0.42812809570277155 -0.6130395196516234 0.23931476380563366 1.3454272824526288 1.8553493467683078 0.7262585485463864 0.8060386596767135
-1 1.509477780297391 3.879562737499862 0.5886532526077162 1.2655619776606024 1.3990929522583664 -0.34170560649024506 1.7418923966881366 1.629417743427085 1.7445593580979215 0.5930685838392928
-1 -0.17633273947080386 1.8278089865738787 1.6079874279761104 2.0641657251872525 0.0013949787963080107 0.9779219807727019 -0.9229761793545943 -1.0291570090345807 1.3628786284816425 0.5752391889181461
-1 -1.0143862085431188 1.1194733654329676 0.372026303777525 0.4779765819717211 0.873963169712578 0.8031044909741862 1.438202993892749 1.483386025663741 0.39707846786644874 -0.5347159094832814
-1 0.11016676987687668 1.44535659616203 0.47296285732106014 0.9569700223555272 0.22754986353621043 1.1107842631735818 -0.20365888995072612 1.7095423750241086 -0.848293390426655 0.857847169492578
-1 0.7508129008937717 2.8747883333024182 0.8289112296791319 1.5951701814113632 0.7420525998761323 1.9537834679324622 0.5603407250007024 0.6017647337718439 0.6431621236261322 1.7673108381156395
-1 -0.1852593368859976 2.2089214215364246 0.17988209448256942 1.720553251777205 1.2120857158218548 1.296273725719677 -0.25129199617788966 2.0013217992492613 0.5065314908683332 0.4536706566267381
-1 0.3257759973178981 0.17932720424930182 1.2245897173975124 1.4392674655132107 -0.19990974032801478 1.616015721370362 1.0976249377861196 2.286751487136163 0.5998423893372578 -0.10744364268832474
-1 -0.18860318421456523 0.6481395082246904 0.8471055242008172 0.8364035710726628 0.5027181893375049 -0.04737632027053729 0.6081198234429218 1.8117061812925739 0.7882062608326725 0.501707612022315
-1 1.4843082385614745 1.1158750459458913 -1.4894665738544455 0.25826376510509763 0.8737547870296022 0.6842381688703825 1.5781821909490459 -0.8859809290045597 2.6448010296898516 1.0451355125183155
-1 1.7920903749688475 2.181377042700981 -0.2580670741698272 0.835878310743556 0.8282113555574907 1.2918481880236576 1.2845735763240005 -0.6226879211726246 1.7452863581983848 0.35415213876681106
-1 1.6059906951044978 0.5477408796911678 2.033456301629621 -0.6056116844976043 2.3157299435817342 1.0282347361444912 -0.37895653151562936 0.9752299146785057 -0.41816188526715736 0.9125445080555991
-1 0.36434340752558814 0.6902917518300258 0.9253611225661063 -0.42114130346772227 2.0970094095591443 2.7085188507498557 1.4289293922116237 0.9542757519821615 1.0546374187652479 1.3258156303811686
-1 1.4902539943349453 1.6573630488454014 -0.3809764834643814 0.9358657723296077 2.7348124001551435 0.9897672456356681 2.560439397267852 2.494870519932018 1.6580041060544213 0.276867359286432
-1 1.1191344811462158 -0.6181668923123884 1.5490411146166472 1.8183809809806493 1.3028570357467482 1.486951380254144 1.1831247980434945 1.780974941037947 -1.827510680099897 2.305550677513012
-1 0.849190160180726 0.927714888220189 0.4152982301284849 1.7201547897444616 1.0010482110516308 0.47888318535920815 1.7303425098316922 1.5212540746719077 1.2164640343110604 0.8672666819224022
-1 1.1818789164071632 2.3299574339825355 -0.2238086965126307 1.0866668603828966 1.777789469252217 -0.2473412361708398 2.4917056426594892 1.0985567817486692 0.8205900594343175 -0.4507497282180284
-1 0.4806312370873962 0.768849921524061 2.2816919830317324 1.8888027374056304 1.3666588628364746 0.313010983641146 -0.9582374160527103 1.7350822166838902 -1.0292285073997203 0.6398099597089605
-1 2.387963695369674 -0.5899448356258876 0.21621305588176487 0.9380272998222627 0.6981388782356867 -0.4629800914467903 0.7722932223610299 1.5585013561079406 0.39398387576565874 1.605900840338324
-1 1.2715952476157897 1.439635629557708 1.0983640636833376 0.9812043919910073 1.5353214720014243 1.0984936772644822 1.1502708274998623 -1.295397653899192 0.2861064908535764 -0.9932837563816654
-1 1.3012696782417956 0.7849306120035814 0.5043907367704977 1.317902271109904 1.2355512152607722 1.7921035283313613 1.3780045579049331 -1.1334086181295735 0.7594490553748667 1.2920327236325173
-1 0.7390703584602525 2.457743695195635 0.3128347254263576 3.2777913748283356 -0.3729594628152144 2.2165912805252592 -0.3208945778133039 0.25945266028499947 0.12129953303222862 0.9577961880424101
-1 0.8445123778336028 1.4240300974070288 0.1873583546229668 0.4955218063785525 0.9094332296150236 1.3540661068354631 0.9171697258910753 0.41888437045897486 2.9462218414395487 0.6502477720645555
-1 1.3877586550503413 0.987611562870769 1.2584972385417663 -0.31990526604547664 1.8690834901315843 1.7043650395994414 -0.9964092334530854 1.1408598689320075 1.4213381391949258 1.3073798077919028
-1 0.06076427697113995 0.42120236957849067 0.592901981159774 1.3720471193027384 0.9036775292098581 0.8953372123185973 1.5452404312257344 2.0708178196722606 -0.8979750106430204 1.6853058787444881
-1 1.1694470503331111 -0.7289698765725721 -0.3241777565346444 -0.02733490335945188 1.8863228847530946 0.8073024667207529 -0.9818689747023401 -0.4283553318571569 0.9994871828689351 0.07075638531545037
-1 1.1047596078086386 1.7708874592017232 -0.1612806069289101 0.08556210685307786 1.8572899576629136 0.7200423074285855 1.2170692625583286 2.0347880443589847 2.7432017121214005 1.3957939162622077
-1 1.197861378414133 1.556444574585297 0.629813576730021 2.4550574210435823 1.9226732616821978 1.9859797173418605 2.186728551603152 2.221928254196631 0.8555508774400884 1.723787004755138
-1 1.161571044817612 0.07979292393847359 0.473025751301427 1.205676831999432 -0.5466232243147817 0.8191419439472176 1.0060075056738604 0.785322530707329 0.22058837011880694 2.6154680787761726
-1 0.17077134170060482 1.1137337091671946 2.318497500926356 0.3973424625226393 1.461779582118195 1.9295571893710908 0.7785519323891255 1.0672230065462434 2.1223852587473258 1.5460766694219767
-1 1.1564652200933274 2.510183232201066 1.6891434345580443 0.13174662119947889 0.8871123877951895 1.4958243544578553 2.9794729912305575 0.901901296036228 1.3871706497633103 2.8969924652525334
-1 -1.0521680406383696 -0.0031861766791221324 -0.10915897400357322 -0.1303567225640898 -0.09337344840645234 0.7148597244723245 1.2180327568998717 3.4184983500514545 1.697740318234704 2.002711960184084
-1 2.376709016910577 0.958001009693663 -0.1081121213002203 1.327468223880286 -0.41205779656829145 1.4289978911250902 0.9819807423748184 2.3188491121493113 0.8657078618437748 0.9391669120890416
-1 0.9776980417955967 -0.6674206197457981 -1.5563935251898675 1.5446269906729104 3.047754956305709 0.3970621484971374 2.7173431471851766 1.7243005353672034 1.9755492634674017 -0.7077753665556163
-1 1.1671355902086602 -0.8193057764678835 1.410567460875851 1.7497653081783076 0.6901637048786208 1.2119799048759736 1.3226344341934888 2.2695811100443404 0.9907324730003678 0.5558635315480431
-1 2.4336171222847973 -0.73180099697987 0.110963544711143 0.2466617891220264 -0.8154643837784403 1.7051343160057892 0.4485983625979719 2.319215306602568 -0.5223921322733727 -0.05099278306658839
-1 1.901698041087508 0.8988295187852892 0.6511477798135669 3.0420349436695076 1.3810269156306683 -0.24628147854970273 0.5188524250377791 1.4141097609090438 0.24777660167964255 1.535797527794107
-1 1.7629403294957187 -0.13022007315691875 1.1647647804960592 0.5890754693324485 2.06533631915097 2.21452694737647 0.673652898562904 2.2005666335367784 1.5261645592168471 0.9017580067794544
-1 1.7376137405520378 1.227528622148764 2.1537333953075093 -0.7244714994487282 0.9737436380972475 1.1956909226237713 2.612848244020281 0.30122025453481716 2.973720741303093 1.8186667174448368
-1 -0.2742361456988558 2.1098716503801613 2.953664212753427 1.574905508426148 1.8552665501344494 1.321110382365208 1.7445198966258182 2.471288236145563 -0.11919705782427648 1.8624551969544791
-1 1.5436386497853212 1.8153339598609863 1.363613793156124 3.0510249899073756 0.5489376037189108 0.007578350689908864 -1.1820947864458877 1.3011272158310803 0.07518458687451968 1.5312667541972245
-1 0.3224512020283108 -0.2209974586026877 2.042104637824572 -0.37728305633852743 -0.5498729693279798 0.7193283373851307 1.2590924907118073 -0.3944236589332939 1.1250230341812884 1.4070211742408931
-1 1.1444341603579156 1.3629504333367566 1.6939924628296188 1.9479380654467797 0.7894876586788064 1.049604859005768 0.3408015558912614 0.6014994900100508 1.4716224256141708 1.185118554114717
-1 1.5859690594959832 0.30570898129196966 0.7464020043785254 2.2285474871009723 2.412881908798376 0.6904305558007539 1.6192643153889568 0.5920043651364744 0.7807197394828229 -0.20297994754139137
-1 1.2950387623080977 1.0916188301034222 0.6600573067651259 1.862615598644322 0.6876153259228353 1.1481594206078056 0.8784422750187779 0.24715809175194348 0.7857238169348668 2.1619479520100247
-1 3.0828763562487733 1.7362496731683166 -0.20896157853930264 1.5332869652046193 -0.21794910668079526 0.9202735211245334 2.574049390833994 1.5268503392385662 -0.38999953644207186 0.22479935308805854
-1 1.7627009184421887 2.2255381870678437 -1.016295091642716 0.6254801643275638 0.6618861479958897 0.9047308122786223 0.852721929456685 -0.7505113940627413 1.7250343985280407 1.8166918481323084
-1 -0.5022420621997736 2.733043970376204 1.5120949360070959 1.9428063677250476 1.3780749670748853 2.2350181236519657 0.8716131236741619 0.2782380235553522 -0.297799811324456 0.16653587974789763
-1 -0.2981918597327633 2.860715416679886 2.1275708273598566 -0.29508534819399324 0.846188811185981 1.8713251354650118 1.0723090993878512 0.4374636574396571 2.210140762205574 0.6809712558014431
-1 1.5619715587750584 1.2704149431309402 1.9712386149819312 0.026280766936758293 0.8206955786918028 1.6318403698412411 -0.5566358146889887 1.7571793612461013 -0.5366638533754291 -0.040269040641153
-1 1.2643496455778207 2.038185139306229 0.6395741359412223 0.27135915089505125 1.4201127961240902 1.5041067668659303 -0.09091064494863543 1.109133071144227 -0.4794905621068224 1.3208155875591663
-1 -0.02895244930542762 -0.49403509214487396 0.712435362084801 2.5460059356446374 0.9396714328426592 -0.7949960754019478 1.6183020075071732 -0.38577084963397135 1.6991710568290967 2.786233832662353
-1 1.261753017958196 1.0918709535770748 1.1265646053317926 0.9867326079450506 0.8288572122803143 2.4418772115091816 1.0454798487585901 -0.19993011811143235 0.14523995518141886 0.866687319252661
-1 1.6985511320556277 0.795437122527888 1.556653786587669 2.1174479278276426 0.3999172845317358 -0.5010796653100276 -0.08438438589923591 1.1138001295987414 -0.30602571964029956 1.4972214829613484
-1 0.41786595805108906 0.6459011706826348 3.657046684462284 0.8222874793996409 0.050062147599186035 0.23963259661744873 3.98442324525362 0.28119552752146837 0.8964441562070578 -0.253526879649719
-1 1.4488020919552733 0.8929138056330631 0.3161270487767218 0.7331766954467245 2.3366307109566495 0.6815405492334983 1.5281435010244593 1.6431760386153362 0.5321346633571438 0.34130859830303917
-1 1.2748486181912866 0.33303368481427886 1.2151848478627916 1.0756517104783787 1.2083219051593854 0.8277625946461055 1.9666455377419778 0.6651325140447175 0.16327294989918317 0.8603717402697098
-1 1.5090300715612457 1.5180463731650495 0.6972598598076571 1.3556192196865902 0.9126434148820246 0.8127664907242128 1.3311309435526322 1.279157714746425 1.7829837559894246 2.988071791570289
-1 0.2727158735259818 1.2998080669104182 1.5121347623238246 -1.5679984907159152 1.515508708019623 -0.15391403969184858 3.1311081089984323 1.847318459389865 1.3425374198002933 1.296082544224974
-1 2.408189206457478 1.2760154921881726 2.1197548437178906 0.05936234352435599 0.19907763560203529 1.5479638808770004 2.471816233765586 2.4680208521093805 1.4113824572688618 0.383801428379995
-1 -0.17965112079351564 -0.3404976625536871 2.7837262771738205 2.6881515223765398 -0.30847324983815394 0.9993265400000024 1.1374605736665502 2.2049953998249694 -0.2513007616550551 0.448830380725894
-1 1.3443693966742452 -0.025711889743784466 2.2443775230207503 0.14834884628873723 0.7271367845373308 2.4714407353590957 2.562158361402452 1.7047011572226343 1.6769293581505482 -7.308081317807247E-4
-1 -0.41870353312467423 1.2877545442386 -0.3164789161896502 1.803839696410392 1.008076378658354 0.10616668976164723 0.4098865481816575 1.146539676959654 1.1538344544688937 0.05907242504921317
-1 1.7936911543812046 1.485342520804878 0.31800311694795325 1.9199555201066274 1.9312631279902837 1.362366670774782 2.6306006265218365 0.133055817623004 2.5078649689837027 1.2068433004457952
-1 -0.1411582634165307 -1.0426813196108524 1.434523926692467 -0.25113509019608093 0.507539296016366 0.23168671363927917 1.1893212121098466 0.8304584451378183 1.4556473134325054 0.6534542423873613
-1 0.6079927716629916 0.09194609771904183 1.6120179701101955 -0.5022953903177365 1.2170945269028797 2.100831302657739 0.8386155807612904 1.5684558466558434 0.27605209581418555 1.5594274213225667
-1 0.07428493649230228 2.293483112741116 0.9708779280979398 -0.45177079067335923 -0.057110219872378076 0.015433876379835065 1.0794154562045615 2.105620271870406 0.9395998613200235 1.2851835351116119
-1 1.578883010870155 1.5609283984502076 1.8223960032380064 2.2142614021520837 0.7130462722633009 0.9252426132551667 2.868560600039225 1.6968141988566166 1.9976720397763048 1.6813323051682774
-1 0.5016495406992045 1.04908195692884 -0.07722896372502253 1.330713406245241 1.1267715047602667 1.6360574586472572 1.2420706446269942 1.9672850660325922 1.054929403781838 1.6077148722801038
-1 2.0538334867970534 1.9213949071716163 1.8934373144800345 1.2381794078176593 0.9175279056098742 0.8206265873347616 -0.8312726444851357 -0.5131966390183769 2.567300850622103 1.6719008505918898
-1 1.2689208746241893 1.4402293624087208 2.7176532271741003 0.01336457957384174 0.1702333910599565 2.3778902914738547 1.7217780353501682 0.7054536312666535 0.3361164972231122 1.1589949811743772
-1 -0.5767062059491888 1.7138887496399136 -1.1154021033816348 0.7168636442060621 2.217046440509127 -0.8161420769580656 1.6271150941587713 -0.09702287214964955 0.22946937882986906 2.7922011937600097
-1 0.9710624979613078 1.5610147329117985 -1.5053608758479413 0.9711728502628203 -0.5150150692664308 0.49562546380947603 1.7163450863443273 1.306018285087743 0.5473958850146698 1.8540315462762198
-1 0.6425941154359618 -0.31480994520520533 -0.056642174933536404 2.2269443093694914 0.6505566385114631 -0.3709635056159635 1.8873810442041976 0.5119563367121428 1.291713540770698 -0.6943082761794022
-1 0.5927308007246384 0.8464951673655936 0.18447571041818456 -0.006190250203252257 -0.012631850494107644 0.81828806055344 0.03231106794400085 2.0927752513240994 -0.12600012916564518 1.9639580630933335
-1 -0.34831756463523855 1.623268907572022 2.1594197097470325 1.0562200902265129 0.9414684460546705 1.4340305236290405 0.7654931413466368 0.01719894816346723 1.5959585538584955 0.2885792827923064
-1 2.2697657120238466 3.1420889453091094 -0.8210208940698709 0.2035264954846796 0.34878833066083437 1.3187569677046596 1.0219701238612262 -0.1213159939916395 1.0802611304225862 1.3078831016284853
-1 1.2480724077104584 1.9077146304274128 0.702946174596962 2.3286147355852034 1.0071749708265634 2.5149204905160154 1.349779745606328 1.044016863507004 0.365723895391459 0.6519926945711725
-1 -0.8985903846454402 -0.5021240182148043 -0.01073065243449256 2.290069714856683 1.9819036535789476 0.03105672582226615 1.339000036426309 0.3323749578280565 0.8021635756060409 1.195220952578341
-1 3.008655872898343 1.0129636641232918 -1.5088469891308582 -0.6947292093040875 1.2487527838514174 0.9032973743393249 1.9979774814850564 0.0435076158833696 0.8478193472405138 0.5026222405279126
-1 -1.0608662183020523 1.511703517053053 0.4555272804535656 2.076056547724862 1.754307244984986 1.3854010129660659 1.8247443481696117 -0.0246162652477655 0.24988078939072067 0.9872960257572898
-1 0.8740725946015646 1.7804072513374016 1.9060935705517543 1.8265003967793456 0.91953745409342 1.3629234354248754 -0.2803757506365385 -1.0129022749852892 2.5019279152710756 1.5245757538298341
-1 0.32688805354617134 1.6000098575767967 -0.1786618864414944 2.3806085458526325 2.3338676324290164 0.7609884113833272 0.1498428862635196 -0.25090796239660373 2.3770456932981814 1.6131488558961797
-1 2.290620763512112 1.3541047134925366 1.2421787622602398 0.8804930591189608 0.6595899728536196 1.6277353547734075 0.18759874372088237 -1.1351531086694964 0.18251082831485133 -0.5713204010530248
-1 -0.22047844715313447 0.8310592465340738 1.7892315227363613 1.1470591393757708 1.0726224455927464 -0.10592031044447459 1.9817888345656018 2.432077040490821 2.2450973493606203 1.3210707817547482
-1 2.070368262568201 2.3671178117141207 0.8627035047548697 1.366475314693422 -0.8331190909005985 0.7551440285820138 2.178737629795865 1.0323167492638525 -0.3148106607913368 0.50662477745953
-1 0.8604853943488086 -0.09592589897715587 2.600032474430587 0.9839706092809413 1.519739305696014 2.1260793286184008 0.03744939964524108 1.2611070446598698 -0.511324151442442 0.5454482162340912
-1 1.8946369523511708 3.362602104881858 1.8838436706953976 1.2491758602363099 0.0054680988441749845 2.651799339501261 0.6411444300353089 1.1035969889037076 0.8324869555591509 1.3031776807447846
-1 2.5154071822014554 1.6803408091264473 0.37434333648729623 2.496324926040323 -0.16401882096773224 -0.5744479735763091 0.9352239350517153 2.442683227544391 -0.5264039462194898 3.015307788051603
-1 1.5111987262832436 0.6410066045062515 1.0002585904405568 -0.8894537972030532 2.8014684904508944 -0.5393437655384221 1.1524079090931012 0.021728095470450404 2.1130698813482622 0.9468113077109184
-1 2.246571391447209 1.2010599601897547 1.234941576895316 -1.7706644509786722 1.471058855485551 0.8939500026890757 3.0844244960496563 0.3937694347012187 2.4529138646148967 1.1858907139355346
-1 2.4615314217465514 2.138799653615231 0.6155097299332213 -0.26863064780465895 1.4804373561575783 1.9409343558847068 0.44935568187190045 1.4016783544796323 0.5844124030092861 3.560614430022461
-1 2.170074376135311 -0.044012090187616204 0.4876588954783079 2.3603606696538524 2.125197091710744 2.4134190214591262 0.41472234938098607 1.9434029103795312 0.10273955644383004 1.235145974467383
-1 1.2969727061242051 3.098685038424812 0.9785969987985332 0.5224703037252412 2.5948178849934393 1.9056896554251344 2.1303162130115787 1.6936027246350522 1.591959269634407 1.3287905654720076
-1 -0.015989877059035873 1.5072072218307366 0.08389293810681375 0.9234581285114085 0.4320229724446347 -0.17718855392460764 0.7238001450159828 1.8397437251675461 0.9523656518925097 2.513817935317845
-1 3.7089889925376345 1.6027646547595036 0.30439608816889874 1.325556017740845 1.5649758448214102 2.0480467830712694 1.4268815678658604 -0.08232989657136769 2.0319641149268852 0.4859663282113227
-1 2.9299411753408178 0.6939333819644463 0.5980477746930858 1.1544643358350055 0.5988463132053894 0.8004691945155193 -0.7969681294710653 -1.246477065340748 0.7551153563842066 2.2320600943025157
-1 1.5618544649786017 -1.2039729275512823 1.9863936078958404 -0.7698679015907834 0.6433908271785455 1.7173978058694828 0.8771509209324759 2.664740793299653 -0.6994627263844606 0.6322436483068374
-1 1.187061394437512 -0.6451485516060627 2.476357446033039 1.7693108617562059 1.3697550089364834 0.40908284287939223 -0.5656163253633264 3.468763307766636 1.617455962016709 0.4894706139195705
-1 -0.4273229723387111 -0.26809867009452515 1.3843160982545846 0.8212240154930317 1.1784396971750364 1.872828424638627 1.3779623371802083 1.1888620042820783 -0.10589695125965615 1.4199981576509952
-1 0.12193951392066005 2.616540426567961 -1.337357835943099 -0.10743949585791679 0.3939788495591735 -0.02266440276523496 2.766246408329433 1.779318925725903 1.1626163281228863 1.1568240129972165
-1 1.4669291522156196 -0.8005956562590923 -0.6879775244399986 3.461310058748968 1.1339641121124138 3.0998254868058384 0.245952923446367 0.7214863675143265 1.0108020940282363 1.8538791497646767
-1 0.37376581529952313 0.3065031814805871 1.3343221577395563 -0.36245405167755473 -0.7157134718616156 0.9091314241626773 0.6213443407765016 -0.3159031135243049 1.0607486905684709 -0.2566933833287508
-1 2.0069622762472235 1.3555276909717138 1.3738458420384927 1.3307981771643953 1.1352058939547374 1.1872314739705727 2.0206074946330155 2.6193996043859977 0.9754506254457527 2.4788773949517737
-1 1.6559576152851871 1.5613387714537157 0.9820632656447196 0.24990370738791912 0.6790482468297928 0.7177001456270966 1.2177661518329543 -0.010128389509312274 0.9949778601566439 0.2730735896651332
-1 3.3541347870312084 1.8903267206950842 1.6609607533550115 0.6313086218186583 1.0174443932043256 2.1002778641752133 -0.7433879263515524 3.6635365130163358 -0.12072379016630852 1.2613991803119946
-1 0.741882011562536 -0.33389745909875646 0.49850980476986007 0.6209294892871532 -0.9345674636388526 1.0706987501267613 0.17174378573602178 1.4966350235504806 1.7786390376763213 1.6231643119303771
-1 0.737851271176944 3.1107332677301804 0.5595554860713969 0.03240910648046724 0.7418890189368929 2.5744268937009354 0.08490736311553437 0.9454019320976027 2.3004255005209213 2.673423266074501
-1 0.9964678056269282 -0.4050367214023043 0.7634512054670727 0.6104047048598984 -0.18420038230329872 2.8225484519075694 -0.17480506682904684 1.188578222519793 2.3609744942610704 2.0104954250932927
-1 0.8561825142599002 1.4715100244558175 1.1551932439330008 -0.866432954658839 0.06672467583391328 0.6567191940892094 2.1238239921343776 1.9236498444842514 1.774783717232303 2.1705643226440356
-1 2.1686685144492652 -0.46548035607855187 1.7905868508290022 1.7291739618095732 1.8420059988367683 1.2812869543894454 0.7094922226284579 4.578093325453002 2.159649972834322 -0.703298751877151
-1 0.01038121312435214 2.041036231629956 1.406313867978486 1.3944476209150578 -0.7450794741024422 0.36098991012411563 -0.8145936978526842 1.0085439903773337 0.6693692426324003 0.6121851518794861
-1 1.8571542967953807 1.4070713551879899 0.5321067816124654 0.6429601839486434 0.9165980917544774 1.071305634192637 -0.06040670535870918 2.5384035240078604 -0.21377477606093764 0.3369977088082866
-1 2.405103563655566 -0.4546855764355364 -0.24489042907792635 1.3318409806777944 1.2523408877207844 0.9313587923017596 1.2089956458520745 3.0921428523894092 1.956850142357836 0.7702767453893322
-1 0.9086347130699683 1.2100828227228213 0.5327052367165771 -0.6550532780225489 2.5505664076947587 1.4300751019325881 -0.9806442677198526 1.9110672232516768 1.956204319904626 -0.6406447989012172
-1 1.750246620105648 1.3081292130126525 1.4716986993259968 -0.3042704857661218 0.2354470475646966 -0.6074481355981227 0.9333801721029178 1.3220227127047701 2.0998355566318203 3.340047345554312
-1 0.8132766080998793 0.345182592805539 -0.08434230880799043 0.371975995128044 1.030128701009812 -0.0838490306566615 1.891400724652641 2.133657072232741 2.4719821498192935 0.9603084853474415
-1 1.426463569977554 2.123479869287884 1.8449734404123337 0.8841571967965259 1.3206820715765568 2.414835584218742 1.129163483268984 -0.8781190476518506 1.5162895167347454 -0.6528866908043633
-1 1.2017423534681941 1.9686754970835203 1.3014044708959847 -1.0240935923675734 0.7502387139905979 0.8253575777839712 1.224646644221756 1.480689489076607 1.7640815996729344 0.2056821278829375
-1 2.7250146939462083 2.227656483011149 2.84947399343455 2.451014425645574 -0.3739053762247364 1.1582450151950303 1.741290414111453 1.376435447217923 0.35033655530431784 0.4806336989868223
-1 1.3542581369916695 0.415546436380271 0.6688613033041042 0.9102881456111578 0.2547986420844246 1.378444594707075 3.43963729226003 1.3067301378198568 1.5647303411064155 2.043293980780698
-1 1.0913358352352922 2.1175733214306947 0.929020839478381 3.090469607746358 0.09151751891798587 1.5634842729294367 1.8016069710014775 1.4861336762215835 1.6076296539436097 -0.26097034661822094
-1 -0.709300017934053 -0.14570511438959777 0.8487791028889955 -0.3957122997819824 0.23663565146376286 2.66035473479832 2.1479897842790923 1.2106691413007877 -0.45712691497148206 2.4225765811823203
-1 0.14756832470608838 2.3704041393692425 0.6496201584931938 -0.11807063222136005 -0.20506086896030706 1.5881151061076393 3.797132222832481 0.943542745977901 0.8565267747881888 1.1864294682583807
-1 -0.3889342935852145 -0.17743324011571104 1.3604682904339318 0.6593714174698198 -0.3584830057001256 3.514136269889732 0.595913513718282 0.1683068614180695 2.0746193584112143 0.6903921573893614
-1 0.2920446897752229 2.9937346155977957 2.251247553131803 0.6975169699248711 0.4494567463916379 1.319277335273955 0.5367328026447278 2.5267557692090836 0.350600102811225 0.5606888320387985
-1 1.228653481176321 1.0182555282617969 -0.5982787788962058 2.6333900117968314 2.0366003161170663 0.5499289981699178 2.542904251265296 2.2146577311919637 0.3954898163391639 0.6205263945903541
-1 -0.0520426119593238 1.590564747318753 1.6958053948956031 1.3511042599706389 -0.047969026912866974 0.55701288765553 0.9263968623271992 0.590838546777129 2.3308650721102633 0.5135257132439688
-1 1.016635594241282 1.8948650280358326 1.440434304566253 1.4592759362683134 1.6827383192498666 -1.0918246492897437 0.43238661798429845 1.5624487435653098 2.220285861909854 1.271128145985624
-1 -0.7222589043422267 0.5115698429182437 1.3516909750379982 1.6184323538658458 0.3138663124851314 -0.02913500500520727 0.8551827087816364 1.6317432725857857 0.6646228309777373 1.886929067576903
-1 1.4628654761642204 1.8652907041028732 0.6622303129185922 0.7509202647315306 -0.036376585463356426 0.7850159634599014 2.2985430427240017 1.0460715145011406 0.8526933674534585 1.1533090709516742
-1 1.0669747034293164 -0.1510400394042828 -0.34893623474816793 1.7754617342041603 1.3436972220233374 3.022419531056307 1.9684180926734447 1.4858550357170357 2.9588700999527395 -0.02437800790558642
-1 0.5379644371164043 -0.27906681292084 0.3380177280655655 0.33722013060203193 0.6571438211538795 1.2052933591547657 1.7731403611930516 0.5077273284789499 1.5626883295465674 -0.050171508356717576
-1 1.2224363031291428 2.179387632259403 1.729844754655598 1.7261086434406607 1.6565721133198088 1.889839925928689 1.8345686999088797 1.051447084834809 0.9359370646456183 0.7645291821631122
-1 2.60292814182841 0.8804157611166004 -0.955075955060207 1.2946117062161222 2.107044588585438 0.2497683006856819 1.6038124754155476 -0.7214552551237594 0.452098771396898 0.6986965061465407
-1 1.0412661702670807 -1.3958762787534025 3.074541266637782 1.76411325380808 -0.39903368929064653 1.3136620541582826 1.1746725568355456 -0.6576469095064521 0.15286303171879478 2.117286307501297
-1 0.31859147805604837 1.2450573919933268 -0.5933863589583486 1.616822450960686 2.3307511175574707 1.4675892671924506 -0.6797208500497198 -0.6357164936808151 2.6616070340209608 0.12503414768311838
-1 0.015640995722970286 0.9521770024879528 -0.021136921124242036 1.5781474391889052 0.7227013060272598 0.7987343733885311 -0.6768705185766593 1.2194260902982417 0.6115575336879959 1.776636860101025
-1 1.7473265876837165 -1.3416662707254097 -0.3178957317552682 -0.7952748363966 -0.0012367493892466719 1.5102140866553868 1.3893554303705593 1.253090374551591 0.37849714433826975 3.8427708908843417
-1 0.1249935088342321 0.9175321556781342 1.2521433252052363 0.10448935908110157 1.748729859258747 1.9013556247400216 2.348145639899152 0.4626753070549736 3.7821319980165344 0.47822934584228827
-1 1.5461491524524733 1.0442419265941036 -0.016418025211677234 -0.6189521317249826 0.9719604409404735 1.1409654487054224 0.5144932080563054 1.677400744669605 1.60852217407324 0.9996875540653996
-1 1.1571589981163284 2.815325710919601 0.20772173229184132 -0.27577989741307296 0.14104944330527658 0.2590225341905401 -0.33859238160667027 2.803757221911037 1.035764969030257 0.16925873998127916
-1 1.8759906736161591 -0.7858122581388844 1.0848147823038492 1.346569014348389 -0.7811951242276918 -0.28091748058441146 0.10734544787850497 1.1946024654289003 1.6406107469177638 1.418186454569726
-1 -0.2974414971504451 -0.7263225506198576 1.667022614186794 1.1033345452667596 -0.2451904831865781 -0.011381119202380274 -0.2081120315941396 0.19505925177058225 1.083883779309256 0.2476147974455678
-1 1.9875844064011776 -1.0551408447589177 0.9235522752742322 -0.1465157757078015 -0.24048981040870454 -0.3751333753617203 1.6243406244366847 -0.38149309424785227 -0.2845380129435624 -0.4586888921471284
-1 -0.43391027275254457 1.3012041634540212 0.34931152784647057 0.2724840573311986 1.895997027401461 0.7955372939424181 2.717841382622603 0.9983211958138658 3.297958269369362 0.28612843397709364
-1 0.09388869926828014 0.7292780962393748 -0.48425219833973965 1.2122506447105803 0.7074049606666732 1.0448613427298579 1.4758560188256675 -0.32361188073438485 2.040268428137505 1.685468904484563
-1 1.0792167846288987 -0.2826348408764243 1.3133025554220168 -0.29264376303967365 0.12334584816456384 1.7916405818476433 2.4401329350478367 1.373668417749465 1.1438238823893943 2.9513159396946955
-1 0.6272602458353195 0.012788348875383604 3.339583303835828 -0.5656471248096915 1.7436358009297308 -0.0849133378284781 1.8766630914593128 0.3286471991737121 0.8557785757636693 1.204343384424849
-1 0.9053623358277365 2.851790381485327 1.0805997920016692 -0.5635383000263379 0.9576644151670836 1.9289302434370748 -0.13805339731578536 3.4861795141210807 0.2005081416731367 1.6544819624039082
-1 0.4910096613955415 1.6681822364133903 0.8202936721704033 2.148200954440342 2.558162860929867 0.6606047330906034 0.7989603259919102 1.0689702044523541 0.7184320065316048 2.023034231513219
-1 1.1256411487276385 0.19900785835501755 1.2085575135898547 -1.356418780267496 0.785218957218392 2.70677848091574 1.9987708656840728 0.6868097252341125 -1.241646154239319 2.9393145029129917
-1 1.9337642982267669 -0.7156557544578908 0.16408179712477566 1.9408268646309592 1.0190820244131475 1.1951052545533123 0.4481509783235238 1.2668590723499928 0.8102310436768919 0.7718152165895394
-1 1.614923882092461 0.19469602471151815 3.766869874799438 -1.3377164159484254 -0.878559530240216 0.3364262245077355 1.8010436667360947 1.777688731609198 2.311140988026292 1.1771602185088652
-1 0.6784758917678138 -0.18464751605809093 1.6835398190359525 0.9616873095363908 1.8625881930711616 1.9970275330538905 1.0465679673330561 1.7874857759504277 1.7797672480031759 0.9806567017840313
-1 1.9543101838028707 -0.44413349405470304 0.3787949477054693 0.09081285199753486 2.460919892284841 0.29445632839265967 0.9120233970904723 1.120046161146032 0.3979415181383884 1.6677498018942478
-1 2.7931886788791984 0.05569901049144255 1.2190718219058607 1.3326923562520578 1.7863786156200971 1.8057619970370333 0.9782497583237075 1.1631245252370526 -0.10647683276082942 0.8291413719741013
-1 0.6746786109931104 0.693150020176567 0.8806942321642721 1.3171663922040504 -0.18964506284133353 1.752816912385852 0.0197418639082243 0.04087366490530042 -0.31356701603876047 1.1688888267402135
-1 -0.8047119894089716 -0.19086822099982692 0.7230280053386025 0.47661575325565886 2.783553868954165 0.39034536568120837 2.4620798409550657 0.3460544872000194 1.6811241975213127 -0.5755589941181993
-1 -0.43736971419082993 0.9731234165917454 0.044303702104787734 1.3285736602137515 1.8134256070231687 4.003995098206477 -0.5823423595861437 1.1000778881670024 2.275332508162996 1.7059404281570498
-1 2.7870499907770374 1.5359115092914868 0.4415840592158585 3.0819184178594012 1.0142235114013434 1.4175457438753696 0.7830675289154578 0.718110803107776 1.752603937821668 0.8681755199560836
-1 1.6629646464798866 1.5720752857585811 1.866918319229821 2.011503983207959 -0.08953127029042407 3.250764941529524 0.8681970712263898 1.8122090555675 0.30361209115382115 1.6190898270526166
-1 0.8689387257925889 1.088532128821611 -0.9638248404112064 -0.03629852962978575 1.5819544244821397 0.533196869581712 1.1629368405935705 0.5952984584910554 0.5901966383762997 0.8680425050414964
-1 0.5657393409043414 0.1269546832382663 -4.0341609669503065E-4 1.1489057321179976 0.25156572912668473 0.48265829258343707 1.051802672080171 -0.797907065268961 0.40336920791124586 0.34951103336108913
-1 2.842259431863403 0.4523061399118463 1.1073417696817962 0.820613792637092 1.2347466769629105 2.445490993196761 -0.1542908283123816 0.8816264920520589 1.7423151819076375 1.6594291913667136
-1 1.5860855260228202 2.8392671863491734 0.5188572450043611 1.047507505252711 3.054126605012979 -0.6006852937930467 0.34982369626834076 0.11607093207054109 1.829510982388106 0.001994427476862848
-1 0.17902283956677512 0.41558050427565774 1.5871923905064695 1.5996558530208187 0.07353003075760078 1.0705630115074813 2.675599132354674 0.7650850730679759 0.8607570887706816 0.9903122299033713
-1 0.7379554955291575 2.072325148209555 0.4462636170973716 0.6880836555742617 0.3535374515580053 0.19240929522338934 2.2791306741261153 1.7199300904991563 2.3790655960655718 -0.4294392660855837
-1 0.5642895627754023 0.9044762545519158 1.4797756442552041 0.6976030137900451 2.5013240752661825 0.8121543920897196 1.864316073466811 1.3213558088397361 2.17814424984865 1.8979547805463015
-1 1.103147738753372 1.616958446219673 2.8479619253624797 3.368348617090012 2.5438833831666434 1.6704650810547208 0.8562521160479526 0.7542938264829215 0.5266574196400498 -0.2890730154742367
-1 1.9142555817765898 0.8049202262783679 2.5019528805928912 0.5238106873271193 1.5359406981988988 2.8356323728714847 3.239716573932437 1.2510518752596296 1.715571851101242 1.222780980267732
-1 0.6041885893884307 0.5707299204297884 1.2540953158421435 1.5510759633503302 -0.4667440237195346 0.26676051631424014 -0.565572799800238 1.4387028778945943 0.9644694652315191 2.1255685675532967
-1 1.7491189390587218 1.2227275279214738 -0.8505836769821726 -0.903216529384467 0.29076052330579005 0.2892222629138922 2.3647508720986217 1.2652921314867005 1.0348376197540348 -0.2562195481430878
-1 2.3800831934663433 -0.010431805594117938 0.8430880161541389 1.278733772829872 1.585905177486663 0.28093811664192425 1.5849634563502026 1.078413585522204 0.4426572711340797 0.6530352928058241
-1 1.7049361022681717 -0.27653366462628215 0.9445796767766628 0.041969783781791725 0.3467762982688263 -0.4874473134901387 0.7531152429497019 0.30167927793354254 2.765258841783637 -0.23618185513880707
-1 0.8097421163995955 0.17729598233902988 2.5214858992792863 1.5180096630697852 1.9899028361613595 0.57436615658855 0.5307905908280097 0.9190155285250498 0.6466076660416842 -0.10626054479014013
-1 2.395022852849255 2.3321432458593208 1.6804528385827555 2.2258435456318937 1.4611936535655663 1.058998523699314 0.31838562794784586 0.39659928716273496 1.4494935872166117 1.391374864616476
-1 1.735291612472487 -0.3191446365558481 0.6607372043463824 1.541446196262466 0.4947578059034823 -0.8293819909066149 0.76596276473359 -0.0851263113957168 1.9200627040331277 1.5173271962047457
-1 0.48007434755469713 0.7936351950677151 1.365699852551887 1.1109515050883414 -0.12031241802004855 -0.18610833660205306 0.2974034656359261 1.3687489920730513 2.1059823724132523 0.941953020877809
-1 2.4520203316077964 1.11003521338105 0.4722773485870979 2.737384705503226 0.7192036221774767 0.6242245483941781 1.2609692406366446 2.0575095746651133 1.3495884659991346 2.0764197346896935
-1 -0.7842236897873944 1.492890069052242 1.765349236922137 1.300042277956386 1.5799338298744416 1.060819121020154 1.1674652333797013 -0.4149263766035056 0.09348961754442264 3.5461008823168543
-1 0.8620605536733185 0.08406312778559633 1.5415557685694021 0.2051913612441839 0.19504752604759068 1.534576255114414 3.107649420779101 1.020214612080108 0.3221723632541289 1.4874661690065234
-1 1.489728417116672 0.06558708406688907 -1.8670045751011424 1.7828483838262912 -0.683023788962926 1.79761793764676 1.5085129455490893 1.2434470961660735 0.5774571270514824 1.4932340982697638
-1 -1.5669127739356443 0.34356934741624334 3.0594253296534424 0.774762761699532 1.0055392162451373 1.3241023069988664 1.1749986092813367 2.19297533155391 1.0435550797072737 2.095514184709966
-1 -0.3634276403952408 1.4409978371532932 0.3823184763192483 0.6254885387609036 -0.35123251562864244 1.819196851350437 2.14116717870738 0.46320929513337494 0.5695755038115515 2.501714843566015
-1 0.013632028077828595 1.8215490521966027 1.7653867346915684 1.4163095749484134 0.25841398470159227 2.2048024054278192 0.9286824219992222 1.133706943250312 1.7330998187732773 1.3552028632095436
-1 1.012536342646575 1.4202805284853588 1.1660963924281333 2.7434608590955594 2.405339566810934 0.35678139532687714 0.7007075773809261 -0.1461824532706133 -0.1116775801341563 2.455669156783493
-1 1.7224210079670872 0.25824562782106875 1.896388948392676 1.5490245294926566 0.566495628127113 1.4439902246901806 -1.1659487820432086 1.2648317293133733 -0.8687762383751962 2.055108054071261
-1 3.5125527162365486 -0.022436189584495336 1.1332983732450903 -0.07450694962415794 0.09001591132041731 0.5853417525905302 3.337681467433381 -0.32222401787392774 2.539181628048545 1.0754745872100386
-1 0.2455099848454918 1.2693508037734986 1.6546347888138584 -2.148792530729241 0.46441142559185566 1.1734134286137057 1.0258039884088828 -0.5586646913499485 -0.3258731206571115 -0.821219883870792
-1 1.827217125452903 1.731864545109457 0.928872208086588 1.2056977735867256 1.818214291632629 0.6585878488136441 1.8002230735809155 0.8708150904043206 -1.5838120389612023 0.8585857536471672
-1 2.2021363682137154 0.4761145331093257 -0.025920931323458296 1.7449566792074553 0.8629966232032906 1.4723084204343524 1.6159540778305606 2.029453834185225 2.26325946376582 1.376244768900244
-1 0.010342658978543584 1.515273076994554 0.19611635441491626 1.654784841440513 -0.033943991780339244 0.6714632219862774 0.2641936457650498 -0.700825233754335 0.23452605282080619 1.621398184902529
-1 1.0480165819981573 0.8797819263901776 -0.641443663240362 0.12817609127433438 1.3647120235220283 -0.48615470921060977 1.0720144074421256 -0.38026314794700733 0.8069083073855456 1.3433152284915995
-1 0.3761857330260455 0.23219703324626284 1.921560210024654 0.38896862067672255 1.1468761246542036 0.8203362705962437 -0.23996402764305458 1.5950906570841252 3.639574852127676 -0.2443366415192889
-1 0.8759552320204246 0.33529291747248857 -0.2551391418074267 0.29090645845832075 -1.1529071816719476 0.7412858224772877 1.2719555749592364 1.3289131183268248 1.3228711885726534 1.5021325652417783
-1 0.439646111605676 0.8753273571625453 -0.5195310985749739 2.656469182704334 0.8907416242841371 1.4150606950578886 3.175298549230411 0.44910268745784754 0.8447367653706002 1.668648718911232
-1 1.1404102468865998 1.4857266483300324 -0.31291554366933605 1.3205568580259288 2.4092775306975023 1.6397731783027976 1.1251407071414252 2.3565497583137436 1.8353622317028138 -1.1683108743275552
-1 2.08122023149769 1.1571239260956436 -0.08056173908716335 0.768249986206349 1.3171573148662759 -0.18023949555734187 -0.25107977208536614 0.3528408329964078 0.7749381509220793 -0.7113421449812265
-1 0.1473845257811165 -1.0521567114122852 -0.47637816156748225 1.4949699096476212 2.271087115324705 1.3826153478446757 2.7436405167916025 -0.02075677223859529 1.1765040243159015 -0.025438785956181542
-1 2.7027482205114826 1.577562959861571 -0.5669337503778331 1.5215534981321372 1.2652067920381662 2.7463387790797444 -0.10995208915345178 -0.9887358827125716 0.7108329384066776 1.3629285100379036
-1 2.9573936017540556 0.1614860515756119 -0.3278573695860796 1.0550562822356224 1.4787913549079965 1.6928275048278305 1.0586362008998798 1.1651361732301 2.361382321862904 2.524722697822938
-1 -0.918683252112166 1.1912188403555544 -0.6386682219001243 0.12852707081177273 1.0186959070915036 -0.7396656648881279 1.390222924345315 -0.6776334559974988 1.6871484268646286 0.9835794195231572
-1 -0.9501651670329723 1.6369415588995389 0.6124916702658543 2.055786019572368 0.20091594691375603 0.27955238962400497 1.8462485957757835 0.766850497882725 0.6439523544318226 -0.45529021581249385
-1 0.08294835997064665 -0.27721496031157833 -0.35456350243850276 0.11228054309930591 3.4737188479123104 0.8438116500646802 1.2682583387249549 2.2187948258289913 1.6181904099869335 2.2762749025565983
-1 1.83339856452743 2.673091344347915 0.7389331991568107 2.067911927048983 1.3782410940205578 2.030974790626103 0.6888073746059981 -0.518101069445974 0.6230936256620102 1.633224100697245
-1 1.7398691778151973 1.1247533360425708 0.2807774763651275 -0.6955611341182046 1.592036824083598 -0.04050352181158767 1.3865010706574772 1.4019929481612587 -0.2642443959402707 0.5934301817863643
-1 -2.019173847473457 2.1681048424611418 1.3422907243645614 0.6467676712250852 0.49642291457381404 1.289806437146178 0.5287383514431835 2.8692305624115457 0.37484482468477054 2.4484351720405875
-1 0.024288362749408376 1.0351720632502537 1.6837605528916666 1.3872579136738206 1.2679651380538202 1.4021182744167016 -0.7041852642469104 1.6806756125489901 0.1307750250742319 2.3317291973580314
-1 -0.06080175616636896 1.0543357215752764 2.099562273809995 0.6174473985800795 0.5458218639483579 -0.1330076265446425 1.782807067124061 3.835868752952487 1.0749746574622228 2.2318191600680155
-1 2.7819388327740797 1.1294517177544148 1.4625685601160094 0.8160359631571115 1.5866067958993928 3.0076062737914184 1.5740992429858394 1.3901837375360109 2.7120095549614893 -0.5329184800190412
-1 -0.08342899095133993 3.2552165445304735 -0.6127389181137219 0.20728621073827602 1.1715077138725913 0.496873621214974 0.7991470651533773 0.5625481785655475 0.7904628851956959 0.485293468158445
-1 0.5879363673253968 0.5480289705171163 0.26878358296170424 0.9493365691333653 0.34421794272116246 1.4045876345319372 0.8323003475233924 1.3822841645472739 1.9408510354113169 2.3160979297534636
-1 2.049725023995715 1.138714228201635 2.228635558152831 1.4833354495511806 0.5549789742701208 1.3850264438047617 1.4418684507619366 3.131909530291612 3.2277156524053705 0.5657214292376471
-1 0.7278339716721132 0.8342775647290255 -0.7804056350094557 1.8999099617115354 1.5129989349558883 1.6238396258236993 -0.13761070763179828 0.6429461405182848 -0.2642956636249272 0.8065034962137944
-1 2.5931023834096854 0.9018261137939111 1.5584456516926881 -0.5802390356360938 1.941618818488975 0.9214260344294213 0.556884632504891 0.26832249168681577 2.4966263079255677 1.1243846486761992
-1 0.14419967158797142 0.9874339005630041 0.8076366869263152 0.515723994659785 -0.9385248237540935 -0.17924876736882722 1.1150091706474443 1.5543894995228547 1.615026336442979 1.1708620595483625
-1 2.1530687310737866 -1.8203657185808888 0.6380519600335401 2.02809789647314 0.30946138948160296 1.7692953099290327 1.0369557864170398 0.3326256746163322 -0.275581422683832 0.21583516634100164
-1 0.896534730391731 2.1309314580821708 0.9688774738233893 0.7810503130534793 1.3417441924762596 0.10748935054015485 0.8725839981470569 2.68470748226214 0.5000051011542708 1.6309858671990054
-1 0.2798388059875424 0.46301766350582063 -0.21330838748068315 1.516256000433057 -0.9521989902404524 1.8668922242244914 -1.429783656173199 0.24500379527846305 1.0717746705573634 2.929223328366103
-1 1.5580038958637812 1.4690967454818293 3.5043865357520065 0.8077006250670602 1.70873452721819 1.725133865080763 -0.17803725982825802 1.2072416111273427 0.7258484330322263 0.9666451576387228
-1 -0.2937927716783808 2.209449837105502 2.471323239279583 1.9931843786987273 0.4670001618859797 1.2200671907651737 1.3884758303330187 1.1014939571310298 1.2017172341718294 2.657179062084367
-1 0.9402246743347112 0.40154461288043775 3.407916599846658 0.732993794216273 0.7120872061718131 0.7443371156456304 0.261691914047522 -1.7816254435328527 1.1872515149455043 1.2859514985608926
-1 1.5116064491281778 2.2468889028407437 0.45828491922709613 1.2192147082911882 0.6354365593721796 -0.2656322662271462 0.22961524227015095 0.6580482520092654 0.8557895993898526 1.1404110974520998
-1 2.738506436693102 1.129940083852354 -0.2531479159181209 -0.3313565595449408 2.157889045868747 0.7757459702743189 2.5165730696859523 -0.504719944568053 0.19221810745654677 0.4962627597149971
-1 3.141323496200573 1.4040718012832414 0.6638624853970507 0.

<TRUNCATED>

[15/21] git commit: spark-shell -> bin/spark-shell

Posted by pw...@apache.org.
spark-shell -> bin/spark-shell


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/b810a85c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/b810a85c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/b810a85c

Branch: refs/heads/master
Commit: b810a85cdddb247e1a104f4daad905b97222ad85
Parents: 980afd2
Author: Prashant Sharma <pr...@imaginea.com>
Authored: Thu Jan 2 18:37:40 2014 +0530
Committer: Prashant Sharma <pr...@imaginea.com>
Committed: Thu Jan 2 18:37:40 2014 +0530

----------------------------------------------------------------------
 README.md                       |  2 +-
 docs/index.md                   |  2 +-
 docs/mllib-guide.md             |  2 +-
 docs/quick-start.md             |  4 ++--
 docs/running-on-yarn.md         |  2 +-
 docs/scala-programming-guide.md | 10 +++++-----
 docs/spark-debugger.md          |  2 +-
 docs/spark-standalone.md        |  4 ++--
 make-distribution.sh            |  2 +-
 9 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b810a85c/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 1550a8b..170e964 100644
--- a/README.md
+++ b/README.md
@@ -19,7 +19,7 @@ which is packaged with it. To build Spark and its example programs, run:
 
 Once you've built Spark, the easiest way to start using it is the shell:
 
-    ./spark-shell
+    ./bin/spark-shell
 
 Or, for the Python API, the Python shell (`./pyspark`).
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b810a85c/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index d3ac696..a5c179c 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -34,7 +34,7 @@ to connect to. This can be a [URL for a distributed cluster](scala-programming-g
 or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using
 `local` for testing.
 
-Finally, you can run Spark interactively through modified versions of the Scala shell (`./spark-shell`) or
+Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or
 Python interpreter (`./pyspark`). These are a great way to learn the framework.
 
 # Launching on a Cluster

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b810a85c/docs/mllib-guide.md
----------------------------------------------------------------------
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md
index c1ff9c4..95537ef 100644
--- a/docs/mllib-guide.md
+++ b/docs/mllib-guide.md
@@ -87,7 +87,7 @@ svmAlg.optimizer.setNumIterations(200)
 val modelL1 = svmAlg.run(parsedData)
 {% endhighlight %}
 
-Both of the code snippets above can be executed in `spark-shell` to generate a
+Both of the code snippets above can be executed in `bin/spark-shell` to generate a
 classifier for the provided dataset.
 
 Available algorithms for binary classification:

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b810a85c/docs/quick-start.md
----------------------------------------------------------------------
diff --git a/docs/quick-start.md b/docs/quick-start.md
index bdf127d..912f332 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -20,7 +20,7 @@ $ sbt/sbt assembly
 ## Basics
 
 Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively.
-Start the shell by running `./spark-shell` in the Spark directory.
+Start the shell by running `./bin/spark-shell` in the Spark directory.
 
 Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory:
 
@@ -99,7 +99,7 @@ scala> linesWithSpark.count()
 res9: Long = 15
 {% endhighlight %}
 
-It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
+It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
 
 # A Standalone App in Scala
 Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b810a85c/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 00cad99..be32034 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -112,7 +112,7 @@ For example:
 
     SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
     SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
-    MASTER=yarn-client ./spark-shell
+    MASTER=yarn-client ./bin/spark-shell
 
 # Building Spark for Hadoop/YARN 2.2.x
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b810a85c/docs/scala-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 1db255c..16ae150 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -13,7 +13,7 @@ At a high level, every Spark application consists of a *driver program* that run
 
 A second abstraction in Spark is *shared variables* that can be used in parallel operations. By default, when Spark runs a function in parallel as a set of tasks on different nodes, it ships a copy of each variable used in the function to each task. Sometimes, a variable needs to be shared across tasks, or between tasks and the driver program. Spark supports two types of shared variables: *broadcast variables*, which can be used to cache a value in memory on all nodes, and *accumulators*, which are variables that are only "added" to, such as counters and sums.
 
-This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `spark-shell` script. We highly recommend doing that to follow along!
+This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `bin/spark-shell` script. We highly recommend doing that to follow along!
 
 # Linking with Spark
 
@@ -54,16 +54,16 @@ object for more advanced configuration.
 
 The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later.
 
-In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `spark-shell` on four cores, use
+In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on four cores, use
 
 {% highlight bash %}
-$ MASTER=local[4] ./spark-shell
+$ MASTER=local[4] ./bin/spark-shell
 {% endhighlight %}
 
 Or, to also add `code.jar` to its classpath, use:
 
 {% highlight bash %}
-$ MASTER=local[4] ADD_JARS=code.jar ./spark-shell
+$ MASTER=local[4] ADD_JARS=code.jar ./bin/spark-shell
 {% endhighlight %}
 
 ### Master URLs
@@ -95,7 +95,7 @@ If you want to run your application on a cluster, you will need to specify the t
 * `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them).
 * `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies.
 
-If you run `spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it.  This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed.
+If you run `bin/spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it.  This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./bin/spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed.
 
 # Resilient Distributed Datasets (RDDs)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b810a85c/docs/spark-debugger.md
----------------------------------------------------------------------
diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md
index d6315d9..11c51d5 100644
--- a/docs/spark-debugger.md
+++ b/docs/spark-debugger.md
@@ -39,7 +39,7 @@ where `path/to/event-log` is where you want the event log to go relative to `$SP
 
 ### Loading the event log into the debugger
 
-1. Run a Spark shell with `MASTER=<i>host</i> ./spark-shell`.
+1. Run a Spark shell with `MASTER=<i>host</i> ./bin/spark-shell`.
 2. Use `EventLogReader` to load the event log as follows:
     {% highlight scala %}
 spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log"))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b810a85c/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index c2ce230..ec5ae5b 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -143,9 +143,9 @@ constructor](scala-programming-guide.html#initializing-spark).
 
 To run an interactive Spark shell against the cluster, run the following command:
 
-    MASTER=spark://IP:PORT ./spark-shell
+    MASTER=spark://IP:PORT ./bin/spark-shell
 
-Note that if you are running spark-shell from one of the spark cluster machines, the `spark-shell` script will
+Note that if you are running spark-shell from one of the spark cluster machines, the `bin/spark-shell` script will
 automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`.
 
 You can also pass an option `-c <numCores>` to control the number of cores that spark-shell uses on the cluster.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b810a85c/make-distribution.sh
----------------------------------------------------------------------
diff --git a/make-distribution.sh b/make-distribution.sh
index 82638a2..8765c7e 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -34,7 +34,7 @@
 # 2) cd to deploy dir; ./bin/start-master.sh
 # 3) Verify master is up by visiting web page, ie http://master-ip:8080.  Note the spark:// URL.
 # 4) ./bin/start-slave.sh 1 <<spark:// URL>>
-# 5) MASTER="spark://my-master-ip:7077" ./spark-shell
+# 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell
 #
 
 # Figure out where the Spark framework is installed


[03/21] moved user scripts to bin folder

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/pagerank_data.txt
----------------------------------------------------------------------
diff --git a/pagerank_data.txt b/pagerank_data.txt
deleted file mode 100644
index 95755ab..0000000
--- a/pagerank_data.txt
+++ /dev/null
@@ -1,6 +0,0 @@
-1 2
-1 3
-1 4
-2 1
-3 1
-4 1

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/pyspark
----------------------------------------------------------------------
diff --git a/pyspark b/pyspark
deleted file mode 100755
index 4941a36..0000000
--- a/pyspark
+++ /dev/null
@@ -1,66 +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.
-#
-
-# Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
-
-# Export this as SPARK_HOME
-export SPARK_HOME="$FWDIR"
-
-SCALA_VERSION=2.9.3
-
-# Exit if the user hasn't compiled Spark
-if [ ! -f "$FWDIR/RELEASE" ]; then
-  # Exit if the user hasn't compiled Spark
-  ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null
-  if [[ $? != 0 ]]; then
-    echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2
-    echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
-    exit 1
-  fi
-fi
-
-# 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
-
-# Figure out which Python executable to use
-if [ -z "$PYSPARK_PYTHON" ] ; then
-  PYSPARK_PYTHON="python"
-fi
-export PYSPARK_PYTHON
-
-# Add the PySpark classes to the Python path:
-export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH
-
-# Load the PySpark shell.py script when ./pyspark is used interactively:
-export OLD_PYTHONSTARTUP=$PYTHONSTARTUP
-export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py
-
-if [ -n "$IPYTHON_OPTS" ]; then
-  IPYTHON=1
-fi
-
-if [[ "$IPYTHON" = "1" ]] ; then
-  IPYTHON_OPTS=${IPYTHON_OPTS:--i}
-  exec ipython "$IPYTHON_OPTS" -c "%run $PYTHONSTARTUP"
-else
-  exec "$PYSPARK_PYTHON" "$@"
-fi

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/pyspark.cmd
----------------------------------------------------------------------
diff --git a/pyspark.cmd b/pyspark.cmd
deleted file mode 100644
index 7c26fbb..0000000
--- a/pyspark.cmd
+++ /dev/null
@@ -1,23 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-rem This is the entry point for running PySpark. To avoid polluting the
-rem environment, it just launches a new cmd to do the real work.
-
-cmd /V /E /C %~dp0pyspark2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/pyspark2.cmd
----------------------------------------------------------------------
diff --git a/pyspark2.cmd b/pyspark2.cmd
deleted file mode 100644
index f58e349..0000000
--- a/pyspark2.cmd
+++ /dev/null
@@ -1,55 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-set SCALA_VERSION=2.9.3
-
-rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
-
-rem Export this as SPARK_HOME
-set SPARK_HOME=%FWDIR%
-
-rem Test whether the user has built Spark
-if exist "%FWDIR%RELEASE" goto skip_build_test
-set FOUND_JAR=0
-for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
-  set FOUND_JAR=1
-)
-if "%FOUND_JAR%"=="0" (
-  echo Failed to find Spark assembly JAR.
-  echo You need to build Spark with sbt\sbt assembly before running this program.
-  goto exit
-)
-:skip_build_test
-
-rem Load environment variables from conf\spark-env.cmd, if it exists
-if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
-
-rem Figure out which Python to use.
-if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python
-
-set PYTHONPATH=%FWDIR%python;%PYTHONPATH%
-
-set OLD_PYTHONSTARTUP=%PYTHONSTARTUP%
-set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py
-
-echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH%
-
-"%PYSPARK_PYTHON%" %*
-:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/run-example
----------------------------------------------------------------------
diff --git a/run-example b/run-example
deleted file mode 100755
index 08ec717..0000000
--- a/run-example
+++ /dev/null
@@ -1,81 +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.
-#
-
-SCALA_VERSION=2.9.3
-
-# Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
-
-# Export this as SPARK_HOME
-export SPARK_HOME="$FWDIR"
-
-# 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 [ -z "$1" ]; then
-  echo "Usage: run-example <example-class> [<args>]" >&2
-  exit 1
-fi
-
-# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
-# to avoid the -sources and -doc packages that are built by publish-local.
-EXAMPLES_DIR="$FWDIR"/examples
-SPARK_EXAMPLES_JAR=""
-if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
-  # Use the JAR from the SBT build
-  export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
-fi
-if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then
-  # Use the JAR from the Maven build
-  # TODO: this also needs to become an assembly!
-  export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar`
-fi
-if [[ -z $SPARK_EXAMPLES_JAR ]]; then
-  echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2
-  echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
-  exit 1
-fi
-
-# Since the examples JAR ideally shouldn't include spark-core (that dependency should be
-# "provided"), also add our standard Spark classpath, built using compute-classpath.sh.
-CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
-CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH"
-
-# Find java binary
-if [ -n "${JAVA_HOME}" ]; then
-  RUNNER="${JAVA_HOME}/bin/java"
-else
-  if [ `command -v java` ]; then
-    RUNNER="java"
-  else
-    echo "JAVA_HOME is not set" >&2
-    exit 1
-  fi
-fi
-
-if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
-  echo -n "Spark Command: "
-  echo "$RUNNER" -cp "$CLASSPATH" "$@"
-  echo "========================================"
-  echo
-fi
-
-exec "$RUNNER" -cp "$CLASSPATH" "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/run-example.cmd
----------------------------------------------------------------------
diff --git a/run-example.cmd b/run-example.cmd
deleted file mode 100644
index 5b2d048..0000000
--- a/run-example.cmd
+++ /dev/null
@@ -1,23 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-rem This is the entry point for running a Spark example. To avoid polluting
-rem the environment, it just launches a new cmd to do the real work.
-
-cmd /V /E /C %~dp0run-example2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/run-example2.cmd
----------------------------------------------------------------------
diff --git a/run-example2.cmd b/run-example2.cmd
deleted file mode 100644
index dbb371e..0000000
--- a/run-example2.cmd
+++ /dev/null
@@ -1,61 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-set SCALA_VERSION=2.9.3
-
-rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
-
-rem Export this as SPARK_HOME
-set SPARK_HOME=%FWDIR%
-
-rem Load environment variables from conf\spark-env.cmd, if it exists
-if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
-
-rem Test that an argument was given
-if not "x%1"=="x" goto arg_given
-  echo Usage: run-example ^<example-class^> [^<args^>]
-  goto exit
-:arg_given
-
-set EXAMPLES_DIR=%FWDIR%examples
-
-rem Figure out the JAR file that our examples were packaged into.
-set SPARK_EXAMPLES_JAR=
-for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*assembly*.jar") do (
-  set SPARK_EXAMPLES_JAR=%%d
-)
-if "x%SPARK_EXAMPLES_JAR%"=="x" (
-  echo Failed to find Spark examples assembly JAR.
-  echo You need to build Spark with sbt\sbt assembly before running this program.
-  goto exit
-)
-
-rem Compute Spark classpath using external script
-set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
-set DONT_PRINT_CLASSPATH=0
-set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH%
-
-rem Figure out where java is.
-set RUNNER=java
-if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java
-
-"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %*
-:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/spark-shell
----------------------------------------------------------------------
diff --git a/spark-shell b/spark-shell
deleted file mode 100755
index 9608bd3..0000000
--- a/spark-shell
+++ /dev/null
@@ -1,87 +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.
-#
-
-#
-# Shell script for starting the Spark Shell REPL
-# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}
-# if those two env vars are set in spark-env.sh but MASTER is not.
-# Options:
-#    -c <cores>    Set the number of cores for REPL to use
-#
-
-# Enter posix mode for bash
-set -o posix
-
-FWDIR="`dirname $0`"
-
-for o in "$@"; do
-  if [ "$1" = "-c" -o "$1" = "--cores" ]; then
-    shift
-    if [ -n "$1" ]; then
-      OPTIONS="-Dspark.cores.max=$1"
-      shift
-    fi
-  fi
-done
-
-# Set MASTER from spark-env if possible
-if [ -z "$MASTER" ]; then
-  if [ -e "$FWDIR/conf/spark-env.sh" ]; then
-    . "$FWDIR/conf/spark-env.sh"
-  fi
-  if [[ "x" != "x$SPARK_MASTER_IP" && "y" != "y$SPARK_MASTER_PORT" ]]; then
-    MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}"
-    export MASTER
-  fi
-fi
-
-# Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in
-# binary distribution of Spark where Scala is not installed
-exit_status=127
-saved_stty=""
-
-# restore stty settings (echo in particular)
-function restoreSttySettings() {
-  stty $saved_stty
-  saved_stty=""
-}
-
-function onExit() {
-  if [[ "$saved_stty" != "" ]]; then
-    restoreSttySettings
-  fi
-  exit $exit_status
-}
-
-# to reenable echo if we are interrupted before completing.
-trap onExit INT
-
-# save terminal settings
-saved_stty=$(stty -g 2>/dev/null)
-# clear on error so we don't later try to restore them
-if [[ ! $? ]]; then
-  saved_stty=""
-fi
-
-$FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
-
-# record the exit status lest it be overwritten:
-# then reenable echo and propagate the code.
-exit_status=$?
-onExit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1d1a6258/spark-shell.cmd
----------------------------------------------------------------------
diff --git a/spark-shell.cmd b/spark-shell.cmd
deleted file mode 100644
index 3e52bf8..0000000
--- a/spark-shell.cmd
+++ /dev/null
@@ -1,22 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-set FWDIR=%~dp0
-
-cmd /V /E /C %FWDIR%spark-class2.cmd org.apache.spark.repl.Main %*


[02/21] git commit: add admin scripts to sbin

Posted by pw...@apache.org.
add admin scripts to sbin

Signed-off-by: shane-huang <sh...@intel.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/fcfe4f92
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/fcfe4f92
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/fcfe4f92

Branch: refs/heads/master
Commit: fcfe4f920484b64b01e4e22219d59c78ffd17054
Parents: dfbdc9d
Author: shane-huang <sh...@intel.com>
Authored: Mon Sep 23 12:42:34 2013 +0800
Committer: shane-huang <sh...@intel.com>
Committed: Mon Sep 23 12:42:34 2013 +0800

----------------------------------------------------------------------
 bin/compute-classpath.cmd  |  69 -----------------
 bin/compute-classpath.sh   |  61 ---------------
 bin/slaves.sh              |  74 ------------------
 bin/spark-config.sh        |  36 ---------
 bin/spark-daemon.sh        | 164 ----------------------------------------
 bin/spark-daemons.sh       |  35 ---------
 bin/start-all.sh           |  34 ---------
 bin/start-master.sh        |  52 -------------
 bin/start-slave.sh         |  35 ---------
 bin/start-slaves.sh        |  48 ------------
 bin/stop-all.sh            |  32 --------
 bin/stop-master.sh         |  27 -------
 bin/stop-slaves.sh         |  37 ---------
 docs/spark-standalone.md   |  12 +--
 sbin/compute-classpath.cmd |  69 +++++++++++++++++
 sbin/compute-classpath.sh  |  61 +++++++++++++++
 sbin/slaves.sh             |  74 ++++++++++++++++++
 sbin/spark-config.sh       |  36 +++++++++
 sbin/spark-daemon.sh       | 164 ++++++++++++++++++++++++++++++++++++++++
 sbin/spark-daemons.sh      |  35 +++++++++
 sbin/start-all.sh          |  34 +++++++++
 sbin/start-master.sh       |  52 +++++++++++++
 sbin/start-slave.sh        |  35 +++++++++
 sbin/start-slaves.sh       |  48 ++++++++++++
 sbin/stop-all.sh           |  32 ++++++++
 sbin/stop-master.sh        |  27 +++++++
 sbin/stop-slaves.sh        |  37 +++++++++
 27 files changed, 710 insertions(+), 710 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/compute-classpath.cmd
----------------------------------------------------------------------
diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd
deleted file mode 100644
index cf38188..0000000
--- a/bin/compute-classpath.cmd
+++ /dev/null
@@ -1,69 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem    http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
-rem script and the ExecutorRunner in standalone cluster mode.
-
-set SCALA_VERSION=2.9.3
-
-rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0..\
-
-rem Load environment variables from conf\spark-env.cmd, if it exists
-if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
-
-rem Build up classpath
-set CLASSPATH=%SPARK_CLASSPATH%;%FWDIR%conf
-if exist "%FWDIR%RELEASE" (
-  for %%d in ("%FWDIR%jars\spark-assembly*.jar") do (
-    set ASSEMBLY_JAR=%%d
-  )
-) else (
-  for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
-    set ASSEMBLY_JAR=%%d
-  )
-)
-set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR%
-
-if "x%SPARK_TESTING%"=="x1" (
-  rem Add test clases to path
-  set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes
-  set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes
-  set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes
-  set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes
-  set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes
-)
-
-rem Add hadoop conf dir - else FileSystem.*, etc fail
-rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
-rem the configurtion files.
-if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir
-  set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR%
-:no_hadoop_conf_dir
-
-if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir
-  set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%
-:no_yarn_conf_dir
-
-rem A bit of a hack to allow calling this script within run2.cmd without seeing output
-if "%DONT_PRINT_CLASSPATH%"=="1" goto exit
-
-echo %CLASSPATH%
-
-:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/compute-classpath.sh
----------------------------------------------------------------------
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
deleted file mode 100755
index c7819d4..0000000
--- a/bin/compute-classpath.sh
+++ /dev/null
@@ -1,61 +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.
-#
-
-# This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
-# script and the ExecutorRunner in standalone cluster mode.
-
-SCALA_VERSION=2.9.3
-
-# Figure out where Spark is installed
-FWDIR="$(cd `dirname $0`/..; pwd)"
-
-# 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
-
-# Build up classpath
-CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
-if [ -f "$FWDIR/RELEASE" ]; then
-  ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
-else
-  ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
-fi
-CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
-
-# Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1
-if [[ $SPARK_TESTING == 1 ]]; then
-  CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes"
-  CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes"
-fi
-
-# Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail !
-# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
-# the configurtion files.
-if [ "x" != "x$HADOOP_CONF_DIR" ]; then
-  CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR"
-fi
-if [ "x" != "x$YARN_CONF_DIR" ]; then
-  CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
-fi
-
-echo "$CLASSPATH"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/slaves.sh
----------------------------------------------------------------------
diff --git a/bin/slaves.sh b/bin/slaves.sh
deleted file mode 100755
index 752565b..0000000
--- a/bin/slaves.sh
+++ /dev/null
@@ -1,74 +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.
-#
-
-# Run a shell command on all slave hosts.
-#
-# Environment Variables
-#
-#   SPARK_SLAVES    File naming remote hosts.
-#     Default is ${SPARK_CONF_DIR}/slaves.
-#   SPARK_CONF_DIR  Alternate conf dir. Default is ${SPARK_HOME}/conf.
-#   SPARK_SLAVE_SLEEP Seconds to sleep between spawning remote commands.
-#   SPARK_SSH_OPTS Options passed to ssh when running remote commands.
-##
-
-usage="Usage: slaves.sh [--config confdir] command..."
-
-# if no args specified, show usage
-if [ $# -le 0 ]; then
-  echo $usage
-  exit 1
-fi
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin/spark-config.sh"
-
-# If the slaves file is specified in the command line,
-# then it takes precedence over the definition in
-# spark-env.sh. Save it here.
-HOSTLIST=$SPARK_SLAVES
-
-if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
-  . "${SPARK_CONF_DIR}/spark-env.sh"
-fi
-
-if [ "$HOSTLIST" = "" ]; then
-  if [ "$SPARK_SLAVES" = "" ]; then
-    export HOSTLIST="${SPARK_CONF_DIR}/slaves"
-  else
-    export HOSTLIST="${SPARK_SLAVES}"
-  fi
-fi
-
-# By default disable strict host key checking
-if [ "$SPARK_SSH_OPTS" = "" ]; then
-  SPARK_SSH_OPTS="-o StrictHostKeyChecking=no"
-fi
-
-for slave in `cat "$HOSTLIST"|sed  "s/#.*$//;/^$/d"`; do
- ssh $SPARK_SSH_OPTS $slave $"${@// /\\ }" \
-   2>&1 | sed "s/^/$slave: /" &
- if [ "$SPARK_SLAVE_SLEEP" != "" ]; then
-   sleep $SPARK_SLAVE_SLEEP
- fi
-done
-
-wait

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/spark-config.sh
----------------------------------------------------------------------
diff --git a/bin/spark-config.sh b/bin/spark-config.sh
deleted file mode 100755
index cd2c7b7..0000000
--- a/bin/spark-config.sh
+++ /dev/null
@@ -1,36 +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.
-#
-
-# included in all the spark scripts with source command
-# should not be executable directly
-# also should not be passed any arguments, since we need original $*
-
-# resolve links - $0 may be a softlink
-this="${BASH_SOURCE-$0}"
-common_bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P)
-script="$(basename -- "$this")"
-this="$common_bin/$script"
-
-# convert relative path to absolute path
-config_bin=`dirname "$this"`
-script=`basename "$this"`
-config_bin=`cd "$config_bin"; pwd`
-this="$config_bin/$script"
-
-export SPARK_PREFIX=`dirname "$this"`/..
-export SPARK_HOME=${SPARK_PREFIX}
-export SPARK_CONF_DIR="$SPARK_HOME/conf"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/spark-daemon.sh
----------------------------------------------------------------------
diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh
deleted file mode 100755
index 6baee0c..0000000
--- a/bin/spark-daemon.sh
+++ /dev/null
@@ -1,164 +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.
-#
-
-# Runs a Spark command as a daemon.
-#
-# Environment Variables
-#
-#   SPARK_CONF_DIR  Alternate conf dir. Default is ${SPARK_PREFIX}/conf.
-#   SPARK_LOG_DIR   Where log files are stored.  PWD by default.
-#   SPARK_MASTER    host:path where spark code should be rsync'd from
-#   SPARK_PID_DIR   The pid files are stored. /tmp by default.
-#   SPARK_IDENT_STRING   A string representing this instance of spark. $USER by default
-#   SPARK_NICENESS The scheduling priority for daemons. Defaults to 0.
-##
-
-usage="Usage: spark-daemon.sh [--config <conf-dir>] [--hosts hostlistfile] (start|stop) <spark-command> <spark-instance-number> <args...>"
-
-# if no args specified, show usage
-if [ $# -le 1 ]; then
-  echo $usage
-  exit 1
-fi
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin/spark-config.sh"
-
-# get arguments
-startStop=$1
-shift
-command=$1
-shift
-instance=$1
-shift
-
-spark_rotate_log ()
-{
-    log=$1;
-    num=5;
-    if [ -n "$2" ]; then
-	num=$2
-    fi
-    if [ -f "$log" ]; then # rotate logs
-	while [ $num -gt 1 ]; do
-	    prev=`expr $num - 1`
-	    [ -f "$log.$prev" ] && mv "$log.$prev" "$log.$num"
-	    num=$prev
-	done
-	mv "$log" "$log.$num";
-    fi
-}
-
-if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
-  . "${SPARK_CONF_DIR}/spark-env.sh"
-fi
-
-if [ "$SPARK_IDENT_STRING" = "" ]; then
-  export SPARK_IDENT_STRING="$USER"
-fi
-
-
-export SPARK_PRINT_LAUNCH_COMMAND="1"
-
-# get log directory
-if [ "$SPARK_LOG_DIR" = "" ]; then
-  export SPARK_LOG_DIR="$SPARK_HOME/logs"
-fi
-mkdir -p "$SPARK_LOG_DIR"
-touch $SPARK_LOG_DIR/.spark_test > /dev/null 2>&1
-TEST_LOG_DIR=$?
-if [ "${TEST_LOG_DIR}" = "0" ]; then
-  rm -f $SPARK_LOG_DIR/.spark_test
-else
-  chown $SPARK_IDENT_STRING $SPARK_LOG_DIR
-fi
-
-if [ "$SPARK_PID_DIR" = "" ]; then
-  SPARK_PID_DIR=/tmp
-fi
-
-# some variables
-export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.log
-export SPARK_ROOT_LOGGER="INFO,DRFA"
-log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out
-pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid
-
-# Set default scheduling priority
-if [ "$SPARK_NICENESS" = "" ]; then
-    export SPARK_NICENESS=0
-fi
-
-
-case $startStop in
-
-  (start)
-
-    mkdir -p "$SPARK_PID_DIR"
-
-    if [ -f $pid ]; then
-      if kill -0 `cat $pid` > /dev/null 2>&1; then
-        echo $command running as process `cat $pid`.  Stop it first.
-        exit 1
-      fi
-    fi
-
-    if [ "$SPARK_MASTER" != "" ]; then
-      echo rsync from $SPARK_MASTER
-      rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME"
-    fi
-
-    spark_rotate_log "$log"
-    echo starting $command, logging to $log
-    cd "$SPARK_PREFIX"
-    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/sbin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
-    newpid=$!
-    echo $newpid > $pid
-    sleep 2
-    # Check if the process has died; in that case we'll tail the log so the user can see
-    if ! kill -0 $newpid >/dev/null 2>&1; then
-      echo "failed to launch $command:"
-      tail -2 "$log" | sed 's/^/  /'
-      echo "full log in $log"
-    fi
-    ;;
-
-  (stop)
-
-    if [ -f $pid ]; then
-      if kill -0 `cat $pid` > /dev/null 2>&1; then
-        echo stopping $command
-        kill `cat $pid`
-      else
-        echo no $command to stop
-      fi
-    else
-      echo no $command to stop
-    fi
-    ;;
-
-  (*)
-    echo $usage
-    exit 1
-    ;;
-
-esac
-
-

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/spark-daemons.sh
----------------------------------------------------------------------
diff --git a/bin/spark-daemons.sh b/bin/spark-daemons.sh
deleted file mode 100755
index 354eb90..0000000
--- a/bin/spark-daemons.sh
+++ /dev/null
@@ -1,35 +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.
-#
-
-# Run a Spark command on all slave hosts.
-
-usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..."
-
-# if no args specified, show usage
-if [ $# -le 1 ]; then
-  echo $usage
-  exit 1
-fi
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin/spark-config.sh"
-
-exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/spark-daemon.sh" "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/start-all.sh
----------------------------------------------------------------------
diff --git a/bin/start-all.sh b/bin/start-all.sh
deleted file mode 100755
index 0182f1a..0000000
--- a/bin/start-all.sh
+++ /dev/null
@@ -1,34 +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.
-#
-
-# Start all spark daemons.
-# Starts the master on this node.
-# Starts a worker on each node specified in conf/slaves
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-# Load the Spark configuration
-. "$bin/spark-config.sh"
-
-# Start Master
-"$bin"/start-master.sh
-
-# Start Workers
-"$bin"/start-slaves.sh

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/start-master.sh
----------------------------------------------------------------------
diff --git a/bin/start-master.sh b/bin/start-master.sh
deleted file mode 100755
index 648c7ae..0000000
--- a/bin/start-master.sh
+++ /dev/null
@@ -1,52 +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.
-#
-
-# Starts the master on the machine this script is executed on.
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin/spark-config.sh"
-
-if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
-  . "${SPARK_CONF_DIR}/spark-env.sh"
-fi
-
-if [ "$SPARK_MASTER_PORT" = "" ]; then
-  SPARK_MASTER_PORT=7077
-fi
-
-if [ "$SPARK_MASTER_IP" = "" ]; then
-  SPARK_MASTER_IP=`hostname`
-fi
-
-if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then
-  SPARK_MASTER_WEBUI_PORT=8080
-fi
-
-# Set SPARK_PUBLIC_DNS so the master report the correct webUI address to the slaves
-if [ "$SPARK_PUBLIC_DNS" = "" ]; then
-    # If we appear to be running on EC2, use the public address by default:
-    # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname
-    if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then
-        export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname`
-    fi
-fi
-
-"$bin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/start-slave.sh
----------------------------------------------------------------------
diff --git a/bin/start-slave.sh b/bin/start-slave.sh
deleted file mode 100755
index 4eefa20..0000000
--- a/bin/start-slave.sh
+++ /dev/null
@@ -1,35 +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.
-#
-
-# Usage: start-slave.sh <worker#> <master-spark-URL>
-#   where <master-spark-URL> is like "spark://localhost:7077"
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-# Set SPARK_PUBLIC_DNS so slaves can be linked in master web UI
-if [ "$SPARK_PUBLIC_DNS" = "" ]; then
-    # If we appear to be running on EC2, use the public address by default:
-    # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname
-    if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then
-        export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname`
-    fi
-fi
-
-"$bin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/start-slaves.sh
----------------------------------------------------------------------
diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh
deleted file mode 100755
index 00dc488..0000000
--- a/bin/start-slaves.sh
+++ /dev/null
@@ -1,48 +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.
-#
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin/spark-config.sh"
-
-if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
-  . "${SPARK_CONF_DIR}/spark-env.sh"
-fi
-
-# Find the port number for the master
-if [ "$SPARK_MASTER_PORT" = "" ]; then
-  SPARK_MASTER_PORT=7077
-fi
-
-if [ "$SPARK_MASTER_IP" = "" ]; then
-  SPARK_MASTER_IP=`hostname`
-fi
-
-# Launch the slaves
-if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
-  exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
-else
-  if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then
-    SPARK_WORKER_WEBUI_PORT=8081
-  fi
-  for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
-    "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" $(( $i + 1 ))  spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
-  done
-fi

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/stop-all.sh
----------------------------------------------------------------------
diff --git a/bin/stop-all.sh b/bin/stop-all.sh
deleted file mode 100755
index b6c83a7..0000000
--- a/bin/stop-all.sh
+++ /dev/null
@@ -1,32 +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.
-#
-
-# Start all spark daemons.
-# Run this on the master nde
-
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-# Load the Spark configuration
-. "$bin/spark-config.sh"
-
-# Stop the slaves, then the master
-"$bin"/stop-slaves.sh
-"$bin"/stop-master.sh

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/stop-master.sh
----------------------------------------------------------------------
diff --git a/bin/stop-master.sh b/bin/stop-master.sh
deleted file mode 100755
index 310e33b..0000000
--- a/bin/stop-master.sh
+++ /dev/null
@@ -1,27 +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.
-#
-
-# Starts the master on the machine this script is executed on.
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin/spark-config.sh"
-
-"$bin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/bin/stop-slaves.sh
----------------------------------------------------------------------
diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh
deleted file mode 100755
index 03e416a..0000000
--- a/bin/stop-slaves.sh
+++ /dev/null
@@ -1,37 +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.
-#
-
-# Starts the master on the machine this script is executed on.
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin/spark-config.sh"
-
-if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
-  . "${SPARK_CONF_DIR}/spark-env.sh"
-fi
-
-if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
-  "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
-else
-  for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
-    "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
-  done
-fi

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 9d4ad1e..b3f9160 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -67,12 +67,12 @@ To launch a Spark standalone cluster with the launch scripts, you need to create
 
 Once you've set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
 
-- `bin/start-master.sh` - Starts a master instance on the machine the script is executed on.
-- `bin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
-- `bin/start-all.sh` - Starts both a master and a number of slaves as described above.
-- `bin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
-- `bin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
-- `bin/stop-all.sh` - Stops both the master and the slaves as described above.
+- `sbin/start-master.sh` - Starts a master instance on the machine the script is executed on.
+- `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
+- `sbin/start-all.sh` - Starts both a master and a number of slaves as described above.
+- `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
+- `sbin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
+- `sbin/stop-all.sh` - Stops both the master and the slaves as described above.
 
 Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine.
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/compute-classpath.cmd
----------------------------------------------------------------------
diff --git a/sbin/compute-classpath.cmd b/sbin/compute-classpath.cmd
new file mode 100644
index 0000000..cf38188
--- /dev/null
+++ b/sbin/compute-classpath.cmd
@@ -0,0 +1,69 @@
+@echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
+rem script and the ExecutorRunner in standalone cluster mode.
+
+set SCALA_VERSION=2.9.3
+
+rem Figure out where the Spark framework is installed
+set FWDIR=%~dp0..\
+
+rem Load environment variables from conf\spark-env.cmd, if it exists
+if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
+
+rem Build up classpath
+set CLASSPATH=%SPARK_CLASSPATH%;%FWDIR%conf
+if exist "%FWDIR%RELEASE" (
+  for %%d in ("%FWDIR%jars\spark-assembly*.jar") do (
+    set ASSEMBLY_JAR=%%d
+  )
+) else (
+  for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
+    set ASSEMBLY_JAR=%%d
+  )
+)
+set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR%
+
+if "x%SPARK_TESTING%"=="x1" (
+  rem Add test clases to path
+  set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes
+  set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes
+  set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes
+  set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes
+  set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes
+)
+
+rem Add hadoop conf dir - else FileSystem.*, etc fail
+rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
+rem the configurtion files.
+if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir
+  set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR%
+:no_hadoop_conf_dir
+
+if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir
+  set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%
+:no_yarn_conf_dir
+
+rem A bit of a hack to allow calling this script within run2.cmd without seeing output
+if "%DONT_PRINT_CLASSPATH%"=="1" goto exit
+
+echo %CLASSPATH%
+
+:exit

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/compute-classpath.sh
----------------------------------------------------------------------
diff --git a/sbin/compute-classpath.sh b/sbin/compute-classpath.sh
new file mode 100755
index 0000000..c7819d4
--- /dev/null
+++ b/sbin/compute-classpath.sh
@@ -0,0 +1,61 @@
+#!/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.
+#
+
+# This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
+# script and the ExecutorRunner in standalone cluster mode.
+
+SCALA_VERSION=2.9.3
+
+# Figure out where Spark is installed
+FWDIR="$(cd `dirname $0`/..; pwd)"
+
+# 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
+
+# Build up classpath
+CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
+if [ -f "$FWDIR/RELEASE" ]; then
+  ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
+else
+  ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar`
+fi
+CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR"
+
+# Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1
+if [[ $SPARK_TESTING == 1 ]]; then
+  CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes"
+  CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes"
+fi
+
+# Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail !
+# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
+# the configurtion files.
+if [ "x" != "x$HADOOP_CONF_DIR" ]; then
+  CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR"
+fi
+if [ "x" != "x$YARN_CONF_DIR" ]; then
+  CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
+fi
+
+echo "$CLASSPATH"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/slaves.sh
----------------------------------------------------------------------
diff --git a/sbin/slaves.sh b/sbin/slaves.sh
new file mode 100755
index 0000000..68408bc
--- /dev/null
+++ b/sbin/slaves.sh
@@ -0,0 +1,74 @@
+#!/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.
+#
+
+# Run a shell command on all slave hosts.
+#
+# Environment Variables
+#
+#   SPARK_SLAVES    File naming remote hosts.
+#     Default is ${SPARK_CONF_DIR}/slaves.
+#   SPARK_CONF_DIR  Alternate conf dir. Default is ${SPARK_HOME}/conf.
+#   SPARK_SLAVE_SLEEP Seconds to sleep between spawning remote commands.
+#   SPARK_SSH_OPTS Options passed to ssh when running remote commands.
+##
+
+usage="Usage: slaves.sh [--config confdir] command..."
+
+# if no args specified, show usage
+if [ $# -le 0 ]; then
+  echo $usage
+  exit 1
+fi
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+. "$sbin/spark-config.sh"
+
+# If the slaves file is specified in the command line,
+# then it takes precedence over the definition in
+# spark-env.sh. Save it here.
+HOSTLIST=$SPARK_SLAVES
+
+if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
+  . "${SPARK_CONF_DIR}/spark-env.sh"
+fi
+
+if [ "$HOSTLIST" = "" ]; then
+  if [ "$SPARK_SLAVES" = "" ]; then
+    export HOSTLIST="${SPARK_CONF_DIR}/slaves"
+  else
+    export HOSTLIST="${SPARK_SLAVES}"
+  fi
+fi
+
+# By default disable strict host key checking
+if [ "$SPARK_SSH_OPTS" = "" ]; then
+  SPARK_SSH_OPTS="-o StrictHostKeyChecking=no"
+fi
+
+for slave in `cat "$HOSTLIST"|sed  "s/#.*$//;/^$/d"`; do
+ ssh $SPARK_SSH_OPTS $slave $"${@// /\\ }" \
+   2>&1 | sed "s/^/$slave: /" &
+ if [ "$SPARK_SLAVE_SLEEP" != "" ]; then
+   sleep $SPARK_SLAVE_SLEEP
+ fi
+done
+
+wait

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/spark-config.sh
----------------------------------------------------------------------
diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh
new file mode 100755
index 0000000..cd2c7b7
--- /dev/null
+++ b/sbin/spark-config.sh
@@ -0,0 +1,36 @@
+#
+# 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.
+#
+
+# included in all the spark scripts with source command
+# should not be executable directly
+# also should not be passed any arguments, since we need original $*
+
+# resolve links - $0 may be a softlink
+this="${BASH_SOURCE-$0}"
+common_bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P)
+script="$(basename -- "$this")"
+this="$common_bin/$script"
+
+# convert relative path to absolute path
+config_bin=`dirname "$this"`
+script=`basename "$this"`
+config_bin=`cd "$config_bin"; pwd`
+this="$config_bin/$script"
+
+export SPARK_PREFIX=`dirname "$this"`/..
+export SPARK_HOME=${SPARK_PREFIX}
+export SPARK_CONF_DIR="$SPARK_HOME/conf"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/spark-daemon.sh
----------------------------------------------------------------------
diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh
new file mode 100755
index 0000000..ae82349
--- /dev/null
+++ b/sbin/spark-daemon.sh
@@ -0,0 +1,164 @@
+#!/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.
+#
+
+# Runs a Spark command as a daemon.
+#
+# Environment Variables
+#
+#   SPARK_CONF_DIR  Alternate conf dir. Default is ${SPARK_PREFIX}/conf.
+#   SPARK_LOG_DIR   Where log files are stored.  PWD by default.
+#   SPARK_MASTER    host:path where spark code should be rsync'd from
+#   SPARK_PID_DIR   The pid files are stored. /tmp by default.
+#   SPARK_IDENT_STRING   A string representing this instance of spark. $USER by default
+#   SPARK_NICENESS The scheduling priority for daemons. Defaults to 0.
+##
+
+usage="Usage: spark-daemon.sh [--config <conf-dir>] [--hosts hostlistfile] (start|stop) <spark-command> <spark-instance-number> <args...>"
+
+# if no args specified, show usage
+if [ $# -le 1 ]; then
+  echo $usage
+  exit 1
+fi
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+. "$sbin/spark-config.sh"
+
+# get arguments
+startStop=$1
+shift
+command=$1
+shift
+instance=$1
+shift
+
+spark_rotate_log ()
+{
+    log=$1;
+    num=5;
+    if [ -n "$2" ]; then
+	num=$2
+    fi
+    if [ -f "$log" ]; then # rotate logs
+	while [ $num -gt 1 ]; do
+	    prev=`expr $num - 1`
+	    [ -f "$log.$prev" ] && mv "$log.$prev" "$log.$num"
+	    num=$prev
+	done
+	mv "$log" "$log.$num";
+    fi
+}
+
+if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
+  . "${SPARK_CONF_DIR}/spark-env.sh"
+fi
+
+if [ "$SPARK_IDENT_STRING" = "" ]; then
+  export SPARK_IDENT_STRING="$USER"
+fi
+
+
+export SPARK_PRINT_LAUNCH_COMMAND="1"
+
+# get log directory
+if [ "$SPARK_LOG_DIR" = "" ]; then
+  export SPARK_LOG_DIR="$SPARK_HOME/logs"
+fi
+mkdir -p "$SPARK_LOG_DIR"
+touch $SPARK_LOG_DIR/.spark_test > /dev/null 2>&1
+TEST_LOG_DIR=$?
+if [ "${TEST_LOG_DIR}" = "0" ]; then
+  rm -f $SPARK_LOG_DIR/.spark_test
+else
+  chown $SPARK_IDENT_STRING $SPARK_LOG_DIR
+fi
+
+if [ "$SPARK_PID_DIR" = "" ]; then
+  SPARK_PID_DIR=/tmp
+fi
+
+# some variables
+export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.log
+export SPARK_ROOT_LOGGER="INFO,DRFA"
+log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out
+pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid
+
+# Set default scheduling priority
+if [ "$SPARK_NICENESS" = "" ]; then
+    export SPARK_NICENESS=0
+fi
+
+
+case $startStop in
+
+  (start)
+
+    mkdir -p "$SPARK_PID_DIR"
+
+    if [ -f $pid ]; then
+      if kill -0 `cat $pid` > /dev/null 2>&1; then
+        echo $command running as process `cat $pid`.  Stop it first.
+        exit 1
+      fi
+    fi
+
+    if [ "$SPARK_MASTER" != "" ]; then
+      echo rsync from $SPARK_MASTER
+      rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME"
+    fi
+
+    spark_rotate_log "$log"
+    echo starting $command, logging to $log
+    cd "$SPARK_PREFIX"
+    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/sbin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
+    newpid=$!
+    echo $newpid > $pid
+    sleep 2
+    # Check if the process has died; in that case we'll tail the log so the user can see
+    if ! kill -0 $newpid >/dev/null 2>&1; then
+      echo "failed to launch $command:"
+      tail -2 "$log" | sed 's/^/  /'
+      echo "full log in $log"
+    fi
+    ;;
+
+  (stop)
+
+    if [ -f $pid ]; then
+      if kill -0 `cat $pid` > /dev/null 2>&1; then
+        echo stopping $command
+        kill `cat $pid`
+      else
+        echo no $command to stop
+      fi
+    else
+      echo no $command to stop
+    fi
+    ;;
+
+  (*)
+    echo $usage
+    exit 1
+    ;;
+
+esac
+
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/spark-daemons.sh
----------------------------------------------------------------------
diff --git a/sbin/spark-daemons.sh b/sbin/spark-daemons.sh
new file mode 100755
index 0000000..d91254b
--- /dev/null
+++ b/sbin/spark-daemons.sh
@@ -0,0 +1,35 @@
+#!/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.
+#
+
+# Run a Spark command on all slave hosts.
+
+usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..."
+
+# if no args specified, show usage
+if [ $# -le 1 ]; then
+  echo $usage
+  exit 1
+fi
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+. "$sbin/spark-config.sh"
+
+exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/spark-daemon.sh" "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/start-all.sh
----------------------------------------------------------------------
diff --git a/sbin/start-all.sh b/sbin/start-all.sh
new file mode 100755
index 0000000..2daf49d
--- /dev/null
+++ b/sbin/start-all.sh
@@ -0,0 +1,34 @@
+#!/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.
+#
+
+# Start all spark daemons.
+# Starts the master on this node.
+# Starts a worker on each node specified in conf/slaves
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+# Load the Spark configuration
+. "$sbin/spark-config.sh"
+
+# Start Master
+"$sbin"/start-master.sh
+
+# Start Workers
+"$sbin"/start-slaves.sh

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/start-master.sh
----------------------------------------------------------------------
diff --git a/sbin/start-master.sh b/sbin/start-master.sh
new file mode 100755
index 0000000..3dcf7cc
--- /dev/null
+++ b/sbin/start-master.sh
@@ -0,0 +1,52 @@
+#!/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.
+#
+
+# Starts the master on the machine this script is executed on.
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+. "$sbin/spark-config.sh"
+
+if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
+  . "${SPARK_CONF_DIR}/spark-env.sh"
+fi
+
+if [ "$SPARK_MASTER_PORT" = "" ]; then
+  SPARK_MASTER_PORT=7077
+fi
+
+if [ "$SPARK_MASTER_IP" = "" ]; then
+  SPARK_MASTER_IP=`hostname`
+fi
+
+if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then
+  SPARK_MASTER_WEBUI_PORT=8080
+fi
+
+# Set SPARK_PUBLIC_DNS so the master report the correct webUI address to the slaves
+if [ "$SPARK_PUBLIC_DNS" = "" ]; then
+    # If we appear to be running on EC2, use the public address by default:
+    # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname
+    if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then
+        export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname`
+    fi
+fi
+
+"$sbin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/start-slave.sh
----------------------------------------------------------------------
diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh
new file mode 100755
index 0000000..524be38
--- /dev/null
+++ b/sbin/start-slave.sh
@@ -0,0 +1,35 @@
+#!/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.
+#
+
+# Usage: start-slave.sh <worker#> <master-spark-URL>
+#   where <master-spark-URL> is like "spark://localhost:7077"
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+# Set SPARK_PUBLIC_DNS so slaves can be linked in master web UI
+if [ "$SPARK_PUBLIC_DNS" = "" ]; then
+    # If we appear to be running on EC2, use the public address by default:
+    # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname
+    if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then
+        export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname`
+    fi
+fi
+
+"$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/start-slaves.sh
----------------------------------------------------------------------
diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh
new file mode 100755
index 0000000..fd5cdeb
--- /dev/null
+++ b/sbin/start-slaves.sh
@@ -0,0 +1,48 @@
+#!/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.
+#
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+. "$sbin/spark-config.sh"
+
+if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
+  . "${SPARK_CONF_DIR}/spark-env.sh"
+fi
+
+# Find the port number for the master
+if [ "$SPARK_MASTER_PORT" = "" ]; then
+  SPARK_MASTER_PORT=7077
+fi
+
+if [ "$SPARK_MASTER_IP" = "" ]; then
+  SPARK_MASTER_IP=`hostname`
+fi
+
+# Launch the slaves
+if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
+  exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
+else
+  if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then
+    SPARK_WORKER_WEBUI_PORT=8081
+  fi
+  for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
+    "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 ))  spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
+  done
+fi

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/stop-all.sh
----------------------------------------------------------------------
diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh
new file mode 100755
index 0000000..60b358d
--- /dev/null
+++ b/sbin/stop-all.sh
@@ -0,0 +1,32 @@
+#!/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.
+#
+
+# Start all spark daemons.
+# Run this on the master nde
+
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+# Load the Spark configuration
+. "$sbin/spark-config.sh"
+
+# Stop the slaves, then the master
+"$sbin"/stop-slaves.sh
+"$sbin"/stop-master.sh

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/stop-master.sh
----------------------------------------------------------------------
diff --git a/sbin/stop-master.sh b/sbin/stop-master.sh
new file mode 100755
index 0000000..2adabd4
--- /dev/null
+++ b/sbin/stop-master.sh
@@ -0,0 +1,27 @@
+#!/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.
+#
+
+# Starts the master on the machine this script is executed on.
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+. "$sbin/spark-config.sh"
+
+"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fcfe4f92/sbin/stop-slaves.sh
----------------------------------------------------------------------
diff --git a/sbin/stop-slaves.sh b/sbin/stop-slaves.sh
new file mode 100755
index 0000000..63802e6
--- /dev/null
+++ b/sbin/stop-slaves.sh
@@ -0,0 +1,37 @@
+#!/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.
+#
+
+# Starts the master on the machine this script is executed on.
+
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
+
+. "$sbin/spark-config.sh"
+
+if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
+  . "${SPARK_CONF_DIR}/spark-env.sh"
+fi
+
+if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
+  "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
+else
+  for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
+    "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
+  done
+fi


[11/21] git commit: Merge branch 'reorgscripts' into scripts-reorg

Posted by pw...@apache.org.
Merge branch 'reorgscripts' into scripts-reorg


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/84849baf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/84849baf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/84849baf

Branch: refs/heads/master
Commit: 84849baf88d31cfaaeee158a947c4db1abe94ce6
Parents: 714fdab 3a5aa92
Author: shane-huang <sh...@intel.com>
Authored: Fri Sep 27 09:28:33 2013 +0800
Committer: shane-huang <sh...@intel.com>
Committed: Fri Sep 27 09:28:33 2013 +0800

----------------------------------------------------------------------
 assembly/src/main/assembly/assembly.xml         |   11 +-
 bin/compute-classpath.cmd                       |   69 --
 bin/compute-classpath.sh                        |   61 --
 bin/pyspark                                     |   66 ++
 bin/pyspark.cmd                                 |   23 +
 bin/pyspark2.cmd                                |   55 +
 bin/run-example                                 |   81 ++
 bin/run-example.cmd                             |   23 +
 bin/run-example2.cmd                            |   61 ++
 bin/slaves.sh                                   |   74 --
 bin/spark                                       |   92 ++
 bin/spark-config.sh                             |   36 -
 bin/spark-daemon.sh                             |  164 ---
 bin/spark-daemons.sh                            |   35 -
 bin/spark-shell                                 |   87 ++
 bin/spark-shell.cmd                             |   23 +
 bin/start-all.sh                                |   34 -
 bin/start-master.sh                             |   52 -
 bin/start-slave.sh                              |   35 -
 bin/start-slaves.sh                             |   48 -
 bin/stop-all.sh                                 |   32 -
 bin/stop-master.sh                              |   27 -
 bin/stop-slaves.sh                              |   37 -
 .../spark/deploy/worker/ExecutorRunner.scala    |    2 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |    4 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |    4 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |    2 +-
 .../scala/org/apache/spark/DriverSuite.scala    |    2 +-
 data/kmeans_data.txt                            |    6 +
 data/lr_data.txt                                | 1000 ++++++++++++++++++
 data/pagerank_data.txt                          |    6 +
 docs/running-on-yarn.md                         |    4 +-
 docs/spark-standalone.md                        |   14 +-
 kmeans_data.txt                                 |    6 -
 lr_data.txt                                     | 1000 ------------------
 make-distribution.sh                            |    5 +-
 pagerank_data.txt                               |    6 -
 pyspark                                         |   66 --
 pyspark.cmd                                     |   23 -
 pyspark2.cmd                                    |   55 -
 python/pyspark/java_gateway.py                  |    2 +-
 python/run-tests                                |    2 +-
 run-example                                     |   81 --
 run-example.cmd                                 |   23 -
 run-example2.cmd                                |   61 --
 sbin/compute-classpath.cmd                      |   69 ++
 sbin/compute-classpath.sh                       |   61 ++
 sbin/slaves.sh                                  |   74 ++
 sbin/spark-class                                |  117 ++
 sbin/spark-class.cmd                            |   23 +
 sbin/spark-class2.cmd                           |   78 ++
 sbin/spark-config.sh                            |   36 +
 sbin/spark-daemon.sh                            |  164 +++
 sbin/spark-daemons.sh                           |   35 +
 sbin/spark-executor                             |   23 +
 sbin/start-all.sh                               |   34 +
 sbin/start-master.sh                            |   52 +
 sbin/start-slave.sh                             |   35 +
 sbin/start-slaves.sh                            |   48 +
 sbin/stop-all.sh                                |   32 +
 sbin/stop-master.sh                             |   27 +
 sbin/stop-slaves.sh                             |   37 +
 spark-class                                     |  117 --
 spark-class.cmd                                 |   23 -
 spark-class2.cmd                                |   78 --
 spark-executor                                  |   22 -
 spark-shell                                     |   87 --
 spark-shell.cmd                                 |   22 -
 68 files changed, 2491 insertions(+), 2403 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/84849baf/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 8f2eef9,0000000..15b3397
mode 100644,000000..100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@@ -1,286 -1,0 +1,286 @@@
 +/*
 + * 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.
 + */
 +
 +package org.apache.spark.scheduler.cluster.mesos
 +
 +import java.io.File
 +import java.util.{ArrayList => JArrayList, List => JList}
 +import java.util.Collections
 +
 +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 +import scala.collection.JavaConversions._
 +
 +import com.google.protobuf.ByteString
 +import org.apache.mesos.{Scheduler => MScheduler}
 +import org.apache.mesos._
 +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _}
 +
 +import org.apache.spark.{SparkException, Logging, SparkContext, TaskState}
 +import org.apache.spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend}
 +
 +/**
 + * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds
 + * onto each Mesos node for the duration of the Spark job instead of relinquishing cores whenever
 + * a task is done. It launches Spark tasks within the coarse-grained Mesos tasks using the
 + * StandaloneBackend mechanism. This class is useful for lower and more predictable latency.
 + *
 + * Unfortunately this has a bit of duplication from MesosSchedulerBackend, but it seems hard to
 + * remove this.
 + */
 +private[spark] class CoarseMesosSchedulerBackend(
 +    scheduler: ClusterScheduler,
 +    sc: SparkContext,
 +    master: String,
 +    appName: String)
 +  extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
 +  with MScheduler
 +  with Logging {
 +
 +  val MAX_SLAVE_FAILURES = 2     // Blacklist a slave after this many failures
 +
 +  // Lock used to wait for scheduler to be registered
 +  var isRegistered = false
 +  val registeredLock = new Object()
 +
 +  // Driver for talking to Mesos
 +  var driver: SchedulerDriver = null
 +
 +  // Maximum number of cores to acquire (TODO: we'll need more flexible controls here)
 +  val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt
 +
 +  // Cores we have acquired with each Mesos task ID
 +  val coresByTaskId = new HashMap[Int, Int]
 +  var totalCoresAcquired = 0
 +
 +  val slaveIdsWithExecutors = new HashSet[String]
 +
 +  val taskIdToSlaveId = new HashMap[Int, String]
 +  val failuresBySlaveId = new HashMap[String, Int] // How many times tasks on each slave failed
 +
 +  val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException(
 +    "Spark home is not set; set it through the spark.home system " +
 +    "property, the SPARK_HOME environment variable or the SparkContext constructor"))
 +
 +  val extraCoresPerSlave = System.getProperty("spark.mesos.extra.cores", "0").toInt
 +
 +  var nextMesosTaskId = 0
 +
 +  def newMesosTaskId(): Int = {
 +    val id = nextMesosTaskId
 +    nextMesosTaskId += 1
 +    id
 +  }
 +
 +  override def start() {
 +    super.start()
 +
 +    synchronized {
 +      new Thread("CoarseMesosSchedulerBackend driver") {
 +        setDaemon(true)
 +        override def run() {
 +          val scheduler = CoarseMesosSchedulerBackend.this
 +          val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build()
 +          driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
 +          try { {
 +            val ret = driver.run()
 +            logInfo("driver.run() returned with code " + ret)
 +          }
 +          } catch {
 +            case e: Exception => logError("driver.run() failed", e)
 +          }
 +        }
 +      }.start()
 +
 +      waitForRegister()
 +    }
 +  }
 +
 +  def createCommand(offer: Offer, numCores: Int): CommandInfo = {
 +    val environment = Environment.newBuilder()
 +    sc.executorEnvs.foreach { case (key, value) =>
 +      environment.addVariables(Environment.Variable.newBuilder()
 +        .setName(key)
 +        .setValue(value)
 +        .build())
 +    }
 +    val command = CommandInfo.newBuilder()
 +      .setEnvironment(environment)
 +    val driverUrl = "akka://spark@%s:%s/user/%s".format(
 +      System.getProperty("spark.driver.host"),
 +      System.getProperty("spark.driver.port"),
 +      StandaloneSchedulerBackend.ACTOR_NAME)
 +    val uri = System.getProperty("spark.executor.uri")
 +    if (uri == null) {
-       val runScript = new File(sparkHome, "spark-class").getCanonicalPath
++      val runScript = new File(sparkHome, "./sbin/spark-class").getCanonicalPath
 +      command.setValue(
 +        "\"%s\" org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
 +          runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
 +    } else {
 +      // Grab everything to the first '.'. We'll use that and '*' to
 +      // glob the directory "correctly".
 +      val basename = uri.split('/').last.split('.').head
 +      command.setValue(
-         "cd %s*; ./spark-class org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
++        "cd %s*; ./sbin/spark-class org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
 +          basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
 +      command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
 +    }
 +    return command.build()
 +  }
 +
 +  override def offerRescinded(d: SchedulerDriver, o: OfferID) {}
 +
 +  override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) {
 +    logInfo("Registered as framework ID " + frameworkId.getValue)
 +    registeredLock.synchronized {
 +      isRegistered = true
 +      registeredLock.notifyAll()
 +    }
 +  }
 +
 +  def waitForRegister() {
 +    registeredLock.synchronized {
 +      while (!isRegistered) {
 +        registeredLock.wait()
 +      }
 +    }
 +  }
 +
 +  override def disconnected(d: SchedulerDriver) {}
 +
 +  override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {}
 +
 +  /**
 +   * Method called by Mesos to offer resources on slaves. We respond by launching an executor,
 +   * unless we've already launched more than we wanted to.
 +   */
 +  override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) {
 +    synchronized {
 +      val filters = Filters.newBuilder().setRefuseSeconds(-1).build()
 +
 +      for (offer <- offers) {
 +        val slaveId = offer.getSlaveId.toString
 +        val mem = getResource(offer.getResourcesList, "mem")
 +        val cpus = getResource(offer.getResourcesList, "cpus").toInt
 +        if (totalCoresAcquired < maxCores && mem >= executorMemory && cpus >= 1 &&
 +            failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES &&
 +            !slaveIdsWithExecutors.contains(slaveId)) {
 +          // Launch an executor on the slave
 +          val cpusToUse = math.min(cpus, maxCores - totalCoresAcquired)
 +          val taskId = newMesosTaskId()
 +          taskIdToSlaveId(taskId) = slaveId
 +          slaveIdsWithExecutors += slaveId
 +          coresByTaskId(taskId) = cpusToUse
 +          val task = MesosTaskInfo.newBuilder()
 +            .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build())
 +            .setSlaveId(offer.getSlaveId)
 +            .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave))
 +            .setName("Task " + taskId)
 +            .addResources(createResource("cpus", cpusToUse))
 +            .addResources(createResource("mem", executorMemory))
 +            .build()
 +          d.launchTasks(offer.getId, Collections.singletonList(task), filters)
 +        } else {
 +          // Filter it out
 +          d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters)
 +        }
 +      }
 +    }
 +  }
 +
 +  /** Helper function to pull out a resource from a Mesos Resources protobuf */
 +  private def getResource(res: JList[Resource], name: String): Double = {
 +    for (r <- res if r.getName == name) {
 +      return r.getScalar.getValue
 +    }
 +    // If we reached here, no resource with the required name was present
 +    throw new IllegalArgumentException("No resource called " + name + " in " + res)
 +  }
 +
 +  /** Build a Mesos resource protobuf object */
 +  private def createResource(resourceName: String, quantity: Double): Protos.Resource = {
 +    Resource.newBuilder()
 +      .setName(resourceName)
 +      .setType(Value.Type.SCALAR)
 +      .setScalar(Value.Scalar.newBuilder().setValue(quantity).build())
 +      .build()
 +  }
 +
 +  /** Check whether a Mesos task state represents a finished task */
 +  private def isFinished(state: MesosTaskState) = {
 +    state == MesosTaskState.TASK_FINISHED ||
 +      state == MesosTaskState.TASK_FAILED ||
 +      state == MesosTaskState.TASK_KILLED ||
 +      state == MesosTaskState.TASK_LOST
 +  }
 +
 +  override def statusUpdate(d: SchedulerDriver, status: TaskStatus) {
 +    val taskId = status.getTaskId.getValue.toInt
 +    val state = status.getState
 +    logInfo("Mesos task " + taskId + " is now " + state)
 +    synchronized {
 +      if (isFinished(state)) {
 +        val slaveId = taskIdToSlaveId(taskId)
 +        slaveIdsWithExecutors -= slaveId
 +        taskIdToSlaveId -= taskId
 +        // Remove the cores we have remembered for this task, if it's in the hashmap
 +        for (cores <- coresByTaskId.get(taskId)) {
 +          totalCoresAcquired -= cores
 +          coresByTaskId -= taskId
 +        }
 +        // If it was a failure, mark the slave as failed for blacklisting purposes
 +        if (state == MesosTaskState.TASK_FAILED || state == MesosTaskState.TASK_LOST) {
 +          failuresBySlaveId(slaveId) = failuresBySlaveId.getOrElse(slaveId, 0) + 1
 +          if (failuresBySlaveId(slaveId) >= MAX_SLAVE_FAILURES) {
 +            logInfo("Blacklisting Mesos slave " + slaveId + " due to too many failures; " +
 +                "is Spark installed on it?")
 +          }
 +        }
 +        driver.reviveOffers() // In case we'd rejected everything before but have now lost a node
 +      }
 +    }
 +  }
 +
 +  override def error(d: SchedulerDriver, message: String) {
 +    logError("Mesos error: " + message)
 +    scheduler.error(message)
 +  }
 +
 +  override def stop() {
 +    super.stop()
 +    if (driver != null) {
 +      driver.stop()
 +    }
 +  }
 +
 +  override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {}
 +
 +  override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) {
 +    logInfo("Mesos slave lost: " + slaveId.getValue)
 +    synchronized {
 +      if (slaveIdsWithExecutors.contains(slaveId.getValue)) {
 +        // Note that the slave ID corresponds to the executor ID on that slave
 +        slaveIdsWithExecutors -= slaveId.getValue
 +        removeExecutor(slaveId.getValue, "Mesos slave lost")
 +      }
 +    }
 +  }
 +
 +  override def executorLost(d: SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int) {
 +    logInfo("Executor lost: %s, marking slave %s as lost".format(e.getValue, s.getValue))
 +    slaveLost(d, s)
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/84849baf/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 50cbc2c,0000000..7e9c05c
mode 100644,000000..100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@@ -1,345 -1,0 +1,345 @@@
 +/*
 + * 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.
 + */
 +
 +package org.apache.spark.scheduler.cluster.mesos
 +
 +import java.io.File
 +import java.util.{ArrayList => JArrayList, List => JList}
 +import java.util.Collections
 +
 +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 +import scala.collection.JavaConversions._
 +
 +import com.google.protobuf.ByteString
 +import org.apache.mesos.{Scheduler => MScheduler}
 +import org.apache.mesos._
 +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _}
 +
 +import org.apache.spark.{Logging, SparkException, SparkContext, TaskState}
 +import org.apache.spark.scheduler.TaskDescription
 +import org.apache.spark.scheduler.cluster.{ClusterScheduler, ExecutorExited, ExecutorLossReason}
 +import org.apache.spark.scheduler.cluster.{SchedulerBackend, SlaveLost, WorkerOffer}
 +import org.apache.spark.util.Utils
 +
 +/**
 + * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a
 + * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks
 + * from multiple apps can run on different cores) and in time (a core can switch ownership).
 + */
 +private[spark] class MesosSchedulerBackend(
 +    scheduler: ClusterScheduler,
 +    sc: SparkContext,
 +    master: String,
 +    appName: String)
 +  extends SchedulerBackend
 +  with MScheduler
 +  with Logging {
 +
 +  // Lock used to wait for scheduler to be registered
 +  var isRegistered = false
 +  val registeredLock = new Object()
 +
 +  // Driver for talking to Mesos
 +  var driver: SchedulerDriver = null
 +
 +  // Which slave IDs we have executors on
 +  val slaveIdsWithExecutors = new HashSet[String]
 +  val taskIdToSlaveId = new HashMap[Long, String]
 +
 +  // An ExecutorInfo for our tasks
 +  var execArgs: Array[Byte] = null
 +
 +  var classLoader: ClassLoader = null
 +
 +  override def start() {
 +    synchronized {
 +      classLoader = Thread.currentThread.getContextClassLoader
 +
 +      new Thread("MesosSchedulerBackend driver") {
 +        setDaemon(true)
 +        override def run() {
 +          val scheduler = MesosSchedulerBackend.this
 +          val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build()
 +          driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
 +          try {
 +            val ret = driver.run()
 +            logInfo("driver.run() returned with code " + ret)
 +          } catch {
 +            case e: Exception => logError("driver.run() failed", e)
 +          }
 +        }
 +      }.start()
 +
 +      waitForRegister()
 +    }
 +  }
 +
 +  def createExecutorInfo(execId: String): ExecutorInfo = {
 +    val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException(
 +      "Spark home is not set; set it through the spark.home system " +
 +      "property, the SPARK_HOME environment variable or the SparkContext constructor"))
 +    val environment = Environment.newBuilder()
 +    sc.executorEnvs.foreach { case (key, value) =>
 +      environment.addVariables(Environment.Variable.newBuilder()
 +        .setName(key)
 +        .setValue(value)
 +        .build())
 +    }
 +    val command = CommandInfo.newBuilder()
 +      .setEnvironment(environment)
 +    val uri = System.getProperty("spark.executor.uri")
 +    if (uri == null) {
-       command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath)
++      command.setValue(new File(sparkHome, "/sbin/spark-executor").getCanonicalPath)
 +    } else {
 +      // Grab everything to the first '.'. We'll use that and '*' to
 +      // glob the directory "correctly".
 +      val basename = uri.split('/').last.split('.').head
-       command.setValue("cd %s*; ./spark-executor".format(basename))
++      command.setValue("cd %s*; ./sbin/spark-executor".format(basename))
 +      command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
 +    }
 +    val memory = Resource.newBuilder()
 +      .setName("mem")
 +      .setType(Value.Type.SCALAR)
 +      .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build())
 +      .build()
 +    ExecutorInfo.newBuilder()
 +      .setExecutorId(ExecutorID.newBuilder().setValue(execId).build())
 +      .setCommand(command)
 +      .setData(ByteString.copyFrom(createExecArg()))
 +      .addResources(memory)
 +      .build()
 +  }
 +
 +  /**
 +   * Create and serialize the executor argument to pass to Mesos. Our executor arg is an array
 +   * containing all the spark.* system properties in the form of (String, String) pairs.
 +   */
 +  private def createExecArg(): Array[Byte] = {
 +    if (execArgs == null) {
 +      val props = new HashMap[String, String]
 +      val iterator = System.getProperties.entrySet.iterator
 +      while (iterator.hasNext) {
 +        val entry = iterator.next
 +        val (key, value) = (entry.getKey.toString, entry.getValue.toString)
 +        if (key.startsWith("spark.")) {
 +          props(key) = value
 +        }
 +      }
 +      // Serialize the map as an array of (String, String) pairs
 +      execArgs = Utils.serialize(props.toArray)
 +    }
 +    return execArgs
 +  }
 +
 +  private def setClassLoader(): ClassLoader = {
 +    val oldClassLoader = Thread.currentThread.getContextClassLoader
 +    Thread.currentThread.setContextClassLoader(classLoader)
 +    return oldClassLoader
 +  }
 +
 +  private def restoreClassLoader(oldClassLoader: ClassLoader) {
 +    Thread.currentThread.setContextClassLoader(oldClassLoader)
 +  }
 +
 +  override def offerRescinded(d: SchedulerDriver, o: OfferID) {}
 +
 +  override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) {
 +    val oldClassLoader = setClassLoader()
 +    try {
 +      logInfo("Registered as framework ID " + frameworkId.getValue)
 +      registeredLock.synchronized {
 +        isRegistered = true
 +        registeredLock.notifyAll()
 +      }
 +    } finally {
 +      restoreClassLoader(oldClassLoader)
 +    }
 +  }
 +
 +  def waitForRegister() {
 +    registeredLock.synchronized {
 +      while (!isRegistered) {
 +        registeredLock.wait()
 +      }
 +    }
 +  }
 +
 +  override def disconnected(d: SchedulerDriver) {}
 +
 +  override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {}
 +
 +  /**
 +   * Method called by Mesos to offer resources on slaves. We resond by asking our active task sets
 +   * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that
 +   * tasks are balanced across the cluster.
 +   */
 +  override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) {
 +    val oldClassLoader = setClassLoader()
 +    try {
 +      synchronized {
 +        // Build a big list of the offerable workers, and remember their indices so that we can
 +        // figure out which Offer to reply to for each worker
 +        val offerableIndices = new ArrayBuffer[Int]
 +        val offerableWorkers = new ArrayBuffer[WorkerOffer]
 +
 +        def enoughMemory(o: Offer) = {
 +          val mem = getResource(o.getResourcesList, "mem")
 +          val slaveId = o.getSlaveId.getValue
 +          mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId)
 +        }
 +
 +        for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) {
 +          offerableIndices += index
 +          offerableWorkers += new WorkerOffer(
 +            offer.getSlaveId.getValue,
 +            offer.getHostname,
 +            getResource(offer.getResourcesList, "cpus").toInt)
 +        }
 +
 +        // Call into the ClusterScheduler
 +        val taskLists = scheduler.resourceOffers(offerableWorkers)
 +
 +        // Build a list of Mesos tasks for each slave
 +        val mesosTasks = offers.map(o => Collections.emptyList[MesosTaskInfo]())
 +        for ((taskList, index) <- taskLists.zipWithIndex) {
 +          if (!taskList.isEmpty) {
 +            val offerNum = offerableIndices(index)
 +            val slaveId = offers(offerNum).getSlaveId.getValue
 +            slaveIdsWithExecutors += slaveId
 +            mesosTasks(offerNum) = new JArrayList[MesosTaskInfo](taskList.size)
 +            for (taskDesc <- taskList) {
 +              taskIdToSlaveId(taskDesc.taskId) = slaveId
 +              mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId))
 +            }
 +          }
 +        }
 +
 +        // Reply to the offers
 +        val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout?
 +        for (i <- 0 until offers.size) {
 +          d.launchTasks(offers(i).getId, mesosTasks(i), filters)
 +        }
 +      }
 +    } finally {
 +      restoreClassLoader(oldClassLoader)
 +    }
 +  }
 +
 +  /** Helper function to pull out a resource from a Mesos Resources protobuf */
 +  def getResource(res: JList[Resource], name: String): Double = {
 +    for (r <- res if r.getName == name) {
 +      return r.getScalar.getValue
 +    }
 +    // If we reached here, no resource with the required name was present
 +    throw new IllegalArgumentException("No resource called " + name + " in " + res)
 +  }
 +
 +  /** Turn a Spark TaskDescription into a Mesos task */
 +  def createMesosTask(task: TaskDescription, slaveId: String): MesosTaskInfo = {
 +    val taskId = TaskID.newBuilder().setValue(task.taskId.toString).build()
 +    val cpuResource = Resource.newBuilder()
 +      .setName("cpus")
 +      .setType(Value.Type.SCALAR)
 +      .setScalar(Value.Scalar.newBuilder().setValue(1).build())
 +      .build()
 +    return MesosTaskInfo.newBuilder()
 +      .setTaskId(taskId)
 +      .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build())
 +      .setExecutor(createExecutorInfo(slaveId))
 +      .setName(task.name)
 +      .addResources(cpuResource)
 +      .setData(ByteString.copyFrom(task.serializedTask))
 +      .build()
 +  }
 +
 +  /** Check whether a Mesos task state represents a finished task */
 +  def isFinished(state: MesosTaskState) = {
 +    state == MesosTaskState.TASK_FINISHED ||
 +      state == MesosTaskState.TASK_FAILED ||
 +      state == MesosTaskState.TASK_KILLED ||
 +      state == MesosTaskState.TASK_LOST
 +  }
 +
 +  override def statusUpdate(d: SchedulerDriver, status: TaskStatus) {
 +    val oldClassLoader = setClassLoader()
 +    try {
 +      val tid = status.getTaskId.getValue.toLong
 +      val state = TaskState.fromMesos(status.getState)
 +      synchronized {
 +        if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) {
 +          // We lost the executor on this slave, so remember that it's gone
 +          slaveIdsWithExecutors -= taskIdToSlaveId(tid)
 +        }
 +        if (isFinished(status.getState)) {
 +          taskIdToSlaveId.remove(tid)
 +        }
 +      }
 +      scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer)
 +    } finally {
 +      restoreClassLoader(oldClassLoader)
 +    }
 +  }
 +
 +  override def error(d: SchedulerDriver, message: String) {
 +    val oldClassLoader = setClassLoader()
 +    try {
 +      logError("Mesos error: " + message)
 +      scheduler.error(message)
 +    } finally {
 +      restoreClassLoader(oldClassLoader)
 +    }
 +  }
 +
 +  override def stop() {
 +    if (driver != null) {
 +      driver.stop()
 +    }
 +  }
 +
 +  override def reviveOffers() {
 +    driver.reviveOffers()
 +  }
 +
 +  override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {}
 +
 +  private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) {
 +    val oldClassLoader = setClassLoader()
 +    try {
 +      logInfo("Mesos slave lost: " + slaveId.getValue)
 +      synchronized {
 +        slaveIdsWithExecutors -= slaveId.getValue
 +      }
 +      scheduler.executorLost(slaveId.getValue, reason)
 +    } finally {
 +      restoreClassLoader(oldClassLoader)
 +    }
 +  }
 +
 +  override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) {
 +    recordSlaveLost(d, slaveId, SlaveLost())
 +  }
 +
 +  override def executorLost(d: SchedulerDriver, executorId: ExecutorID,
 +                            slaveId: SlaveID, status: Int) {
 +    logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue,
 +                                                                 slaveId.getValue))
 +    recordSlaveLost(d, slaveId, ExecutorExited(status))
 +  }
 +
 +  // TODO: query Mesos for number of cores
 +  override def defaultParallelism() = System.getProperty("spark.default.parallelism", "8").toInt
 +}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/84849baf/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
----------------------------------------------------------------------


[12/21] git commit: refactor $FWD variable

Posted by pw...@apache.org.
refactor $FWD variable


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/cc37b315
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/cc37b315
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/cc37b315

Branch: refs/heads/master
Commit: cc37b3151cb606ecf1c35865202dc2a08741281c
Parents: 84849ba
Author: Andrew xia <ju...@intel.com>
Authored: Sun Sep 29 22:00:19 2013 +0800
Committer: Andrew xia <ju...@intel.com>
Committed: Sun Sep 29 22:00:19 2013 +0800

----------------------------------------------------------------------
 bin/pyspark               | 2 +-
 bin/run-example           | 2 +-
 bin/spark                 | 4 ++--
 sbin/compute-classpath.sh | 2 +-
 sbin/spark-class          | 4 ++--
 5 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cc37b315/bin/pyspark
----------------------------------------------------------------------
diff --git a/bin/pyspark b/bin/pyspark
index 45a2308..392a92b 100755
--- a/bin/pyspark
+++ b/bin/pyspark
@@ -37,7 +37,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then
 fi
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cc37b315/bin/run-example
----------------------------------------------------------------------
diff --git a/bin/run-example b/bin/run-example
index f29bb2c..ded08a8 100755
--- a/bin/run-example
+++ b/bin/run-example
@@ -26,7 +26,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)"
 export SPARK_HOME="$FWDIR"
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cc37b315/bin/spark
----------------------------------------------------------------------
diff --git a/bin/spark b/bin/spark
index aa005a5..f5f7440 100755
--- a/bin/spark
+++ b/bin/spark
@@ -26,7 +26,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)"
 export SPARK_HOME="$FWDIR"
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
@@ -62,7 +62,7 @@ JAVA_OPTS="$OUR_JAVA_OPTS"
 JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
 JAVA_OPTS="$JAVA_OPTS -Xms$APP_MEM -Xmx$APP_MEM"
 # Load extra JAVA_OPTS from conf/java-opts, if it exists
-if [ -e $FWDIR/conf/java-opts ] ; then
+if [ -e "$FWDIR/conf/java-opts" ] ; then
   JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
 fi
 export JAVA_OPTS

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cc37b315/sbin/compute-classpath.sh
----------------------------------------------------------------------
diff --git a/sbin/compute-classpath.sh b/sbin/compute-classpath.sh
index c7819d4..d9217ec 100755
--- a/sbin/compute-classpath.sh
+++ b/sbin/compute-classpath.sh
@@ -26,7 +26,7 @@ SCALA_VERSION=2.9.3
 FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cc37b315/sbin/spark-class
----------------------------------------------------------------------
diff --git a/sbin/spark-class b/sbin/spark-class
index 7f25fe1..3bdc29e 100755
--- a/sbin/spark-class
+++ b/sbin/spark-class
@@ -26,7 +26,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)"
 export SPARK_HOME="$FWDIR"
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
@@ -87,7 +87,7 @@ JAVA_OPTS="$OUR_JAVA_OPTS"
 JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
 JAVA_OPTS="$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
+if [ -e "$FWDIR/conf/java-opts" ] ; then
   JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
 fi
 export JAVA_OPTS


[13/21] git commit: deprecate "spark" script and SPAKR_CLASSPATH environment variable

Posted by pw...@apache.org.
deprecate "spark" script and SPAKR_CLASSPATH environment variable


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/52ccf4f8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/52ccf4f8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/52ccf4f8

Branch: refs/heads/master
Commit: 52ccf4f859d92ed9e86d3720a983ac2c4a1c23bf
Parents: cc37b31
Author: Andrew xia <ju...@intel.com>
Authored: Sat Oct 12 14:34:14 2013 +0800
Committer: Andrew xia <ju...@intel.com>
Committed: Sat Oct 12 14:34:14 2013 +0800

----------------------------------------------------------------------
 bin/spark                                       | 92 --------------------
 core/pom.xml                                    |  1 -
 .../scala/org/apache/spark/SparkContext.scala   |  2 +-
 repl-bin/src/deb/bin/run                        |  3 +-
 repl/pom.xml                                    |  1 -
 sbin/compute-classpath.cmd                      |  2 +-
 sbin/compute-classpath.sh                       |  2 +-
 7 files changed, 4 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/52ccf4f8/bin/spark
----------------------------------------------------------------------
diff --git a/bin/spark b/bin/spark
deleted file mode 100755
index f5f7440..0000000
--- a/bin/spark
+++ /dev/null
@@ -1,92 +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.
-#
-
-SCALA_VERSION=2.9.3
-
-# Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`/..; pwd)"
-
-# Export this as SPARK_HOME
-export SPARK_HOME="$FWDIR"
-
-# 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 [ -z "$1" ]; then
-  echo "Usage: spark <class> [<args>]" >&2
-  echo "Usage: export SPARK_CLASSPATH before running the command" >&2
-  exit 1
-fi
-
-
-# Find the java binary
-if [ -n "${JAVA_HOME}" ]; then
-  RUNNER="${JAVA_HOME}/bin/java"
-else
-  if [ `command -v java` ]; then
-    RUNNER="java"
-  else
-    echo "JAVA_HOME is not set" >&2
-    exit 1
-  fi
-fi
-
-# Set SPARK_MEM if it isn't already set
-SPARK_MEM=${SPARK_MEM:-512m}
-export SPARK_MEM
-
-# Set APP_MEM if it isn't already set, we use this for this process as the app driver process may need 
-# as much memory as specified in SPARK_MEM
-APP_MEM=${APP_MEM:-512m}
-
-# Set JAVA_OPTS to be able to load native libraries and to set heap size
-JAVA_OPTS="$OUR_JAVA_OPTS"
-JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
-JAVA_OPTS="$JAVA_OPTS -Xms$APP_MEM -Xmx$APP_MEM"
-# Load extra JAVA_OPTS from conf/java-opts, if it exists
-if [ -e "$FWDIR/conf/java-opts" ] ; then
-  JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
-fi
-export JAVA_OPTS
-# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
-
-if [ ! -f "$FWDIR/RELEASE" ]; then
-  # Exit if the user hasn't compiled Spark
-  ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null
-  if [[ $? != 0 ]]; then
-    echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2
-    echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
-    exit 1
-  fi
-fi
-
-# Compute classpath using external script
-CLASSPATH=`$FWDIR/sbin/compute-classpath.sh`
-export CLASSPATH
-
-if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
-  echo -n "Spark Command: "
-  echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
-  echo "========================================"
-  echo
-fi
-
-exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/52ccf4f8/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 9c2d604..8359fef 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -226,7 +226,6 @@
           <environmentVariables>
             <SPARK_HOME>${basedir}/..</SPARK_HOME>
             <SPARK_TESTING>1</SPARK_TESTING>
-            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
           </environmentVariables>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/52ccf4f8/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 912ce75..ce7c4fe 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -132,7 +132,7 @@ class SparkContext(
   // Environment variables to pass to our executors
   private[spark] val executorEnvs = HashMap[String, String]()
   // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner
-  for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) {
+  for (key <- Seq("SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) {
     val value = System.getenv(key)
     if (value != null) {
       executorEnvs(key) = value

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/52ccf4f8/repl-bin/src/deb/bin/run
----------------------------------------------------------------------
diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run
index 8b5d830..d34f189 100755
--- a/repl-bin/src/deb/bin/run
+++ b/repl-bin/src/deb/bin/run
@@ -48,8 +48,7 @@ fi
 export JAVA_OPTS
 
 # Build up classpath
-CLASSPATH="$SPARK_CLASSPATH"
-CLASSPATH+=":$FWDIR/conf"
+CLASSPATH=":$FWDIR/conf"
 for jar in `find $FWDIR -name '*jar'`; do
   CLASSPATH+=":$jar"
 done

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/52ccf4f8/repl/pom.xml
----------------------------------------------------------------------
diff --git a/repl/pom.xml b/repl/pom.xml
index 2826c07..f71184f 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -125,7 +125,6 @@
           <environmentVariables>
             <SPARK_HOME>${basedir}/..</SPARK_HOME>
             <SPARK_TESTING>1</SPARK_TESTING>
-            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
           </environmentVariables>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/52ccf4f8/sbin/compute-classpath.cmd
----------------------------------------------------------------------
diff --git a/sbin/compute-classpath.cmd b/sbin/compute-classpath.cmd
index cf38188..e0b8a8e 100644
--- a/sbin/compute-classpath.cmd
+++ b/sbin/compute-classpath.cmd
@@ -29,7 +29,7 @@ rem Load environment variables from conf\spark-env.cmd, if it exists
 if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
 
 rem Build up classpath
-set CLASSPATH=%SPARK_CLASSPATH%;%FWDIR%conf
+set CLASSPATH=%FWDIR%conf
 if exist "%FWDIR%RELEASE" (
   for %%d in ("%FWDIR%jars\spark-assembly*.jar") do (
     set ASSEMBLY_JAR=%%d

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/52ccf4f8/sbin/compute-classpath.sh
----------------------------------------------------------------------
diff --git a/sbin/compute-classpath.sh b/sbin/compute-classpath.sh
index d9217ec..cfe5fe7 100755
--- a/sbin/compute-classpath.sh
+++ b/sbin/compute-classpath.sh
@@ -31,7 +31,7 @@ if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
 fi
 
 # Build up classpath
-CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
+CLASSPATH="$FWDIR/conf"
 if [ -f "$FWDIR/RELEASE" ]; then
   ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar`
 else