You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2008/12/08 21:35:29 UTC

svn commit: r724473 [1/3] - in /hadoop/hive/trunk: ./ ant/ ant/src/org/apache/hadoop/hive/ant/ cli/ common/ common/src/java/org/apache/hadoop/hive/conf/ conf/ data/conf/ hadoopcore/bin/ hadoopcore/conf/ hadoopcore/lib/ ivy/ metastore/ ql/ ql/src/java/o...

Author: zshao
Date: Mon Dec  8 12:35:28 2008
New Revision: 724473

URL: http://svn.apache.org/viewvc?rev=724473&view=rev
Log:
HIVE-98. Dependency management with hadoop core using ivy. (Ashish Thusoo through zshao)

Added:
    hadoop/hive/trunk/ant/ivy.xml
    hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/GetVersionPref.java
    hadoop/hive/trunk/cli/ivy.xml
    hadoop/hive/trunk/common/ivy.xml
    hadoop/hive/trunk/ivy/
    hadoop/hive/trunk/ivy/get_ivy.xml
    hadoop/hive/trunk/ivy/ivysettings.xml
    hadoop/hive/trunk/metastore/ivy.xml
    hadoop/hive/trunk/ql/ivy.xml
    hadoop/hive/trunk/serde/ivy.xml
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/antlib.xml
    hadoop/hive/trunk/build-common.xml
    hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hadoop/hive/trunk/conf/hive-default.xml
    hadoop/hive/trunk/data/conf/hive-site.xml
    hadoop/hive/trunk/hadoopcore/bin/hadoop
    hadoop/hive/trunk/hadoopcore/bin/hadoop-config.sh
    hadoop/hive/trunk/hadoopcore/bin/hadoop-daemon.sh
    hadoop/hive/trunk/hadoopcore/bin/hadoop-daemons.sh
    hadoop/hive/trunk/hadoopcore/bin/rcc
    hadoop/hive/trunk/hadoopcore/bin/slaves.sh
    hadoop/hive/trunk/hadoopcore/bin/start-all.sh
    hadoop/hive/trunk/hadoopcore/bin/start-balancer.sh
    hadoop/hive/trunk/hadoopcore/bin/start-dfs.sh
    hadoop/hive/trunk/hadoopcore/bin/start-mapred.sh
    hadoop/hive/trunk/hadoopcore/bin/stop-all.sh
    hadoop/hive/trunk/hadoopcore/bin/stop-balancer.sh
    hadoop/hive/trunk/hadoopcore/bin/stop-dfs.sh
    hadoop/hive/trunk/hadoopcore/bin/stop-mapred.sh
    hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml
    hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml.template
    hadoop/hive/trunk/hadoopcore/conf/configuration.xsl
    hadoop/hive/trunk/hadoopcore/conf/hadoop-default.xml
    hadoop/hive/trunk/hadoopcore/conf/hadoop-env.sh
    hadoop/hive/trunk/hadoopcore/conf/hadoop-env.sh.template
    hadoop/hive/trunk/hadoopcore/conf/hadoop-metrics.properties
    hadoop/hive/trunk/hadoopcore/conf/hadoop-site.xml
    hadoop/hive/trunk/hadoopcore/conf/hadoop-site.xml.template
    hadoop/hive/trunk/hadoopcore/conf/log4j.properties
    hadoop/hive/trunk/hadoopcore/conf/masters
    hadoop/hive/trunk/hadoopcore/conf/masters.template
    hadoop/hive/trunk/hadoopcore/conf/slaves
    hadoop/hive/trunk/hadoopcore/conf/slaves.template
    hadoop/hive/trunk/hadoopcore/conf/ssl-client.xml.example
    hadoop/hive/trunk/hadoopcore/conf/ssl-server.xml.example
    hadoop/hive/trunk/hadoopcore/lib/hsqldb-1.8.0.10.LICENSE.txt
    hadoop/hive/trunk/hadoopcore/lib/jetty-5.1.4.LICENSE.txt
    hadoop/hive/trunk/hadoopcore/lib/junit-3.8.1.LICENSE.txt
    hadoop/hive/trunk/hadoopcore/lib/kfs-0.2.LICENSE.txt
    hadoop/hive/trunk/hadoopcore/lib/slf4j-LICENSE.txt
    hadoop/hive/trunk/ql/build.xml
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/input13.q
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/input3_limit.q
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/mapreduce2.q
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/subq.q
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/union.q
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input3_limit.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/mapreduce2.q.out

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Mon Dec  8 12:35:28 2008
@@ -6,6 +6,9 @@
 
   NEW FEATURES
 
+    HIVE-98. Dependency management with hadoop core using ivy.
+    (Ashish Thusoo through zshao)
+
     HIVE-73. Thrift Server and Client for Hive (Raghu through zshao)
 
     HIVE-113. Distribute by and sort by support. (zshao)

Added: hadoop/hive/trunk/ant/ivy.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ant/ivy.xml?rev=724473&view=auto
==============================================================================
--- hadoop/hive/trunk/ant/ivy.xml (added)
+++ hadoop/hive/trunk/ant/ivy.xml Mon Dec  8 12:35:28 2008
@@ -0,0 +1,3 @@
+<ivy-module version="2.0">
+    <info organisation="org.apache.hadoop.hive" module="ant"/>
+</ivy-module>

