You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hama.apache.org by hy...@apache.org on 2010/04/07 18:01:46 UTC

svn commit: r931602 - in /incubator/hama/trunk: ./ bin/ src/java/org/apache/hama/ src/java/org/apache/hama/graph/

Author: hyunsik
Date: Wed Apr  7 16:01:45 2010
New Revision: 931602

URL: http://svn.apache.org/viewvc?rev=931602&view=rev
Log:
HAMA-246: Current shell scripts for bsp daemon have several problems.

Added:
    incubator/hama/trunk/bin/grooms.sh   (with props)
    incubator/hama/trunk/bin/start-bspd.sh   (with props)
    incubator/hama/trunk/bin/stop-bspd.sh   (with props)
Removed:
    incubator/hama/trunk/bin/groomservers.sh
    incubator/hama/trunk/bin/start-all.sh
    incubator/hama/trunk/bin/stop-all.sh
Modified:
    incubator/hama/trunk/CHANGES.txt
    incubator/hama/trunk/bin/hama
    incubator/hama/trunk/bin/hama-config.sh
    incubator/hama/trunk/bin/hama-daemon.sh
    incubator/hama/trunk/bin/hama-daemons.sh
    incubator/hama/trunk/src/java/org/apache/hama/HamaMaster.java
    incubator/hama/trunk/src/java/org/apache/hama/graph/GroomServer.java

Modified: incubator/hama/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/CHANGES.txt?rev=931602&r1=931601&r2=931602&view=diff
==============================================================================
--- incubator/hama/trunk/CHANGES.txt (original)
+++ incubator/hama/trunk/CHANGES.txt Wed Apr  7 16:01:45 2010
@@ -137,6 +137,7 @@ Trunk (unreleased changes)
 
   BUG FIXES
  
+    HAMA-246: Current shell scripts for bsp daemon have several problems. (hyunsik)
     HAMA-238: Example fail when performing sparse matrices addition (edwardyoon)
     HAMA-225: Jacobi iteration is in a infinite loop (edwardyoon)
     HAMA-214: Can't run the examples (edwardyoon)

Added: incubator/hama/trunk/bin/grooms.sh
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/bin/grooms.sh?rev=931602&view=auto
==============================================================================
--- incubator/hama/trunk/bin/grooms.sh (added)
+++ incubator/hama/trunk/bin/grooms.sh Wed Apr  7 16:01:45 2010
@@ -0,0 +1,68 @@
+#!/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
+#
+#   HAMA_GROOMS  File naming remote hosts.
+#     Default is ${HAMA_CONF_DIR}/groomservers.
+#   HAMA_CONF_DIR  Alternate conf dir. Default is ${HAMA_HOME}/conf.
+#   HAMA_GROOM_SLEEP Seconds to sleep between spawning remote commands.
+#   HAMA_SSH_OPTS Options passed to ssh when running remote commands.
+##
+
+usage="Usage: grooms.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"/hama-config.sh
+
+# If the groomservers file is specified in the command line,
+# then it takes precedence over the definition in 
+# hama-env.sh. Save it here.
+HOSTLIST=$HAMA_GROOMS
+
+if [ -f "${HAMA_CONF_DIR}/hama-env.sh" ]; then
+  . "${HAMA_CONF_DIR}/hama-env.sh"
+fi
+
+if [ "$HOSTLIST" = "" ]; then
+  if [ "$HAMA_GROOMS" = "" ]; then
+    export HOSTLIST="${HAMA_CONF_DIR}/groomservers"
+  else
+    export HOSTLIST="${HAMA_GROOMS}"
+  fi
+fi
+
+for groom in `cat "$HOSTLIST"|sed  "s/#.*$//;/^$/d"`; do
+ ssh $HAMA_SSH_OPTS $groom $"${@// /\\ }" \
+   2>&1 | sed "s/^/$groom: /" &
+ if [ "$HAMA_GROOM_SLEEP" != "" ]; then
+   sleep $HAMA_GROOM_SLEEP
+ fi
+done
+
+wait