Added: hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/GetVersionPref.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/GetVersionPref.java?rev=724473&view=auto
==============================================================================
--- hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/GetVersionPref.java (added)
+++ hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/GetVersionPref.java Mon Dec  8 12:35:28 2008
@@ -0,0 +1,94 @@
+/**
+ * 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.hadoop.hive.ant;
+
+import org.apache.tools.ant.AntClassLoader;
+import org.apache.tools.ant.BuildException;
+import org.apache.tools.ant.Task;
+import org.apache.tools.ant.Project;
+
+import java.util.regex.Pattern;
+import java.util.regex.Matcher;
+import java.io.*;
+
+/**
+ * Implementation of the ant task <getversionpref property="nameoftheproperty" input="versionstring"/>.
+ *
+ * This ant task takes an input version string (e.g. 0.17.2) and set an ant property (whose name
+ * is specified in the property attribute) with the version prefix. For 0.17.2, the version prefix
+ * is 0.17. Similarly, for 0.18.0, the version prefix is 0.18. The version prefix is the first two
+ * components of the version string.
+ */
+public class GetVersionPref extends Task {
+
+  /**
+   * The name of the property that gets the version prefix.
+   */
+  protected String property;
+
+  /**
+   * The input string that contains the version string.
+   */
+  protected String input;
+ 
+  public void setProperty(String property) {
+    this.property = property;
+  }
+
+  public String getProperty() {
+    return property;
+  }
+
+  public void setInput(String input) {
+    this.input = input;
+  }
+
+  public String getInput() {
+    return input;
+  }
+
+  /**
+   * Executes the ant task <getversionperf>.
+   *
+   * It extracts the version prefix using regular expressions on the version string. It then sets
+   * the property in the project with the extracted prefix. The property is set to an empty string
+   * in case no match is found for the prefix regular expression (which will happen in case the
+   * version string does not conform to the version format).
+   */
+  @Override
+  public void execute() throws BuildException {
+
+    if (property == null) {
+      throw new BuildException("No property specified");
+    }
+
+    if (input == null) {
+      throw new BuildException("No input stringspecified");
+    }
+
+    try {
+      Pattern p = Pattern.compile("^(\\d+\\.\\d+).*");
+      Matcher m = p.matcher(input);
+      getProject().setProperty(property, m.matches() ? m.group(1) : "");
+    }
+    catch (Exception e) {
+      throw new BuildException("Failed with: " + e.getMessage());
+    }
+  }
+}

Modified: hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/antlib.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/antlib.xml?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/antlib.xml (original)
+++ hadoop/hive/trunk/ant/src/org/apache/hadoop/hive/ant/antlib.xml Mon Dec  8 12:35:28 2008
@@ -21,4 +21,6 @@
 <antlib>
   <taskdef name="qtestgen"
            classname="org.apache.hadoop.hive.ant.QTestGenTask" />
+  <taskdef name="getversionpref"
+           classname="org.apache.hadoop.hive.ant.GetVersionPref" />
 </antlib>

Modified: hadoop/hive/trunk/build-common.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/build-common.xml?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/build-common.xml (original)
+++ hadoop/hive/trunk/build-common.xml Mon Dec  8 12:35:28 2008
@@ -18,7 +18,7 @@
 -->
 
 
-<project name="hivecommon" default="jar">
+<project xmlns:ivy="antlib:org.apache.ivy.ant" name="hivecommon" default="jar">
 
   <property name="name" value="${ant.project.name}"/>
 
@@ -26,15 +26,19 @@
   <property name="hive.conf.dir" value="${hive.root}/conf"/>
   <property name="dist.dir" location="${hive.root}"/>
 
-  <property name="hadoop.jar" location="${hive.root}/hadoopcore/lib/hadoop-0.20.0-dev-core.jar"/>
-
   <property name="src.dir.hive" location="${hive.root}"/>
   <property name="build.dir.hive" location="${hive.root}/build"/>
+  <property name="build.dir.hadoop" location="${hive.root}/build/hadoopcore"/>
   <property name="build.dir" location="${build.dir.hive}/${name}"/>
   <property name="build.classes" location="${build.dir}/classes"/>
   <property name="build.encoding" value="ISO-8859-1"/>
   <property name="deploy.dir" location="${build.dir.hive}"/>
 
+  <property name="hadoop.mirror" value="http://archive.apache.org/dist"/>
+  <property name="hadoop.version" value="0.19.0"/>
+  <property name="hadoop.root" location="${build.dir.hadoop}/hadoop-${hadoop.version}"/>
+  <property name="hadoop.jar" location="${hadoop.root}/hadoop-${hadoop.version}-core.jar"/>
+
   <property name="javac.debug" value="on"/>
   <property name="javac.version" value="1.5"/>
   <property name="javac.optimize" value="on"/>
@@ -68,6 +72,21 @@
     <path refid="classpath"/>
   </path>
 
+  <import file="${hive.root}/ivy/get_ivy.xml"/>
+
+  <target name="settings-ivy" depends="init-ivy">
+    <ivy:settings file="${hive.root}/ivy/ivysettings.xml"/>
+  </target>
+
+  <target name="resolve" description="--> retrieve dependencies with ivy" depends="settings-ivy">
+    <ivy:retrieve pattern="${build.dir.hadoop}/[artifact]-[revision].[ext]"/>
+  </target>
+
+  <target name="install-hadoopcore" depends="resolve">
+    <untar src="${build.dir.hadoop}/hadoop-${hadoop.version}.tar.gz" dest="${build.dir.hadoop}" compression="gzip"/>
+    <chmod file="${hadoop.root}/bin/hadoop" perm="+x"/>
+  </target>
+
   <!-- I am not sure whether we need this target any more since that package does what is needed -->
   <target name="deploy" depends="jar">
     <echo message="hive: ${name}"/>
@@ -107,6 +126,7 @@
     <mkdir dir="${build.dir}"/>
     <mkdir dir="${build.classes}"/>
     <mkdir dir="${build.dir.hive}/jexl/classes"/>
+    <mkdir dir="${build.dir.hadoop}"/>
     <mkdir dir="${test.build.dir}"/>
     <mkdir dir="${test.build.src}"/>
     <mkdir dir="${test.build.classes}"/>
@@ -121,7 +141,7 @@
     </copy>
   </target>
 
-  <target name="compile" depends="init">
+  <target name="compile" depends="init, resolve, install-hadoopcore">
     <echo message="Compiling: ${name}"/>
     <javac
      encoding="${build.encoding}"
@@ -220,6 +240,7 @@
       <!--
       <jvmarg value="-Xdebug"/>
       <jvmarg value="-Xrunjdwp:transport=dt_socket,address=8000,server=y,suspend=y"/> -->