Propchange: incubator/hama/trunk/bin/grooms.sh
------------------------------------------------------------------------------
    svn:executable = *

Modified: incubator/hama/trunk/bin/hama
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/bin/hama?rev=931602&r1=931601&r2=931602&view=diff
==============================================================================
--- incubator/hama/trunk/bin/hama (original)
+++ incubator/hama/trunk/bin/hama Wed Apr  7 16:01:45 2010
@@ -1,49 +1,49 @@
-#! /usr/bin/env bash
+#!/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
 #
-#/**
-# * Copyright 2007 The Apache Software Foundation
-# *
-# * 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.
-# */
-# 
-# The hama command script.  Based on the hbase command script putting
-# in hama classes, libs and configurations ahead of hbase's.
+#     http://www.apache.org/licenses/LICENSE-2.0
 #
-# TODO: Narrow the amount of duplicated code.
+# 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.
+
+
+# The Hama command script
 #
-# Environment Variables:
+# Environment Variables
 #
 #   JAVA_HOME        The java implementation to use.  Overrides JAVA_HOME.
 #
-#   HAMA_CLASSPATH   Extra Java CLASSPATH entries.
+#   HAMA_CLASSPATH Extra Java CLASSPATH entries.
 #
-#   HAMA_HEAPSIZE    The maximum amount of heap to use, in MB. 
+#   HAMA_HEAPSIZE  The maximum amount of heap to use, in MB. 
 #                    Default is 1000.
 #
-#   HAMA_CONF_DIR    Alternate conf dir. Default is ${HAMA_HOME}/conf.
+#   HAMA_OPTS      Extra Java runtime options.
 #   
-#   HAMA_JARS        Additional jars to be included for task execution.
+#   HAMA_NAMENODE_OPTS       These options are added to HAMA_OPTS 
+#   HAMA_CLIENT_OPTS         when the respective command is run.
+#   HAMA_{COMMAND}_OPTS etc  HAMA_JT_OPTS applies to JobTracker 
+#                              for e.g.  HAMA_CLIENT_OPTS applies to 
+#                              more than one command (fs, dfs, fsck, 
+#                              dfsadmin etc)  
+#
+#   HAMA_CONF_DIR  Alternate conf dir. Default is ${HAMA_HOME}/conf.
 #
 #   HAMA_ROOT_LOGGER The root appender. Default is INFO,console
 #
+
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 
-# This will set HAMA_HOME, etc.
 . "$bin"/hama-config.sh
 
 cygwin=false