+      <env key="HADOOP_HOME" value="${hadoop.root}"/>
       <sysproperty key="test.output.overwrite" value="${overwrite}"/>
       <sysproperty key="test.service.standalone.server" value="${standalone}"/>
       <sysproperty key="log4j.configuration" value="file://${test.data.dir}/conf/hive-log4j.properties"/>

Added: hadoop/hive/trunk/cli/ivy.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/cli/ivy.xml?rev=724473&view=auto
==============================================================================
--- hadoop/hive/trunk/cli/ivy.xml (added)
+++ hadoop/hive/trunk/cli/ivy.xml Mon Dec  8 12:35:28 2008
@@ -0,0 +1,8 @@
+<ivy-module version="2.0">
+    <info organisation="org.apache.hadoop.hive" module="cli"/>
+    <dependencies>
+        <dependency org="hadoop" name="core" rev="${hadoop.version}">
+          <artifact name="hadoop" type="source" ext="tar.gz"/>
+        </dependency> 
+    </dependencies>
+</ivy-module>

Added: hadoop/hive/trunk/common/ivy.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/common/ivy.xml?rev=724473&view=auto
==============================================================================
--- hadoop/hive/trunk/common/ivy.xml (added)
+++ hadoop/hive/trunk/common/ivy.xml Mon Dec  8 12:35:28 2008
@@ -0,0 +1,8 @@
+<ivy-module version="2.0">
+    <info organisation="org.apache.hadoop.hive" module="common"/>
+    <dependencies>
+        <dependency org="hadoop" name="core" rev="${hadoop.version}">
+          <artifact name="hadoop" type="source" ext="tar.gz"/>
+        </dependency> 
+    </dependencies>
+</ivy-module>

Modified: hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Mon Dec  8 12:35:28 2008
@@ -53,7 +53,7 @@
 
     // hadoop stuff
     HADOOPBIN("hadoop.bin.path", System.getenv("HADOOP_HOME") + "/bin/hadoop"),
-    HADOOPCONF("hadoop.config.dir", System.getProperty("user.dir") + "/../../../conf"),
+    HADOOPCONF("hadoop.config.dir", System.getenv("HADOOP_HOME") + "/conf"),
     HADOOPFS("fs.default.name", "file:///"),
     HADOOPMAPFILENAME("map.input.file", null),
     HADOOPJT("mapred.job.tracker", "local"),

Modified: hadoop/hive/trunk/conf/hive-default.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/conf/hive-default.xml?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/conf/hive-default.xml (original)
+++ hadoop/hive/trunk/conf/hive-default.xml Mon Dec  8 12:35:28 2008
@@ -9,21 +9,6 @@
 <!-- users do not have to edit hadoop configuration files (that may be managed as a centralized -->
 <!-- resource).                                                                                 -->
 
-<!-- Hadoop Setup -->
-<property>
-  <name>hadoop.bin.path</name>
-  <value>${user.dir}/hadoopcore/bin/hadoop</value>
-  <!-- note that the hive shell script also uses this property name -->
-  <description>Path to hadoop binary. Assumes that by default we are executing from hive</description>
-</property>
-
-<property>
-  <name>hadoop.config.dir</name>
-  <value>${user.dir}/hadoopcore/conf</value>
-  <!-- note that the hive shell script also uses this property name -->
-  <description>Path to hadoop configuration. Again assumes that by default we are executing from hive/</description>
-</property>
-
 <!-- Hive Execution Parameters -->
 <property>
   <name>hive.exec.scratchdir</name>

Modified: hadoop/hive/trunk/data/conf/hive-site.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/data/conf/hive-site.xml?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/data/conf/hive-site.xml (original)
+++ hadoop/hive/trunk/data/conf/hive-site.xml Mon Dec  8 12:35:28 2008
@@ -9,20 +9,6 @@
 <!-- users do not have to edit hadoop configuration files (that may be managed as a centralized -->
 <!-- resource).                                                                                 -->
 
-<!-- Hadoop Setup -->
-
-<property>
-  <name>hadoop.bin.path</name>
-  <value>${user.dir}/../hadoopcore/bin/hadoop</value>
-  <description>Path to hadoop binary. Assumes that by default we are executing from hive/</description>
-</property>
-
-<property>
-  <name>hadoop.config.dir</name>
-  <value>${user.dir}/../hadoopcore/conf</value>
-  <description>Path to hadoop configuration. Again assumes that by default we are executing from hive/</description>
-</property>
-
 <!-- Hive Execution Parameters -->
 <property>
   <name>hive.exec.scratchdir</name>