@@ -53,13 +53,14 @@ esac
 
 # if no args specified, show usage
 if [ $# = 0 ]; then
-  echo "Usage: hama <command>"
-  echo "where <command> is one of:"
-  echo "  master           run the master server"
-  echo "  groomserver      run the groom server"
-  echo "  examples         run the HAMA examples"
+  echo "Usage: hama [--config confdir] COMMAND"
+  echo "where COMMAND is one of:"
+  echo "  bspmaster            run the BSP Master node"
+  echo "  groom                run the Groom node"
+  echo "  version              print the version"
+  echo "  jar <jar>            run a jar file"
   echo " or"
-  echo "  CLASSNAME        run the class named CLASSNAME"
+  echo "  CLASSNAME            run the class named CLASSNAME"
   echo "Most commands print help when invoked w/o parameters."
   exit 1
 fi
@@ -68,7 +69,6 @@ fi
 COMMAND=$1
 shift
 
-# Source the hama-env.sh.  Will have JAVA_HOME defined.
 if [ -f "${HAMA_CONF_DIR}/hama-env.sh" ]; then
   . "${HAMA_CONF_DIR}/hama-env.sh"
 fi
@@ -78,7 +78,7 @@ if [ "$JAVA_HOME" != "" ]; then
   #echo "run java in $JAVA_HOME"
   JAVA_HOME=$JAVA_HOME
 fi
-
+  
 if [ "$JAVA_HOME" = "" ]; then
   echo "Error: JAVA_HOME is not set."
   exit 1
@@ -94,66 +94,49 @@ if [ "$HAMA_HEAPSIZE" != "" ]; then
   #echo $JAVA_HEAP_MAX
 fi
 
-# so that filenames w/ spaces are handled correctly in loops below
-IFS=
-
 # CLASSPATH initially contains $HAMA_CONF_DIR
 CLASSPATH="${HAMA_CONF_DIR}"
-
-
 CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/tools.jar
 
-# for developers, add hama classes to CLASSPATH
+# for developers, add Hama classes to CLASSPATH
 if [ -d "$HAMA_HOME/build/classes" ]; then
   CLASSPATH=${CLASSPATH}:$HAMA_HOME/build/classes
 fi
-if [ -d "$HAMA_HOME/build/test" ]; then
-  CLASSPATH=${CLASSPATH}:$HAMA_HOME/build/test
-fi
-if [ -d "$HAMA_HOME/build/examples" ]; then
+if [ -d "$HAMA_HOME/build/webapps" ]; then
   CLASSPATH=${CLASSPATH}:$HAMA_HOME/build
 fi
+if [ -d "$HAMA_HOME/build/test/classes" ]; then
+  CLASSPATH=${CLASSPATH}:$HAMA_HOME/build/test/classes
+fi
+if [ -d "$HAMA_HOME/build/tools" ]; then
+  CLASSPATH=${CLASSPATH}:$HAMA_HOME/build/tools
+fi
 
-# for releases, add hama to CLASSPATH
-for f in $HAMA_HOME/hama*.jar; do
-  if [ -f $f ]; then
-    CLASSPATH=${CLASSPATH}:$f;
-  fi
-  if [[ `echo $f | grep 'examples.jar$'` ]]; then
-    HAMA_EXAMPLES_JAR=$f;
-  else
-    if [[ `echo $f | grep 'test.jar$'` ]]; then
-      HAMA_TEST_JAR=$f;
-    else
-      if [[ `echo $f | grep 'hama*'` ]]; then
-        HAMA_JAR=$f;
-      fi
-    fi
-  fi
-done
+# so that filenames w/ spaces are handled correctly in loops below
+IFS=
 
-# Add libs to CLASSPATH
-for f in $HAMA_HOME/lib/*.jar; do
+# for releases, add core hama jar & webapps to CLASSPATH
+if [ -d "$HAMA_HOME/webapps" ]; then
+  CLASSPATH=${CLASSPATH}:$HAMA_HOME
+fi
+for f in $HAMA_HOME/hama-*-core.jar; do # TODO - should be changed before initial release
   CLASSPATH=${CLASSPATH}:$f;
-  if [[ `echo $f | grep 'hbase*'` ]]; then
-    if [ "$HAMA_JARS" == "" ] ; then
-      HAMA_JARS=$f;
-    else
-      HAMA_JARS=${HAMA_JARS},$f;
-    fi
-  fi
 done
 
-for f in $HAMA_HOME/lib/jetty-ext/*.jar; do
+# add libs to CLASSPATH
+for f in $HAMA_HOME/lib/*.jar; do
   CLASSPATH=${CLASSPATH}:$f;
 done
 
-for f in $HAMA_HOME/lib/findbugs/*.jar; do
+for f in $HAMA_HOME/lib/jsp-2.1/*.jar; do
   CLASSPATH=${CLASSPATH}:$f;
 done
 
-for f in $HAMA_HOME/lib/findbugs/plugin/*.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
+for f in $HAMA_HOME/hama-*-tools.jar; do
+  TOOL_PATH=${TOOL_PATH}:$f;
+done
+for f in $HAMA_HOME/build/hama-*-tools.jar; do
+  TOOL_PATH=${TOOL_PATH}:$f;
 done
 
 # add user-specified CLASSPATH last
@@ -169,37 +152,41 @@ if [ "$HAMA_LOGFILE" = "" ]; then
   HAMA_LOGFILE='hama.log'
 fi
 
-# cygwin path translation
-if $cygwin; then
-  CLASSPATH=`cygpath -p -w "$CLASSPATH"`
-  HAMA_HOME=`cygpath -d "$HAMA_HOME"`
-  HAMA_LOG_DIR=`cygpath -d "$HAMA_LOG_DIR"`
-fi
-
-# cygwin path translation
-if $cygwin; then
-  JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"`
+# default policy file for service-level authorization
+if [ "$HAMA_POLICYFILE" = "" ]; then
+  HAMA_POLICYFILE="hama-policy.xml"
 fi
 
 # restore ordinary behaviour
 unset IFS
 
 # figure out which class to run
-if [ "$COMMAND" = "examples" ] ; then
-  CLASS="org.apache.hama.examples.ExampleDriver"
-elif [ "$COMMAND" = "master" ] ; then
+if [ "$COMMAND" = "bspmaster" ] ; then
   CLASS='org.apache.hama.HamaMaster'
-elif [ "$COMMAND" = "groomserver" ] ; then
+  BSP_OPTS="$BSP_OPTS $BSP_BSPMASTER_OPTS"
+elif [ "$COMMAND" = "groom" ] ; then
   CLASS='org.apache.hama.graph.GroomServer'
+  BSP_OPTS="$BSP_OPTS $BSP_GROOMSERVER_OPTS"
+elif [ "$COMMAND" = "version" ] ; then
+  CLASS=org.apache.bsp.util.VersionInfo
+  BSP_OPTS="$BSP_OPTS $BSP_CLIENT_OPTS"
 else
   CLASS=$COMMAND
 fi
 
-# Have JVM dump heap if we run out of memory.  Files will be 'launch directory'
-# and are named like the following: java_pid21612.hprof. Apparently it doesn't
-# 'cost' to have this flag enabled. Its a 1.6 flag only. See:
-# http://blogs.sun.com/alanb/entry/outofmemoryerror_looks_a_bit_better 
-HAMA_OPTS="$HAMA_OPTS -XX:+HeapDumpOnOutOfMemoryError"
+# cygwin path translation
+if $cygwin; then
+  CLASSPATH=`cygpath -p -w "$CLASSPATH"`
+  HAMA_HOME=`cygpath -w "$HAMA_HOME"`
+  HAMA_LOG_DIR=`cygpath -w "$HAMA_LOG_DIR"`
+  TOOL_PATH=`cygpath -p -w "$TOOL_PATH"`
+fi
+
+# cygwin path translation
+if $cygwin; then
+  JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"`
+fi
+
 HAMA_OPTS="$HAMA_OPTS -Dhama.log.dir=$HAMA_LOG_DIR"
 HAMA_OPTS="$HAMA_OPTS -Dhama.log.file=$HAMA_LOGFILE"
 HAMA_OPTS="$HAMA_OPTS -Dhama.home.dir=$HAMA_HOME"
@@ -208,6 +195,7 @@ HAMA_OPTS="$HAMA_OPTS -Dhama.root.logger
 if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
   HAMA_OPTS="$HAMA_OPTS -Djava.library.path=$JAVA_LIBRARY_PATH"
 fi  
+HAMA_OPTS="$HAMA_OPTS -Dhama.policy.file=$HAMA_POLICYFILE"
 
 # run it
 exec "$JAVA" $JAVA_HEAP_MAX $HAMA_OPTS -classpath "$CLASSPATH" $CLASS "$@"

Modified: incubator/hama/trunk/bin/hama-config.sh
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/bin/hama-config.sh?rev=931602&r1=931601&r2=931602&view=diff
==============================================================================
--- incubator/hama/trunk/bin/hama-config.sh (original)
+++ incubator/hama/trunk/bin/hama-config.sh Wed Apr  7 16:01:45 2010
@@ -1,28 +1,21 @@
+# 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
 #
-#/**
-# * Copyright 2007 The Apache Software Foundation
-# *
-# * 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.
-# */
+#     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 hama scripts with source command
 # should not be executable directly
 # also should not be passed any arguments, since we need original $*
-# Modelled after $HBASE_HOME/bin/hbase-config.sh.
 
 # resolve links - $0 may be a softlink
 
@@ -43,31 +36,33 @@ script=`basename "$this"`
 bin=`cd "$bin"; pwd`
 this="$bin/$script"
 
-# the root of the hama installation
+# the root of the Hama installation
 export HAMA_HOME=`dirname "$this"`/..
 
-#check to see if the conf dir or hama home are given as an optional arguments
-while [ $# -gt 1 ]
-do
-  if [ "--config" = "$1" ]
-  then
-    shift
-    confdir=$1
-    shift
-    HAMA_CONF_DIR=$confdir
-  elif [ "--hosts" = "$1" ]
-  then
-    shift
-    hosts=$1
-    shift
-    HAMA_GROOMSERVERS=$hosts
-  else
-    # Presume we are at end of options and break
-    break
-  fi
-done
-
-# Allow alternate hama conf dir location.
+#check to see if the conf dir is given as an optional argument
+if [ $# -gt 1 ]
+then
+    if [ "--config" = "$1" ]
+	  then
+	      shift
+	      confdir=$1
+	      shift
+	      HAMA_CONF_DIR=$confdir
+    fi
+fi
+ 
+# Allow alternate conf dir location.
 HAMA_CONF_DIR="${HAMA_CONF_DIR:-$HAMA_HOME/conf}"
-# List of hama groom servers.
-HAMA_GROOMSERVERS="${HAMA_GROOMSERVERS:-$HAMA_CONF_DIR/groomservers}"
\ No newline at end of file
+
+#check to see it is specified whether to use the grooms or the
+# masters file
+if [ $# -gt 1 ]
+then
+    if [ "--hosts" = "$1" ]
+    then
+        shift
+        grooms=$1
+        shift
+        export HAMA_SLAVES="${HAMA_CONF_DIR}/$grooms"
+    fi
+fi

Modified: incubator/hama/trunk/bin/hama-daemon.sh
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/bin/hama-daemon.sh?rev=931602&r1=931601&r2=931602&view=diff
==============================================================================
--- incubator/hama/trunk/bin/hama-daemon.sh (original)
+++ incubator/hama/trunk/bin/hama-daemon.sh Wed Apr  7 16:01:45 2010
@@ -1,40 +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
 #
-#/**
-# * Copyright 2007 The Apache Software Foundation
-# *
-# * 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 Hadoop hbase command as a daemon.
+# 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 Hama command as a daemon.
 #
 # Environment Variables
 #
-#   HAMA_CONF_DIR   Alternate hbase conf dir. Default is ${HAMA_HOME}/conf.
-#   HAMA_LOG_DIR    Where log files are stored.  PWD by default.
-#   HAMA_PID_DIR    The pid files are stored. /tmp by default.
-#   HAMA_IDENT_STRING   A string representing this instance of hadoop. $USER by default
+#   HAMA_CONF_DIR  Alternate conf dir. Default is ${HAMA_HOME}/conf.
+#   HAMA_LOG_DIR   Where log files are stored.  PWD by default.
+#   HAMA_MASTER    host:path where hama code should be rsync'd from
+#   HAMA_PID_DIR   The pid files are stored. /tmp by default.
+#   HAMA_IDENT_STRING   A string representing this instance of hama. $USER by default
 #   HAMA_NICENESS The scheduling priority for daemons. Defaults to 0.
-#
-# Modelled after $HADOOP_HOME/bin/hadoop-daemon.sh
+##
 
-usage="Usage: hama-daemon.sh [--config <conf-dir>]\
- (start|stop) <hama-command> \
- <args...>"
+usage="Usage: hama-daemon.sh [--config <conf-dir>] [--hosts hostlistfile] (start|stop) <hama-command> <args...>"
 
 # if no args specified, show usage
 if [ $# -le 1 ]; then
@@ -50,7 +44,6 @@ bin=`cd "$bin"; pwd`
 # get arguments
 startStop=$1
 shift
-
 command=$1
 shift
 
@@ -59,15 +52,15 @@ hama_rotate_log ()
     log=$1;
     num=5;
     if [ -n "$2" ]; then
-    num=$2
+	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";
+	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
 }
 
@@ -89,21 +82,10 @@ if [ "$HAMA_IDENT_STRING" = "" ]; then
   export HAMA_IDENT_STRING="$USER"
 fi
 
-# Some variables
-# Work out java location so can print version into log.
-if [ "$JAVA_HOME" != "" ]; then
-  #echo "run java in $JAVA_HOME"
-  JAVA_HOME=$JAVA_HOME
-fi
-if [ "$JAVA_HOME" = "" ]; then
-  echo "Error: JAVA_HOME is not set."
-  exit 1
-fi
-JAVA=$JAVA_HOME/bin/java
+# some variables
 export HAMA_LOGFILE=hama-$HAMA_IDENT_STRING-$command-$HOSTNAME.log
 export HAMA_ROOT_LOGGER="INFO,DRFA"
-logout=$HAMA_LOG_DIR/hama-$HAMA_IDENT_STRING-$command-$HOSTNAME.out  
-loglog="${HAMA_LOG_DIR}/${HAMA_LOGFILE}"
+log=$HAMA_LOG_DIR/hama-$HAMA_IDENT_STRING-$command-$HOSTNAME.out
 pid=$HAMA_PID_DIR/hama-$HAMA_IDENT_STRING-$command.pid
 
 # Set default scheduling priority
@@ -114,7 +96,9 @@ fi
 case $startStop in
 
   (start)
+
     mkdir -p "$HAMA_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.
@@ -122,42 +106,30 @@ case $startStop in
       fi
     fi
 
-    hama_rotate_log $logout
-    echo starting $command, logging to $logout
-    # Add to the command log file vital stats on our environment.
-    echo "`date` Starting $command on `hostname`" >> $loglog
-    echo "ulimit -n `ulimit -n`" >> $loglog 2>&1
-    nohup nice -n $HAMA_NICENESS "$HAMA_HOME"/bin/hama \
-        --config "${HAMA_CONF_DIR}" \
-        $command $startStop "$@" > "$logout" 2>&1 < /dev/null &
+    if [ "$HAMA_MASTER" != "" ]; then
+      echo rsync from $HAMA_MASTER
+      rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $HAMA_MASTER/ "$HAMA_HOME"
+    fi
+
+    hama_rotate_log $log
+    echo starting $command, logging to $log
+    cd "$HAMA_HOME"
+    nohup nice -n $HAMA_NICENESS "$HAMA_HOME"/bin/hama --config $HAMA_CONF_DIR $command "$@" > "$log" 2>&1 < /dev/null &
     echo $! > $pid
-    sleep 1; head "$logout"
+    sleep 1; head "$log"
     ;;
-
+          
   (stop)
+
     if [ -f $pid ]; then
       if kill -0 `cat $pid` > /dev/null 2>&1; then
-        echo -n stopping $command
-        echo "`date` Stopping $command" >> $loglog
-        if [ "$command" = "master" ]; then
-          nohup nice -n $HAMA_NICENESS "$HAMA_HOME"/bin/hama \
-              --config "${HAMA_CONF_DIR}" \
-              $command $startStop "$@" > "$logout" 2>&1 < /dev/null &
-        else
-          echo "`date` Killing $command" >> $loglog
-          kill `cat $pid` > /dev/null 2>&1
-        fi
-        while kill -0 `cat $pid` > /dev/null 2>&1; do
-          echo -n "."
-          sleep 1;
-        done
-        echo
+        echo stopping $command
+        kill `cat $pid`
       else
-        retval=$?
-        echo no $command to stop because kill of pid `cat $pid` failed with status $retval
+        echo no $command to stop
       fi
     else
-      echo no $command to stop because no pid file $pid
+      echo no $command to stop
     fi
     ;;
 

Modified: incubator/hama/trunk/bin/hama-daemons.sh
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/bin/hama-daemons.sh?rev=931602&r1=931601&r2=931602&view=diff
==============================================================================
--- incubator/hama/trunk/bin/hama-daemons.sh (original)
+++ incubator/hama/trunk/bin/hama-daemons.sh Wed Apr  7 16:01:45 2010
@@ -1,30 +1,24 @@
 #!/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
 #
-#/**
-# * Copyright 2007 The Apache Software Foundation
-# *
-# * 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 hama command on all slave hosts.
-# Modelled after $HADOOP_HOME/bin/hadoop-daemons.sh
+# 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="Usage: hama-daemons.sh [--config <hama-confdir>] \
- [--hosts groomserversfile] [start|stop] command args..."
+# Run a Hama command on all slave hosts.
+
+usage="Usage: hama-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command args..."
 
 # if no args specified, show usage
 if [ $# -le 1 ]; then
@@ -37,15 +31,4 @@ bin=`cd "$bin"; pwd`
 
 . $bin/hama-config.sh
 
-remote_cmd="cd ${HAMA_HOME}; $bin/hama-daemon.sh --config ${HAMA_CONF_DIR} $@"
-args="--config ${HAMA_CONF_DIR} $remote_cmd"
-
-command=$2
-case $command in
-  (zookeeper)
-    exec "$bin/zookeepers.sh" $args
-    ;;
-  (*)
-    exec "$bin/groomservers.sh" $args
-    ;;
-esac
+exec "$bin/grooms.sh" --config $HAMA_CONF_DIR cd "$HAMA_HOME" \; "$bin/hama-daemon.sh" --config $HAMA_CONF_DIR "$@"

Added: incubator/hama/trunk/bin/start-bspd.sh
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/bin/start-bspd.sh?rev=931602&view=auto
==============================================================================
--- incubator/hama/trunk/bin/start-bspd.sh (added)
+++ incubator/hama/trunk/bin/start-bspd.sh Wed Apr  7 16:01:45 2010
@@ -0,0 +1,29 @@
+#!/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 hama map reduce daemons.  Run this on master node.
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/hama-config.sh
+
+# start mapred daemons
+# start jobtracker first to minimize connection errors at startup
+"$bin"/hama-daemon.sh --config $HAMA_CONF_DIR start bspmaster
+"$bin"/hama-daemons.sh --config $HAMA_CONF_DIR start groom

Propchange: incubator/hama/trunk/bin/start-bspd.sh
------------------------------------------------------------------------------
    svn:executable = *

Added: incubator/hama/trunk/bin/stop-bspd.sh
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/bin/stop-bspd.sh?rev=931602&view=auto
==============================================================================
--- incubator/hama/trunk/bin/stop-bspd.sh (added)
+++ incubator/hama/trunk/bin/stop-bspd.sh Wed Apr  7 16:01:45 2010
@@ -0,0 +1,28 @@
+#!/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.
+
+
+# Stop hama map reduce daemons.  Run this on master node.
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/hama-config.sh
+
+"$bin"/hama-daemon.sh --config $HAMA_CONF_DIR stop bspmaster
+"$bin"/hama-daemons.sh --config $HAMA_CONF_DIR stop groom
+

Propchange: incubator/hama/trunk/bin/stop-bspd.sh
------------------------------------------------------------------------------
    svn:executable = *

Modified: incubator/hama/trunk/src/java/org/apache/hama/HamaMaster.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/HamaMaster.java?rev=931602&r1=931601&r2=931602&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/HamaMaster.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/HamaMaster.java Wed Apr  7 16:01:45 2010
@@ -164,7 +164,7 @@ public class HamaMaster implements JobSu
   
   public static void main(String [] args) {
     StringUtils.startupShutdownMessage(HamaMaster.class, args, LOG);
-    if (args.length != 1) {
+    if (args.length != 0) {
       System.out.println("usage: HamaMaster");
       System.exit(-1);
     }

Modified: incubator/hama/trunk/src/java/org/apache/hama/graph/GroomServer.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/graph/GroomServer.java?rev=931602&r1=931601&r2=931602&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/graph/GroomServer.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/graph/GroomServer.java Wed Apr  7 16:01:45 2010
@@ -273,7 +273,7 @@ public class GroomServer implements Runn
 
   public static void main(String[] args) {
     StringUtils.startupShutdownMessage(GroomServer.class, args, LOG);
-    if (args.length != 1) {
+    if (args.length != 0) {
       System.out.println("usage: GroomServer");
       System.exit(-1);
     }