Modified: hadoop/hive/trunk/hadoopcore/bin/hadoop
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/hadoop?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/hadoop (original)
+++ hadoop/hive/trunk/hadoopcore/bin/hadoop Mon Dec  8 12:35:28 2008
@@ -1,273 +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.
-
-
-# The Hadoop command script
-#
-# Environment Variables
-#
-#   JAVA_HOME        The java implementation to use.  Overrides JAVA_HOME.
-#
-#   HADOOP_CLASSPATH Extra Java CLASSPATH entries.
-#
-#   HADOOP_HEAPSIZE  The maximum amount of heap to use, in MB. 
-#                    Default is 1000.
-#
-#   HADOOP_OPTS      Extra Java runtime options.
-#   
-#   HADOOP_NAMENODE_OPTS       These options are added to HADOOP_OPTS 
-#   HADOOP_CLIENT_OPTS         when the respective command is run.
-#   HADOOP_{COMMAND}_OPTS etc  HADOOP_JT_OPTS applies to JobTracker 
-#                              for e.g.  HADOOP_CLIENT_OPTS applies to 
-#                              more than one command (fs, dfs, fsck, 
-#                              dfsadmin etc)  
-#
-#   HADOOP_CONF_DIR  Alternate conf dir. Default is ${HADOOP_HOME}/conf.
-#
-#   HADOOP_ROOT_LOGGER The root appender. Default is INFO,console
-#
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin"/hadoop-config.sh
-
-cygwin=false
-case "`uname`" in
-CYGWIN*) cygwin=true;;
-esac
-
-# if no args specified, show usage
-if [ $# = 0 ]; then
-  echo "Usage: hadoop [--config confdir] COMMAND"
-  echo "where COMMAND is one of:"
-  echo "  namenode -format     format the DFS filesystem"
-  echo "  secondarynamenode    run the DFS secondary namenode"
-  echo "  namenode             run the DFS namenode"
-  echo "  datanode             run a DFS datanode"
-  echo "  dfsadmin             run a DFS admin client"
-  echo "  fsck                 run a DFS filesystem checking utility"
-  echo "  fs                   run a generic filesystem user client"
-  echo "  balancer             run a cluster balancing utility"
-  echo "  jobtracker           run the MapReduce job Tracker node" 
-  echo "  pipes                run a Pipes job"
-  echo "  tasktracker          run a MapReduce task Tracker node" 
-  echo "  job                  manipulate MapReduce jobs"
-  echo "  queue                get information regarding JobQueues" 
-  echo "  version              print the version"
-  echo "  jar <jar>            run a jar file"
-  echo "  distcp <srcurl> <desturl> copy file or directories recursively"
-  echo "  archive -archiveName NAME <src>* <dest> create a hadoop archive"
-  echo "  daemonlog            get/set the log level for each daemon"
-  echo " or"
-  echo "  CLASSNAME            run the class named CLASSNAME"
-  echo "Most commands print help when invoked w/o parameters."
-  exit 1
-fi
-
-# get arguments
-COMMAND=$1
-shift
-
-if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
-  . "${HADOOP_CONF_DIR}/hadoop-env.sh"
-fi
-
-# some Java parameters
-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
-JAVA_HEAP_MAX=-Xmx1000m 
-
-# check envvars which might override default args
-if [ "$HADOOP_HEAPSIZE" != "" ]; then
-  #echo "run with heapsize $HADOOP_HEAPSIZE"
-  JAVA_HEAP_MAX="-Xmx""$HADOOP_HEAPSIZE""m"
-  #echo $JAVA_HEAP_MAX
-fi
-
-# CLASSPATH initially contains $HADOOP_CONF_DIR
-CLASSPATH="${HADOOP_CONF_DIR}"
-CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/tools.jar
-
-# for developers, add Hadoop classes to CLASSPATH
-if [ -d "$HADOOP_HOME/build/classes" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/classes
-fi
-if [ -d "$HADOOP_HOME/build/webapps" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build
-fi
-if [ -d "$HADOOP_HOME/build/test/classes" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/test/classes
-fi
-if [ -d "$HADOOP_HOME/build/tools" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/tools
-fi
-
-# so that filenames w/ spaces are handled correctly in loops below
-IFS=
-
-# for releases, add core hadoop jar & webapps to CLASSPATH
-if [ -d "$HADOOP_HOME/webapps" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME
-fi
-for f in $HADOOP_HOME/hadoop-*-core.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
-
-# add libs to CLASSPATH
-for f in $HADOOP_HOME/lib/*.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
-
-for f in $HADOOP_HOME/lib/jetty-ext/*.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
-
-for f in $HADOOP_HOME/hadoop-*-tools.jar; do
-  TOOL_PATH=${TOOL_PATH}:$f;
-done
-for f in $HADOOP_HOME/build/hadoop-*-tools.jar; do
-  TOOL_PATH=${TOOL_PATH}:$f;
-done
-
-# add user-specified CLASSPATH last
-if [ "$HADOOP_CLASSPATH" != "" ]; then
-  CLASSPATH=${CLASSPATH}:${HADOOP_CLASSPATH}
-fi
-
-# default log directory & file
-if [ "$HADOOP_LOG_DIR" = "" ]; then
-  HADOOP_LOG_DIR="$HADOOP_HOME/logs"
-fi
-if [ "$HADOOP_LOGFILE" = "" ]; then
-  HADOOP_LOGFILE='hadoop.log'
-fi
-
-# restore ordinary behaviour
-unset IFS
-
-# figure out which class to run
-if [ "$COMMAND" = "namenode" ] ; then
-  CLASS='org.apache.hadoop.hdfs.server.namenode.NameNode'
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_NAMENODE_OPTS"
-elif [ "$COMMAND" = "secondarynamenode" ] ; then
-  CLASS='org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode'
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_SECONDARYNAMENODE_OPTS"
-elif [ "$COMMAND" = "datanode" ] ; then
-  CLASS='org.apache.hadoop.hdfs.server.datanode.DataNode'
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_DATANODE_OPTS"
-elif [ "$COMMAND" = "fs" ] ; then
-  CLASS=org.apache.hadoop.fs.FsShell
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-elif [ "$COMMAND" = "dfs" ] ; then
-  CLASS=org.apache.hadoop.fs.FsShell
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-elif [ "$COMMAND" = "dfsadmin" ] ; then
-  CLASS=org.apache.hadoop.hdfs.tools.DFSAdmin
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-elif [ "$COMMAND" = "fsck" ] ; then
-  CLASS=org.apache.hadoop.hdfs.tools.DFSck
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-elif [ "$COMMAND" = "balancer" ] ; then
-  CLASS=org.apache.hadoop.hdfs.server.balancer.Balancer
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_BALANCER_OPTS"
-elif [ "$COMMAND" = "jobtracker" ] ; then
-  CLASS=org.apache.hadoop.mapred.JobTracker
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_JOBTRACKER_OPTS"
-elif [ "$COMMAND" = "tasktracker" ] ; then
-  CLASS=org.apache.hadoop.mapred.TaskTracker
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_TASKTRACKER_OPTS"
-elif [ "$COMMAND" = "job" ] ; then
-  CLASS=org.apache.hadoop.mapred.JobClient
-elif [ "$COMMAND" = "queue" ] ; then
-  CLASS=org.apache.hadoop.mapred.JobQueueClient
-elif [ "$COMMAND" = "pipes" ] ; then
-  CLASS=org.apache.hadoop.mapred.pipes.Submitter
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-elif [ "$COMMAND" = "version" ] ; then
-  CLASS=org.apache.hadoop.util.VersionInfo
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-elif [ "$COMMAND" = "jar" ] ; then
-  CLASS=org.apache.hadoop.mapred.JobShell
-elif [ "$COMMAND" = "distcp" ] ; then
-  CLASS=org.apache.hadoop.tools.DistCp
-  CLASSPATH=${CLASSPATH}:${TOOL_PATH}
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-elif [ "$COMMAND" = "daemonlog" ] ; then
-  CLASS=org.apache.hadoop.log.LogLevel
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-elif [ "$COMMAND" = "archive" ] ; then
-  CLASS=org.apache.hadoop.tools.HadoopArchives
-  CLASSPATH=${CLASSPATH}:${TOOL_PATH}
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-elif [ "$COMMAND" = "sampler" ] ; then
-  CLASS=org.apache.hadoop.mapred.lib.InputSampler
-  HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
-else
-  CLASS=$COMMAND
-fi
-
-# cygwin path translation
-if $cygwin; then
-  CLASSPATH=`cygpath -p -w "$CLASSPATH"`
-  HADOOP_HOME=`cygpath -d "$HADOOP_HOME"`
-  HADOOP_LOG_DIR=`cygpath -d "$HADOOP_LOG_DIR"`
-  TOOL_PATH=`cygpath -p -w "$TOOL_PATH"`
-fi
-# setup 'java.library.path' for native-hadoop code if necessary
-JAVA_LIBRARY_PATH=''
-if [ -d "${HADOOP_HOME}/build/native" -o -d "${HADOOP_HOME}/lib/native" ]; then
-  JAVA_PLATFORM=`CLASSPATH=${CLASSPATH} ${JAVA} org.apache.hadoop.util.PlatformName | sed -e "s/ /_/g"`
-  
-  if [ -d "$HADOOP_HOME/build/native" ]; then
-    JAVA_LIBRARY_PATH=${HADOOP_HOME}/build/native/${JAVA_PLATFORM}/lib
-  fi
-  
-  if [ -d "${HADOOP_HOME}/lib/native" ]; then
-    if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
-      JAVA_LIBRARY_PATH=${JAVA_LIBRARY_PATH}:${HADOOP_HOME}/lib/native/${JAVA_PLATFORM}
-    else
-      JAVA_LIBRARY_PATH=${HADOOP_HOME}/lib/native/${JAVA_PLATFORM}
-    fi
-  fi
-fi
-
-# cygwin path translation
-if $cygwin; then
-  JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"`
-fi
-
-HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.log.dir=$HADOOP_LOG_DIR"
-HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.log.file=$HADOOP_LOGFILE"
-HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.home.dir=$HADOOP_HOME"
-HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.id.str=$HADOOP_IDENT_STRING"
-HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.root.logger=${HADOOP_ROOT_LOGGER:-INFO,console}"
-if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
-  HADOOP_OPTS="$HADOOP_OPTS -Djava.library.path=$JAVA_LIBRARY_PATH"
-fi  
-
-# run it
-exec "$JAVA" $JAVA_HEAP_MAX $HADOOP_OPTS -classpath "$CLASSPATH" $CLASS "$@"

Modified: hadoop/hive/trunk/hadoopcore/bin/hadoop-config.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/hadoop-config.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/hadoop-config.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/hadoop-config.sh Mon Dec  8 12:35:28 2008
@@ -1,68 +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 hadoop 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="$0"
-while [ -h "$this" ]; do
-  ls=`ls -ld "$this"`
-  link=`expr "$ls" : '.*-> \(.*\)$'`
-  if expr "$link" : '.*/.*' > /dev/null; then
-    this="$link"
-  else
-    this=`dirname "$this"`/"$link"
-  fi
-done
-
-# convert relative path to absolute path
-bin=`dirname "$this"`
-script=`basename "$this"`
-bin=`cd "$bin"; pwd`
-this="$bin/$script"
-
-# the root of the Hadoop installation
-export HADOOP_HOME=`dirname "$this"`/..
-
-#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
-	      HADOOP_CONF_DIR=$confdir
-    fi
-fi
- 
-# Allow alternate conf dir location.
-HADOOP_CONF_DIR="${HADOOP_CONF_DIR:-$HADOOP_HOME/conf}"
-
-#check to see it is specified whether to use the slaves or the
-# masters file
-if [ $# -gt 1 ]
-then
-    if [ "--hosts" = "$1" ]
-    then
-        shift
-        slavesfile=$1
-        shift
-        export HADOOP_SLAVES="${HADOOP_CONF_DIR}/$slavesfile"
-    fi
-fi

Modified: hadoop/hive/trunk/hadoopcore/bin/hadoop-daemon.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/hadoop-daemon.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/hadoop-daemon.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/hadoop-daemon.sh Mon Dec  8 12:35:28 2008
@@ -1,143 +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 Hadoop command as a daemon.
-#
-# Environment Variables
-#
-#   HADOOP_CONF_DIR  Alternate conf dir. Default is ${HADOOP_HOME}/conf.
-#   HADOOP_LOG_DIR   Where log files are stored.  PWD by default.
-#   HADOOP_MASTER    host:path where hadoop code should be rsync'd from
-#   HADOOP_PID_DIR   The pid files are stored. /tmp by default.
-#   HADOOP_IDENT_STRING   A string representing this instance of hadoop. $USER by default
-#   HADOOP_NICENESS The scheduling priority for daemons. Defaults to 0.
-##
-
-usage="Usage: hadoop-daemon.sh [--config <conf-dir>] [--hosts hostlistfile] (start|stop) <hadoop-command> <args...>"
-
-# if no args specified, show usage
-if [ $# -le 1 ]; then
-  echo $usage
-  exit 1
-fi
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin"/hadoop-config.sh
-
-# get arguments
-startStop=$1
-shift
-command=$1
-shift
-
-hadoop_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 "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
-  . "${HADOOP_CONF_DIR}/hadoop-env.sh"
-fi
-
-# get log directory
-if [ "$HADOOP_LOG_DIR" = "" ]; then
-  export HADOOP_LOG_DIR="$HADOOP_HOME/logs"
-fi
-mkdir -p "$HADOOP_LOG_DIR"
-
-if [ "$HADOOP_PID_DIR" = "" ]; then
-  HADOOP_PID_DIR=/tmp
-fi
-
-if [ "$HADOOP_IDENT_STRING" = "" ]; then
-  export HADOOP_IDENT_STRING="$USER"
-fi
-
-# some variables
-export HADOOP_LOGFILE=hadoop-$HADOOP_IDENT_STRING-$command-$HOSTNAME.log
-export HADOOP_ROOT_LOGGER="INFO,DRFA"
-log=$HADOOP_LOG_DIR/hadoop-$HADOOP_IDENT_STRING-$command-$HOSTNAME.out
-pid=$HADOOP_PID_DIR/hadoop-$HADOOP_IDENT_STRING-$command.pid
-
-# Set default scheduling priority
-if [ "$HADOOP_NICENESS" = "" ]; then
-    export HADOOP_NICENESS=0
-fi
-
-case $startStop in
-
-  (start)
-
-    mkdir -p "$HADOOP_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 [ "$HADOOP_MASTER" != "" ]; then
-      echo rsync from $HADOOP_MASTER
-      rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $HADOOP_MASTER/ "$HADOOP_HOME"
-    fi
-
-    hadoop_rotate_log $log
-    echo starting $command, logging to $log
-    cd "$HADOOP_HOME"
-    nohup nice -n $HADOOP_NICENESS "$HADOOP_HOME"/bin/hadoop --config $HADOOP_CONF_DIR $command "$@" > "$log" 2>&1 < /dev/null &
-    echo $! > $pid
-    sleep 1; head "$log"
-    ;;
-          
-  (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
-
-

Modified: hadoop/hive/trunk/hadoopcore/bin/hadoop-daemons.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/hadoop-daemons.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/hadoop-daemons.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/hadoop-daemons.sh Mon Dec  8 12:35:28 2008
@@ -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.
-
-
-# Run a Hadoop command on all slave hosts.
-
-usage="Usage: hadoop-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command args..."
-
-# if no args specified, show usage
-if [ $# -le 1 ]; then
-  echo $usage
-  exit 1
-fi
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. $bin/hadoop-config.sh
-
-exec "$bin/slaves.sh" --config $HADOOP_CONF_DIR cd "$HADOOP_HOME" \; "$bin/hadoop-daemon.sh" --config $HADOOP_CONF_DIR "$@"

Modified: hadoop/hive/trunk/hadoopcore/bin/rcc
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/rcc?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/rcc (original)
+++ hadoop/hive/trunk/hadoopcore/bin/rcc Mon Dec  8 12:35:28 2008
@@ -1,99 +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.
-
-
-# The Hadoop record compiler
-#
-# Environment Variables
-#
-#   JAVA_HOME        The java implementation to use.  Overrides JAVA_HOME.
-#
-#   HADOOP_OPTS      Extra Java runtime options.
-#
-#   HADOOP_CONF_DIR  Alternate conf dir. Default is ${HADOOP_HOME}/conf.
-#
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin"/hadoop-config.sh
-
-if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
-  . "${HADOOP_CONF_DIR}/hadoop-env.sh"
-fi
-
-# some Java parameters
-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
-JAVA_HEAP_MAX=-Xmx1000m 
-
-# CLASSPATH initially contains $HADOOP_CONF_DIR
-CLASSPATH="${HADOOP_CONF_DIR}"
-CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/tools.jar
-
-# for developers, add Hadoop classes to CLASSPATH
-if [ -d "$HADOOP_HOME/build/classes" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/classes
-fi
-if [ -d "$HADOOP_HOME/build/webapps" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build
-fi
-if [ -d "$HADOOP_HOME/build/test/classes" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/test/classes
-fi
-
-# so that filenames w/ spaces are handled correctly in loops below
-IFS=
-
-# for releases, add core hadoop jar & webapps to CLASSPATH
-if [ -d "$HADOOP_HOME/webapps" ]; then
-  CLASSPATH=${CLASSPATH}:$HADOOP_HOME
-fi
-for f in $HADOOP_HOME/hadoop-*-core.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
-
-# add libs to CLASSPATH
-for f in $HADOOP_HOME/lib/*.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
-
-for f in $HADOOP_HOME/lib/jetty-ext/*.jar; do
-  CLASSPATH=${CLASSPATH}:$f;
-done
-
-# restore ordinary behaviour
-unset IFS
-
-CLASS='org.apache.hadoop.record.compiler.generated.Rcc'
-
-# cygwin path translation
-if expr `uname` : 'CYGWIN*' > /dev/null; then
-  CLASSPATH=`cygpath -p -w "$CLASSPATH"`
-fi
-
-# run it
-exec "$JAVA" $HADOOP_OPTS -classpath "$CLASSPATH" $CLASS "$@"

Modified: hadoop/hive/trunk/hadoopcore/bin/slaves.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/slaves.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/slaves.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/slaves.sh Mon Dec  8 12:35:28 2008
@@ -1,68 +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
-#
-#   HADOOP_SLAVES    File naming remote hosts.
-#     Default is ${HADOOP_CONF_DIR}/slaves.
-#   HADOOP_CONF_DIR  Alternate conf dir. Default is ${HADOOP_HOME}/conf.
-#   HADOOP_SLAVE_SLEEP Seconds to sleep between spawning remote commands.
-#   HADOOP_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"/hadoop-config.sh
-
-# If the slaves file is specified in the command line,
-# then it takes precedence over the definition in 
-# hadoop-env.sh. Save it here.
-HOSTLIST=$HADOOP_SLAVES
-
-if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
-  . "${HADOOP_CONF_DIR}/hadoop-env.sh"
-fi
-
-if [ "$HOSTLIST" = "" ]; then
-  if [ "$HADOOP_SLAVES" = "" ]; then
-    export HOSTLIST="${HADOOP_CONF_DIR}/slaves"
-  else
-    export HOSTLIST="${HADOOP_SLAVES}"
-  fi
-fi
-
-for slave in `cat "$HOSTLIST"|sed  "s/#.*$//;/^$/d"`; do
- ssh $HADOOP_SSH_OPTS $slave $"${@// /\\ }" \
-   2>&1 | sed "s/^/$slave: /" &
- if [ "$HADOOP_SLAVE_SLEEP" != "" ]; then
-   sleep $HADOOP_SLAVE_SLEEP
- fi
-done
-
-wait

Modified: hadoop/hive/trunk/hadoopcore/bin/start-all.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/start-all.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/start-all.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/start-all.sh Mon Dec  8 12:35:28 2008
@@ -1,30 +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 hadoop daemons.  Run this on master node.
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin"/hadoop-config.sh
-
-# start dfs daemons
-"$bin"/start-dfs.sh --config $HADOOP_CONF_DIR
-
-# start mapred daemons
-"$bin"/start-mapred.sh --config $HADOOP_CONF_DIR

Modified: hadoop/hive/trunk/hadoopcore/bin/start-balancer.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/start-balancer.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/start-balancer.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/start-balancer.sh Mon Dec  8 12:35:28 2008
@@ -1,25 +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"/hadoop-config.sh
-
-# Start balancer daemon.
-
-"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR start balancer $@

Modified: hadoop/hive/trunk/hadoopcore/bin/start-dfs.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/start-dfs.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/start-dfs.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/start-dfs.sh Mon Dec  8 12:35:28 2008
@@ -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.
-
-
-# Start hadoop dfs daemons.
-# Optinally upgrade or rollback dfs state.
-# Run this on master node.
-
-usage="Usage: start-dfs.sh [-upgrade|-rollback]"
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin"/hadoop-config.sh
-
-# get arguments
-if [ $# -ge 1 ]; then
-	nameStartOpt=$1
-	shift
-	case $nameStartOpt in
-	  (-upgrade)
-	  	;;
-	  (-rollback) 
-	  	dataStartOpt=$nameStartOpt
-	  	;;
-	  (*)
-		  echo $usage
-		  exit 1
-	    ;;
-	esac
-fi
-
-# start dfs daemons
-# start namenode after datanodes, to minimize time namenode is up w/o data
-# note: datanodes will log connection errors until namenode starts
-"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR start namenode $nameStartOpt
-"$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR start datanode $dataStartOpt
-"$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR --hosts masters start secondarynamenode

Modified: hadoop/hive/trunk/hadoopcore/bin/start-mapred.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/start-mapred.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/start-mapred.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/start-mapred.sh Mon Dec  8 12:35:28 2008
@@ -1,29 +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 hadoop map reduce daemons.  Run this on master node.
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin"/hadoop-config.sh
-
-# start mapred daemons
-# start jobtracker first to minimize connection errors at startup
-"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR start jobtracker
-"$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR start tasktracker

Modified: hadoop/hive/trunk/hadoopcore/bin/stop-all.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/stop-all.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/stop-all.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/stop-all.sh Mon Dec  8 12:35:28 2008
@@ -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.
-
-
-# Stop all hadoop daemons.  Run this on master node.
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin"/hadoop-config.sh
-
-"$bin"/stop-mapred.sh --config $HADOOP_CONF_DIR
-"$bin"/stop-dfs.sh --config $HADOOP_CONF_DIR

Modified: hadoop/hive/trunk/hadoopcore/bin/stop-balancer.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/stop-balancer.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/stop-balancer.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/stop-balancer.sh Mon Dec  8 12:35:28 2008
@@ -1,26 +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"/hadoop-config.sh
-
-# Stop balancer daemon.
-# Run this on the machine where the balancer is running
-
-"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop balancer

Modified: hadoop/hive/trunk/hadoopcore/bin/stop-dfs.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/stop-dfs.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/stop-dfs.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/stop-dfs.sh Mon Dec  8 12:35:28 2008
@@ -1,29 +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.
-
-
-# Stop hadoop DFS daemons.  Run this on master node.
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin"/hadoop-config.sh
-
-"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop namenode
-"$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR stop datanode
-"$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR --hosts masters stop secondarynamenode
-

Modified: hadoop/hive/trunk/hadoopcore/bin/stop-mapred.sh
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/bin/stop-mapred.sh?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/bin/stop-mapred.sh (original)
+++ hadoop/hive/trunk/hadoopcore/bin/stop-mapred.sh Mon Dec  8 12:35:28 2008
@@ -1,28 +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.
-
-
-# Stop hadoop map reduce daemons.  Run this on master node.
-
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
-
-. "$bin"/hadoop-config.sh
-
-"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR stop jobtracker
-"$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR stop tasktracker
-

Modified: hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml (original)
+++ hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml Mon Dec  8 12:35:28 2008
@@ -1,77 +0,0 @@
-<?xml version="1.0"?>
-
-<!-- This is the configuration file for the resource manager in Hadoop. -->
-<!-- You can configure various scheduling parameters related to queues. -->
-<!-- The properties for a queue follow a naming convention,such as, -->
-<!-- mapred.capacity-scheduler.queue.<queue-name>.property-name. -->
-
-<configuration>
-
-  <property>
-    <name>mapred.capacity-scheduler.queue.default.guaranteed-capacity</name>
-    <value>100</value>
-    <description>Percentage of the number of slots in the cluster that are
-      guaranteed to be available for jobs in this queue.
-    </description>    
-  </property>
-  
-  <property>
-    <name>mapred.capacity-scheduler.queue.default.reclaim-time-limit</name>
-    <value>300</value>
-    <description>The amount of time, in seconds, before which 
-      resources distributed to other queues will be reclaimed.
-    </description>
-  </property>
-
-  <property>
-    <name>mapred.capacity-scheduler.queue.default.supports-priority</name>
-    <value>false</value>
-    <description>If true, priorities of jobs will be taken into 
-      account in scheduling decisions.
-    </description>
-  </property>
-
-  <property>
-    <name>mapred.capacity-scheduler.queue.default.minimum-user-limit-percent</name>
-    <value>100</value>
-    <description> Each queue enforces a limit on the percentage of resources 
-    allocated to a user at any given time, if there is competition for them. 
-    This user limit can vary between a minimum and maximum value. The former
-    depends on the number of users who have submitted jobs, and the latter is
-    set to this property value. For example, suppose the value of this 
-    property is 25. If two users have submitted jobs to a queue, no single 
-    user can use more than 50% of the queue resources. If a third user submits
-    a job, no single user can use more than 33% of the queue resources. With 4 
-    or more users, no user can use more than 25% of the queue's resources. A 
-    value of 100 implies no user limits are imposed. 
-    </description>
-  </property>
-  
-  <!-- The default configuration settings for the capacity task scheduler -->
-  <!-- The default values would be applied to all the queues which don't have -->
-  <!-- the appropriate property for the particular queue -->
-  <property>
-    <name>mapred.capacity-scheduler.default-reclaim-time-limit</name>
-    <value>300</value>
-    <description>The amount of time, in seconds, before which 
-    resources distributed to other queues will be reclaimed by default
-    in a job queue.
-    </description>
-  </property>
-  
-  <property>
-    <name>mapred.capacity-scheduler.default-supports-priority</name>
-    <value>false</value>
-    <description>If true, priorities of jobs will be taken into 
-      account in scheduling decisions by default in a job queue.
-    </description>
-  </property>
-  
-  <property>
-    <name>mapred.capacity-scheduler.default-minimum-user-limit-percent</name>
-    <value>100</value>
-    <description>The percentage of the resources limited to a particular user
-      for the job queue at any given point of time by default.
-    </description>
-  </property>
-</configuration>

Modified: hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml.template
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml.template?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml.template (original)
+++ hadoop/hive/trunk/hadoopcore/conf/capacity-scheduler.xml.template Mon Dec  8 12:35:28 2008
@@ -1,77 +0,0 @@
-<?xml version="1.0"?>
-
-<!-- This is the configuration file for the resource manager in Hadoop. -->
-<!-- You can configure various scheduling parameters related to queues. -->
-<!-- The properties for a queue follow a naming convention,such as, -->
-<!-- mapred.capacity-scheduler.queue.<queue-name>.property-name. -->
-
-<configuration>
-
-  <property>
-    <name>mapred.capacity-scheduler.queue.default.guaranteed-capacity</name>
-    <value>100</value>
-    <description>Percentage of the number of slots in the cluster that are
-      guaranteed to be available for jobs in this queue.
-    </description>    
-  </property>
-  
-  <property>
-    <name>mapred.capacity-scheduler.queue.default.reclaim-time-limit</name>
-    <value>300</value>
-    <description>The amount of time, in seconds, before which 
-      resources distributed to other queues will be reclaimed.
-    </description>
-  </property>
-
-  <property>
-    <name>mapred.capacity-scheduler.queue.default.supports-priority</name>
-    <value>false</value>
-    <description>If true, priorities of jobs will be taken into 
-      account in scheduling decisions.
-    </description>
-  </property>
-
-  <property>
-    <name>mapred.capacity-scheduler.queue.default.minimum-user-limit-percent</name>
-    <value>100</value>
-    <description> Each queue enforces a limit on the percentage of resources 
-    allocated to a user at any given time, if there is competition for them. 
-    This user limit can vary between a minimum and maximum value. The former
-    depends on the number of users who have submitted jobs, and the latter is
-    set to this property value. For example, suppose the value of this 
-    property is 25. If two users have submitted jobs to a queue, no single 
-    user can use more than 50% of the queue resources. If a third user submits
-    a job, no single user can use more than 33% of the queue resources. With 4 
-    or more users, no user can use more than 25% of the queue's resources. A 
-    value of 100 implies no user limits are imposed. 
-    </description>
-  </property>
-  
-  <!-- The default configuration settings for the capacity task scheduler -->
-  <!-- The default values would be applied to all the queues which don't have -->
-  <!-- the appropriate property for the particular queue -->
-  <property>
-    <name>mapred.capacity-scheduler.default-reclaim-time-limit</name>
-    <value>300</value>
-    <description>The amount of time, in seconds, before which 
-    resources distributed to other queues will be reclaimed by default
-    in a job queue.
-    </description>
-  </property>
-  
-  <property>
-    <name>mapred.capacity-scheduler.default-supports-priority</name>
-    <value>false</value>
-    <description>If true, priorities of jobs will be taken into 
-      account in scheduling decisions by default in a job queue.
-    </description>
-  </property>
-  
-  <property>
-    <name>mapred.capacity-scheduler.default-minimum-user-limit-percent</name>
-    <value>100</value>
-    <description>The percentage of the resources limited to a particular user
-      for the job queue at any given point of time by default.
-    </description>
-  </property>
-</configuration>

Modified: hadoop/hive/trunk/hadoopcore/conf/configuration.xsl
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hadoopcore/conf/configuration.xsl?rev=724473&r1=724472&r2=724473&view=diff
==============================================================================
--- hadoop/hive/trunk/hadoopcore/conf/configuration.xsl (original)
+++ hadoop/hive/trunk/hadoopcore/conf/configuration.xsl Mon Dec  8 12:35:28 2008
@@ -1,24 +0,0 @@
-<?xml version="1.0"?>
-<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
-<xsl:output method="html"/>
-<xsl:template match="configuration">
-<html>
-<body>
-<table border="1">
-<tr>
- <td>name</td>
- <td>value</td>
- <td>description</td>
-</tr>
-<xsl:for-each select="property">
-<tr>
-  <td><a name="{name}"><xsl:value-of select="name"/></a></td>
-  <td><xsl:value-of select="value"/></td>
-  <td><xsl:value-of select="description"/></td>
-</tr>
-</xsl:for-each>
-</table>
-</body>
-</html>
-</xsl:template>
-</xsl:stylesheet>