MAPREDUCE-233 sync branch with trunk
git-svn-id: https://svn.apache.org/repos/asf/hadoop/mapreduce/branches/MAPREDUCE-233@903227 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/.eclipse.templates/.classpath b/.eclipse.templates/.classpath
index aff92ca..9bd0107 100644
--- a/.eclipse.templates/.classpath
+++ b/.eclipse.templates/.classpath
@@ -2,6 +2,7 @@
<classpath>
<classpathentry kind="src" path="src/java"/>
<classpathentry kind="src" path="src/test/mapred"/>
+ <classpathentry kind="src" path="src/test/unit"/>
<classpathentry kind="src" path="src/benchmarks/gridmix2/src/java"/>
<classpathentry kind="src" path="src/examples"/>
<classpathentry kind="src" path="src/tools"/>
@@ -25,17 +26,16 @@
<classpathentry kind="src" path="src/contrib/vaidya/src/java"/>
<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER"/>
<classpathentry kind="var" path="ANT_HOME/lib/ant.jar"/>
- <classpathentry kind="lib" path="lib/hadoop-core-0.22.0-dev.jar"/>
- <classpathentry kind="lib" path="lib/hadoop-core-test-0.22.0-dev.jar"/>
- <classpathentry kind="lib" path="lib/hadoop-hdfs-0.22.0-dev.jar"/>
- <classpathentry kind="lib" path="lib/hadoop-hdfs-test-0.22.0-dev.jar"/>
+ <classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/hadoop-core-0.22.0-SNAPSHOT.jar"/>
+ <classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/hadoop-core-test-0.22.0-SNAPSHOT.jar"/>
+ <classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/hadoop-hdfs-0.22.0-SNAPSHOT.jar"/>
+ <classpathentry kind="lib" path="build/ivy/lib/Hadoop/test/hadoop-hdfs-test-0.22.0-SNAPSHOT.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/avro-1.2.0.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/commons-cli-1.2.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/commons-codec-1.3.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/commons-el-1.0.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/commons-httpclient-3.0.1.jar"/>
- <classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/commons-logging-1.0.4.jar"/>
- <classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/commons-logging-api-1.0.4.jar"/>
+ <classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/commons-logging-1.1.1.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/commons-net-1.4.1.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/core-3.1.1.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/hsqldb-1.8.0.10.jar"/>
@@ -43,7 +43,7 @@
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/jackson-mapper-asl-1.0.1.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/jasper-compiler-5.5.12.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/jasper-runtime-5.5.12.jar"/>
- <classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/jets3t-0.6.1.jar"/>
+ <classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/jets3t-0.7.1.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/jetty-6.1.14.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/jetty-util-6.1.14.jar"/>
<classpathentry kind="lib" path="build/ivy/lib/Hadoop/common/jsp-2.1-6.1.14.jar"/>
diff --git a/.gitignore b/.gitignore
index f488b16..addaeb1 100644
--- a/.gitignore
+++ b/.gitignore
@@ -36,6 +36,7 @@
conf/mapred-queues.xml
docs/api/
logs/
+src/contrib/capacity-scheduler/src/java/mapred-queues.xml
src/contrib/index/conf/index-config.xml
src/docs/build
src/docs/cn/build
diff --git a/CHANGES.txt b/CHANGES.txt
index 760da6a..96c8e92 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,8 +4,14 @@
INCOMPATIBLE CHANGES
+ MAPREDUCE-1287. Only call the partitioner with more than one reducer.
+ (cdouglas)
+
NEW FEATURES
+ MAPREDUCE-698. Per-pool task limits for the fair scheduler.
+ (Kevin Peterson via matei)
+
MAPREDUCE-1017. Compression and output splitting for Sqoop.
(Aaron Kimball via tomwhite)
@@ -13,6 +19,20 @@
transfers using a shared JobTracker generated key.
(Boris Shkolnik via ddas)
+ MAPREDUCE-1168. Export data to databases via Sqoop. (Aaron Kimball via
+ tomwhite)
+
+ MAPREDUCE-744. Introduces the notion of a public distributed cache.
+ (Devaraj Das)
+
+ MAPREDUCE-1338. Introduces the notion of token cache using which
+ tokens and secrets can be sent by the Job client to the JobTracker.
+ (Boris Shkolnik via ddas)
+
+ HDFS-503. This patch implements an optional layer over HDFS that
+ implements offline erasure-coding. It can be used to reduce the
+ total storage requirements of HDFS. (dhruba)
+
IMPROVEMENTS
MAPREDUCE-1198. Alternatively schedule different types of tasks in
@@ -54,6 +74,62 @@
MAPREDUCE-1231. Added a new DistCp option, -skipcrccheck, so that the CRC
check during setup can be skipped. (Jothi Padmanabhan via szetszwo)
+ MAPREDUCE-1190. Add package documentation for BBP example.
+ (Tsz Wo (Nicholas) Sze via cdouglas)
+
+ MAPREDUCE-1119. When tasks fail to report status, show tasks's stack dump
+ before killing. (Aaron Kimball via tomwhite)
+
+ MAPREDUCE-1185. Redirect running job url to history url if job is already
+ retired. (Amareshwari Sriramadasu and Sharad Agarwal via sharad)
+
+ MAPREDUCE-1050. Introduce a mock object testing framework. (tomwhite)
+
+ MAPREDUCE-1084. Implementing aspects development and fault injeciton
+ framework for MapReduce. (Sreekanth Ramakrishnan via cos)
+
+ MAPREDUCE-1209. Move common specific part of the test TestReflectionUtils
+ out of mapred into common. (Todd Lipcon via tomwhite)
+
+ MAPREDUCE-967. TaskTracker does not need to fully unjar job jars.
+ (Todd Lipcon via tomwhite)
+
+ MAPREDUCE-1083. Changes in MapReduce so that group information of users
+ can be refreshed in the JobTracker via command line.
+ (Boris Shkolnik via ddas)
+
+ MAPREDUCE-181. Changes the job submission process to be secure.
+ (Devaraj Das)
+
+ MAPREDUCE-1250. Refactors the JobToken to use Common's Token interface.
+ (Kan Zhang via ddas)
+
+ MAPREDUCE-896. Enhance tasktracker to cleanup files that might have
+ been created by user tasks with non-writable permissions.
+ (Ravi Gummadi via yhemanth)
+
+ MAPREDUCE-372. Change org.apache.hadoop.mapred.lib.ChainMapper/Reducer
+ to use new mapreduce api. (Amareshwari Sriramadasu via sharad)
+
+ MAPREDUCE-1295. Add a tool in Rumen for folding and manipulating job
+ traces. (Dick King via cdouglas)
+
+ MAPREDUCE-1302. TrackerDistributedCacheManager deletes file
+ asynchronously, thus reducing task initialization delays.
+ (Zheng Shao via dhruba)
+
+ MAPREDUCE-1218. TaskTrackers send cpu and memory usage of
+ node to JobTracker. (Scott Chen via dhruba)
+
+ MAPREDUCE-847. Fix Releaseaudit warning count to zero
+ (Giridharan Kesavan)
+
+ MAPREDUCE-1337. Use generics in StreamJob to improve readability of that
+ class. (Kay Kay via cdouglas)
+
+ MAPREDUCE-361. Port terasort example to the new mapreduce API. (Amareshwari
+ Sriramadasu via cdouglas)
+
OPTIMIZATIONS
MAPREDUCE-270. Fix the tasktracker to optionally send an out-of-band
@@ -61,8 +137,18 @@
Configuration changes:
add mapreduce.tasktracker.outofband.heartbeat
+ MAPREDUCE-1224. Calling "SELECT t.* from <table> AS t" to get meta
+ information is too expensive for big tables. (Spencer Ho via tomwhite)
+
+ MAPREDUCE-1186. Modified code in distributed cache to set permissions
+ only on required set of localized paths.
+ (Amareshwari Sriramadasu via yhemanth)
+
BUG FIXES
+ MAPREDUCE-1258. Fix fair scheduler event log not logging job info.
+ (matei)
+
MAPREDUCE-1089. Fix NPE in fair scheduler preemption when tasks are
scheduled but not running. (Todd Lipcon via matei)
@@ -101,6 +187,75 @@
unreferenced files in error conditions.
(Amareshwari Sriramadasu via yhemanth)
+ MAPREDUCE-1245. Fix TestFairScheduler failures by instantiating lightweight
+ Jobtracker. (sharad)
+
+ MAPREDUCE-1260. Update Eclipse configuration to match changes to Ivy
+ configuration. (Edwin Chan via cos)
+
+ MAPREDUCE-1152. Distinguish between failed and killed tasks in
+ JobTrackerInstrumentation. (Sharad Agarwal via cdouglas)
+
+ MAPREDUCE-1285. In DistCp.deleteNonexisting(..), get class from the
+ parameter instead of using FileStatus.class. (Peter Romianowski via
+ szetszwo)
+
+ MAPREDUCE-1251. c++ utils doesn't compile. (Eli Collins via tomwhite)
+
+ MAPREDUCE-1148. SQL identifiers are a superset of Java identifiers.
+ (Aaron Kimball via tomwhite)
+
+ MAPREDUCE-1294. Build fails to pull latest hadoop-core-* artifacts (cos)
+
+ MAPREDUCE-1213. TaskTrackers restart is faster because it deletes
+ distributed cache directory asynchronously. (Zheng Shao via dhruba)
+
+ MAPREDUCE-1146. Sqoop dependencies break Eclipse build on Linux.
+ (Aaron Kimball via tomwhite)
+
+ MAPREDUCE-1174. Sqoop improperly handles table/column names which are
+ reserved sql words. (Aaron Kimball via tomwhite)
+
+ MAPREDUCE-1265. The task attempt error log prints the name of the
+ tasktracker machine. (Scott Chen via dhruba)
+
+ MAPREDUCE-1201. ProcfsBasedProcessTree collects CPU usage information.
+ (Scott Chen via dhruba)
+
+ MAPREDUCE-1326. fi tests don't use fi-site.xml (cos)
+
+ MAPREDUCE-1235. Fix a MySQL timestamp incompatibility in Sqoop. (Aaron
+ Kimball via cdouglas)
+
+ MAPREDUCE-1165. Replace non-portable function name with C99 equivalent.
+ (Allen Wittenauer via cdouglas)
+
+ MAPREDUCE-1331. Fixes a typo in a testcase (Devaraj Das)
+
+ MAPREDUCE-1293. AutoInputFormat doesn't work with non-default FileSystems.
+ (Andrew Hitchcock via tomwhite)
+
+ MAPREDUCE-1131. Using profilers other than hprof can cause JobClient to
+ report job failure. (Aaron Kimball via tomwhite)
+
+ MAPREDUCE-1155. Streaming tests swallow exceptions.
+ (Todd Lipcon via tomwhite)
+
+ MAPREDUCE-1310. CREATE TABLE statements for Hive do not correctly specify
+ delimiters. (Aaron Kimball via tomwhite)
+
+ MAPREDUCE-1212. Mapreduce contrib project ivy dependencies are not included
+ in binary target. (Aaron Kimball via tomwhite)
+
+ MAPREDUCE-1388. Move the HDFS RAID package from HDFS to MAPREDUCE.
+ (Eli Collins via dhruba)
+
+ MAPREDUCE-1313. Fix NPE in Sqoop when table with null fields uses escape
+ during import. (Aaron Kimball via cdouglas)
+
+ MAPREDUCE-1327. Fix Sqoop handling of Oracle timezone with timestamp data
+ types in import. (Leonid Furman via cdouglas)
+
Release 0.21.0 - Unreleased
INCOMPATIBLE CHANGES
@@ -534,6 +689,15 @@
HADOOP-5107. Use Maven ant tasks to publish artifacts. (Giridharan Kesavan
via omalley)
+ MAPREDUCE-1229. Allow customization of job submission policy in Mumak.
+ (Hong Tang via cdouglas)
+
+ MAPREDUCE-1317. Reduce the memory footprint of Rumen objects by interning
+ host Strings. (Hong Tang via cdouglas)
+
+ MAPREDUCE-1097. Add support for Vertica 3.5 to its contrib module. (Omer
+ Trajman via cdouglas)
+
BUG FIXES
MAPREDUCE-878. Rename fair scheduler design doc to
@@ -931,3 +1095,78 @@
MAPREDUCE-787. Fix JobSubmitter to honor user given symlink path.
(Amareshwari Sriramadasu via sharad)
+ MAPREDUCE-1249. Update config default value for socket read timeout to
+ match code default. (Amareshwari Sriramadasu via cdouglas)
+
+ MAPREDUCE-1161. Remove ineffective synchronization in NotificationTestCase.
+ (Owen O'Malley via cdouglas)
+
+ MAPREDUCE-1244. Fix eclipse-plugin's build dependencies. (gkesavan)
+
+ MAPREDUCE-1075. Fix JobTracker to not throw an NPE for a non-existent
+ queue. (V.V.Chaitanya Krishna via yhemanth)
+
+ MAPREDUCE-754. Fix NPE in expiry thread when a TT is lost. (Amar Kamat
+ via sharad)
+
+ MAPREDUCE-1074. Document Reducer mark/reset functionality. (Jothi
+ Padmanabhan via cdouglas)
+
+ MAPREDUCE-1267. Fix typo in mapred-default.xml. (Todd Lipcon via cdouglas)
+
+ MAPREDUCE-952. Remove inadvertently reintroduced Task.Counter enum. (Jothi
+ Padmanabhan via cdouglas)
+
+ MAPREDUCE-1230. Fix handling of null records in VerticaInputFormat. (Omer
+ Trajman via cdouglas)
+
+ MAPREDUCE-1171. Allow shuffle retries and read-error reporting to be
+ configurable. (Amareshwari Sriramadasu via acmurthy)
+
+ MAPREDUCE-879. Fix broken unit test TestTaskTrackerLocalization on MacOS.
+ (Sreekanth Ramakrishnan via yhemanth)
+
+ MAPREDUCE-1124. Fix imprecise byte counts in Gridmix. (cdouglas)
+
+ MAPREDUCE-1222. Add an option to exclude numeric IP addresses in topologies
+ processed by Mumak. (Hong Tang via cdouglas)
+
+ MAPREDUCE-1284. Fix fts_open() call in task-controller that was failing
+ LinuxTaskController unit tests. (Ravi Gummadi via yhemanth)
+
+ MAPREDUCE-1143. Fix running task counters to be updated correctly
+ when speculative attempts are running for a TIP.
+ (Rahul Kumar Singh via yhemanth)
+
+ MAPREDUCE-1241. Use a default queue configuration in JobTracker when
+ mapred-queues.xml is unavailable. (Todd Lipcon via cdouglas)
+
+ MAPREDUCE-1301. Fix set up of permission checking script used in
+ localization tests. (Amareshwari Sriramadasu via yhemanth)
+
+ MAPREDUCE-1286. Remove quoting from client opts in TaskRunner. (Yuri
+ Pradkin via cdouglas)
+
+ MAPREDUCE-1059. Use distcp.bytes.per.map when adding sync markers in
+ distcp. (Aaron Kimball via cdouglas)
+
+ MAPREDUCE-1009. Update forrest documentation describing hierarchical
+ queues. (Vinod Kumar Vavilapalli via yhemanth)
+
+ MAPREDUCE-1342. Fixed deadlock in global blacklisting of tasktrackers.
+ (Amareshwari Sriramadasu via acmurthy)
+
+ MAPREDUCE-1316. Fixes a memory leak of TaskInProgress instances in
+ the jobtracker. (Amar Kamat via yhemanth)
+
+ MAPREDUCE-1359. TypedBytes TestIO doesn't mkdir its test dir first.
+ (Anatoli Fomenko via cos)
+
+ MAPREDUCE-1314. Correct errant mapreduce.x.mapreduce.x replacements from
+ bulk change. (Amareshwari Sriramadasu via cdouglas)
+
+ MAPREDUCE-1365. Restore accidentally renamed test in
+ TestTaskTrackerBloacklisting. (Amareshwari Sriramadasu via cdouglas)
+
+ MAPREDUCE-1406. Fix spelling of JobContext.MAP_COMBINE_MIN_SPILLS.
+ (cdouglas)
diff --git a/build-utils.xml b/build-utils.xml
new file mode 100644
index 0000000..8a9356a
--- /dev/null
+++ b/build-utils.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!--
+Contains utilities that are common for the main and contrib builds.
+-->
+<project name="build-utils">
+
+ <target name="java5.check" unless="java5.home">
+ <fail message="'java5.home' is not defined. Forrest requires Java 5. Please pass -Djava5.home=<base of Java 5 distribution> to Ant on the command-line." />
+ </target>
+
+ <target name="forrest.check" unless="forrest.home" depends="java5.check">
+ <fail message="'forrest.home' is not defined. Please pass -Dforrest.home=<base of Apache Forrest installation> to Ant on the command-line." />
+ </target>
+
+</project>
\ No newline at end of file
diff --git a/build.xml b/build.xml
index 2682b13..151692c 100644
--- a/build.xml
+++ b/build.xml
@@ -21,6 +21,8 @@
xmlns:artifact="urn:maven-artifact-ant"
xmlns:ivy="antlib:org.apache.ivy.ant">
+ <import file="build-utils.xml" />
+
<!-- Load all the default properties, and any the user wants -->
<!-- to contribute (without having to type -D or edit this file -->
<property file="${user.home}/build.properties" />
@@ -361,6 +363,7 @@
<copy todir="${build.classes}">
<fileset dir="${mapred.src.dir}" includes="**/*.properties"/>
<fileset dir="${mapred.src.dir}" includes="mapred-default.xml"/>
+ <fileset dir="${mapred.src.dir}" includes="mapred-queues-default.xml"/>
</copy>
</target>
@@ -439,6 +442,7 @@
<fileset file="${conf.dir}/log4j.properties"/>
<fileset file="${conf.dir}/hadoop-metrics.properties"/> -->
<zipfileset dir="${build.webapps}" prefix="webapps"/>
+ <fileset file="${jar.extra.properties.list}" />
</jar>
</target>
@@ -476,7 +480,7 @@
<javac
encoding="${build.encoding}"
- srcdir="${test.src.dir}/mapred"
+ srcdir="${test.src.dir}/mapred;${test.src.dir}/unit"
includes="org/apache/hadoop/**/*.java"
destdir="${test.mapred.build.classes}"
debug="${javac.debug}"
@@ -565,6 +569,7 @@
<macrodef name="macro-test-runner">
<attribute name="test.file" />
+ <attribute name="suite.type" />
<sequential>
<delete dir="${test.build.data}"/>
<mkdir dir="${test.build.data}"/>
@@ -572,6 +577,8 @@
<mkdir dir="${test.log.dir}"/>
<copy file="${test.src.dir}/hadoop-policy.xml"
todir="${test.build.extraconf}" />
+ <copy file="${test.src.dir}/fi-site.xml"
+ todir="${test.build.extraconf}" />
<junit showoutput="${test.output}"
printsummary="${test.junit.printsummary}"
haltonfailure="${test.junit.haltonfailure}"
@@ -600,10 +607,16 @@
<syspropertyset dynamic="no">
<propertyref name="compile.c++"/>
</syspropertyset>
+
+ <!-- Pass probability specifications to the spawn JVM -->
+ <syspropertyset id="FaultProbabilityProperties">
+ <propertyref regex="fi.*"/>
+ </syspropertyset>
+
<classpath refid="test.classpath"/>
<formatter type="${test.junit.output.format}" />
<batchtest todir="${test.build.dir}" unless="testcase">
- <fileset dir="${test.src.dir}/mapred" excludes="**/${test.exclude}.java">
+ <fileset dir="${test.src.dir}/@{suite.type}" excludes="**/${test.exclude}.java">
<patternset>
<includesfile name="@{test.file}"/>
</patternset>
@@ -611,20 +624,42 @@
</batchtest>
<batchtest todir="${test.build.dir}" if="testcase">
<fileset dir="${test.src.dir}/mapred" includes="**/${testcase}.java"/>
+ <fileset dir="${test.src.dir}/unit" includes="**/${testcase}.java"/>
+ </batchtest>
+ <!--batch test to test all the testcases in aop folder with fault
+ injection-->
+ <batchtest todir="${test.build.dir}" if="tests.notestcase.fi">
+ <fileset dir="${test.src.dir}/aop"
+ includes="**/${test.include}.java"
+ excludes="**/${test.exclude}.java" />
+ </batchtest>
+ <!-- batch test for testing a single test case in aop folder with
+ fault injection-->
+ <batchtest todir="${test.build.dir}" if="tests.testcase.fi">
+ <fileset dir="${test.src.dir}/aop" includes="**/${testcase}.java"/>
+ </batchtest>
+ <!--The following batch is for very special occasions only when
+ a non-FI tests are needed to be executed against FI-environment -->
+ <batchtest todir="${test.build.dir}" if="tests.testcaseonly.fi">
+ <fileset dir="${test.src.dir}/mapred"
+ includes="**/${testcase}.java"/>
</batchtest>
</junit>
<antcall target="checkfailure"/>
</sequential>
</macrodef>
- <target name="run-test-mapred" depends="compile-mapred-test" description="Run mapred unit tests">
- <macro-test-runner test.file="${test.mapred.all.tests.file}" />
+ <target name="run-test-mapred" depends="compile-mapred-test" description="Run mapred functional and system tests">
+ <macro-test-runner test.file="${test.mapred.all.tests.file}" suite.type="mapred"/>
</target>
<target name="run-commit-test" depends="compile-mapred-test" description="Run approximate 10-minute set of unit tests prior to commiting">
- <macro-test-runner test.file="${test.mapred.commit.tests.file}" />
+ <macro-test-runner test.file="${test.mapred.commit.tests.file}" suite.type="mapred"/>
</target>
+ <target name="run-test-unit" depends="compile-mapred-test" description="Run unit tests">
+ <macro-test-runner test.file="${test.mapred.all.tests.file}" suite.type="unit"/>
+ </target>
<target name="checkfailure" if="tests.failed">
@@ -754,6 +789,8 @@
<!-- ================================================================== -->
<target name="docs" depends="forrest.check" description="Generate forrest-based documentation. To use, specify -Dforrest.home=<base of Apache Forrest installation> on the command line." if="forrest.home">
+ <copy file="${conf.dir}/mapred-queues.xml.template"
+ tofile="${build.docs}/mapred-queues.xml"/>
<exec dir="${docs.src}" executable="${forrest.home}/bin/forrest"
failonerror="true">
<env key="JAVA_HOME" value="${java5.home}"/>
@@ -765,16 +802,12 @@
<style basedir="${mapred.src.dir}" destdir="${build.docs}"
includes="mapred-default.xml" style="conf/configuration.xsl"/>
<antcall target="changes-to-html"/>
+ <subant target="docs">
+ <property name="build.docs" value="${build.docs}"/>
+ <fileset file="${contrib.dir}/build.xml"/>
+ </subant>
</target>
- <target name="forrest.check" unless="forrest.home" depends="java5.check">
- <fail message="'forrest.home' is not defined. Please pass -Dforrest.home=<base of Apache Forrest installation> to Ant on the command-line." />
- </target>
-
- <target name="java5.check" unless="java5.home">
- <fail message="'java5.home' is not defined. Forrest requires Java 5. Please pass -Djava5.home=<base of Java 5 distribution> to Ant on the command-line." />
- </target>
-
<target name="javadoc-dev" depends="compile, ivy-retrieve-javadoc" description="Generate javadoc for hadoop developers">
<mkdir dir="${build.javadoc.dev}"/>
<javadoc
@@ -1235,14 +1268,33 @@
classpathref="releaseaudit-classpath"/>
</target>
+ <!--<target name="releaseaudit" depends="package, rats-taskdef" description="Release Audit activities"> -->
<target name="releaseaudit" depends="package, rats-taskdef" description="Release Audit activities">
<rat:report xmlns:rat="antlib:org.apache.rat.anttasks">
<fileset dir="${dist.dir}">
<exclude name="CHANGES.txt"/>
- <exclude name="docs/"/>
+ <exclude name="**/conf/*"/>
+ <exclude name="**/docs/"/>
+ <exclude name="**/VERSION"/>
+ <exclude name="webapps/**/WEB-INF/web.xml"/>
<exclude name="lib/jdiff/"/>
<exclude name="src/test/all-tests"/>
<exclude name="src/test/commit-tests"/>
+ <exclude name="src/test/mapred/org/apache/hadoop/mapred/test.tgz"/>
+ <exclude name="src/test/tools/data/rumen/**/*"/>
+ <exclude name="src/test/mapred/org/apache/hadoop/mapred/*.txt"/>
+ <exclude name="src/contrib/sqoop/testdata/hive/scripts/*.q"/>
+ <exclude name="src/contrib/mumak/src/test/data/*.json"/>
+ <exclude name="src/contrib/index/sample/*.txt"/>
+ <exclude name="src/test/mapred/org/apache/hadoop/cli/data60bytes"/>
+ <exclude name="src/examples/org/apache/hadoop/examples/dancing/puzzle1.dta"/>
+ <exclude name="src/contrib/eclipse-plugin/META-INF/MANIFEST.MF"/>
+ <exclude name="src/c++/librecordio/*"/>
+ <exclude name="src/c++/pipes/*"/>
+ <exclude name="src/c++/utils/*"/>
+ <exclude name="src/c++/task-controller/*"/>
+ <exclude name="src/examples/pipes/*"/>
+ <exclude name="src/c++/pipes/debug/*"/>
</fileset>
</rat:report>
</target>
@@ -1720,6 +1772,86 @@
failonerror="yes">
</exec>
</target>
-
<!-- end of task-controller targets -->
+
+ <!-- Begining of fault-injection targets-->
+ <import file="${test.src.dir}/aop/build/aop.xml"/>
+
+ <!-- declaring mapred.src.dir as java.src.dir for aop.xml -->
+ <property name="java.src.dir" value="${src.dir}/java"/>
+
+ <!-- target dependency from aop.xml -->
+ <target name="-classes-compilation"
+ depends="compile-mapred-classes, compile-mapred-test"/>
+
+ <target name="jar-test-fault-inject" depends="jar-mapred-test-fault-inject"
+ description="Make hadoop-mapred-test-fi.jar files"/>
+
+ <!-- target to build test-fi.jar-->
+ <target name="jar-mapred-test-fault-inject" depends="injectfaults"
+ description="Make hadoop-mapred-test-fi.jar">
+ <macro-jar-test-fault-inject target.name="jar-test"
+ jar.final.name="test.final.name"
+ jar.final.value="${name}-test-${version}-fi"/>
+ </target>
+
+ <!-- target to build the hadoop-fi.jar -->
+ <target name="jar-fault-inject" depends="injectfaults"
+ description="Make hadoop-fi.jar">
+ <macro-jar-fault-inject
+ target.name="jar"
+ jar.final.name="final.name"
+ jar.final.value="${final.name}-fi" />
+ </target>
+
+ <!-- target to run fault injected test cases will run entire mapred test
+ suite-->
+ <target name="run-test-mapred-fault-inject" depends="injectfaults"
+ description="Run full suite of unit tests with fault injection">
+ <macro-run-tests-fault-inject target.name="run-test-mapred"
+ testcasesonly="false"/>
+ </target>
+
+ <!-- target to run non-FI tests in a FI environment-->
+ <target name="run-fault-inject-with-testcaseonly" depends="injectfaults">
+ <fail unless="testcase">
+ Can't run this target without -Dtestcase setting!
+ </fail>
+ <macro-run-tests-fault-inject target.name="run-test-mapred"
+ testcasesonly="true"/>
+ </target>
+ <condition property="tests.notestcase">
+ <and>
+ <isfalse value="${test.fault.inject}"/>
+ <not>
+ <isset property="testcase"/>
+ </not>
+ </and>
+ </condition>
+ <condition property="tests.notestcase.fi">
+ <and>
+ <not>
+ <isset property="testcase"/>
+ </not>
+ <istrue value="${test.fault.inject}"/>
+ </and>
+ </condition>
+ <condition property="test.testcase">
+ <and>
+ <isfalse value="${test.fault.inject}"/>
+ <isset property="testcase"/>
+ </and>
+ </condition>
+ <condition property="tests.testcaseonly.fi">
+ <istrue value="${special.fi.testcasesonly}" />
+ </condition>
+ <condition property="tests.testcase.fi">
+ <and>
+ <istrue value="${test.fault.inject}" />
+ <isset property="testcase" />
+ <isfalse value="${special.fi.testcasesonly}" />
+ </and>
+ </condition>
+ <!-- End of fault injection targets-->
+
</project>
diff --git a/conf/capacity-scheduler.xml.template b/conf/capacity-scheduler.xml.template
index 588a2fd..68572ac 100644
--- a/conf/capacity-scheduler.xml.template
+++ b/conf/capacity-scheduler.xml.template
@@ -1,78 +1,39 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
-<!-- 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. -->
+ http://www.apache.org/licenses/LICENSE-2.0
-<configuration>
+ 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.
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
- <property>
- <name>mapred.capacity-scheduler.queue.default.capacity</name>
- <value>100</value>
- <description>Percentage of the number of slots in the cluster that are
- to be available for jobs in this queue.
- </description>
- </property>
+<!-- This is one of the configuration files for capacity-scheduler
+ (org.apache.hadoop.mapred.CapacityTaskScheduler), a TaskScheduler
+ for Map/Reduce system. The other configuration file is
+ conf/mapred-queues.xml which it shares with the framework for
+ configuring queues in the system. -->
- <property>
- <name>mapred.capacity-scheduler.queue.default.subQueues</name>
- <value></value>
- <description>Sub-queues are queues configured within queues.
- They provide a mechanism for administrators to link logically related queues
- Sub-queues can be nested. So there can be queues within a sub-queue.
- </description>
- </property>
+<!-- This file can be used to configure (1) job-initialization-poller
+ related properties and (2) the default values for various properties
+ for all the queues.-->
- <property>
- <name>mapred.capacity-scheduler.queue.default.maximum-capacity</name>
- <value>-1</value>
- <description>
- maximum-capacity-stretch defines a limit beyond which a sub-queue cannot use the capacity of its parent queue.
- This provides a means to limit how much excess capacity a sub-queue can use. By default, there is no limit.
- The maximum-capacity-stretch of a queue can only be greater than or equal to its minimum capacity.
- Default value of 100 implies , sub-queue can use complete capacity of its parent.
- This property could be to curtail certain jobs which are long running in nature from occupying more than a
- certain percentage of the cluster, which in the absence of pre-emption, could lead to capacity guarantees of
- other queues being affected.
- </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>
+<configuration>
+ <!-- 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 configured in the -->
+ <!-- queue-configuration file conf/mapred-queues.xml -->
<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>
- <property>
- <name>mapred.capacity-scheduler.queue.default.maximum-initialized-jobs-per-user</name>
- <value>2</value>
- <description>The maximum number of jobs to be pre-initialized for a user
- of the job queue.
- </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-supports-priority</name>
<value>false</value>
<description>If true, priorities of jobs will be taken into
diff --git a/conf/configuration.xsl b/conf/configuration.xsl
index 377cdbe..d50d80b 100644
--- a/conf/configuration.xsl
+++ b/conf/configuration.xsl
@@ -1,4 +1,20 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
<xsl:output method="html"/>
<xsl:template match="configuration">
diff --git a/conf/mapred-queues.xml.template b/conf/mapred-queues.xml.template
index 443f2d9..252f5af 100644
--- a/conf/mapred-queues.xml.template
+++ b/conf/mapred-queues.xml.template
@@ -1,4 +1,20 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
<!-- This is the template for queue configuration. The format supports nesting of
queues within queues - a feature called hierarchical queues. All queues are
defined within the 'queues' tag which is the top level element for this
diff --git a/ivy.xml b/ivy.xml
index 956df33..8592a4a 100644
--- a/ivy.xml
+++ b/ivy.xml
@@ -94,6 +94,14 @@
rev="${lucene-core.version}" conf="javadoc->default"/>
<dependency org="org.apache.hadoop" name="avro" rev="${avro.version}"
conf="common->default"/>
+ <dependency org="org.mockito" name="mockito-all" rev="${mockito-all.version}"
+ conf="test->default"/>
+ <!-- dependency addition for the fault injection -->
+ <dependency org="org.aspectj" name="aspectjrt" rev="${aspectj.version}"
+ conf="common->default"/>
+ <dependency org="org.aspectj" name="aspectjtools" rev="${aspectj.version}"
+ conf="common->default"/>
+
</dependencies>
</ivy-module>
diff --git a/ivy/ivysettings.xml b/ivy/ivysettings.xml
index 40b5843..9e10e41 100644
--- a/ivy/ivysettings.xml
+++ b/ivy/ivysettings.xml
@@ -39,14 +39,15 @@
<resolvers>
<ibiblio name="maven2" root="${repo.maven.org}" pattern="${maven2.pattern.ext}" m2compatible="true"/>
- <ibiblio name="apache-snapshot" root="${snapshot.apache.org}" m2compatible="true"/>
+ <ibiblio name="apache-snapshot" root="${snapshot.apache.org}" m2compatible="true"
+ checkmodified="true" changingPattern=".*SNAPSHOT"/>
<filesystem name="fs" m2compatible="true" force="true">
<artifact pattern="${repo.dir}/org/apache/hadoop/[module]/[revision]/[module]-[revision].[ext]"/>
<ivy pattern="${repo.dir}/org/apache/hadoop/[module]/[revision]/[module]-[revision].pom"/>
</filesystem>
- <chain name="default" dual="true">
+ <chain name="default" dual="true" checkmodified="true" changingPattern=".*SNAPSHOT">
<resolver ref="apache-snapshot"/>
<resolver ref="maven2"/>
</chain>
diff --git a/ivy/libraries.properties b/ivy/libraries.properties
index d70d601..2047239 100644
--- a/ivy/libraries.properties
+++ b/ivy/libraries.properties
@@ -16,6 +16,9 @@
#These are the versions of our dependencies (in alphabetical order)
apacheant.version=1.7.1
ant-task.version=2.0.10
+#Aspectj depedency for Fault injection
+aspectj.version=1.6.5
+
avro.version=1.2.0
checkstyle.version=4.2
@@ -64,6 +67,8 @@
mina-core.version=2.0.0-M5
+mockito-all.version=1.8.0
+
oro.version=2.0.8
paranamer.version=1.5
paranamer.version=1.5
@@ -78,3 +83,4 @@
xmlenc.version=0.52
xerces.version=1.4.4
+
diff --git a/ivybuild.xml b/ivybuild.xml
index 9571387..7d9be0f 100644
--- a/ivybuild.xml
+++ b/ivybuild.xml
@@ -341,8 +341,12 @@
<arg value="diff" />
<arg value="${trunk}/src" />
<arg value="${branch}/src" />
- <arg value="--changelist" />
- <arg value="${issue}"/>
+ <arg value="${hadoop/mapred}/JobEndNotifier.java" />
+ <arg value="${hadoop/mapred}/JobTracker.java" />
+ <arg value="${hadoop/mapred}/TaskTracker.java" />
+ <arg value="${hadoop/mapred}/TaskTrackerAction.java" />
+ <arg value="${hadoop/mapred}/TaskTrackerStatus.java" />
+ <arg value="${test/mapred}/TestTaskTrackerLifecycle.java" />
</svn>
</target>
diff --git a/src/benchmarks/gridmix/generateData.sh b/src/benchmarks/gridmix/generateData.sh
index 890309e..8170423 100644
--- a/src/benchmarks/gridmix/generateData.sh
+++ b/src/benchmarks/gridmix/generateData.sh
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/gridmix-env b/src/benchmarks/gridmix/gridmix-env
index befb59e..bacb554 100644
--- a/src/benchmarks/gridmix/gridmix-env
+++ b/src/benchmarks/gridmix/gridmix-env
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
## Environment configuration
diff --git a/src/benchmarks/gridmix/javasort/text-sort.large b/src/benchmarks/gridmix/javasort/text-sort.large
index 30bde98..bc6e1b2 100644
--- a/src/benchmarks/gridmix/javasort/text-sort.large
+++ b/src/benchmarks/gridmix/javasort/text-sort.large
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/javasort/text-sort.medium b/src/benchmarks/gridmix/javasort/text-sort.medium
index 4cb1448..718ca76 100644
--- a/src/benchmarks/gridmix/javasort/text-sort.medium
+++ b/src/benchmarks/gridmix/javasort/text-sort.medium
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/javasort/text-sort.small b/src/benchmarks/gridmix/javasort/text-sort.small
index 92cab7d..544973f 100644
--- a/src/benchmarks/gridmix/javasort/text-sort.small
+++ b/src/benchmarks/gridmix/javasort/text-sort.small
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/maxent/maxent.large b/src/benchmarks/gridmix/maxent/maxent.large
index ca6475b..6cd6a6a 100644
--- a/src/benchmarks/gridmix/maxent/maxent.large
+++ b/src/benchmarks/gridmix/maxent/maxent.large
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/monsterQuery/monster_query.large b/src/benchmarks/gridmix/monsterQuery/monster_query.large
index 0796382..fcd861f 100644
--- a/src/benchmarks/gridmix/monsterQuery/monster_query.large
+++ b/src/benchmarks/gridmix/monsterQuery/monster_query.large
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/monsterQuery/monster_query.medium b/src/benchmarks/gridmix/monsterQuery/monster_query.medium
index dfd1b73..10355e0 100644
--- a/src/benchmarks/gridmix/monsterQuery/monster_query.medium
+++ b/src/benchmarks/gridmix/monsterQuery/monster_query.medium
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/monsterQuery/monster_query.small b/src/benchmarks/gridmix/monsterQuery/monster_query.small
index 946939d..26e515a 100644
--- a/src/benchmarks/gridmix/monsterQuery/monster_query.small
+++ b/src/benchmarks/gridmix/monsterQuery/monster_query.small
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/pipesort/text-sort.large b/src/benchmarks/gridmix/pipesort/text-sort.large
index 83be735..57aa215 100644
--- a/src/benchmarks/gridmix/pipesort/text-sort.large
+++ b/src/benchmarks/gridmix/pipesort/text-sort.large
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/pipesort/text-sort.medium b/src/benchmarks/gridmix/pipesort/text-sort.medium
index 81f4d69..aa040ae 100644
--- a/src/benchmarks/gridmix/pipesort/text-sort.medium
+++ b/src/benchmarks/gridmix/pipesort/text-sort.medium
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/pipesort/text-sort.small b/src/benchmarks/gridmix/pipesort/text-sort.small
index b1bcf19..84519b7 100644
--- a/src/benchmarks/gridmix/pipesort/text-sort.small
+++ b/src/benchmarks/gridmix/pipesort/text-sort.small
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/streamsort/text-sort.large b/src/benchmarks/gridmix/streamsort/text-sort.large
index 43115bc..3050157 100644
--- a/src/benchmarks/gridmix/streamsort/text-sort.large
+++ b/src/benchmarks/gridmix/streamsort/text-sort.large
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/streamsort/text-sort.medium b/src/benchmarks/gridmix/streamsort/text-sort.medium
index 66a333b..8be615b 100644
--- a/src/benchmarks/gridmix/streamsort/text-sort.medium
+++ b/src/benchmarks/gridmix/streamsort/text-sort.medium
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/streamsort/text-sort.small b/src/benchmarks/gridmix/streamsort/text-sort.small
index 425a1fc..e913103 100644
--- a/src/benchmarks/gridmix/streamsort/text-sort.small
+++ b/src/benchmarks/gridmix/streamsort/text-sort.small
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/allThroughHod b/src/benchmarks/gridmix/submissionScripts/allThroughHod
index c2b8786..527f468 100644
--- a/src/benchmarks/gridmix/submissionScripts/allThroughHod
+++ b/src/benchmarks/gridmix/submissionScripts/allThroughHod
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/allToSameCluster b/src/benchmarks/gridmix/submissionScripts/allToSameCluster
index 923a202..7cad567 100644
--- a/src/benchmarks/gridmix/submissionScripts/allToSameCluster
+++ b/src/benchmarks/gridmix/submissionScripts/allToSameCluster
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/maxentHod b/src/benchmarks/gridmix/submissionScripts/maxentHod
index 7af1de6..a107ff3 100644
--- a/src/benchmarks/gridmix/submissionScripts/maxentHod
+++ b/src/benchmarks/gridmix/submissionScripts/maxentHod
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/maxentToSameCluster b/src/benchmarks/gridmix/submissionScripts/maxentToSameCluster
index 276a82c..6463c3d 100644
--- a/src/benchmarks/gridmix/submissionScripts/maxentToSameCluster
+++ b/src/benchmarks/gridmix/submissionScripts/maxentToSameCluster
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/monsterQueriesHod b/src/benchmarks/gridmix/submissionScripts/monsterQueriesHod
index 8d803b5..506d4bc 100644
--- a/src/benchmarks/gridmix/submissionScripts/monsterQueriesHod
+++ b/src/benchmarks/gridmix/submissionScripts/monsterQueriesHod
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/monsterQueriesToSameCluster b/src/benchmarks/gridmix/submissionScripts/monsterQueriesToSameCluster
index 14d8edf..881c9ef 100644
--- a/src/benchmarks/gridmix/submissionScripts/monsterQueriesToSameCluster
+++ b/src/benchmarks/gridmix/submissionScripts/monsterQueriesToSameCluster
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/sleep_if_too_busy b/src/benchmarks/gridmix/submissionScripts/sleep_if_too_busy
index 390d07b..9021c82 100644
--- a/src/benchmarks/gridmix/submissionScripts/sleep_if_too_busy
+++ b/src/benchmarks/gridmix/submissionScripts/sleep_if_too_busy
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
sleep 1
for ((java_process=$((`ps -ef|grep java|wc -l`-1)); \
diff --git a/src/benchmarks/gridmix/submissionScripts/textSortHod b/src/benchmarks/gridmix/submissionScripts/textSortHod
index 9462a95..ef73d82 100644
--- a/src/benchmarks/gridmix/submissionScripts/textSortHod
+++ b/src/benchmarks/gridmix/submissionScripts/textSortHod
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/textSortToSameCluster b/src/benchmarks/gridmix/submissionScripts/textSortToSameCluster
index 90a848d..44ca639 100644
--- a/src/benchmarks/gridmix/submissionScripts/textSortToSameCluster
+++ b/src/benchmarks/gridmix/submissionScripts/textSortToSameCluster
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/webdataScanHod b/src/benchmarks/gridmix/submissionScripts/webdataScanHod
index 56ffea1..e8f65cb 100644
--- a/src/benchmarks/gridmix/submissionScripts/webdataScanHod
+++ b/src/benchmarks/gridmix/submissionScripts/webdataScanHod
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/webdataScanToSameCluster b/src/benchmarks/gridmix/submissionScripts/webdataScanToSameCluster
index 238eaee..1a0040e 100644
--- a/src/benchmarks/gridmix/submissionScripts/webdataScanToSameCluster
+++ b/src/benchmarks/gridmix/submissionScripts/webdataScanToSameCluster
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/webdataSortHod b/src/benchmarks/gridmix/submissionScripts/webdataSortHod
index 5908583..6202778 100644
--- a/src/benchmarks/gridmix/submissionScripts/webdataSortHod
+++ b/src/benchmarks/gridmix/submissionScripts/webdataSortHod
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/submissionScripts/webdataSortToSameCluster b/src/benchmarks/gridmix/submissionScripts/webdataSortToSameCluster
index 237014d..bf43fec 100644
--- a/src/benchmarks/gridmix/submissionScripts/webdataSortToSameCluster
+++ b/src/benchmarks/gridmix/submissionScripts/webdataSortToSameCluster
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/webdatascan/webdata_scan.large b/src/benchmarks/gridmix/webdatascan/webdata_scan.large
index d2f8a14..0d2a452 100644
--- a/src/benchmarks/gridmix/webdatascan/webdata_scan.large
+++ b/src/benchmarks/gridmix/webdatascan/webdata_scan.large
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/webdatascan/webdata_scan.medium b/src/benchmarks/gridmix/webdatascan/webdata_scan.medium
index 08eb03d..0481ae7 100644
--- a/src/benchmarks/gridmix/webdatascan/webdata_scan.medium
+++ b/src/benchmarks/gridmix/webdatascan/webdata_scan.medium
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/webdatascan/webdata_scan.small b/src/benchmarks/gridmix/webdatascan/webdata_scan.small
index ab1b202..e692724 100644
--- a/src/benchmarks/gridmix/webdatascan/webdata_scan.small
+++ b/src/benchmarks/gridmix/webdatascan/webdata_scan.small
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/webdatasort/webdata_sort.large b/src/benchmarks/gridmix/webdatasort/webdata_sort.large
index 6dddee1..455d91b 100644
--- a/src/benchmarks/gridmix/webdatasort/webdata_sort.large
+++ b/src/benchmarks/gridmix/webdatasort/webdata_sort.large
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/webdatasort/webdata_sort.medium b/src/benchmarks/gridmix/webdatasort/webdata_sort.medium
index 4b92e11..f426cf1 100644
--- a/src/benchmarks/gridmix/webdatasort/webdata_sort.medium
+++ b/src/benchmarks/gridmix/webdatasort/webdata_sort.medium
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix/webdatasort/webdata_sort.small b/src/benchmarks/gridmix/webdatasort/webdata_sort.small
index 1504f9d..40fa316 100644
--- a/src/benchmarks/gridmix/webdatasort/webdata_sort.small
+++ b/src/benchmarks/gridmix/webdatasort/webdata_sort.small
@@ -1,4 +1,15 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
GRID_DIR=`dirname "$0"`
GRID_DIR=`cd "$GRID_DIR"; pwd`
diff --git a/src/benchmarks/gridmix2/README.gridmix2 b/src/benchmarks/gridmix2/README.gridmix2
index 05d4f30..71cf77d 100644
--- a/src/benchmarks/gridmix2/README.gridmix2
+++ b/src/benchmarks/gridmix2/README.gridmix2
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
### "Gridmix" Benchmark ###
Contents:
diff --git a/src/benchmarks/gridmix2/build.xml b/src/benchmarks/gridmix2/build.xml
index f4425d1..a80ce5f 100644
--- a/src/benchmarks/gridmix2/build.xml
+++ b/src/benchmarks/gridmix2/build.xml
@@ -1,4 +1,20 @@
<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
<project default="main" basedir=".">
<property name="Name" value="gridmix"/>
<property name="version" value="0.1"/>
diff --git a/src/benchmarks/gridmix2/gridmix_config.xml b/src/benchmarks/gridmix2/gridmix_config.xml
index 1523de2..82f212b 100644
--- a/src/benchmarks/gridmix2/gridmix_config.xml
+++ b/src/benchmarks/gridmix2/gridmix_config.xml
@@ -1,4 +1,21 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<?xml-stylesheet type="text/xsl" href="nutch-conf.xsl"?>
<!-- Put site-specific property overrides in this file. -->
diff --git a/src/c++/pipes/api/hadoop/Pipes.hh b/src/c++/pipes/api/hadoop/Pipes.hh
index 9a785d9..b5d0ddd 100644
--- a/src/c++/pipes/api/hadoop/Pipes.hh
+++ b/src/c++/pipes/api/hadoop/Pipes.hh
@@ -31,6 +31,8 @@
#include <string>
#endif
+#include <stdint.h>
+
namespace HadoopPipes {
/**
diff --git a/src/c++/task-controller/Makefile.in b/src/c++/task-controller/Makefile.in
index 671065a..bbcd367 100644
--- a/src/c++/task-controller/Makefile.in
+++ b/src/c++/task-controller/Makefile.in
@@ -26,22 +26,22 @@
TESTBINARY=${testdir}/test-task-controller
all: $(OBJS)
- $(CC) $(CFLAG) -o $(BINARY) $(OBJS)
+ $(CC) $(CFLAGS) -o $(BINARY) $(OBJS)
main.o: main.c task-controller.h
- $(CC) $(CFLAG) -o main.o -c main.c
+ $(CC) $(CFLAGS) -o main.o -c main.c
task-controller.o: task-controller.c task-controller.h
- $(CC) $(CFLAG) -o task-controller.o -c task-controller.c
+ $(CC) $(CFLAGS) -o task-controller.o -c task-controller.c
configuration.o: configuration.h configuration.c
- $(CC) $(CFLAG) -o configuration.o -c configuration.c
+ $(CC) $(CFLAGS) -o configuration.o -c configuration.c
${testdir}/test-task-controller.o: task-controller.c task-controller.h
- $(CC) $(CFLAG) -o ${testdir}/test-task-controller.o -c ${testdir}/test-task-controller.c
+ $(CC) $(CFLAGS) -o ${testdir}/test-task-controller.o -c ${testdir}/test-task-controller.c
test: $(TESTOBJS)
- $(CC) $(CFLAG) -o $(TESTBINARY) $(TESTOBJS)
+ $(CC) $(CFLAGS) -o $(TESTBINARY) $(TESTOBJS)
clean:
rm -rf $(BINARY) $(OBJS) $(TESTOBJS)
diff --git a/src/c++/task-controller/main.c b/src/c++/task-controller/main.c
index 8bd095e..0c4d238 100644
--- a/src/c++/task-controller/main.c
+++ b/src/c++/task-controller/main.c
@@ -51,6 +51,7 @@
const char * task_id = NULL;
const char * tt_root = NULL;
const char *log_dir = NULL;
+ const char * unique_string = NULL;
int exit_code = 0;
const char * task_pid = NULL;
const char* const short_options = "l:";
@@ -58,6 +59,8 @@
NULL, 0 } };
const char* log_file = NULL;
+ char * dir_to_be_deleted = NULL;
+ int conf_dir_len = 0;
//Minimum number of arguments required to run the task-controller
//command-name user command tt-root
@@ -67,10 +70,17 @@
}
#ifndef HADOOP_CONF_DIR
- hadoop_conf_dir = (char *) malloc (sizeof(char) *
- (strlen(argv[0]) - strlen(EXEC_PATTERN)) + 1);
- strncpy(hadoop_conf_dir,argv[0],(strlen(argv[0]) - strlen(EXEC_PATTERN)));
- hadoop_conf_dir[(strlen(argv[0]) - strlen(EXEC_PATTERN))] = '\0';
+ conf_dir_len = (strlen(argv[0]) - strlen(EXEC_PATTERN)) + 1;
+ if (conf_dir_len < 1) {
+ // We didn't get an absolute path to our argv[0]; bail.
+ printf("Cannot find configuration directory.\n");
+ printf("This program must be run with its full absolute path.\n");
+ return INVALID_CONF_DIR;
+ } else {
+ hadoop_conf_dir = (char *) malloc (sizeof(char) * conf_dir_len);
+ strncpy(hadoop_conf_dir,argv[0],(strlen(argv[0]) - strlen(EXEC_PATTERN)));
+ hadoop_conf_dir[(strlen(argv[0]) - strlen(EXEC_PATTERN))] = '\0';
+ }
#endif
do {
next_option = getopt_long(argc, argv, short_options, long_options, NULL);
@@ -112,8 +122,11 @@
job_id = argv[optind++];
exit_code = initialize_job(job_id, user_detail->pw_name);
break;
- case INITIALIZE_DISTRIBUTEDCACHE:
- exit_code = initialize_distributed_cache(user_detail->pw_name);
+ case INITIALIZE_DISTRIBUTEDCACHE_FILE:
+ tt_root = argv[optind++];
+ unique_string = argv[optind++];
+ exit_code = initialize_distributed_cache_file(tt_root, unique_string,
+ user_detail->pw_name);
break;
case LAUNCH_TASK_JVM:
tt_root = argv[optind++];
@@ -142,6 +155,17 @@
exit_code
= run_debug_script_as_user(user_detail->pw_name, job_id, task_id, tt_root);
break;
+ case SIGQUIT_TASK_JVM:
+ task_pid = argv[optind++];
+ exit_code = kill_user_task(user_detail->pw_name, task_pid, SIGQUIT);
+ break;
+ case ENABLE_TASK_FOR_CLEANUP:
+ tt_root = argv[optind++];
+ job_id = argv[optind++];
+ dir_to_be_deleted = argv[optind++];
+ exit_code = enable_task_for_cleanup(tt_root, user_detail->pw_name, job_id,
+ dir_to_be_deleted);
+ break;
default:
exit_code = INVALID_COMMAND_PROVIDED;
}
diff --git a/src/c++/task-controller/task-controller.c b/src/c++/task-controller/task-controller.c
index aa28ecb..a342499 100644
--- a/src/c++/task-controller/task-controller.c
+++ b/src/c++/task-controller/task-controller.c
@@ -35,6 +35,7 @@
}
if(initgroups(user_detail->pw_name, user_detail->pw_gid) != 0) {
+ fprintf(LOGFILE, "unable to initgroups : %s\n", strerror(errno));
cleanup();
return SETUID_OPER_FAILED;
}
@@ -172,9 +173,10 @@
/**
* Get the distributed cache directory for a particular user
*/
-char *get_distributed_cache_directory(const char *tt_root, const char *user) {
- return concatenate(USER_DISTRIBUTED_CACHE_DIR_PATTERN, "dist_cache_path", 2,
- tt_root, user);
+char *get_distributed_cache_directory(const char *tt_root, const char *user,
+ const char* unique_string) {
+ return concatenate(USER_DISTRIBUTED_CACHE_DIR_PATTERN,
+ "dist_cache_unique_path", 3, tt_root, user, unique_string);
}
char *get_job_work_directory(const char *job_dir) {
@@ -197,6 +199,17 @@
attempt_dir);
}
+/*
+ * Builds the full path of the dir(localTaskDir or localWorkDir)
+ * tt_root : is the base path(i.e. mapred-local-dir) sent to task-controller
+ * dir_to_be_deleted : is either taskDir($taskId) OR taskWorkDir($taskId/work)
+ */
+char *get_task_dir_path(const char *tt_root, const char *user,
+ const char *jobid, const char *dir_to_be_deleted) {
+ return concatenate(TT_LOCAL_TASK_DIR_PATTERN, "task_dir_full_path", 4,
+ tt_root, user, jobid, dir_to_be_deleted);
+}
+
/**
* Get the log directory for the given attempt.
*/
@@ -218,17 +231,17 @@
* launcher file resolve to one and same. This is done so as to avoid
* security pitfalls because of relative path components in the file name.
*/
-int check_task_launcher_path(char *path) {
+int check_path_for_relative_components(char *path) {
char * resolved_path = (char *) canonicalize_file_name(path);
if (resolved_path == NULL) {
fprintf(LOGFILE,
- "Error resolving the task launcher file path: %s. Passed path: %s\n",
+ "Error resolving the path: %s. Passed path: %s\n",
strerror(errno), path);
return ERROR_RESOLVING_FILE_PATH;
}
if (strcmp(resolved_path, path) != 0) {
fprintf(LOGFILE,
- "Relative path components in the file path: %s. Resolved path: %s\n",
+ "Relative path components in the path: %s. Resolved path: %s\n",
path, resolved_path);
free(resolved_path);
return RELATIVE_PATH_COMPONENTS_IN_FILE_PATH;
@@ -255,23 +268,26 @@
static int change_mode(const char *path, mode_t mode) {
int exit_code = chmod(path, mode);
if (exit_code != 0) {
- fprintf(LOGFILE, "chown %d of path %s failed: %s.\n", mode, path,
+ fprintf(LOGFILE, "chmod %d of path %s failed: %s.\n", mode, path,
strerror(errno));
}
return exit_code;
}
/**
- * Function to secure the given path. It does the following recursively:
+ * Function to change permissions of the given path. It does the following
+ * recursively:
* 1) changes the owner/group of the paths to the passed owner/group
* 2) changes the file permission to the passed file_mode and directory
* permission to the passed dir_mode
+ *
+ * should_check_ownership : boolean to enable checking of ownership of each path
*/
static int secure_path(const char *path, uid_t uid, gid_t gid,
- mode_t file_mode, mode_t dir_mode) {
+ mode_t file_mode, mode_t dir_mode, int should_check_ownership) {
FTS *tree = NULL; // the file hierarchy
FTSENT *entry = NULL; // a file in the hierarchy
- char *paths[] = { (char *) path };
+ char *paths[] = { (char *) path, NULL };//array needs to be NULL-terminated
int process_path = 0;
int dir = 0;
int error_code = 0;
@@ -361,7 +377,8 @@
if (!process_path) {
continue;
}
- if (compare_ownership(uid, gid, entry->fts_path) == 0) {
+ if (should_check_ownership &&
+ (compare_ownership(uid, gid, entry->fts_path) == 0)) {
// already set proper permissions.
// This might happen with distributed cache.
#ifdef DEBUG
@@ -373,7 +390,7 @@
continue;
}
- if (check_ownership(entry->fts_path) != 0) {
+ if (should_check_ownership && (check_ownership(entry->fts_path) != 0)) {
fprintf(LOGFILE,
"Invalid file path. %s not user/group owned by the tasktracker.\n",
entry->fts_path);
@@ -466,8 +483,9 @@
free(job_dir);
break;
}
- } else if (secure_path(attempt_dir, user_detail->pw_uid, tasktracker_gid,
- S_IRWXU | S_IRWXG, S_ISGID | S_IRWXU | S_IRWXG) != 0) {
+ } else if (secure_path(attempt_dir, user_detail->pw_uid,
+ tasktracker_gid, S_IRWXU | S_IRWXG, S_ISGID | S_IRWXU | S_IRWXG,
+ 1) != 0) {
// No setgid on files and setgid on dirs, 770
fprintf(LOGFILE, "Failed to secure the attempt_dir %s\n", attempt_dir);
failed = 1;
@@ -526,8 +544,8 @@
}
gid_t tasktracker_gid = getegid(); // the group permissions of the binary.
- if (secure_path(task_log_dir, user_detail->pw_uid, tasktracker_gid, S_IRWXU
- | S_IRWXG, S_ISGID | S_IRWXU | S_IRWXG) != 0) {
+ if (secure_path(task_log_dir, user_detail->pw_uid, tasktracker_gid,
+ S_IRWXU | S_IRWXG, S_ISGID | S_IRWXU | S_IRWXG, 1) != 0) {
// setgid on dirs but not files, 770. As of now, there are no files though
fprintf(LOGFILE, "Failed to secure the log_dir %s\n", task_log_dir);
return -1;
@@ -639,9 +657,9 @@
free(user_dir);
break;
}
- } else if (secure_path(user_dir, user_detail->pw_uid, tasktracker_gid,
- S_IRUSR | S_IXUSR | S_IRWXG, S_ISGID | S_IRUSR | S_IXUSR | S_IRWXG)
- != 0) {
+ } else if (secure_path(user_dir, user_detail->pw_uid,
+ tasktracker_gid, S_IRUSR | S_IXUSR | S_IRWXG, S_ISGID | S_IRUSR |
+ S_IXUSR | S_IRWXG, 1) != 0) {
// No setgid on files and setgid on dirs, 570
fprintf(LOGFILE, "Failed to secure the user_dir %s\n",
user_dir);
@@ -721,7 +739,7 @@
break;
}
} else if (secure_path(job_dir, user_detail->pw_uid, tasktracker_gid,
- S_IRUSR | S_IXUSR | S_IRWXG, S_ISGID | S_IRUSR | S_IXUSR | S_IRWXG)
+ S_IRUSR | S_IXUSR | S_IRWXG, S_ISGID | S_IRUSR | S_IXUSR | S_IRWXG, 1)
!= 0) {
// No setgid on files and setgid on dirs, 570
fprintf(LOGFILE, "Failed to secure the job_dir %s\n", job_dir);
@@ -778,22 +796,25 @@
}
/**
- * Function to initialize the distributed cache files of a user.
+ * Function to initialize the distributed cache file for a user.
* It does the following:
- * * sudo chown user:mapred -R taskTracker/$user/distcache
- * * sudo chmod 2570 -R taskTracker/$user/distcache
- * This is done once per every JVM launch. Tasks reusing JVMs just create
+ * * sudo chown user:mapred -R taskTracker/$user/distcache/<randomdir>
+ * * sudo chmod 2570 -R taskTracker/$user/distcache/<randomdir>
+ * This is done once per localization. Tasks reusing JVMs just create
* symbolic links themselves and so there isn't anything specific to do in
* that case.
- * Sometimes, it happens that a task uses the whole or part of a directory
- * structure in taskTracker/$user/distcache. In this case, some paths are
- * already set proper private permissions by this same function called during
- * a previous JVM launch. In the current invocation, we only do the
- * chown/chmod operation of files/directories that are newly created by the
- * TaskTracker (i.e. those that still are not owned by user:mapred)
*/
-int initialize_distributed_cache(const char *user) {
-
+int initialize_distributed_cache_file(const char *tt_root,
+ const char *unique_string, const char *user) {
+ if (tt_root == NULL) {
+ fprintf(LOGFILE, "tt_root passed is null.\n");
+ return INVALID_ARGUMENT_NUMBER;
+ }
+ if (unique_string == NULL) {
+ fprintf(LOGFILE, "unique_string passed is null.\n");
+ return INVALID_ARGUMENT_NUMBER;
+ }
+
if (user == NULL) {
fprintf(LOGFILE, "user passed is null.\n");
return INVALID_ARGUMENT_NUMBER;
@@ -803,69 +824,41 @@
fprintf(LOGFILE, "Couldn't get the user details of %s", user);
return INVALID_USER_NAME;
}
-
- gid_t tasktracker_gid = getegid(); // the group permissions of the binary.
-
- char **local_dir = (char **) get_values(TT_SYS_DIR_KEY);
- if (local_dir == NULL) {
- fprintf(LOGFILE, "%s is not configured.\n", TT_SYS_DIR_KEY);
+ //Check tt_root
+ if (check_tt_root(tt_root) < 0) {
+ fprintf(LOGFILE, "invalid tt root passed %s\n", tt_root);
cleanup();
return INVALID_TT_ROOT;
}
- char *full_local_dir_str = (char *) get_value(TT_SYS_DIR_KEY);
-#ifdef DEBUG
- fprintf(LOGFILE, "Value from config for %s is %s.\n", TT_SYS_DIR_KEY,
- full_local_dir_str);
-#endif
+ // set permission on the unique directory
+ char *localized_unique_dir = get_distributed_cache_directory(tt_root, user,
+ unique_string);
+ if (localized_unique_dir == NULL) {
+ fprintf(LOGFILE, "Couldn't get unique distcache directory for %s.\n", user);
+ cleanup();
+ return INITIALIZE_DISTCACHEFILE_FAILED;
+ }
- char *distcache_dir;
- char **local_dir_ptr = local_dir;
+ gid_t binary_gid = getegid(); // the group permissions of the binary.
int failed = 0;
- while (*local_dir_ptr != NULL) {
- distcache_dir = get_distributed_cache_directory(*local_dir_ptr, user);
- if (distcache_dir == NULL) {
- fprintf(LOGFILE, "Couldn't get distcache directory for %s.\n", user);
- failed = 1;
- break;
- }
-
- struct stat filestat;
- if (stat(distcache_dir, &filestat) != 0) {
- if (errno == ENOENT) {
-#ifdef DEBUG
- fprintf(LOGFILE, "distcache_dir %s doesn't exist. Not doing anything.\n",
- distcache_dir);
-#endif
- } else {
- // stat failed because of something else!
- fprintf(LOGFILE, "Failed to stat the distcache_dir %s\n",
- distcache_dir);
- failed = 1;
- free(distcache_dir);
- break;
- }
- } else if (secure_path(distcache_dir, user_detail->pw_uid,
- tasktracker_gid, S_IRUSR | S_IXUSR | S_IRWXG, S_ISGID | S_IRUSR
- | S_IXUSR | S_IRWXG) != 0) {
- // No setgid on files and setgid on dirs, 570
- fprintf(LOGFILE, "Failed to secure the distcache_dir %s\n",
- distcache_dir);
- failed = 1;
- free(distcache_dir);
- break;
- }
-
- local_dir_ptr++;
- free(distcache_dir);
+ struct stat filestat;
+ if (stat(localized_unique_dir, &filestat) != 0) {
+ // stat on distcache failed because of something
+ fprintf(LOGFILE, "Failed to stat the localized_unique_dir %s\n",
+ localized_unique_dir);
+ failed = INITIALIZE_DISTCACHEFILE_FAILED;
+ } else if (secure_path(localized_unique_dir, user_detail->pw_uid,
+ binary_gid, S_IRUSR | S_IXUSR | S_IRWXG, S_ISGID | S_IRUSR
+ | S_IXUSR | S_IRWXG, 1) != 0) {
+ // No setgid on files and setgid on dirs, 570
+ fprintf(LOGFILE, "Failed to secure the localized_unique_dir %s\n",
+ localized_unique_dir);
+ failed = INITIALIZE_DISTCACHEFILE_FAILED;
}
- free(local_dir);
- free(full_local_dir_str);
+ free(localized_unique_dir);
cleanup();
- if (failed) {
- return INITIALIZE_DISTCACHE_FAILED;
- }
- return 0;
+ return failed;
}
/**
@@ -980,7 +973,7 @@
}
errno = 0;
- exit_code = check_task_launcher_path(task_script_path);
+ exit_code = check_path_for_relative_components(task_script_path);
if(exit_code != 0) {
goto cleanup;
}
@@ -1029,7 +1022,8 @@
return run_process_as_user(user, jobid, taskid, tt_root, RUN_DEBUG_SCRIPT);
}
/**
- * Function used to terminate/kill a task launched by the user.
+ * Function used to terminate/kill a task launched by the user,
+ * or dump the process' stack (by sending SIGQUIT).
* The function sends appropriate signal to the process group
* specified by the task_pid.
*/
@@ -1076,3 +1070,60 @@
cleanup();
return 0;
}
+
+/**
+ * Enables the path for deletion by changing the owner, group and permissions
+ * of the specified path and all the files/directories in the path recursively.
+ * * sudo chown user:mapred -R full_path
+ * * sudo chmod 2770 -R full_path
+ * Before changing permissions, makes sure that the given path doesn't contain
+ * any relative components.
+ * tt_root : is the base path(i.e. mapred-local-dir) sent to task-controller
+ * dir_to_be_deleted : is either taskDir OR taskWorkDir that is to be deleted
+ */
+int enable_task_for_cleanup(const char *tt_root, const char *user,
+ const char *jobid, const char *dir_to_be_deleted) {
+ int exit_code = 0;
+ gid_t tasktracker_gid = getegid(); // the group permissions of the binary.
+
+ char * full_path = NULL;
+ if (check_tt_root(tt_root) < 0) {
+ fprintf(LOGFILE, "invalid tt root passed %s\n", tt_root);
+ cleanup();
+ return INVALID_TT_ROOT;
+ }
+
+ full_path = get_task_dir_path(tt_root, user, jobid, dir_to_be_deleted);
+ if (full_path == NULL) {
+ fprintf(LOGFILE,
+ "Could not build the full path. Not deleting the dir %s\n",
+ dir_to_be_deleted);
+ exit_code = UNABLE_TO_BUILD_PATH; // may be malloc failed
+ }
+ // Make sure that the path given is not having any relative components
+ else if ((exit_code = check_path_for_relative_components(full_path)) != 0) {
+ fprintf(LOGFILE,
+ "Not changing permissions. Path may contain relative components.\n",
+ full_path);
+ }
+ else if (get_user_details(user) < 0) {
+ fprintf(LOGFILE, "Couldn't get the user details of %s.\n", user);
+ exit_code = INVALID_USER_NAME;
+ }
+ else if (exit_code = secure_path(full_path, user_detail->pw_uid,
+ tasktracker_gid,
+ S_IRWXU | S_IRWXG, S_ISGID | S_IRWXU | S_IRWXG, 0) != 0) {
+ // No setgid on files and setgid on dirs, 770.
+ // set 770 permissions for user, TTgroup for all files/directories in
+ // 'full_path' recursively sothat deletion of path by TaskTracker succeeds.
+
+ fprintf(LOGFILE, "Failed to set permissions for %s\n", full_path);
+ }
+
+ if (full_path != NULL) {
+ free(full_path);
+ }
+ // free configurations
+ cleanup();
+ return exit_code;
+}
diff --git a/src/c++/task-controller/task-controller.h b/src/c++/task-controller/task-controller.h
index f270d54..26c96ff 100644
--- a/src/c++/task-controller/task-controller.h
+++ b/src/c++/task-controller/task-controller.h
@@ -39,12 +39,14 @@
enum command {
INITIALIZE_USER,
INITIALIZE_JOB,
- INITIALIZE_DISTRIBUTEDCACHE,
+ INITIALIZE_DISTRIBUTEDCACHE_FILE,
LAUNCH_TASK_JVM,
INITIALIZE_TASK,
TERMINATE_TASK_JVM,
KILL_TASK_JVM,
RUN_DEBUG_SCRIPT,
+ SIGQUIT_TASK_JVM,
+ ENABLE_TASK_FOR_CLEANUP
};
enum errorcodes {
@@ -66,16 +68,18 @@
PREPARE_TASK_LOGS_FAILED, //16
INVALID_TT_LOG_DIR, //17
OUT_OF_MEMORY, //18
- INITIALIZE_DISTCACHE_FAILED, //19
+ INITIALIZE_DISTCACHEFILE_FAILED, //19
INITIALIZE_USER_FAILED, //20
UNABLE_TO_EXECUTE_DEBUG_SCRIPT, //21
+ INVALID_CONF_DIR, //22
+ UNABLE_TO_BUILD_PATH //23
};
#define USER_DIR_PATTERN "%s/taskTracker/%s"
#define TT_JOB_DIR_PATTERN USER_DIR_PATTERN"/jobcache/%s"
-#define USER_DISTRIBUTED_CACHE_DIR_PATTERN USER_DIR_PATTERN"/distcache"
+#define USER_DISTRIBUTED_CACHE_DIR_PATTERN USER_DIR_PATTERN"/distcache/%s"
#define JOB_DIR_TO_JOB_WORK_PATTERN "%s/work"
@@ -85,6 +89,8 @@
#define TASK_SCRIPT_PATTERN "%s/%s/taskjvm.sh"
+#define TT_LOCAL_TASK_DIR_PATTERN "%s/taskTracker/%s/jobcache/%s/%s"
+
#define TT_SYS_DIR_KEY "mapreduce.cluster.local.dir"
#define TT_LOG_DIR_KEY "hadoop.log.dir"
@@ -110,10 +116,14 @@
int initialize_job(const char *jobid, const char *user);
-int initialize_distributed_cache(const char *user);
+int initialize_distributed_cache_file(const char *tt_root,
+ const char* unique_string, const char *user);
int kill_user_task(const char *user, const char *task_pid, int sig);
+int enable_task_for_cleanup(const char *tt_root, const char *user,
+ const char *jobid, const char *dir_to_be_deleted);
+
int prepare_attempt_directory(const char *attempt_dir, const char *user);
// The following functions are exposed for testing
diff --git a/src/c++/utils/api/hadoop/SerialUtils.hh b/src/c++/utils/api/hadoop/SerialUtils.hh
index 16cbab6..cadfd76 100644
--- a/src/c++/utils/api/hadoop/SerialUtils.hh
+++ b/src/c++/utils/api/hadoop/SerialUtils.hh
@@ -19,6 +19,7 @@
#define HADOOP_SERIAL_UTILS_HH
#include <string>
+#include <stdint.h>
namespace HadoopUtils {
@@ -57,7 +58,7 @@
{ \
if (!(CONDITION)) { \
throw HadoopUtils::Error((MESSAGE), __FILE__, __LINE__, \
- __PRETTY_FUNCTION__); \
+ __func__); \
} \
}
diff --git a/src/contrib/build-contrib.xml b/src/contrib/build-contrib.xml
index e6edd48..1b445f4 100644
--- a/src/contrib/build-contrib.xml
+++ b/src/contrib/build-contrib.xml
@@ -21,6 +21,8 @@
<project name="hadoopbuildcontrib" xmlns:ivy="antlib:org.apache.ivy.ant">
+ <import file="../../build-utils.xml" />
+
<property name="name" value="${ant.project.name}"/>
<property name="root" value="${basedir}"/>
@@ -106,6 +108,7 @@
<pathelement location="${hadoop.root}/build"/>
<pathelement location="${build.examples}"/>
<pathelement location="${hadoop.root}/build/examples"/>
+ <path refid="${ant.project.name}.test-classpath"/>
<path refid="contrib-classpath"/>
</path>
@@ -165,7 +168,7 @@
<!-- ================================================================== -->
<!-- Compile test code -->
<!-- ================================================================== -->
- <target name="compile-test" depends="compile-examples" if="test.available">
+ <target name="compile-test" depends="compile-examples, ivy-retrieve-test" if="test.available">
<echo message="contrib: ${name}"/>
<javac
encoding="${build.encoding}"
@@ -214,6 +217,17 @@
<include name="${dest.jar}" />
</fileset>
</copy>
+
+ <!-- copy the dependency libraries into the contrib/lib dir -->
+ <mkdir dir="${dist.dir}/contrib/${name}/lib"/>
+ <copy todir="${dist.dir}/contrib/${name}/lib" includeEmptyDirs="false" flatten="true">
+ <fileset dir="${common.ivy.lib.dir}">
+ <!-- except for those already present due to Hadoop -->
+ <present present="srconly" targetdir="${dist.dir}/lib" />
+ </fileset>
+ </copy>
+ <!-- if the lib dir is empty, remove it. -->
+ <delete dir="${dist.dir}/contrib/${name}/lib" includeEmptyDirs="true" excludes="*.jar" />
</target>
<!-- ================================================================== -->
@@ -259,6 +273,10 @@
<antcall target="checkfailure"/>
</target>
+ <target name="docs" depends="forrest.check" description="Generate forrest-based documentation. To use, specify -Dforrest.home=<base of Apache Forrest installation> on the command line." if="forrest.home">
+ <!-- Nothing by default -->
+ </target>
+
<target name="checkfailure" if="tests.failed">
<touch file="${build.contrib.dir}/testsfailed"/>
<fail unless="continueOnFailure">Contrib Tests failed!</fail>
@@ -315,4 +333,15 @@
pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}" sync="true" />
<ivy:cachepath pathid="${ant.project.name}.common-classpath" conf="common" />
</target>
+
+ <target name="ivy-resolve-test" depends="ivy-init">
+ <ivy:resolve settingsRef="${ant.project.name}.ivy.settings" conf="test" />
+ </target>
+
+ <target name="ivy-retrieve-test" depends="ivy-resolve-test"
+ description="Retrieve Ivy-managed artifacts for the test configuration">
+ <ivy:retrieve settingsRef="${ant.project.name}.ivy.settings"
+ pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}" sync="true" />
+ <ivy:cachepath pathid="${ant.project.name}.test-classpath" conf="test" />
+ </target>
</project>
diff --git a/src/contrib/build.xml b/src/contrib/build.xml
index 7a87a93..7a4c770 100644
--- a/src/contrib/build.xml
+++ b/src/contrib/build.xml
@@ -59,11 +59,18 @@
<fileset dir="." includes="gridmix/build.xml"/>
<fileset dir="." includes="vertica/build.xml"/>
<fileset dir="." includes="mumak/build.xml"/>
+ <fileset dir="." includes="raid/build.xml"/>
</subant>
<available file="${build.contrib.dir}/testsfailed" property="testsfailed"/>
<fail if="testsfailed">Tests failed!</fail>
</target>
+ <target name="docs">
+ <subant target="docs">
+ <fileset dir="." includes="capacity-scheduler/build.xml"/>
+ </subant>
+ </target>
+
<!-- ====================================================== -->
<!-- Clean all the contribs. -->
<!-- ====================================================== -->
diff --git a/src/contrib/capacity-scheduler/build.xml b/src/contrib/capacity-scheduler/build.xml
index f299abd..dd62fd8 100644
--- a/src/contrib/capacity-scheduler/build.xml
+++ b/src/contrib/capacity-scheduler/build.xml
@@ -25,4 +25,12 @@
<import file="../build-contrib.xml"/>
+ <target name="docs" depends="forrest.check" description="Generate forrest-based documentation. To use, specify -Dforrest.home=<base of Apache Forrest installation> on the command line." if="forrest.home">
+ <copy file="src/java/mapred-queues.xml.template"
+ tofile="${build.docs}/mapred-queues-capacity-scheduler.xml"/>
+ <xslt in="${conf.dir}/capacity-scheduler.xml.template"
+ out="${build.docs}/capacity-scheduler-conf.html"
+ style="${conf.dir}/configuration.xsl"/>
+ </target>
+
</project>
diff --git a/src/contrib/capacity-scheduler/ivy.xml b/src/contrib/capacity-scheduler/ivy.xml
index 4ff3345..6e2f58b 100644
--- a/src/contrib/capacity-scheduler/ivy.xml
+++ b/src/contrib/capacity-scheduler/ivy.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<ivy-module version="1.0">
<info organisation="org.apache.hadoop" module="${ant.project.name}">
<license name="Apache 2.0"/>
@@ -27,11 +44,11 @@
<dependency org="org.apache.hadoop" name="hadoop-core"
rev="${hadoop-core.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-core-test"
- rev="${hadoop-core.version}" conf="common->default"/>
+ rev="${hadoop-core.version}" conf="test->default"/>
<dependency org="org.apache.hadoop" name="hadoop-hdfs"
rev="${hadoop-hdfs.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-hdfs-test"
- rev="${hadoop-hdfs.version}" conf="common->default"/>
+ rev="${hadoop-hdfs.version}" conf="test->default"/>
<dependency org="commons-cli" name="commons-cli"
rev="${commons-cli.version}" conf="common->default"/>
<dependency org="commons-logging" name="commons-logging"
diff --git a/src/contrib/capacity-scheduler/ivy/libraries.properties b/src/contrib/capacity-scheduler/ivy/libraries.properties
index a470b37..8a80dd8 100644
--- a/src/contrib/capacity-scheduler/ivy/libraries.properties
+++ b/src/contrib/capacity-scheduler/ivy/libraries.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
#This properties file lists the versions of the various artifacts used by streaming.
#It drives ivy and the generation of a maven POM
diff --git a/src/contrib/capacity-scheduler/src/java/mapred-queues.xml.template b/src/contrib/capacity-scheduler/src/java/mapred-queues.xml.template
new file mode 100644
index 0000000..21b8919
--- /dev/null
+++ b/src/contrib/capacity-scheduler/src/java/mapred-queues.xml.template
@@ -0,0 +1,164 @@
+<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<!--
+ This is the template for queue configuration when the configured
+ scheduler to use is capacity-scheduler
+ (org.apache.hadoop.mapred.CapacityTaskScheduler). To use this,
+ copy this file into conf directory renaming it to mapred-queues.xml.
+-->
+<queues aclsEnabled="false">
+ <queue>
+
+ <name>default</name>
+
+ <state>running</state>
+
+ <acl-submit-job>*</acl-submit-job>
+ <acl-administer-jobs>*</acl-administer-jobs>
+
+ <properties>
+
+ <property key="capacity" value="100">
+ <!--
+ <description>
+ For a root-level container queue, this is the percentage of the
+ number of slots in the cluster that will be available for all its
+ immediate children together. For a root-level leaf-queue, this is
+ the percentage of the number of slots in the cluster that will be
+ available for all its jobs. For a non-root level container queue,
+ this is the percentage of the number of slots in its parent queue
+ that will be available for all its children together. For a
+ non-root-level leaf queue, this is the percentage of the number of
+ slots in its parent queue that will be available for jobs in this
+ queue. The sum of capacities for all children of a container queue
+ should be less than or equal 100. The sum of capacities of all the
+ root-level queues should be less than or equal to 100.
+
+ This property can be refreshed.
+ </description>
+ -->
+ </property>
+
+ <property key="maximum-capacity" value="-1">
+ <!--
+ <description>
+ A limit in percentage beyond which a non-root-level queue cannot use
+ the capacity of its parent queue; for a root-level queue, this is
+ the limit in percentage beyond which it cannot use the
+ cluster-capacity. This property provides a means to limit how much
+ excess capacity a queue can use. It can be used to prevent queues
+ with long running jobs from occupying more than a certain percentage
+ of the parent-queue or the cluster, which, in the absence of
+ pre-emption, can lead to capacity guarantees of other queues getting
+ affected.
+
+ The maximum-capacity of a queue can only be greater than or equal to
+ its capacity. By default, there is no limit for a queue. For a
+ non-root-level queue this means it can occupy till the
+ maximum-capacity of its parent, for a root-level queue, it means that
+ it can occupy the whole cluster. A value of 100 implies that a queue
+ can use the complete capacity of its parent, or the complete
+ cluster-capacity in case of root-level-queues.
+
+ This property can be refreshed.
+ </description>
+ -->
+ </property>
+
+ <property key="supports-priority" value="false">
+ <!--
+ <description>This is only applicable to leaf queues. If true,
+ priorities of jobs will be taken into account in scheduling
+ decisions.
+
+ This property CANNOT be refreshed.
+ </description>
+ -->
+ </property>
+
+ <property key="minimum-user-limit-percent" value="100">
+ <!--
+ <description>This is only applicable to leaf queues. 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.
+
+ This property can be refreshed.
+ </description>
+ -->
+ </property>
+
+ <property key="maximum-initialized-jobs-per-user" value="2">
+ <!--
+ <description>This is only applicable to leaf queues. The maximum number
+ of jobs to be pre-initialized for a user of the job queue.
+
+ This property can be refreshed.
+ </description>
+ -->
+ </property>
+
+ </properties>
+ </queue>
+
+ <!-- Here is a sample of a hierarchical queue configuration
+ where q2 and q3 are children of q1 sharing the capacity
+ of q1. In this example, q2 and q3 are leaf level
+ queues as it has no queues configured within it. Currently, ACLs
+ and state are only supported for the leaf level queues.
+ <queue>
+ <name>q1</name>
+ <properties>
+ <property key="capacity" value="100"/>
+ </properties>
+ <queue>
+ <name>q2</name>
+ <state>stopped</state>
+ <acl-submit-job>*</acl-submit-job>
+ <acl-administer-jobs>*</acl-administer-jobs>
+ <properties>
+ <property key="capacity" value="50"/>
+ <property key="maximum-capacity" value="60"/>
+ <property key="supports-priority" value="false"/>
+ <property key="minimum-user-limit-percent" value="100"/>
+ <property key="maximum-initialized-jobs-per-user" value="2"/>
+ </properties>
+ </queue>
+ <queue>
+ <name>q3</name>
+ <state>stopped</state>
+ <acl-submit-job>*</acl-submit-job>
+ <acl-administer-jobs>*</acl-administer-jobs>
+ <properties>
+ <property key="capacity" value="50"/>
+ <property key="maximum-capacity" value="-1"/>
+ <property key="supports-priority" value="false"/>
+ <property key="minimum-user-limit-percent" value="100"/>
+ <property key="maximum-initialized-jobs-per-user" value="2"/>
+ </properties>
+ </queue>
+ </queue>
+ -->
+</queues>
diff --git a/src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java b/src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java
index 04c6f2d..c13324c 100644
--- a/src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java
+++ b/src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java
@@ -601,7 +601,7 @@
//Check if job supports speculative map execution first then
//check if job has speculative maps.
return (job.getJobConf().getMapSpeculativeExecution())&& (
- hasSpeculativeTask(job.getMapTasks(),
+ hasSpeculativeTask(job.getTasks(TaskType.MAP),
tts));
}
@@ -644,7 +644,7 @@
//check if the job supports reduce speculative execution first then
//check if the job has speculative tasks.
return (job.getJobConf().getReduceSpeculativeExecution()) && (
- hasSpeculativeTask(job.getReduceTasks(),
+ hasSpeculativeTask(job.getTasks(TaskType.REDUCE),
tts));
}
diff --git a/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/CapacityTestUtils.java b/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/CapacityTestUtils.java
index 2cbcf5f..6542cab 100644
--- a/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/CapacityTestUtils.java
+++ b/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/CapacityTestUtils.java
@@ -42,6 +42,7 @@
import org.apache.hadoop.mapred.FakeObjectUtilities.FakeJobHistory;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
+import org.apache.hadoop.mapreduce.split.JobSplit;
import org.apache.hadoop.security.SecurityUtil.AccessControlList;
@@ -349,9 +350,9 @@
}
}
TaskAttemptID attemptId = getTaskAttemptID(true, areAllMapsRunning);
+ JobSplit.TaskSplitMetaInfo split = JobSplit.EMPTY_TASK_SPLIT;
Task task = new MapTask(
- "", attemptId, 0, "", new BytesWritable(),
- super.numSlotsPerMap) {
+ "", attemptId, 0, split.getSplitIndex(), super.numSlotsPerMap) {
@Override
public String toString() {
return String.format("%s on %s", getTaskID(), tts.getTrackerName());
@@ -362,7 +363,7 @@
// create a fake TIP and keep track of it
FakeTaskInProgress mapTip = new FakeTaskInProgress(
getJobID(),
- getJobConf(), task, true, this);
+ getJobConf(), task, true, this, split);
mapTip.taskStatus.setRunState(TaskStatus.State.RUNNING);
if (areAllMapsRunning) {
speculativeMapTasks++;
@@ -408,7 +409,7 @@
// create a fake TIP and keep track of it
FakeTaskInProgress reduceTip = new FakeTaskInProgress(
getJobID(),
- getJobConf(), task, false, this);
+ getJobConf(), task, false, this, null);
reduceTip.taskStatus.setRunState(TaskStatus.State.RUNNING);
if (areAllReducesRunning) {
speculativeReduceTasks++;
@@ -499,8 +500,9 @@
FakeTaskInProgress(
JobID jId, JobConf jobConf, Task t,
- boolean isMap, FakeJobInProgress job) {
- super(jId, "", new Job.RawSplit(), null, jobConf, job, 0, 1);
+ boolean isMap, FakeJobInProgress job,
+ JobSplit.TaskSplitMetaInfo split) {
+ super(jId, "", split, null, jobConf, job, 0, 1);
this.isMap = isMap;
this.fakeJob = job;
activeTasks = new TreeMap<TaskAttemptID, String>();
diff --git a/src/contrib/data_join/ivy.xml b/src/contrib/data_join/ivy.xml
index 94401c6..de01782 100644
--- a/src/contrib/data_join/ivy.xml
+++ b/src/contrib/data_join/ivy.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<ivy-module version="1.0">
<info organisation="org.apache.hadoop" module="${ant.project.name}">
<license name="Apache 2.0"/>
diff --git a/src/contrib/data_join/ivy/libraries.properties b/src/contrib/data_join/ivy/libraries.properties
index a470b37..8a80dd8 100644
--- a/src/contrib/data_join/ivy/libraries.properties
+++ b/src/contrib/data_join/ivy/libraries.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
#This properties file lists the versions of the various artifacts used by streaming.
#It drives ivy and the generation of a maven POM
diff --git a/src/contrib/dynamic-scheduler/ivy.xml b/src/contrib/dynamic-scheduler/ivy.xml
index 53b2cc8..4bb8b88 100644
--- a/src/contrib/dynamic-scheduler/ivy.xml
+++ b/src/contrib/dynamic-scheduler/ivy.xml
@@ -29,6 +29,7 @@
<conf name="runtime" description="runtime but not the artifact" />
<conf name="common" visibility="private" extends="runtime" description="common artifacts"/>
+ <conf name="test" visibility="private" extends="runtime"/>
</configurations>
<publications>
@@ -48,8 +49,6 @@
<dependency org="commons-net" name="commons-net" rev="${commons-net.version}" conf="common->master"/>
<dependency org="org.mortbay.jetty" name="servlet-api-2.5" rev="${servlet-api-2.5.version}" conf="common->master"/>
<dependency org="junit" name="junit" rev="${junit.version}" conf="common->default"/>
- <dependency org="commons-logging" name="commons-logging-api" rev="${commons-logging-api.version}" conf="common->default"/>
<dependency org="org.slf4j" name="slf4j-api" rev="${slf4j-api.version}" conf="common->master"/>
- <dependency org="org.slf4j" name="slf4j-log4j12" rev="${slf4j-log4j12.version}" conf="common->master"/>
</dependencies>
</ivy-module>
diff --git a/src/contrib/eclipse-plugin/.classpath b/src/contrib/eclipse-plugin/.classpath
index bf69ddb..6a2ab98 100644
--- a/src/contrib/eclipse-plugin/.classpath
+++ b/src/contrib/eclipse-plugin/.classpath
@@ -1,4 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
<classpath>
<classpathentry excluding="org/apache/hadoop/eclipse/server/CopyOfHadoopServer.java" kind="src" path="src/java"/>
<classpathentry exported="true" kind="lib" path="classes" sourcepath="classes"/>
diff --git a/src/contrib/eclipse-plugin/.project b/src/contrib/eclipse-plugin/.project
index 2f8ba51..6f6058c 100644
--- a/src/contrib/eclipse-plugin/.project
+++ b/src/contrib/eclipse-plugin/.project
@@ -1,4 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
<projectDescription>
<name>MapReduceTools</name>
<comment></comment>
diff --git a/src/contrib/eclipse-plugin/.settings/org.eclipse.jdt.core.prefs b/src/contrib/eclipse-plugin/.settings/org.eclipse.jdt.core.prefs
index f22875e..6ec7145 100644
--- a/src/contrib/eclipse-plugin/.settings/org.eclipse.jdt.core.prefs
+++ b/src/contrib/eclipse-plugin/.settings/org.eclipse.jdt.core.prefs
@@ -1,4 +1,16 @@
#Sat Oct 13 13:37:43 CEST 2007
+# Licensed under the Apache License, Version 2.0 (the "License");
+# # you may not use this file except in compliance with the License.
+# # You may obtain a copy of the License at
+# #
+# # http://www.apache.org/licenses/LICENSE-2.0
+# #
+# # Unless required by applicable law or agreed to in writing, software
+# # distributed under the License is distributed on an "AS IS" BASIS,
+# # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# # See the License for the specific language governing permissions and
+# # limitations under the License.
+#
eclipse.preferences.version=1
instance/org.eclipse.core.net/org.eclipse.core.net.hasMigrated=true
org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
diff --git a/src/contrib/eclipse-plugin/.settings/org.eclipse.jdt.ui.prefs b/src/contrib/eclipse-plugin/.settings/org.eclipse.jdt.ui.prefs
index ea2f6cf..df64c00 100644
--- a/src/contrib/eclipse-plugin/.settings/org.eclipse.jdt.ui.prefs
+++ b/src/contrib/eclipse-plugin/.settings/org.eclipse.jdt.ui.prefs
@@ -1,4 +1,15 @@
#Tue Aug 14 19:41:15 PDT 2007
+## Licensed under the Apache License, Version 2.0 (the "License");
+## you may not use this file except in compliance with the License.
+## You may obtain a copy of the License at
+##
+## http://www.apache.org/licenses/LICENSE-2.0
+##
+## Unless required by applicable law or agreed to in writing, software
+## distributed under the License is distributed on an "AS IS" BASIS,
+## WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+## See the License for the specific language governing permissions and
+## limitations under the License.
eclipse.preferences.version=1
formatter_profile=_Lucene
formatter_settings_version=11
diff --git a/src/contrib/eclipse-plugin/.settings/org.eclipse.wst.validation.prefs b/src/contrib/eclipse-plugin/.settings/org.eclipse.wst.validation.prefs
index dd7d0cf..80040d4 100644
--- a/src/contrib/eclipse-plugin/.settings/org.eclipse.wst.validation.prefs
+++ b/src/contrib/eclipse-plugin/.settings/org.eclipse.wst.validation.prefs
@@ -1,4 +1,15 @@
#Tue Aug 14 19:41:15 PDT 2007
+## Licensed under the Apache License, Version 2.0 (the "License");
+## you may not use this file except in compliance with the License.
+## You may obtain a copy of the License at
+##
+## http://www.apache.org/licenses/LICENSE-2.0
+##
+## Unless required by applicable law or agreed to in writing, software
+## distributed under the License is distributed on an "AS IS" BASIS,
+## WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+## See the License for the specific language governing permissions and
+## limitations under the License.
DELEGATES_PREFERENCE=delegateValidatorListorg.eclipse.wst.xsd.core.internal.validation.eclipse.XSDDelegatingValidator\=org.eclipse.wst.xsd.core.internal.validation.eclipse.Validator;org.eclipse.wst.wsdl.validation.internal.eclipse.WSDLDelegatingValidator\=org.eclipse.wst.wsdl.validation.internal.eclipse.Validator;
USER_BUILD_PREFERENCE=enabledBuildValidatorListorg.eclipse.wst.xsd.core.internal.validation.eclipse.XSDDelegatingValidator;org.eclipse.jst.jsp.core.internal.validation.JSPContentValidator;org.eclipse.wst.html.internal.validation.HTMLValidator;org.eclipse.wst.xml.core.internal.validation.eclipse.Validator;org.eclipse.jst.jsf.validation.internal.appconfig.AppConfigValidator;org.eclipse.jst.jsp.core.internal.validation.JSPBatchValidator;org.eclipse.wst.dtd.core.internal.validation.eclipse.Validator;org.eclipse.wst.wsi.ui.internal.WSIMessageValidator;org.eclipse.wst.wsdl.validation.internal.eclipse.WSDLDelegatingValidator;org.eclipse.jst.jsf.validation.internal.JSPSemanticsValidator;
USER_MANUAL_PREFERENCE=enabledManualValidatorListorg.eclipse.wst.xsd.core.internal.validation.eclipse.XSDDelegatingValidator;org.eclipse.jst.jsp.core.internal.validation.JSPContentValidator;org.eclipse.wst.html.internal.validation.HTMLValidator;org.eclipse.wst.xml.core.internal.validation.eclipse.Validator;org.eclipse.jst.jsf.validation.internal.appconfig.AppConfigValidator;org.eclipse.jst.jsp.core.internal.validation.JSPBatchValidator;org.eclipse.wst.dtd.core.internal.validation.eclipse.Validator;org.eclipse.wst.wsi.ui.internal.WSIMessageValidator;org.eclipse.wst.wsdl.validation.internal.eclipse.WSDLDelegatingValidator;org.eclipse.jst.jsf.validation.internal.JSPSemanticsValidator;
diff --git a/src/contrib/eclipse-plugin/build.properties b/src/contrib/eclipse-plugin/build.properties
index b632af3..3aa55dd 100644
--- a/src/contrib/eclipse-plugin/build.properties
+++ b/src/contrib/eclipse-plugin/build.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
output.. = bin/
bin.includes = META-INF/,\
plugin.xml,\
diff --git a/src/contrib/eclipse-plugin/ivy.xml b/src/contrib/eclipse-plugin/ivy.xml
index fee8874..588583e 100644
--- a/src/contrib/eclipse-plugin/ivy.xml
+++ b/src/contrib/eclipse-plugin/ivy.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<ivy-module version="1.0">
<info organisation="org.apache.hadoop" module="${ant.project.name}">
<license name="Apache 2.0"/>
diff --git a/src/contrib/eclipse-plugin/ivy/libraries.properties b/src/contrib/eclipse-plugin/ivy/libraries.properties
index a470b37..8a80dd8 100644
--- a/src/contrib/eclipse-plugin/ivy/libraries.properties
+++ b/src/contrib/eclipse-plugin/ivy/libraries.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
#This properties file lists the versions of the various artifacts used by streaming.
#It drives ivy and the generation of a maven POM
diff --git a/src/contrib/eclipse-plugin/plugin.xml b/src/contrib/eclipse-plugin/plugin.xml
index 193734e..9b4165f 100644
--- a/src/contrib/eclipse-plugin/plugin.xml
+++ b/src/contrib/eclipse-plugin/plugin.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
<?eclipse version="3.2"?>
<plugin>
diff --git a/src/contrib/eclipse-plugin/resources/ConnectDFS.xml b/src/contrib/eclipse-plugin/resources/ConnectDFS.xml
index f9ca772..03be2b7 100644
--- a/src/contrib/eclipse-plugin/resources/ConnectDFS.xml
+++ b/src/contrib/eclipse-plugin/resources/ConnectDFS.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
<cheatsheet title="Set default Hadoop path tutorial">
<intro>
diff --git a/src/contrib/eclipse-plugin/resources/CreateProj.xml b/src/contrib/eclipse-plugin/resources/CreateProj.xml
index 652c997..4a71dcb 100644
--- a/src/contrib/eclipse-plugin/resources/CreateProj.xml
+++ b/src/contrib/eclipse-plugin/resources/CreateProj.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
<cheatsheet title="MapReduce project creation tutorial">
<intro>
<description>
diff --git a/src/contrib/eclipse-plugin/resources/HelloWorld.xml b/src/contrib/eclipse-plugin/resources/HelloWorld.xml
index 563a2a7..bae2815 100644
--- a/src/contrib/eclipse-plugin/resources/HelloWorld.xml
+++ b/src/contrib/eclipse-plugin/resources/HelloWorld.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
<compositeCheatsheet name="IBM MapReduce Tools for Eclipse">
<taskGroup name="Develop Hadoop Applications" kind="set">
<intro
@@ -118,4 +135,4 @@
</task>
</taskGroup>
</taskGroup>
-</compositeCheatsheet>
\ No newline at end of file
+</compositeCheatsheet>
diff --git a/src/contrib/eclipse-plugin/resources/RunProj.xml b/src/contrib/eclipse-plugin/resources/RunProj.xml
index f22ef47..db9c181 100644
--- a/src/contrib/eclipse-plugin/resources/RunProj.xml
+++ b/src/contrib/eclipse-plugin/resources/RunProj.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
<cheatsheet title="MapReduce project run tutorial">
<intro>
<description>
diff --git a/src/contrib/eclipse-plugin/resources/SetHadoopPath.xml b/src/contrib/eclipse-plugin/resources/SetHadoopPath.xml
index 743d4c1..9e5fd14 100644
--- a/src/contrib/eclipse-plugin/resources/SetHadoopPath.xml
+++ b/src/contrib/eclipse-plugin/resources/SetHadoopPath.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
<cheatsheet title="Set default Hadoop path tutorial">
<intro>
diff --git a/src/contrib/eclipse-plugin/resources/Setup.xml b/src/contrib/eclipse-plugin/resources/Setup.xml
index 677a151..ca7cfe8 100644
--- a/src/contrib/eclipse-plugin/resources/Setup.xml
+++ b/src/contrib/eclipse-plugin/resources/Setup.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+
<cheatsheet title="Open Browser">
<intro>
<description>This cheat sheet launches a browser to the Hadoop website.</description>
diff --git a/src/contrib/fairscheduler/ivy.xml b/src/contrib/fairscheduler/ivy.xml
index abd059f..8d85e04 100644
--- a/src/contrib/fairscheduler/ivy.xml
+++ b/src/contrib/fairscheduler/ivy.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<ivy-module version="1.0">
<info organisation="org.apache.hadoop" module="${ant.project.name}">
<license name="Apache 2.0"/>
@@ -15,6 +32,7 @@
<conf name="common" visibility="private"
description="artifacts needed to compile/test the application"/>
+ <conf name="test" visibility="private" extends="runtime"/>
</configurations>
<publications>
@@ -25,11 +43,11 @@
<dependency org="org.apache.hadoop" name="hadoop-core"
rev="${hadoop-core.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-core-test"
- rev="${hadoop-core.version}" conf="common->default"/>
+ rev="${hadoop-core.version}" conf="test->default"/>
<dependency org="org.apache.hadoop" name="hadoop-hdfs"
rev="${hadoop-core.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-hdfs-test"
- rev="${hadoop-core.version}" conf="common->default"/>
+ rev="${hadoop-core.version}" conf="test->default"/>
<dependency org="commons-logging"
name="commons-logging"
rev="${commons-logging.version}"
diff --git a/src/contrib/fairscheduler/ivy/libraries.properties b/src/contrib/fairscheduler/ivy/libraries.properties
index a470b37..8a80dd8 100644
--- a/src/contrib/fairscheduler/ivy/libraries.properties
+++ b/src/contrib/fairscheduler/ivy/libraries.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
#This properties file lists the versions of the various artifacts used by streaming.
#It drives ivy and the generation of a maven POM
diff --git a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java
index 5da002d..bc79326 100644
--- a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java
+++ b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java
@@ -927,14 +927,15 @@
for (JobInProgress job: jobs) {
JobProfile profile = job.getProfile();
JobInfo info = infos.get(job);
- // TODO: Fix
- /*eventLog.log("JOB",
+ Schedulable ms = info.mapSchedulable;
+ Schedulable rs = info.reduceSchedulable;
+ eventLog.log("JOB",
profile.getJobID(), profile.name, profile.user,
job.getPriority(), poolMgr.getPoolName(job),
- job.numMapTasks, info.runningMaps, info.neededMaps,
- info.mapFairShare, info.mapWeight, info.mapDeficit,
- job.numReduceTasks, info.runningReduces, info.neededReduces,
- info.reduceFairShare, info.reduceWeight, info.reduceDeficit);*/
+ job.numMapTasks, ms.getRunningTasks(),
+ ms.getDemand(), ms.getFairShare(), ms.getWeight(),
+ job.numReduceTasks, rs.getRunningTasks(),
+ rs.getDemand(), rs.getFairShare(), rs.getWeight());
}
// List pools in alphabetical order
List<Pool> pools = new ArrayList<Pool>(poolMgr.getPools());
diff --git a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerServlet.java b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerServlet.java
index 78d7828..afaa57d 100644
--- a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerServlet.java
+++ b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerServlet.java
@@ -151,16 +151,17 @@
*/
private void showPools(PrintWriter out, boolean advancedView) {
synchronized(scheduler) {
+ boolean warnInverted = false;
PoolManager poolManager = scheduler.getPoolManager();
out.print("<h2>Pools</h2>\n");
out.print("<table border=\"2\" cellpadding=\"5\" cellspacing=\"2\">\n");
out.print("<tr><th rowspan=2>Pool</th>" +
"<th rowspan=2>Running Jobs</th>" +
- "<th colspan=3>Map Tasks</th>" +
- "<th colspan=3>Reduce Tasks</th>" +
+ "<th colspan=4>Map Tasks</th>" +
+ "<th colspan=4>Reduce Tasks</th>" +
"<th rowspan=2>Scheduling Mode</th></tr>\n<tr>" +
- "<th>Min Share</th><th>Running</th><th>Fair Share</th>" +
- "<th>Min Share</th><th>Running</th><th>Fair Share</th></tr>\n");
+ "<th>Min Share</th><th>Max Share</th><th>Running</th><th>Fair Share</th>" +
+ "<th>Min Share</th><th>Max Share</th><th>Running</th><th>Fair Share</th></tr>\n");
List<Pool> pools = new ArrayList<Pool>(poolManager.getPools());
Collections.sort(pools, new Comparator<Pool>() {
public int compare(Pool p1, Pool p2) {
@@ -174,21 +175,51 @@
String name = pool.getName();
int runningMaps = pool.getMapSchedulable().getRunningTasks();
int runningReduces = pool.getReduceSchedulable().getRunningTasks();
+ int maxMaps = poolManager.getMaxSlots(name, TaskType.MAP);
+ int maxReduces = poolManager.getMaxSlots(name, TaskType.REDUCE);
+ boolean invertedMaps = poolManager.invertedMinMax(TaskType.MAP, name);
+ boolean invertedReduces = poolManager.invertedMinMax(TaskType.REDUCE, name);
+ warnInverted = warnInverted || invertedMaps || invertedReduces;
out.print("<tr>");
out.printf("<td>%s</td>", name);
out.printf("<td>%d</td>", pool.getJobs().size());
+ // Map Tasks
out.printf("<td>%d</td>", poolManager.getAllocation(name,
TaskType.MAP));
+ out.print("<td>");
+ if(maxMaps == Integer.MAX_VALUE) {
+ out.print("-");
+ } else {
+ out.print(maxMaps);
+ }
+ if(invertedMaps) {
+ out.print("*");
+ }
+ out.print("</td>");
out.printf("<td>%d</td>", runningMaps);
out.printf("<td>%.1f</td>", pool.getMapSchedulable().getFairShare());
+ // Reduce Tasks
out.printf("<td>%d</td>", poolManager.getAllocation(name,
TaskType.REDUCE));
+ out.print("<td>");
+ if(maxReduces == Integer.MAX_VALUE) {
+ out.print("-");
+ } else {
+ out.print(maxReduces);
+ }
+ if(invertedReduces) {
+ out.print("*");
+ }
+ out.print("</td>");
out.printf("<td>%d</td>", runningReduces);
out.printf("<td>%.1f</td>", pool.getReduceSchedulable().getFairShare());
out.printf("<td>%s</td>", pool.getSchedulingMode());
out.print("</tr>\n");
}
out.print("</table>\n");
+ if(warnInverted) {
+ out.print("<p>* One or more pools have max share set lower than min share. Max share will be used and minimum will be treated as if set equal to max.</p>");
+ }
}
}
diff --git a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/JobSchedulable.java b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/JobSchedulable.java
index bff248c..42039c9 100644
--- a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/JobSchedulable.java
+++ b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/JobSchedulable.java
@@ -58,7 +58,7 @@
// - have N attempts running, in which case it demands N slots, and may
// potentially demand one more slot if it needs to be speculated
TaskInProgress[] tips = (taskType == TaskType.MAP ?
- job.getMapTasks() : job.getReduceTasks());
+ job.getTasks(TaskType.MAP) : job.getTasks(TaskType.REDUCE));
boolean speculationEnabled = (taskType == TaskType.MAP ?
job.hasSpeculativeMaps() : job.hasSpeculativeReduces());
long time = scheduler.getClock().getTime();
diff --git a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java
index f3fde46..fbf3cb8 100644
--- a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java
+++ b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java
@@ -69,6 +69,10 @@
private Map<String, Integer> mapAllocs = new HashMap<String, Integer>();
private Map<String, Integer> reduceAllocs = new HashMap<String, Integer>();
+ // If set, cap number of map and reduce tasks in a pool
+ private Map<String, Integer> poolMaxMaps = new HashMap<String, Integer>();
+ private Map<String, Integer> poolMaxReduces = new HashMap<String, Integer>();
+
// Sharing weights for each pool
private Map<String, Double> poolWeights = new HashMap<String, Double>();
@@ -221,6 +225,8 @@
Map<String, Integer> reduceAllocs = new HashMap<String, Integer>();
Map<String, Integer> poolMaxJobs = new HashMap<String, Integer>();
Map<String, Integer> userMaxJobs = new HashMap<String, Integer>();
+ Map<String, Integer> poolMaxMaps = new HashMap<String, Integer>();
+ Map<String, Integer> poolMaxReduces = new HashMap<String, Integer>();
Map<String, Double> poolWeights = new HashMap<String, Double>();
Map<String, SchedulingMode> poolModes = new HashMap<String, SchedulingMode>();
Map<String, Long> minSharePreemptionTimeouts = new HashMap<String, Long>();
@@ -271,6 +277,14 @@
String text = ((Text)field.getFirstChild()).getData().trim();
int val = Integer.parseInt(text);
reduceAllocs.put(poolName, val);
+ } else if ("maxMaps".equals(field.getTagName())) {
+ String text = ((Text)field.getFirstChild()).getData().trim();
+ int val = Integer.parseInt(text);
+ poolMaxMaps.put(poolName, val);
+ } else if ("maxReduces".equals(field.getTagName())) {
+ String text = ((Text)field.getFirstChild()).getData().trim();
+ int val = Integer.parseInt(text);
+ poolMaxReduces.put(poolName, val);
} else if ("maxRunningJobs".equals(field.getTagName())) {
String text = ((Text)field.getFirstChild()).getData().trim();
int val = Integer.parseInt(text);
@@ -288,6 +302,16 @@
poolModes.put(poolName, parseSchedulingMode(text));
}
}
+ if (poolMaxMaps.containsKey(poolName) && mapAllocs.containsKey(poolName)
+ && poolMaxMaps.get(poolName) < mapAllocs.get(poolName)) {
+ LOG.warn(String.format("Pool %s has max maps %d less than min maps %d",
+ poolName, poolMaxMaps.get(poolName), mapAllocs.get(poolName)));
+ }
+ if(poolMaxReduces.containsKey(poolName) && reduceAllocs.containsKey(poolName)
+ && poolMaxReduces.get(poolName) < reduceAllocs.get(poolName)) {
+ LOG.warn(String.format("Pool %s has max reduces %d less than min reduces %d",
+ poolName, poolMaxReduces.get(poolName), reduceAllocs.get(poolName)));
+ }
} else if ("user".equals(element.getTagName())) {
String userName = element.getAttribute("name");
NodeList fields = element.getChildNodes();
@@ -331,6 +355,8 @@
synchronized(this) {
this.mapAllocs = mapAllocs;
this.reduceAllocs = reduceAllocs;
+ this.poolMaxMaps = poolMaxMaps;
+ this.poolMaxReduces = poolMaxReduces;
this.poolMaxJobs = poolMaxJobs;
this.userMaxJobs = userMaxJobs;
this.poolWeights = poolWeights;
@@ -351,6 +377,25 @@
}
}
+ /**
+ * Does the pool have incompatible max and min allocation set.
+ *
+ * @param type
+ * {@link TaskType#MAP} or {@link TaskType#REDUCE}
+ * @param pool
+ * the pool name
+ * @return true if the max is less than the min
+ */
+ boolean invertedMinMax(TaskType type, String pool) {
+ Map<String, Integer> max = TaskType.MAP == type ? poolMaxMaps : poolMaxReduces;
+ Map<String, Integer> min = TaskType.MAP == type ? mapAllocs : reduceAllocs;
+ if (max.containsKey(pool) && min.containsKey(pool)
+ && max.get(pool) < min.get(pool)) {
+ return true;
+ }
+ return false;
+ }
+
private SchedulingMode parseSchedulingMode(String text)
throws AllocationConfigurationException {
text = text.toLowerCase();
@@ -373,7 +418,20 @@
Integer alloc = allocationMap.get(pool);
return (alloc == null ? 0 : alloc);
}
-
+
+ /**
+ * Get the maximum map or reduce slots for the given pool.
+ * @return the cap set on this pool, or Integer.MAX_VALUE if not set.
+ */
+ int getMaxSlots(String poolName, TaskType taskType) {
+ Map<String, Integer> maxMap = (taskType == TaskType.MAP ? poolMaxMaps : poolMaxReduces);
+ if (maxMap.containsKey(poolName)) {
+ return maxMap.get(poolName);
+ } else {
+ return Integer.MAX_VALUE;
+ }
+ }
+
/**
* Add a job in the appropriate pool
*/
diff --git a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolSchedulable.java b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolSchedulable.java
index e612ce9..3fbcab3 100644
--- a/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolSchedulable.java
+++ b/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolSchedulable.java
@@ -82,6 +82,11 @@
sched.updateDemand();
demand += sched.getDemand();
}
+ // if demand exceeds the cap for this pool, limit to the max
+ int maxTasks = poolMgr.getMaxSlots(pool.getName(), taskType);
+ if(demand > maxTasks) {
+ demand = maxTasks;
+ }
}
/**
@@ -135,6 +140,10 @@
@Override
public Task assignTask(TaskTrackerStatus tts, long currentTime,
Collection<JobInProgress> visited) throws IOException {
+ int runningTasks = getRunningTasks();
+ if (runningTasks >= poolMgr.getMaxSlots(pool.getName(), taskType)) {
+ return null;
+ }
SchedulingMode mode = pool.getSchedulingMode();
Comparator<Schedulable> comparator;
if (mode == SchedulingMode.FIFO) {
diff --git a/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java b/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java
index 27660f7..19306ae 100644
--- a/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java
+++ b/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java
@@ -43,6 +43,7 @@
import org.apache.hadoop.mapred.UtilsForTests.FakeClock;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
+import org.apache.hadoop.mapreduce.split.JobSplit;
import org.apache.hadoop.net.Node;
public class TestFairScheduler extends TestCase {
@@ -111,12 +112,14 @@
// create maps
numMapTasks = conf.getNumMapTasks();
maps = new TaskInProgress[numMapTasks];
+ // empty format
+ JobSplit.TaskSplitMetaInfo split = JobSplit.EMPTY_TASK_SPLIT;
for (int i = 0; i < numMapTasks; i++) {
String[] inputLocations = null;
if (mapInputLocations != null)
inputLocations = mapInputLocations[i];
maps[i] = new FakeTaskInProgress(getJobID(), i,
- getJobConf(), this, inputLocations);
+ getJobConf(), this, inputLocations, split);
if (mapInputLocations == null) // Job has no locality info
nonLocalMaps.add(maps[i]);
}
@@ -137,7 +140,8 @@
if (!tip.isRunning() && !tip.isComplete() &&
getLocalityLevel(tip, tts) < localityLevel) {
TaskAttemptID attemptId = getTaskAttemptID(tip);
- Task task = new MapTask("", attemptId, 0, "", new BytesWritable(), 1) {
+ JobSplit.TaskSplitMetaInfo split = JobSplit.EMPTY_TASK_SPLIT;
+ Task task = new MapTask("", attemptId, 0, split.getSplitIndex(), 1) {
@Override
public String toString() {
return String.format("%s on %s", getTaskID(), tts.getTrackerName());
@@ -231,8 +235,9 @@
// Constructor for map
FakeTaskInProgress(JobID jId, int id, JobConf jobConf,
- FakeJobInProgress job, String[] inputLocations) {
- super(jId, "", new Job.RawSplit(), null, jobConf, job, id, 1);
+ FakeJobInProgress job, String[] inputLocations,
+ JobSplit.TaskSplitMetaInfo split) {
+ super(jId, "", split, null, jobConf, job, id, 1);
this.isMap = true;
this.fakeJob = job;
this.inputLocations = inputLocations;
@@ -829,7 +834,8 @@
// Finish up the tasks and advance time again. Note that we must finish
// the task since FakeJobInProgress does not properly maintain running
// tasks, so the scheduler will always get an empty task list from
- // the JobInProgress's getMapTasks/getReduceTasks and think they finished.
+ // the JobInProgress's getTasks(TaskType.MAP)/getTasks(TaskType.REDUCE) and
+ // think they finished.
taskTrackerManager.finishTask("tt1", "attempt_test_0001_m_000000_0");
taskTrackerManager.finishTask("tt1", "attempt_test_0002_m_000000_0");
taskTrackerManager.finishTask("tt1", "attempt_test_0001_r_000000_0");
@@ -937,7 +943,8 @@
// Finish up the tasks and advance time again. Note that we must finish
// the task since FakeJobInProgress does not properly maintain running
// tasks, so the scheduler will always get an empty task list from
- // the JobInProgress's getMapTasks/getReduceTasks and think they finished.
+ // the JobInProgress's getTasks(TaskType.MAP)/getTasks(TaskType.REDUCE) and
+ // think they finished.
taskTrackerManager.finishTask("tt1", "attempt_test_0001_m_000000_0");
taskTrackerManager.finishTask("tt1", "attempt_test_0002_m_000000_0");
taskTrackerManager.finishTask("tt1", "attempt_test_0001_r_000000_0");
@@ -1640,6 +1647,45 @@
assertEquals(1.33, info3.reduceSchedulable.getFairShare(), 0.01);
}
+ public void testPoolMaxMapsReduces() throws Exception {
+ PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+ out.println("<?xml version=\"1.0\"?>");
+ out.println("<allocations>");
+ // Pool with upper bound
+ out.println("<pool name=\"poolLimited\">");
+ out.println("<weight>1.0</weight>");
+ out.println("<maxMaps>2</maxMaps>");
+ out.println("<maxReduces>1</maxReduces>");
+ out.println("</pool>");
+ out.println("</allocations>");
+ out.close();
+ scheduler.getPoolManager().reloadAllocs();
+ // Create two jobs with ten maps
+ JobInProgress job1 = submitJob(JobStatus.RUNNING, 10, 5, "poolLimited");
+ advanceTime(10);
+ JobInProgress job2 = submitJob(JobStatus.RUNNING, 10, 5);
+ checkAssignment("tt1", "attempt_test_0002_m_000000_0 on tt1");
+ checkAssignment("tt1", "attempt_test_0002_r_000000_0 on tt1");
+ checkAssignment("tt1", "attempt_test_0001_m_000000_0 on tt1");
+ checkAssignment("tt1", "attempt_test_0001_r_000000_0 on tt1");
+ checkAssignment("tt2", "attempt_test_0002_m_000001_0 on tt2");
+ checkAssignment("tt2", "attempt_test_0002_r_000001_0 on tt2");
+ checkAssignment("tt2", "attempt_test_0001_m_000001_0 on tt2");
+ checkAssignment("tt2", "attempt_test_0002_r_000002_0 on tt2");
+
+ Pool limited = scheduler.getPoolManager().getPool("poolLimited");
+ assertEquals(2, limited.getSchedulable(TaskType.MAP).getRunningTasks());
+ assertEquals(1, limited.getSchedulable(TaskType.REDUCE).getRunningTasks());
+ Pool defaultPool = scheduler.getPoolManager().getPool("default");
+ assertEquals(2, defaultPool.getSchedulable(TaskType.MAP).getRunningTasks());
+ assertEquals(3, defaultPool.getSchedulable(TaskType.REDUCE)
+ .getRunningTasks());
+ assertEquals(2, job1.runningMapTasks);
+ assertEquals(1, job1.runningReduceTasks);
+ assertEquals(2, job2.runningMapTasks);
+ assertEquals(3, job2.runningReduceTasks);
+ }
+
/**
* Tests that max-running-tasks per node are set by assigning load
* equally accross the cluster in CapBasedLoadManager.
diff --git a/src/contrib/gridmix/ivy.xml b/src/contrib/gridmix/ivy.xml
index 31fca1c..7510fa5 100644
--- a/src/contrib/gridmix/ivy.xml
+++ b/src/contrib/gridmix/ivy.xml
@@ -28,6 +28,7 @@
<conf name="common" visibility="private" extends="runtime"
description="artifacts needed to compile/test the application"/>
+ <conf name="test" visibility="private" extends="runtime"/>
</configurations>
<publications>
@@ -42,7 +43,7 @@
<dependency org="org.apache.hadoop"
name="hadoop-core-test"
rev="${hadoop-core.version}"
- conf="common->default"/>
+ conf="test->default"/>
<dependency org="org.apache.hadoop"
name="hadoop-hdfs"
rev="${hadoop-hdfs.version}"
@@ -50,7 +51,7 @@
<dependency org="org.apache.hadoop"
name="hadoop-hdfs-test"
rev="${hadoop-hdfs.version}"
- conf="common->default"/>
+ conf="test->default"/>
<dependency org="commons-logging"
name="commons-logging"
rev="${commons-logging.version}"
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/AvgRecordFactory.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/AvgRecordFactory.java
new file mode 100644
index 0000000..f46faea
--- /dev/null
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/AvgRecordFactory.java
@@ -0,0 +1,91 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Given byte and record targets, emit roughly equal-sized records satisfying
+ * the contract.
+ */
+class AvgRecordFactory extends RecordFactory {
+
+ /**
+ * Percentage of record for key data.
+ */
+ public static final String GRIDMIX_KEY_FRC = "gridmix.key.fraction";
+
+
+ private final long targetBytes;
+ private final long targetRecords;
+ private final long step;
+ private final int avgrec;
+ private final int keyLen;
+ private long accBytes = 0L;
+ private long accRecords = 0L;
+
+ /**
+ * @param targetBytes Expected byte count.
+ * @param targetRecords Expected record count.
+ * @param conf Used to resolve edge cases @see #GRIDMIX_KEY_FRC
+ */
+ public AvgRecordFactory(long targetBytes, long targetRecords,
+ Configuration conf) {
+ this.targetBytes = targetBytes;
+ this.targetRecords = targetRecords <= 0 && this.targetBytes >= 0
+ ? Math.max(1,
+ this.targetBytes / conf.getInt("gridmix.missing.rec.size", 64 * 1024))
+ : targetRecords;
+ final long tmp = this.targetBytes / this.targetRecords;
+ step = this.targetBytes - this.targetRecords * tmp;
+ avgrec = (int) Math.min(Integer.MAX_VALUE, tmp + 1);
+ keyLen = Math.max(1,
+ (int)(tmp * Math.min(1.0f, conf.getFloat(GRIDMIX_KEY_FRC, 0.1f))));
+ }
+
+ @Override
+ public boolean next(GridmixKey key, GridmixRecord val) throws IOException {
+ if (accBytes >= targetBytes) {
+ return false;
+ }
+ final int reclen = accRecords++ >= step ? avgrec - 1 : avgrec;
+ final int len = (int) Math.min(targetBytes - accBytes, reclen);
+ // len != reclen?
+ if (key != null) {
+ key.setSize(keyLen);
+ val.setSize(len - key.getSize());
+ } else {
+ val.setSize(len);
+ }
+ accBytes += len;
+ return true;
+ }
+
+ @Override
+ public float getProgress() throws IOException {
+ return Math.min(1.0f, accBytes / ((float)targetBytes));
+ }
+
+ @Override
+ public void close() throws IOException {
+ // noop
+ }
+
+}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FilePool.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FilePool.java
index fca1cbb..ab65489 100644
--- a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FilePool.java
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FilePool.java
@@ -223,7 +223,6 @@
return getSize();
}
- // TODO sort, pick rand pairs of kth large/small in dir
IndexMapper mapping;
if ((curdir.size() < 200) || ((double) targetSize / getSize() > 0.5)) {
mapping = new DenseIndexMapper(curdir.size());
@@ -234,13 +233,13 @@
ArrayList<Integer> selected = new ArrayList<Integer>();
long ret = 0L;
int poolSize = curdir.size();
- while (ret < targetSize) {
+ do {
int pos = rand.nextInt(poolSize);
int index = mapping.get(pos);
selected.add(index);
ret += curdir.get(index).getLen();
mapping.swap(pos, --poolSize);
- }
+ } while (ret < targetSize);
for (Integer i : selected) {
files.add(curdir.get(i));
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FileQueue.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FileQueue.java
new file mode 100644
index 0000000..840f83c
--- /dev/null
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FileQueue.java
@@ -0,0 +1,105 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
+
+/**
+ * Given a {@link org.apache.hadoop.mapreduce.lib.input.CombineFileSplit},
+ * circularly read through each input source.
+ */
+class FileQueue extends InputStream {
+
+ private int idx = -1;
+ private long curlen = -1L;
+ private FSDataInputStream input;
+ private final byte[] z = new byte[1];
+ private final Path[] paths;
+ private final long[] lengths;
+ private final long[] startoffset;
+ private final Configuration conf;
+
+ /**
+ * @param split Description of input sources.
+ * @param conf Used to resolve FileSystem instances.
+ */
+ public FileQueue(CombineFileSplit split, Configuration conf)
+ throws IOException {
+ this.conf = conf;
+ paths = split.getPaths();
+ startoffset = split.getStartOffsets();
+ lengths = split.getLengths();
+ nextSource();
+ }
+
+ protected void nextSource() throws IOException {
+ if (0 == paths.length) {
+ return;
+ }
+ if (input != null) {
+ input.close();
+ }
+ idx = (idx + 1) % paths.length;
+ curlen = lengths[idx];
+ final Path file = paths[idx];
+ final FileSystem fs = file.getFileSystem(conf);
+ input = fs.open(file);
+ input.seek(startoffset[idx]);
+ }
+
+ @Override
+ public int read() throws IOException {
+ final int tmp = read(z);
+ return tmp == -1 ? -1 : (0xFF & z[0]);
+ }
+
+ @Override
+ public int read(byte[] b) throws IOException {
+ return read(b, 0, b.length);
+ }
+
+ @Override
+ public int read(byte[] b, int off, int len) throws IOException {
+ int kvread = 0;
+ while (kvread < len) {
+ if (curlen <= 0) {
+ nextSource();
+ continue;
+ }
+ final int srcRead = (int) Math.min(len - kvread, curlen);
+ IOUtils.readFully(input, b, kvread, srcRead);
+ curlen -= srcRead;
+ kvread += srcRead;
+ }
+ return kvread;
+ }
+
+ @Override
+ public void close() throws IOException {
+ input.close();
+ }
+
+}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateData.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateData.java
index ae8b137..931fb54 100644
--- a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateData.java
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateData.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.io.DataInput;
import java.io.DataOutput;
+import java.io.OutputStream;
import java.util.Arrays;
import java.util.ArrayList;
import java.util.List;
@@ -28,7 +29,6 @@
import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
@@ -52,10 +52,30 @@
// TODO can replace with form of GridmixJob
class GenerateData extends GridmixJob {
+ /**
+ * Total bytes to write.
+ */
+ public static final String GRIDMIX_GEN_BYTES = "gridmix.gen.bytes";
+
+ /**
+ * Maximum size per file written.
+ */
+ public static final String GRIDMIX_GEN_CHUNK = "gridmix.gen.bytes.per.file";
+
+ /**
+ * Size of writes to output file.
+ */
+ public static final String GRIDMIX_VAL_BYTES = "gendata.val.bytes";
+
+ /**
+ * Status reporting interval, in megabytes.
+ */
+ public static final String GRIDMIX_GEN_INTERVAL = "gendata.interval.mb";
+
public GenerateData(Configuration conf, Path outdir, long genbytes)
throws IOException {
super(conf, 0L, "GRIDMIX_GENDATA");
- job.getConfiguration().setLong("gridmix.gendata.bytes", genbytes);
+ job.getConfiguration().setLong(GRIDMIX_GEN_BYTES, genbytes);
FileOutputFormat.setOutputPath(job, outdir);
}
@@ -84,7 +104,7 @@
protected void setup(Context context)
throws IOException, InterruptedException {
val = new BytesWritable(new byte[
- context.getConfiguration().getInt("gendata.val.bytes", 1024 * 1024)]);
+ context.getConfiguration().getInt(GRIDMIX_VAL_BYTES, 1024 * 1024)]);
}
@Override
@@ -106,7 +126,7 @@
final JobClient client = new JobClient(jobCtxt.getConfiguration());
ClusterStatus stat = client.getClusterStatus(true);
final long toGen =
- jobCtxt.getConfiguration().getLong("gridmix.gendata.bytes", -1);
+ jobCtxt.getConfiguration().getLong(GRIDMIX_GEN_BYTES, -1);
if (toGen < 0) {
throw new IOException("Invalid/missing generation bytes: " + toGen);
}
@@ -144,7 +164,7 @@
throws IOException, InterruptedException {
toWrite = split.getLength();
RINTERVAL = ctxt.getConfiguration().getInt(
- "gendata.report.interval.mb", 10) << 20;
+ GRIDMIX_GEN_INTERVAL, 10) << 20;
}
@Override
public boolean nextKeyValue() throws IOException {
@@ -219,20 +239,52 @@
public RecordWriter<NullWritable,BytesWritable> getRecordWriter(
TaskAttemptContext job) throws IOException {
- Path file = getDefaultWorkFile(job, "");
- FileSystem fs = file.getFileSystem(job.getConfiguration());
- final FSDataOutputStream fileOut = fs.create(file, false);
- return new RecordWriter<NullWritable,BytesWritable>() {
- @Override
- public void write(NullWritable key, BytesWritable value)
- throws IOException {
- fileOut.write(value.getBytes(), 0, value.getLength());
- }
- @Override
- public void close(TaskAttemptContext ctxt) throws IOException {
+ return new ChunkWriter(getDefaultWorkFile(job, ""),
+ job.getConfiguration());
+ }
+
+ static class ChunkWriter extends RecordWriter<NullWritable,BytesWritable> {
+ private final Path outDir;
+ private final FileSystem fs;
+ private final long maxFileBytes;
+
+ private long accFileBytes = 0L;
+ private long fileIdx = -1L;
+ private OutputStream fileOut = null;
+
+ public ChunkWriter(Path outDir, Configuration conf) throws IOException {
+ this.outDir = outDir;
+ fs = outDir.getFileSystem(conf);
+ maxFileBytes = conf.getLong(GRIDMIX_GEN_CHUNK, 1L << 30);
+ nextDestination();
+ }
+ private void nextDestination() throws IOException {
+ if (fileOut != null) {
fileOut.close();
}
- };
+ fileOut = fs.create(new Path(outDir, "segment-" + (++fileIdx)), false);
+ accFileBytes = 0L;
+ }
+ @Override
+ public void write(NullWritable key, BytesWritable value)
+ throws IOException {
+ int written = 0;
+ final int total = value.getLength();
+ while (written < total) {
+ final int write = (int)
+ Math.min(total - written, maxFileBytes - accFileBytes);
+ fileOut.write(value.getBytes(), written, write);
+ written += write;
+ accFileBytes += write;
+ if (accFileBytes >= maxFileBytes) {
+ nextDestination();
+ }
+ }
+ }
+ @Override
+ public void close(TaskAttemptContext ctxt) throws IOException {
+ fileOut.close();
+ }
}
}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java
index 95597f5..40393fe 100644
--- a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java
@@ -26,7 +26,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.mapreduce.Job;
@@ -73,12 +72,11 @@
"gridmix.client.pending.queue.depth";
/**
- * Size of key data in synthetic jobs. At present, key length is not
- * available in job traces. Since all solutions are equally bad, globally
- * specifying the amount of each record that is key data is the simplest
- * to implement and the method chosen.
+ * Multiplier to accelerate or decelerate job submission. As a crude means of
+ * sizing a job trace to a cluster, the time separating two jobs is
+ * multiplied by this factor.
*/
- public static final String GRIDMIX_KEY_LEN = "gridmix.min.key.length";
+ public static final String GRIDMIX_SUB_MUL = "gridmix.submit.multiplier";
// Submit data structures
private JobFactory factory;
@@ -135,7 +133,7 @@
submitter = createJobSubmitter(monitor,
conf.getInt(GRIDMIX_SUB_THR,
Runtime.getRuntime().availableProcessors() + 1),
- conf.getInt(GRIDMIX_QUE_DEP, 100),
+ conf.getInt(GRIDMIX_QUE_DEP, 5),
new FilePool(conf, ioPath));
factory = createJobFactory(submitter, traceIn, scratchDir, conf, startFlag);
monitor.start();
@@ -182,12 +180,10 @@
printUsage(System.err);
return 1;
}
- FileSystem fs = null;
InputStream trace = null;
try {
final Configuration conf = getConf();
Path scratchDir = new Path(ioPath, conf.get(GRIDMIX_OUT_DIR, "gridmix"));
- fs = scratchDir.getFileSystem(conf);
// add shutdown hook for SIGINT, etc.
Runtime.getRuntime().addShutdownHook(sdh);
CountDownLatch startFlag = new CountDownLatch(1);
@@ -210,7 +206,7 @@
if (factory != null) {
// wait for input exhaustion
- factory.join();
+ factory.join(Long.MAX_VALUE);
final Throwable badTraceException = factory.error();
if (null != badTraceException) {
LOG.error("Error in trace", badTraceException);
@@ -218,10 +214,10 @@
}
// wait for pending tasks to be submitted
submitter.shutdown();
- submitter.join();
+ submitter.join(Long.MAX_VALUE);
// wait for running tasks to complete
monitor.shutdown();
- monitor.join();
+ monitor.join(Long.MAX_VALUE);
}
} finally {
IOUtils.cleanup(LOG, trace);
@@ -236,13 +232,17 @@
*/
class Shutdown extends Thread {
- private void killComponent(Component<?> component) {
+ static final long FAC_SLEEP = 1000;
+ static final long SUB_SLEEP = 4000;
+ static final long MON_SLEEP = 15000;
+
+ private void killComponent(Component<?> component, long maxwait) {
if (component == null) {
return;
}
- component.abort(); // read no more tasks
+ component.abort();
try {
- component.join();
+ component.join(maxwait);
} catch (InterruptedException e) {
LOG.warn("Interrupted waiting for " + component);
}
@@ -253,9 +253,9 @@
public void run() {
LOG.info("Exiting...");
try {
- killComponent(factory); // read no more tasks
- killComponent(submitter); // submit no more tasks
- killComponent(monitor); // process remaining jobs in this thread
+ killComponent(factory, FAC_SLEEP); // read no more tasks
+ killComponent(submitter, SUB_SLEEP); // submit no more tasks
+ killComponent(monitor, MON_SLEEP); // process remaining jobs here
} finally {
if (monitor == null) {
return;
@@ -306,7 +306,8 @@
out.printf(" %-40s : Output directory\n", GRIDMIX_OUT_DIR);
out.printf(" %-40s : Submitting threads\n", GRIDMIX_SUB_THR);
out.printf(" %-40s : Queued job desc\n", GRIDMIX_QUE_DEP);
- out.printf(" %-40s : Key size\n", GRIDMIX_KEY_LEN);
+ out.printf(" %-40s : Key fraction of rec\n",
+ AvgRecordFactory.GRIDMIX_KEY_FRC);
}
/**
@@ -331,7 +332,7 @@
* Wait until the service completes. It is assumed that either a
* {@link #shutdown} or {@link #abort} has been requested.
*/
- void join() throws InterruptedException;
+ void join(long millis) throws InterruptedException;
/**
* Shut down gracefully, finishing all pending work. Reject new requests.
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixJob.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixJob.java
index 63b8312..1b5c104 100644
--- a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixJob.java
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixJob.java
@@ -17,22 +17,10 @@
*/
package org.apache.hadoop.mapred.gridmix;
-import java.io.DataInput;
-import java.io.DataOutput;
import java.io.IOException;
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-import java.nio.IntBuffer;
-import java.nio.LongBuffer;
import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.EnumSet;
import java.util.Formatter;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map.Entry;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
@@ -40,15 +28,10 @@
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.Writable;
@@ -65,7 +48,6 @@
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.tools.rumen.JobStory;
@@ -80,6 +62,7 @@
class GridmixJob implements Callable<Job>, Delayed {
public static final String JOBNAME = "GRIDMIX";
+ public static final String ORIGNAME = "gridmix.job.name.original";
public static final Log LOG = LogFactory.getLog(GridmixJob.class);
private static final ThreadLocal<Formatter> nameFormat =
@@ -180,14 +163,17 @@
job.setReducerClass(GridmixReducer.class);
job.setNumReduceTasks(jobdesc.getNumberReduces());
job.setMapOutputKeyClass(GridmixKey.class);
- job.setMapOutputValueClass(BytesWritable.class);
- job.setSortComparatorClass(BytesWritable.Comparator.class);
+ job.setMapOutputValueClass(GridmixRecord.class);
+ job.setSortComparatorClass(GridmixKey.Comparator.class);
job.setGroupingComparatorClass(SpecGroupingComparator.class);
job.setInputFormatClass(GridmixInputFormat.class);
job.setOutputFormatClass(RawBytesOutputFormat.class);
job.setPartitionerClass(DraftPartitioner.class);
job.setJarByClass(GridmixJob.class);
job.getConfiguration().setInt("gridmix.job.seq", seq);
+ job.getConfiguration().set(ORIGNAME, null == jobdesc.getJobID()
+ ? "<unknown>" : jobdesc.getJobID().toString());
+ job.getConfiguration().setBoolean(Job.USED_GENERIC_PARSER, true);
FileInputFormat.addInputPath(job, new Path("ignored"));
FileOutputFormat.setOutputPath(job, outdir);
job.submit();
@@ -200,11 +186,10 @@
}
}
- /**
- * Group REDUCE_SPEC records together
- */
public static class SpecGroupingComparator
- implements RawComparator<GridmixKey>, Serializable {
+ implements RawComparator<GridmixKey> {
+ private final DataInputBuffer di = new DataInputBuffer();
+ private final byte[] reset = di.getData();
@Override
public int compare(GridmixKey g1, GridmixKey g2) {
final byte t1 = g1.getType();
@@ -215,284 +200,128 @@
}
assert t1 == GridmixKey.DATA;
assert t2 == GridmixKey.DATA;
- return WritableComparator.compareBytes(
- g1.getBytes(), 0, g1.getLength(),
- g2.getBytes(), 0, g2.getLength());
+ return g1.compareTo(g2);
}
@Override
- public int compare(byte[] b1, int s1, int l1,
- byte[] b2, int s2, int l2) {
- final byte t1 = b1[s1 + 4];
- final byte t2 = b2[s2 + 4];
- if (t1 == GridmixKey.REDUCE_SPEC ||
- t2 == GridmixKey.REDUCE_SPEC) {
- return t1 - t2;
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ try {
+ final int ret;
+ di.reset(b1, s1, l1);
+ final int x1 = WritableUtils.readVInt(di);
+ di.reset(b2, s2, l2);
+ final int x2 = WritableUtils.readVInt(di);
+ final int t1 = b1[s1 + x1];
+ final int t2 = b2[s2 + x2];
+ if (t1 == GridmixKey.REDUCE_SPEC ||
+ t2 == GridmixKey.REDUCE_SPEC) {
+ ret = t1 - t2;
+ } else {
+ assert t1 == GridmixKey.DATA;
+ assert t2 == GridmixKey.DATA;
+ ret =
+ WritableComparator.compareBytes(b1, s1, x1, b2, s2, x2);
+ }
+ di.reset(reset, 0, 0);
+ return ret;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
}
- assert t1 == GridmixKey.DATA;
- assert t2 == GridmixKey.DATA;
- return WritableComparator.compareBytes(
- b1, s1 + 4, l1 - 4,
- b2, s2 + 4, l2 - 4);
- }
- }
-
- /**
- * Keytype for synthetic jobs, some embedding instructions for the reduce.
- */
- public static class GridmixKey extends BytesWritable {
- // long fields specifying reduce contract
- private enum RSpec { REC_IN, REC_OUT, BYTES_OUT };
- private static final int SPEC_START = 5; // type + partition len
- private static final int NUMFIELDS = RSpec.values().length;
- private static final int SPEC_SIZE = NUMFIELDS * 8;
-
- // Key types
- static final byte REDUCE_SPEC = 0;
- static final byte DATA = 1;
-
- private IntBuffer partition;
- private LongBuffer spec;
-
- public GridmixKey() {
- super(new byte[SPEC_START]);
- }
-
- public GridmixKey(byte type, byte[] b) {
- super(b);
- setType(type);
- }
-
- public byte getType() {
- return getBytes()[0];
- }
- public void setPartition(int partition) {
- this.partition.put(0, partition);
- }
- public int getPartition() {
- return partition.get(0);
- }
- public long getReduceInputRecords() {
- checkState(REDUCE_SPEC);
- return spec.get(RSpec.REC_IN.ordinal());
- }
- public long getReduceOutputBytes() {
- checkState(REDUCE_SPEC);
- return spec.get(RSpec.BYTES_OUT.ordinal());
- }
- public long getReduceOutputRecords() {
- checkState(REDUCE_SPEC);
- return spec.get(RSpec.REC_OUT.ordinal());
- }
- public void setType(byte b) {
- switch (b) {
- case REDUCE_SPEC:
- if (getCapacity() < SPEC_START + SPEC_SIZE) {
- setSize(SPEC_START + SPEC_SIZE);
- }
- spec =
- ByteBuffer.wrap(getBytes(), SPEC_START, SPEC_SIZE).asLongBuffer();
- break;
- case DATA:
- if (getCapacity() < SPEC_START) {
- setSize(SPEC_START);
- }
- spec = null;
- break;
- default:
- throw new IllegalArgumentException("Illegal type " + b);
- }
- getBytes()[0] = b;
- partition =
- ByteBuffer.wrap(getBytes(), 1, SPEC_START - 1).asIntBuffer();
- }
- public void setReduceInputRecords(long records) {
- checkState(REDUCE_SPEC);
- spec.put(RSpec.REC_IN.ordinal(), records);
- }
- public void setReduceOutputBytes(long bytes) {
- checkState(REDUCE_SPEC);
- spec.put(RSpec.BYTES_OUT.ordinal(), bytes);
- }
- public void setReduceOutputRecords(long records) {
- checkState(REDUCE_SPEC);
- spec.put(RSpec.REC_OUT.ordinal(), records);
- }
- private void checkState(byte b) {
- if (getLength() < SPEC_START || getType() != b) {
- throw new IllegalStateException("Expected " + b + ", was " + getType());
- }
- }
- @Override
- public void readFields(DataInput in) throws IOException {
- super.readFields(in);
- if (getLength() < SPEC_START) {
- throw new IOException("Invalid GridmixKey, len " + getLength());
- }
- partition =
- ByteBuffer.wrap(getBytes(), 1, SPEC_START - 1).asIntBuffer();
- spec = getType() == REDUCE_SPEC
- ? ByteBuffer.wrap(getBytes(), SPEC_START, SPEC_SIZE).asLongBuffer()
- : null;
- }
- @Override
- public void write(DataOutput out) throws IOException {
- super.write(out);
- if (getType() == REDUCE_SPEC) {
- LOG.debug("SPEC(" + getPartition() + ") " + getReduceInputRecords() +
- " -> " + getReduceOutputRecords() + "/" + getReduceOutputBytes());
- }
- }
- @Override
- public boolean equals(Object other) {
- if (other instanceof GridmixKey) {
- return super.equals(other);
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return super.hashCode();
}
}
public static class GridmixMapper
- extends Mapper<IntWritable,BytesWritable,GridmixKey,BytesWritable> {
+ extends Mapper<NullWritable,GridmixRecord,GridmixKey,GridmixRecord> {
- private final Random r = new Random();
- private GridmixKey key;
- private final BytesWritable val = new BytesWritable();
-
- private int keyLen;
private double acc;
private double ratio;
- private int[] reduceRecordSize;
- private long[] reduceRecordCount;
- private long[] reduceRecordRemaining;
+ private final ArrayList<RecordFactory> reduces =
+ new ArrayList<RecordFactory>();
+ private final Random r = new Random();
+
+ private final GridmixKey key = new GridmixKey();
+ private final GridmixRecord val = new GridmixRecord();
@Override
- protected void setup(Context context)
+ protected void setup(Context ctxt)
throws IOException, InterruptedException {
- // TODO clearly job-specific, but no data at present
- keyLen = context.getConfiguration().getInt(Gridmix.GRIDMIX_KEY_LEN, 20);
- key = new GridmixKey(GridmixKey.DATA, new byte[keyLen]);
- final GridmixSplit split = (GridmixSplit) context.getInputSplit();
- LOG.info("ID: " + split.getId());
- reduceRecordCount = split.getOutputRecords();
- reduceRecordRemaining =
- Arrays.copyOf(reduceRecordCount, reduceRecordCount.length);
- reduceRecordSize = new int[reduceRecordCount.length];
- int valsize = -1;
+ final Configuration conf = ctxt.getConfiguration();
+ final GridmixSplit split = (GridmixSplit) ctxt.getInputSplit();
+ final int maps = split.getMapCount();
final long[] reduceBytes = split.getOutputBytes();
+ final long[] reduceRecords = split.getOutputRecords();
+
long totalRecords = 0L;
- for (int i = 0; i < reduceBytes.length; ++i) {
- reduceRecordSize[i] = Math.max(0,
- Math.round(reduceBytes[i] / (1.0f * reduceRecordCount[i])) - keyLen);
- valsize = Math.max(reduceRecordSize[i], valsize);
- totalRecords += reduceRecordCount[i];
+ final int nReduces = ctxt.getNumReduceTasks();
+ if (nReduces > 0) {
+ int idx = 0;
+ int id = split.getId();
+ for (int i = 0; i < nReduces; ++i) {
+ final GridmixKey.Spec spec = new GridmixKey.Spec();
+ if (i == id) {
+ spec.bytes_out = split.getReduceBytes(idx);
+ spec.rec_out = split.getReduceRecords(idx);
+ ++idx;
+ id += maps;
+ }
+ reduces.add(new IntermediateRecordFactory(
+ new AvgRecordFactory(reduceBytes[i], reduceRecords[i], conf),
+ i, reduceRecords[i], spec, conf));
+ totalRecords += reduceRecords[i];
+ }
+ } else {
+ reduces.add(new AvgRecordFactory(reduceBytes[0], reduceRecords[0],
+ conf));
+ totalRecords = reduceRecords[0];
}
- valsize = Math.max(0, valsize - 4); // BW len encoding
- val.setCapacity(valsize);
- val.setSize(valsize);
- ratio = totalRecords / (1.0 * split.getInputRecords());
+ final long splitRecords = split.getInputRecords();
+ final long inputRecords = splitRecords <= 0 && split.getLength() >= 0
+ ? Math.max(1,
+ split.getLength() / conf.getInt("gridmix.missing.rec.size", 64*1024))
+ : splitRecords;
+ ratio = totalRecords / (1.0 * inputRecords);
acc = 0.0;
}
- protected void fillBytes(BytesWritable val, int len) {
- r.nextBytes(val.getBytes());
- val.setSize(len);
- }
-
- /** Find next non-empty partition after start. */
- private int getNextPart(final int start) {
- int p = start;
- do {
- p = (p + 1) % reduceRecordSize.length;
- } while (0 == reduceRecordRemaining[p] && p != start);
- return 0 == reduceRecordRemaining[p] ? -1 : p;
- }
-
@Override
- public void map(IntWritable ignored, BytesWritable bytes,
+ public void map(NullWritable ignored, GridmixRecord rec,
Context context) throws IOException, InterruptedException {
- int p = getNextPart(r.nextInt(reduceRecordSize.length));
- if (-1 == p) {
- return;
- }
acc += ratio;
- while (acc >= 1.0) {
- fillBytes(key, key.getLength());
- key.setType(GridmixKey.DATA);
- key.setPartition(p);
- --reduceRecordRemaining[p];
- fillBytes(val, reduceRecordSize[p]);
+ while (acc >= 1.0 && !reduces.isEmpty()) {
+ key.setSeed(r.nextLong());
+ val.setSeed(r.nextLong());
+ final int idx = r.nextInt(reduces.size());
+ final RecordFactory f = reduces.get(idx);
+ if (!f.next(key, val)) {
+ reduces.remove(idx);
+ continue;
+ }
context.write(key, val);
acc -= 1.0;
- if (0 == reduceRecordRemaining[p] && -1 == (p = getNextPart(p))) {
- return;
- }
}
}
@Override
public void cleanup(Context context)
throws IOException, InterruptedException {
- // output any remaining records
- // TODO include reduce spec in remaining records if avail
- // (i.e. move this to map)
- for (int i = 0; i < reduceRecordSize.length; ++i) {
- for (long j = reduceRecordRemaining[i]; j > 0; --j) {
- fillBytes(key, key.getLength());
- key.setType(GridmixKey.DATA);
- key.setPartition(i);
- fillBytes(val, reduceRecordSize[i]);
+ for (RecordFactory factory : reduces) {
+ key.setSeed(r.nextLong());
+ while (factory.next(key, val)) {
context.write(key, val);
+ key.setSeed(r.nextLong());
}
}
- val.setSize(0);
- key.setType(GridmixKey.REDUCE_SPEC);
- final int reduces = context.getNumReduceTasks();
- final GridmixSplit split = (GridmixSplit) context.getInputSplit();
- final int maps = split.getMapCount();
- int idx = 0;
- int id = split.getId();
- for (int i = 0; i < reduces; ++i) {
- key.setPartition(i);
- key.setReduceInputRecords(reduceRecordCount[i]);
- // Write spec for all red st r_id % id == 0
- if (i == id) {
- key.setReduceOutputBytes(split.getReduceBytes(idx));
- key.setReduceOutputRecords(split.getReduceRecords(idx));
- LOG.debug(String.format("SPEC'D %d / %d to %d",
- split.getReduceRecords(idx), split.getReduceBytes(idx), i));
- ++idx;
- id += maps;
- } else {
- key.setReduceOutputBytes(0);
- key.setReduceOutputRecords(0);
- }
- context.write(key, val);
- }
}
}
public static class GridmixReducer
- extends Reducer<GridmixKey,BytesWritable,NullWritable,BytesWritable> {
+ extends Reducer<GridmixKey,GridmixRecord,NullWritable,GridmixRecord> {
private final Random r = new Random();
- private final BytesWritable val = new BytesWritable();
+ private final GridmixRecord val = new GridmixRecord();
private double acc;
private double ratio;
- private long written;
- private long inRecords = 0L;
- private long outBytes = 0L;
- private long outRecords = 0L;
-
- protected void fillBytes(BytesWritable val, int len) {
- r.nextBytes(val.getBytes());
- val.setSize(len);
- }
+ private RecordFactory factory;
@Override
protected void setup(Context context)
@@ -501,62 +330,52 @@
context.getCurrentKey().getType() != GridmixKey.REDUCE_SPEC) {
throw new IOException("Missing reduce spec");
}
- for (BytesWritable ignored : context.getValues()) {
+ long outBytes = 0L;
+ long outRecords = 0L;
+ long inRecords = 0L;
+ for (GridmixRecord ignored : context.getValues()) {
final GridmixKey spec = context.getCurrentKey();
inRecords += spec.getReduceInputRecords();
- LOG.debug("GOT COUNT " + spec.getReduceInputRecords());
outBytes += spec.getReduceOutputBytes();
outRecords += spec.getReduceOutputRecords();
}
- LOG.debug("GOT SPEC " + outRecords + "/" + outBytes);
- val.setCapacity(Math.round(outBytes / (1.0f * outRecords)));
+ if (0 == outRecords && inRecords > 0) {
+ LOG.info("Spec output bytes w/o records. Using input record count");
+ outRecords = inRecords;
+ }
+ factory =
+ new AvgRecordFactory(outBytes, outRecords, context.getConfiguration());
ratio = outRecords / (1.0 * inRecords);
acc = 0.0;
- LOG.debug(String.format("RECV %d -> %10d/%10d %d %f", inRecords,
- outRecords, outBytes, val.getCapacity(), ratio));
}
@Override
- protected void reduce(GridmixKey key, Iterable<BytesWritable> values,
+ protected void reduce(GridmixKey key, Iterable<GridmixRecord> values,
Context context) throws IOException, InterruptedException {
- for (BytesWritable ignored : values) {
+ for (GridmixRecord ignored : values) {
acc += ratio;
- while (acc >= 1.0 && written < outBytes) {
- final int len = (int) Math.min(outBytes - written, val.getCapacity());
- fillBytes(val, len);
+ while (acc >= 1.0 && factory.next(null, val)) {
context.write(NullWritable.get(), val);
acc -= 1.0;
- written += len;
- LOG.debug(String.format("%f %d/%d", acc, written, outBytes));
}
}
}
@Override
protected void cleanup(Context context)
throws IOException, InterruptedException {
- while (written < outBytes) {
- final int len = (int) Math.min(outBytes - written, val.getCapacity());
- fillBytes(val, len);
+ val.setSeed(r.nextLong());
+ while (factory.next(null, val)) {
context.write(NullWritable.get(), val);
- written += len;
+ val.setSeed(r.nextLong());
}
}
}
static class GridmixRecordReader
- extends RecordReader<IntWritable,BytesWritable> {
+ extends RecordReader<NullWritable,GridmixRecord> {
- private long bytesRead = 0;
- private long bytesTotal;
- private Configuration conf;
- private final IntWritable key = new IntWritable();
- private final BytesWritable inBytes = new BytesWritable();
-
- private FSDataInputStream input;
- private int idx = -1;
- private int capacity;
- private Path[] paths;
- private long[] startoffset;
- private long[] lengths;
+ private RecordFactory factory;
+ private final Random r = new Random();
+ private final GridmixRecord val = new GridmixRecord();
public GridmixRecordReader() { }
@@ -564,178 +383,36 @@
public void initialize(InputSplit genericSplit, TaskAttemptContext ctxt)
throws IOException, InterruptedException {
final GridmixSplit split = (GridmixSplit)genericSplit;
- this.conf = ctxt.getConfiguration();
- paths = split.getPaths();
- startoffset = split.getStartOffsets();
- lengths = split.getLengths();
- bytesTotal = split.getLength();
- capacity = (int) Math.round(bytesTotal / (1.0 * split.getInputRecords()));
- inBytes.setCapacity(capacity);
- nextSource();
+ final Configuration conf = ctxt.getConfiguration();
+ factory = new ReadRecordFactory(split.getLength(),
+ split.getInputRecords(), new FileQueue(split, conf), conf);
}
- private void nextSource() throws IOException {
- idx = (idx + 1) % paths.length;
- final Path file = paths[idx];
- final FileSystem fs = file.getFileSystem(conf);
- input = fs.open(file, capacity);
- input.seek(startoffset[idx]);
- }
+
@Override
public boolean nextKeyValue() throws IOException {
- if (bytesRead >= bytesTotal) {
- return false;
- }
- final int len = (int)
- Math.min(bytesTotal - bytesRead, inBytes.getCapacity());
- int kvread = 0;
- while (kvread < len) {
- assert lengths[idx] >= 0;
- if (lengths[idx] <= 0) {
- nextSource();
- continue;
- }
- final int srcRead = (int) Math.min(len - kvread, lengths[idx]);
- IOUtils.readFully(input, inBytes.getBytes(), kvread, srcRead);
- //LOG.trace("Read " + srcRead + " bytes from " + paths[idx]);
- lengths[idx] -= srcRead;
- kvread += srcRead;
- }
- bytesRead += kvread;
- return true;
+ val.setSeed(r.nextLong());
+ return factory.next(null, val);
}
@Override
public float getProgress() throws IOException {
- return bytesRead / ((float)bytesTotal);
+ return factory.getProgress();
}
@Override
- public IntWritable getCurrentKey() { return key; }
+ public NullWritable getCurrentKey() {
+ return NullWritable.get();
+ }
@Override
- public BytesWritable getCurrentValue() { return inBytes; }
+ public GridmixRecord getCurrentValue() {
+ return val;
+ }
@Override
public void close() throws IOException {
- IOUtils.cleanup(null, input);
- }
- }
-
- static class GridmixSplit extends CombineFileSplit {
- private int id;
- private int nSpec;
- private int maps;
- private int reduces;
- private long inputRecords;
- private long outputBytes;
- private long outputRecords;
- private long maxMemory;
- private double[] reduceBytes = new double[0];
- private double[] reduceRecords = new double[0];
-
- // Spec for reduces id mod this
- private long[] reduceOutputBytes = new long[0];
- private long[] reduceOutputRecords = new long[0];
-
- GridmixSplit() {
- super();
- }
-
- public GridmixSplit(CombineFileSplit cfsplit, int maps, int id,
- long inputBytes, long inputRecords, long outputBytes,
- long outputRecords, double[] reduceBytes, double[] reduceRecords,
- long[] reduceOutputBytes, long[] reduceOutputRecords)
- throws IOException {
- super(cfsplit);
- this.id = id;
- this.maps = maps;
- reduces = reduceBytes.length;
- this.inputRecords = inputRecords;
- this.outputBytes = outputBytes;
- this.outputRecords = outputRecords;
- this.reduceBytes = Arrays.copyOf(reduceBytes, reduces);
- this.reduceRecords = Arrays.copyOf(reduceRecords, reduces);
- nSpec = reduceOutputBytes.length;
- this.reduceOutputBytes = reduceOutputBytes;
- this.reduceOutputRecords = reduceOutputRecords;
- }
- public int getId() {
- return id;
- }
- public int getMapCount() {
- return maps;
- }
- public long getInputRecords() {
- return inputRecords;
- }
- public long[] getOutputBytes() {
- final long[] ret = new long[reduces];
- for (int i = 0; i < reduces; ++i) {
- ret[i] = Math.round(outputBytes * reduceBytes[i]);
- }
- return ret;
- }
- public long[] getOutputRecords() {
- final long[] ret = new long[reduces];
- for (int i = 0; i < reduces; ++i) {
- ret[i] = Math.round(outputRecords * reduceRecords[i]);
- }
- return ret;
- }
- public long getReduceBytes(int i) {
- return reduceOutputBytes[i];
- }
- public long getReduceRecords(int i) {
- return reduceOutputRecords[i];
- }
- @Override
- public void write(DataOutput out) throws IOException {
- super.write(out);
- WritableUtils.writeVInt(out, id);
- WritableUtils.writeVInt(out, maps);
- WritableUtils.writeVLong(out, inputRecords);
- WritableUtils.writeVLong(out, outputBytes);
- WritableUtils.writeVLong(out, outputRecords);
- WritableUtils.writeVLong(out, maxMemory);
- WritableUtils.writeVInt(out, reduces);
- for (int i = 0; i < reduces; ++i) {
- out.writeDouble(reduceBytes[i]);
- out.writeDouble(reduceRecords[i]);
- }
- WritableUtils.writeVInt(out, nSpec);
- for (int i = 0; i < nSpec; ++i) {
- out.writeLong(reduceOutputBytes[i]);
- out.writeLong(reduceOutputRecords[i]);
- }
- }
- @Override
- public void readFields(DataInput in) throws IOException {
- super.readFields(in);
- id = WritableUtils.readVInt(in);
- maps = WritableUtils.readVInt(in);
- inputRecords = WritableUtils.readVLong(in);
- outputBytes = WritableUtils.readVLong(in);
- outputRecords = WritableUtils.readVLong(in);
- maxMemory = WritableUtils.readVLong(in);
- reduces = WritableUtils.readVInt(in);
- if (reduceBytes.length < reduces) {
- reduceBytes = new double[reduces];
- reduceRecords = new double[reduces];
- }
- for (int i = 0; i < reduces; ++i) {
- reduceBytes[i] = in.readDouble();
- reduceRecords[i] = in.readDouble();
- }
- nSpec = WritableUtils.readVInt(in);
- if (reduceOutputBytes.length < nSpec) {
- reduceOutputBytes = new long[nSpec];
- reduceOutputRecords = new long[nSpec];
- }
- for (int i = 0; i < nSpec; ++i) {
- reduceOutputBytes[i] = in.readLong();
- reduceOutputRecords[i] = in.readLong();
- }
+ factory.close();
}
}
static class GridmixInputFormat
- extends InputFormat<IntWritable,BytesWritable> {
+ extends InputFormat<NullWritable,GridmixRecord> {
@Override
public List<InputSplit> getSplits(JobContext jobCtxt) throws IOException {
@@ -743,29 +420,28 @@
"gridmix.job.seq", -1));
}
@Override
- public RecordReader<IntWritable,BytesWritable> createRecordReader(
+ public RecordReader<NullWritable,GridmixRecord> createRecordReader(
InputSplit split, final TaskAttemptContext taskContext)
throws IOException {
return new GridmixRecordReader();
}
}
- static class RawBytesOutputFormat
- extends FileOutputFormat<NullWritable,BytesWritable> {
+ static class RawBytesOutputFormat<K>
+ extends FileOutputFormat<K,GridmixRecord> {
@Override
- public RecordWriter<NullWritable,BytesWritable> getRecordWriter(
+ public RecordWriter<K,GridmixRecord> getRecordWriter(
TaskAttemptContext job) throws IOException {
Path file = getDefaultWorkFile(job, "");
FileSystem fs = file.getFileSystem(job.getConfiguration());
final FSDataOutputStream fileOut = fs.create(file, false);
- return new RecordWriter<NullWritable,BytesWritable>() {
+ return new RecordWriter<K,GridmixRecord>() {
@Override
- public void write(NullWritable key, BytesWritable value)
+ public void write(K ignored, GridmixRecord value)
throws IOException {
- //LOG.trace("WROTE " + value.getLength() + " bytes");
- fileOut.write(value.getBytes(), 0, value.getLength());
+ value.writeRandom(fileOut, value.getSize());
}
@Override
public void close(TaskAttemptContext ctxt) throws IOException {
@@ -829,8 +505,10 @@
jobdesc.getTaskInfo(TaskType.REDUCE, i + j * maps);
specBytes[j] = info.getOutputBytes();
specRecords[j] = info.getOutputRecords();
- LOG.debug(String.format("SPEC(%d) %d -> %d %d %d", id(), i,
- i + j * maps, info.getOutputRecords(), info.getOutputBytes()));
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(String.format("SPEC(%d) %d -> %d %d %d", id(), i,
+ i + j * maps, info.getOutputRecords(), info.getOutputBytes()));
+ }
}
final TaskInfo info = jobdesc.getTaskInfo(TaskType.MAP, i);
splits.add(new GridmixSplit(striper.splitFor(inputDir,
@@ -842,77 +520,4 @@
pushDescription(id(), splits);
}
- static class InputStriper {
- int idx;
- long currentStart;
- FileStatus current;
- final List<FileStatus> files = new ArrayList<FileStatus>();
-
- InputStriper(FilePool inputDir, long mapBytes)
- throws IOException {
- final long inputBytes = inputDir.getInputFiles(mapBytes, files);
- if (mapBytes > inputBytes) {
- LOG.warn("Using " + inputBytes + "/" + mapBytes + " bytes");
- }
- current = files.get(0);
- }
-
- CombineFileSplit splitFor(FilePool inputDir, long bytes, int nLocs)
- throws IOException {
- final ArrayList<Path> paths = new ArrayList<Path>();
- final ArrayList<Long> start = new ArrayList<Long>();
- final ArrayList<Long> length = new ArrayList<Long>();
- final HashMap<String,Double> sb = new HashMap<String,Double>();
- while (bytes > 0) {
- paths.add(current.getPath());
- start.add(currentStart);
- final long fromFile = Math.min(bytes, current.getLen() - currentStart);
- length.add(fromFile);
- for (BlockLocation loc :
- inputDir.locationsFor(current, currentStart, fromFile)) {
- final double tedium = loc.getLength() / (1.0 * bytes);
- for (String l : loc.getHosts()) {
- Double j = sb.get(l);
- if (null == j) {
- sb.put(l, tedium);
- } else {
- sb.put(l, j.doubleValue() + tedium);
- }
- }
- }
- currentStart += fromFile;
- bytes -= fromFile;
- if (current.getLen() - currentStart == 0) {
- current = files.get(++idx % files.size());
- currentStart = 0;
- }
- }
- final ArrayList<Entry<String,Double>> sort =
- new ArrayList<Entry<String,Double>>(sb.entrySet());
- Collections.sort(sort, hostRank);
- final String[] hosts = new String[Math.min(nLocs, sort.size())];
- for (int i = 0; i < nLocs && i < sort.size(); ++i) {
- hosts[i] = sort.get(i).getKey();
- }
- return new CombineFileSplit(paths.toArray(new Path[0]),
- toLongArray(start), toLongArray(length), hosts);
- }
-
- private long[] toLongArray(final ArrayList<Long> sigh) {
- final long[] ret = new long[sigh.size()];
- for (int i = 0; i < ret.length; ++i) {
- ret[i] = sigh.get(i);
- }
- return ret;
- }
-
- final Comparator<Entry<String,Double>> hostRank =
- new Comparator<Entry<String,Double>>() {
- public int compare(Entry<String,Double> a, Entry<String,Double> b) {
- final double va = a.getValue();
- final double vb = b.getValue();
- return va > vb ? -1 : va < vb ? 1 : 0;
- }
- };
- }
}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixKey.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixKey.java
new file mode 100644
index 0000000..4b21265
--- /dev/null
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixKey.java
@@ -0,0 +1,258 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.WritableComparator;
+
+class GridmixKey extends GridmixRecord {
+ static final byte REDUCE_SPEC = 0;
+ static final byte DATA = 1;
+
+ static final int META_BYTES = 1;
+
+ private byte type;
+ private int partition; // NOT serialized
+ private Spec spec = new Spec();
+
+ GridmixKey() {
+ this(DATA, 1, 0L);
+ }
+ GridmixKey(byte type, int size, long seed) {
+ super(size, seed);
+ this.type = type;
+ // setting type may change pcnt random bytes
+ setSize(size);
+ }
+
+ @Override
+ public int getSize() {
+ switch (type) {
+ case REDUCE_SPEC:
+ return super.getSize() + spec.getSize() + META_BYTES;
+ case DATA:
+ return super.getSize() + META_BYTES;
+ default:
+ throw new IllegalStateException("Invalid type: " + type);
+ }
+ }
+
+ @Override
+ public void setSize(int size) {
+ switch (type) {
+ case REDUCE_SPEC:
+ super.setSize(size - (META_BYTES + spec.getSize()));
+ break;
+ case DATA:
+ super.setSize(size - META_BYTES);
+ break;
+ default:
+ throw new IllegalStateException("Invalid type: " + type);
+ }
+ }
+
+ /**
+ * Partition is not serialized.
+ */
+ public int getPartition() {
+ return partition;
+ }
+ public void setPartition(int partition) {
+ this.partition = partition;
+ }
+
+ public long getReduceInputRecords() {
+ assert REDUCE_SPEC == getType();
+ return spec.rec_in;
+ }
+ public void setReduceInputRecords(long rec_in) {
+ assert REDUCE_SPEC == getType();
+ final int origSize = getSize();
+ spec.rec_in = rec_in;
+ setSize(origSize);
+ }
+
+ public long getReduceOutputRecords() {
+ assert REDUCE_SPEC == getType();
+ return spec.rec_out;
+ }
+ public void setReduceOutputRecords(long rec_out) {
+ assert REDUCE_SPEC == getType();
+ final int origSize = getSize();
+ spec.rec_out = rec_out;
+ setSize(origSize);
+ }
+
+ public long getReduceOutputBytes() {
+ assert REDUCE_SPEC == getType();
+ return spec.bytes_out;
+ };
+ public void setReduceOutputBytes(long b_out) {
+ assert REDUCE_SPEC == getType();
+ final int origSize = getSize();
+ spec.bytes_out = b_out;
+ setSize(origSize);
+ }
+
+ public byte getType() {
+ return type;
+ }
+ public void setType(byte type) throws IOException {
+ final int origSize = getSize();
+ switch (type) {
+ case REDUCE_SPEC:
+ case DATA:
+ this.type = type;
+ break;
+ default:
+ throw new IOException("Invalid type: " + type);
+ }
+ setSize(origSize);
+ }
+
+ public void setSpec(Spec spec) {
+ assert REDUCE_SPEC == getType();
+ final int origSize = getSize();
+ this.spec.set(spec);
+ setSize(origSize);
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ super.readFields(in);
+ setType(in.readByte());
+ if (REDUCE_SPEC == getType()) {
+ spec.readFields(in);
+ }
+ }
+ @Override
+ public void write(DataOutput out) throws IOException {
+ super.write(out);
+ final byte t = getType();
+ out.writeByte(t);
+ if (REDUCE_SPEC == t) {
+ spec.write(out);
+ }
+ }
+ int fixedBytes() {
+ return super.fixedBytes() +
+ (REDUCE_SPEC == getType() ? spec.getSize() : 0) + META_BYTES;
+ }
+ @Override
+ public int compareTo(GridmixRecord other) {
+ final GridmixKey o = (GridmixKey) other;
+ final byte t1 = getType();
+ final byte t2 = o.getType();
+ if (t1 != t2) {
+ return t1 - t2;
+ }
+ return super.compareTo(other);
+ }
+
+ /**
+ * Note that while the spec is not explicitly included, changing the spec
+ * may change its size, which will affect equality.
+ */
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (other != null && other.getClass() == getClass()) {
+ final GridmixKey o = ((GridmixKey)other);
+ return getType() == o.getType() && super.equals(o);
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return super.hashCode() ^ getType();
+ }
+
+ public static class Spec implements Writable {
+ long rec_in;
+ long rec_out;
+ long bytes_out;
+ public Spec() { }
+
+ public void set(Spec other) {
+ rec_in = other.rec_in;
+ bytes_out = other.bytes_out;
+ rec_out = other.rec_out;
+ }
+
+ public int getSize() {
+ return WritableUtils.getVIntSize(rec_in) +
+ WritableUtils.getVIntSize(rec_out) +
+ WritableUtils.getVIntSize(bytes_out);
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ rec_in = WritableUtils.readVLong(in);
+ rec_out = WritableUtils.readVLong(in);
+ bytes_out = WritableUtils.readVLong(in);
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ WritableUtils.writeVLong(out, rec_in);
+ WritableUtils.writeVLong(out, rec_out);
+ WritableUtils.writeVLong(out, bytes_out);
+ }
+ }
+
+ public static class Comparator extends GridmixRecord.Comparator {
+
+ private final DataInputBuffer di = new DataInputBuffer();
+ private final byte[] reset = di.getData();
+
+ public Comparator() {
+ super(GridmixKey.class);
+ }
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ try {
+ di.reset(b1, s1, l1);
+ final int x1 = WritableUtils.readVInt(di);
+ di.reset(b2, s2, l2);
+ final int x2 = WritableUtils.readVInt(di);
+ final int ret = (b1[s1 + x1] != b2[s2 + x2])
+ ? b1[s1 + x1] - b2[s2 + x2]
+ : super.compare(b1, s1, x1, b2, s2, x2);
+ di.reset(reset, 0, 0);
+ return ret;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ static {
+ WritableComparator.define(GridmixKey.class, new Comparator());
+ }
+ }
+}
+
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java
new file mode 100644
index 0000000..c4409e0
--- /dev/null
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java
@@ -0,0 +1,215 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+
+class GridmixRecord implements WritableComparable<GridmixRecord> {
+
+ private static final int FIXED_BYTES = 1;
+ private int size = -1;
+ private long seed;
+ private final DataInputBuffer dib =
+ new DataInputBuffer();
+ private final DataOutputBuffer dob =
+ new DataOutputBuffer(Long.SIZE / Byte.SIZE);
+ private byte[] literal = dob.getData();
+
+ GridmixRecord() {
+ this(1, 0L);
+ }
+
+ GridmixRecord(int size, long seed) {
+ this.seed = seed;
+ setSizeInternal(size);
+ }
+
+ public int getSize() {
+ return size;
+ }
+
+ public void setSize(int size) {
+ setSizeInternal(size);
+ }
+
+ private void setSizeInternal(int size) {
+ this.size = Math.max(1, size);
+ try {
+ seed = maskSeed(seed, this.size);
+ dob.reset();
+ dob.writeLong(seed);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public final void setSeed(long seed) {
+ this.seed = seed;
+ }
+
+ /** Marsaglia, 2003. */
+ long nextRand(long x) {
+ x ^= (x << 13);
+ x ^= (x >>> 7);
+ return (x ^= (x << 17));
+ }
+
+ public void writeRandom(DataOutput out, final int size) throws IOException {
+ long tmp = seed;
+ out.writeLong(tmp);
+ int i = size - (Long.SIZE / Byte.SIZE);
+ while (i > Long.SIZE / Byte.SIZE - 1) {
+ tmp = nextRand(tmp);
+ out.writeLong(tmp);
+ i -= Long.SIZE / Byte.SIZE;
+ }
+ for (tmp = nextRand(tmp); i > 0; --i) {
+ out.writeByte((int)(tmp & 0xFF));
+ tmp >>>= Byte.SIZE;
+ }
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ size = WritableUtils.readVInt(in);
+ int payload = size - WritableUtils.getVIntSize(size);
+ if (payload > Long.SIZE / Byte.SIZE) {
+ seed = in.readLong();
+ payload -= Long.SIZE / Byte.SIZE;
+ } else {
+ Arrays.fill(literal, (byte)0);
+ in.readFully(literal, 0, payload);
+ dib.reset(literal, 0, literal.length);
+ seed = dib.readLong();
+ payload = 0;
+ }
+ final int vBytes = in.skipBytes(payload);
+ if (vBytes != payload) {
+ throw new EOFException("Expected " + payload + ", read " + vBytes);
+ }
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ // data bytes including vint encoding
+ WritableUtils.writeVInt(out, size);
+ final int payload = size - WritableUtils.getVIntSize(size);
+ if (payload > Long.SIZE / Byte.SIZE) {
+ writeRandom(out, payload);
+ } else if (payload > 0) {
+ out.write(literal, 0, payload);
+ }
+ }
+
+ @Override
+ public int compareTo(GridmixRecord other) {
+ return compareSeed(other.seed,
+ Math.max(0, other.getSize() - other.fixedBytes()));
+ }
+
+ int fixedBytes() {
+ // min vint size
+ return FIXED_BYTES;
+ }
+
+ private static long maskSeed(long sd, int sz) {
+ // Don't use fixedBytes here; subclasses will set intended random len
+ if (sz <= FIXED_BYTES) {
+ sd = 0L;
+ } else if (sz < Long.SIZE / Byte.SIZE + FIXED_BYTES) {
+ final int tmp = sz - FIXED_BYTES;
+ final long mask = (1L << (Byte.SIZE * tmp)) - 1;
+ sd &= mask << (Byte.SIZE * (Long.SIZE / Byte.SIZE - tmp));
+ }
+ return sd;
+ }
+
+ int compareSeed(long jSeed, int jSize) {
+ final int iSize = Math.max(0, getSize() - fixedBytes());
+ final int seedLen = Math.min(iSize, jSize) + FIXED_BYTES;
+ jSeed = maskSeed(jSeed, seedLen);
+ long iSeed = maskSeed(seed, seedLen);
+ final int cmplen = Math.min(iSize, jSize);
+ for (int i = 0; i < cmplen; i += Byte.SIZE) {
+ final int k = cmplen - i;
+ for (long j = Long.SIZE - Byte.SIZE;
+ j >= Math.max(0, Long.SIZE / Byte.SIZE - k) * Byte.SIZE;
+ j -= Byte.SIZE) {
+ final int xi = (int)((iSeed >>> j) & 0xFFL);
+ final int xj = (int)((jSeed >>> j) & 0xFFL);
+ if (xi != xj) {
+ return xi - xj;
+ }
+ }
+ iSeed = nextRand(iSeed);
+ jSeed = nextRand(jSeed);
+ }
+ return iSize - jSize;
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (other != null && other.getClass() == getClass()) {
+ final GridmixRecord o = ((GridmixRecord)other);
+ return getSize() == o.getSize() && seed == o.seed;
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return (int)(seed * getSize());
+ }
+
+ public static class Comparator extends WritableComparator {
+
+ public Comparator() {
+ super(GridmixRecord.class);
+ }
+
+ public Comparator(Class<? extends WritableComparable<?>> sub) {
+ super(sub);
+ }
+
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ int n1 = WritableUtils.decodeVIntSize(b1[s1]);
+ int n2 = WritableUtils.decodeVIntSize(b2[s2]);
+ n1 -= WritableUtils.getVIntSize(n1);
+ n2 -= WritableUtils.getVIntSize(n2);
+ return compareBytes(b1, s1+n1, l1-n1, b2, s2+n2, l2-n2);
+ }
+
+ static {
+ WritableComparator.define(GridmixRecord.class, new Comparator());
+ }
+ }
+
+}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixSplit.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixSplit.java
new file mode 100644
index 0000000..b611c9d
--- /dev/null
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixSplit.java
@@ -0,0 +1,148 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
+
+class GridmixSplit extends CombineFileSplit {
+ private int id;
+ private int nSpec;
+ private int maps;
+ private int reduces;
+ private long inputRecords;
+ private long outputBytes;
+ private long outputRecords;
+ private long maxMemory;
+ private double[] reduceBytes = new double[0];
+ private double[] reduceRecords = new double[0];
+
+ // Spec for reduces id mod this
+ private long[] reduceOutputBytes = new long[0];
+ private long[] reduceOutputRecords = new long[0];
+
+ GridmixSplit() {
+ super();
+ }
+
+ public GridmixSplit(CombineFileSplit cfsplit, int maps, int id,
+ long inputBytes, long inputRecords, long outputBytes,
+ long outputRecords, double[] reduceBytes, double[] reduceRecords,
+ long[] reduceOutputBytes, long[] reduceOutputRecords)
+ throws IOException {
+ super(cfsplit);
+ this.id = id;
+ this.maps = maps;
+ reduces = reduceBytes.length;
+ this.inputRecords = inputRecords;
+ this.outputBytes = outputBytes;
+ this.outputRecords = outputRecords;
+ this.reduceBytes = reduceBytes;
+ this.reduceRecords = reduceRecords;
+ nSpec = reduceOutputBytes.length;
+ this.reduceOutputBytes = reduceOutputBytes;
+ this.reduceOutputRecords = reduceOutputRecords;
+ }
+ public int getId() {
+ return id;
+ }
+ public int getMapCount() {
+ return maps;
+ }
+ public long getInputRecords() {
+ return inputRecords;
+ }
+ public long[] getOutputBytes() {
+ if (0 == reduces) {
+ return new long[] { outputBytes };
+ }
+ final long[] ret = new long[reduces];
+ for (int i = 0; i < reduces; ++i) {
+ ret[i] = Math.round(outputBytes * reduceBytes[i]);
+ }
+ return ret;
+ }
+ public long[] getOutputRecords() {
+ if (0 == reduces) {
+ return new long[] { outputRecords };
+ }
+ final long[] ret = new long[reduces];
+ for (int i = 0; i < reduces; ++i) {
+ ret[i] = Math.round(outputRecords * reduceRecords[i]);
+ }
+ return ret;
+ }
+ public long getReduceBytes(int i) {
+ return reduceOutputBytes[i];
+ }
+ public long getReduceRecords(int i) {
+ return reduceOutputRecords[i];
+ }
+ @Override
+ public void write(DataOutput out) throws IOException {
+ super.write(out);
+ WritableUtils.writeVInt(out, id);
+ WritableUtils.writeVInt(out, maps);
+ WritableUtils.writeVLong(out, inputRecords);
+ WritableUtils.writeVLong(out, outputBytes);
+ WritableUtils.writeVLong(out, outputRecords);
+ WritableUtils.writeVLong(out, maxMemory);
+ WritableUtils.writeVInt(out, reduces);
+ for (int i = 0; i < reduces; ++i) {
+ out.writeDouble(reduceBytes[i]);
+ out.writeDouble(reduceRecords[i]);
+ }
+ WritableUtils.writeVInt(out, nSpec);
+ for (int i = 0; i < nSpec; ++i) {
+ WritableUtils.writeVLong(out, reduceOutputBytes[i]);
+ WritableUtils.writeVLong(out, reduceOutputRecords[i]);
+ }
+ }
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ super.readFields(in);
+ id = WritableUtils.readVInt(in);
+ maps = WritableUtils.readVInt(in);
+ inputRecords = WritableUtils.readVLong(in);
+ outputBytes = WritableUtils.readVLong(in);
+ outputRecords = WritableUtils.readVLong(in);
+ maxMemory = WritableUtils.readVLong(in);
+ reduces = WritableUtils.readVInt(in);
+ if (reduceBytes.length < reduces) {
+ reduceBytes = new double[reduces];
+ reduceRecords = new double[reduces];
+ }
+ for (int i = 0; i < reduces; ++i) {
+ reduceBytes[i] = in.readDouble();
+ reduceRecords[i] = in.readDouble();
+ }
+ nSpec = WritableUtils.readVInt(in);
+ if (reduceOutputBytes.length < nSpec) {
+ reduceOutputBytes = new long[nSpec];
+ reduceOutputRecords = new long[nSpec];
+ }
+ for (int i = 0; i < nSpec; ++i) {
+ reduceOutputBytes[i] = WritableUtils.readVLong(in);
+ reduceOutputRecords[i] = WritableUtils.readVLong(in);
+ }
+ }
+}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/InputStriper.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/InputStriper.java
new file mode 100644
index 0000000..0576162
--- /dev/null
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/InputStriper.java
@@ -0,0 +1,127 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Given a {@link #FilePool}, obtain a set of files capable of satisfying
+ * a full set of splits, then iterate over each source to fill the request.
+ */
+class InputStriper {
+ public static final Log LOG = LogFactory.getLog(InputStriper.class);
+ int idx;
+ long currentStart;
+ FileStatus current;
+ final List<FileStatus> files = new ArrayList<FileStatus>();
+
+ /**
+ * @param inputDir Pool from which files are requested.
+ * @param mapBytes Sum of all expected split requests.
+ */
+ InputStriper(FilePool inputDir, long mapBytes)
+ throws IOException {
+ final long inputBytes = inputDir.getInputFiles(mapBytes, files);
+ if (mapBytes > inputBytes) {
+ LOG.warn("Using " + inputBytes + "/" + mapBytes + " bytes");
+ }
+ if (files.isEmpty() && mapBytes > 0) {
+ throw new IOException("Failed to satisfy request for " + mapBytes);
+ }
+ current = files.isEmpty() ? null : files.get(0);
+ }
+
+ /**
+ * @param inputDir Pool used to resolve block locations.
+ * @param bytes Target byte count
+ * @param nLocs Number of block locations per split.
+ * @return A set of files satisfying the byte count, with locations weighted
+ * to the dominating proportion of input bytes.
+ */
+ CombineFileSplit splitFor(FilePool inputDir, long bytes, int nLocs)
+ throws IOException {
+ final ArrayList<Path> paths = new ArrayList<Path>();
+ final ArrayList<Long> start = new ArrayList<Long>();
+ final ArrayList<Long> length = new ArrayList<Long>();
+ final HashMap<String,Double> sb = new HashMap<String,Double>();
+ do {
+ paths.add(current.getPath());
+ start.add(currentStart);
+ final long fromFile = Math.min(bytes, current.getLen() - currentStart);
+ length.add(fromFile);
+ for (BlockLocation loc :
+ inputDir.locationsFor(current, currentStart, fromFile)) {
+ final double tedium = loc.getLength() / (1.0 * bytes);
+ for (String l : loc.getHosts()) {
+ Double j = sb.get(l);
+ if (null == j) {
+ sb.put(l, tedium);
+ } else {
+ sb.put(l, j.doubleValue() + tedium);
+ }
+ }
+ }
+ currentStart += fromFile;
+ bytes -= fromFile;
+ if (current.getLen() - currentStart == 0) {
+ current = files.get(++idx % files.size());
+ currentStart = 0;
+ }
+ } while (bytes > 0);
+ final ArrayList<Entry<String,Double>> sort =
+ new ArrayList<Entry<String,Double>>(sb.entrySet());
+ Collections.sort(sort, hostRank);
+ final String[] hosts = new String[Math.min(nLocs, sort.size())];
+ for (int i = 0; i < nLocs && i < sort.size(); ++i) {
+ hosts[i] = sort.get(i).getKey();
+ }
+ return new CombineFileSplit(paths.toArray(new Path[0]),
+ toLongArray(start), toLongArray(length), hosts);
+ }
+
+ private long[] toLongArray(final ArrayList<Long> sigh) {
+ final long[] ret = new long[sigh.size()];
+ for (int i = 0; i < ret.length; ++i) {
+ ret[i] = sigh.get(i);
+ }
+ return ret;
+ }
+
+ static final Comparator<Entry<String,Double>> hostRank =
+ new Comparator<Entry<String,Double>>() {
+ public int compare(Entry<String,Double> a, Entry<String,Double> b) {
+ final double va = a.getValue();
+ final double vb = b.getValue();
+ return va > vb ? -1 : va < vb ? 1 : 0;
+ }
+ };
+}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/IntermediateRecordFactory.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/IntermediateRecordFactory.java
new file mode 100644
index 0000000..a6fc6c6
--- /dev/null
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/IntermediateRecordFactory.java
@@ -0,0 +1,110 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Factory passing reduce specification as its last record.
+ */
+class IntermediateRecordFactory extends RecordFactory {
+
+ private final GridmixKey.Spec spec;
+ private final RecordFactory factory;
+ private final int partition;
+ private final long targetRecords;
+ private boolean done = false;
+ private long accRecords = 0L;
+
+ /**
+ * @param targetBytes Expected byte count.
+ * @param targetRecords Expected record count; will emit spec records after
+ * this boundary is passed.
+ * @param partition Reduce to which records are emitted.
+ * @param spec Specification to emit.
+ * @param conf Unused.
+ */
+ public IntermediateRecordFactory(long targetBytes, long targetRecords,
+ int partition, GridmixKey.Spec spec, Configuration conf) {
+ this(new AvgRecordFactory(targetBytes, targetRecords, conf), partition,
+ targetRecords, spec, conf);
+ }
+
+ /**
+ * @param factory Factory from which byte/record counts are obtained.
+ * @param partition Reduce to which records are emitted.
+ * @param targetRecords Expected record count; will emit spec records after
+ * this boundary is passed.
+ * @param spec Specification to emit.
+ * @param conf Unused.
+ */
+ public IntermediateRecordFactory(RecordFactory factory, int partition,
+ long targetRecords, GridmixKey.Spec spec, Configuration conf) {
+ this.spec = spec;
+ this.factory = factory;
+ this.partition = partition;
+ this.targetRecords = targetRecords;
+ }
+
+ @Override
+ public boolean next(GridmixKey key, GridmixRecord val) throws IOException {
+ assert key != null;
+ final boolean rslt = factory.next(key, val);
+ ++accRecords;
+ if (rslt) {
+ if (accRecords < targetRecords) {
+ key.setType(GridmixKey.DATA);
+ } else {
+ final int orig = key.getSize();
+ key.setType(GridmixKey.REDUCE_SPEC);
+ spec.rec_in = accRecords;
+ key.setSpec(spec);
+ val.setSize(val.getSize() - (key.getSize() - orig));
+ // reset counters
+ accRecords = 0L;
+ spec.bytes_out = 0L;
+ spec.rec_out = 0L;
+ done = true;
+ }
+ } else if (!done) {
+ // ensure spec emitted
+ key.setType(GridmixKey.REDUCE_SPEC);
+ key.setPartition(partition);
+ key.setSize(0);
+ val.setSize(0);
+ spec.rec_in = 0L;
+ key.setSpec(spec);
+ done = true;
+ return true;
+ }
+ key.setPartition(partition);
+ return rslt;
+ }
+
+ @Override
+ public float getProgress() throws IOException {
+ return factory.getProgress();
+ }
+
+ @Override
+ public void close() throws IOException {
+ factory.close();
+ }
+}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobFactory.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobFactory.java
index ff2df1d..859d406 100644
--- a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobFactory.java
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobFactory.java
@@ -26,9 +26,16 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.tools.rumen.JobStory;
import org.apache.hadoop.tools.rumen.JobStoryProducer;
+import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants.Values;
import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants;
+import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
+import org.apache.hadoop.tools.rumen.TaskInfo;
import org.apache.hadoop.tools.rumen.ZombieJobProducer;
import org.apache.commons.logging.Log;
@@ -47,6 +54,7 @@
public static final Log LOG = LogFactory.getLog(JobFactory.class);
private final Path scratch;
+ private final float rateFactor;
private final Configuration conf;
private final ReaderThread rThread;
private final AtomicInteger sequence;
@@ -83,6 +91,7 @@
Path scratch, Configuration conf, CountDownLatch startFlag) {
sequence = new AtomicInteger(0);
this.scratch = scratch;
+ this.rateFactor = conf.getFloat(Gridmix.GRIDMIX_SUB_MUL, 1.0f);
this.jobProducer = jobProducer;
this.conf = new Configuration(conf);
this.submitter = submitter;
@@ -90,6 +99,61 @@
this.rThread = new ReaderThread();
}
+ static class MinTaskInfo extends TaskInfo {
+ public MinTaskInfo(TaskInfo info) {
+ super(info.getInputBytes(), info.getInputRecords(),
+ info.getOutputBytes(), info.getOutputRecords(),
+ info.getTaskMemory());
+ }
+ public long getInputBytes() {
+ return Math.max(0, super.getInputBytes());
+ }
+ public int getInputRecords() {
+ return Math.max(0, super.getInputRecords());
+ }
+ public long getOutputBytes() {
+ return Math.max(0, super.getOutputBytes());
+ }
+ public int getOutputRecords() {
+ return Math.max(0, super.getOutputRecords());
+ }
+ public long getTaskMemory() {
+ return Math.max(0, super.getTaskMemory());
+ }
+ }
+
+ static class FilterJobStory implements JobStory {
+
+ protected final JobStory job;
+
+ public FilterJobStory(JobStory job) {
+ this.job = job;
+ }
+ public JobConf getJobConf() { return job.getJobConf(); }
+ public String getName() { return job.getName(); }
+ public JobID getJobID() { return job.getJobID(); }
+ public String getUser() { return job.getUser(); }
+ public long getSubmissionTime() { return job.getSubmissionTime(); }
+ public InputSplit[] getInputSplits() { return job.getInputSplits(); }
+ public int getNumberMaps() { return job.getNumberMaps(); }
+ public int getNumberReduces() { return job.getNumberReduces(); }
+ public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {
+ return job.getTaskInfo(taskType, taskNumber);
+ }
+ public TaskAttemptInfo getTaskAttemptInfo(TaskType taskType, int taskNumber,
+ int taskAttemptNumber) {
+ return job.getTaskAttemptInfo(taskType, taskNumber, taskAttemptNumber);
+ }
+ public TaskAttemptInfo getMapTaskAttemptInfoAdjusted(
+ int taskNumber, int taskAttemptNumber, int locality) {
+ return job.getMapTaskAttemptInfoAdjusted(
+ taskNumber, taskAttemptNumber, locality);
+ }
+ public Values getOutcome() {
+ return job.getOutcome();
+ }
+ }
+
/**
* Worker thread responsible for reading descriptions, assigning sequence
* numbers, and normalizing time.
@@ -107,7 +171,12 @@
} while (job != null
&& (job.getOutcome() != Pre21JobHistoryConstants.Values.SUCCESS ||
job.getSubmissionTime() < 0));
- return job;
+ return null == job ? null : new FilterJobStory(job) {
+ @Override
+ public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {
+ return new MinTaskInfo(this.job.getTaskInfo(taskType, taskNumber));
+ }
+ };
}
@Override
@@ -133,11 +202,12 @@
}
final long current = job.getSubmissionTime();
if (current < last) {
- throw new IOException(
- "JobStories are not ordered by submission time.");
+ LOG.warn("Job " + job.getJobID() + " out of order");
+ continue;
}
last = current;
- submitter.add(new GridmixJob(conf, initTime + (current - first),
+ submitter.add(new GridmixJob(conf, initTime +
+ Math.round(rateFactor * (current - first)),
job, scratch, sequence.getAndIncrement()));
} catch (IOException e) {
JobFactory.this.error = e;
@@ -179,8 +249,8 @@
/**
* Wait for the reader thread to exhaust the job trace.
*/
- public void join() throws InterruptedException {
- rThread.join();
+ public void join(long millis) throws InterruptedException {
+ rThread.join(millis);
}
/**
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobMonitor.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobMonitor.java
index 850e6eb..3b917af 100644
--- a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobMonitor.java
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobMonitor.java
@@ -47,7 +47,8 @@
private final MonitorThread mThread;
private final BlockingQueue<Job> runningJobs;
private final long pollDelayMillis;
- private volatile boolean graceful = false;
+ private boolean graceful = false;
+ private boolean shutdown = false;
/**
* Create a JobMonitor with a default polling interval of 5s.
@@ -59,7 +60,7 @@
/**
* Create a JobMonitor that sleeps for the specified duration after
* polling a still-running job.
- * @param pollDelaySec Delay after polling a running job
+ * @param pollDelay Delay after polling a running job
* @param unit Time unit for pollDelaySec (rounded to milliseconds)
*/
public JobMonitor(int pollDelay, TimeUnit unit) {
@@ -80,14 +81,14 @@
* Temporary hook for recording job success.
*/
protected void onSuccess(Job job) {
- LOG.info(job.getJobName() + " succeeded");
+ LOG.info(job.getJobName() + " (" + job.getID() + ")" + " success");
}
/**
* Temporary hook for recording job failure.
*/
protected void onFailure(Job job) {
- LOG.info(job.getJobName() + " failed");
+ LOG.info(job.getJobName() + " (" + job.getID() + ")" + " failure");
}
/**
@@ -128,20 +129,30 @@
@Override
public void run() {
- boolean interrupted = false;
+ boolean graceful;
+ boolean shutdown;
while (true) {
try {
synchronized (mJobs) {
+ graceful = JobMonitor.this.graceful;
+ shutdown = JobMonitor.this.shutdown;
runningJobs.drainTo(mJobs);
}
- final boolean graceful = JobMonitor.this.graceful;
// shutdown conditions; either shutdown requested and all jobs
// have completed or abort requested and there are recently
- // submitted jobs not yet accounted for
- if (interrupted && ((!graceful && runningJobs.isEmpty()) ||
- (graceful && mJobs.isEmpty()))) {
- break;
+ // submitted jobs not in the monitored set
+ if (shutdown) {
+ if (!graceful) {
+ while (!runningJobs.isEmpty()) {
+ synchronized (mJobs) {
+ runningJobs.drainTo(mJobs);
+ }
+ }
+ break;
+ } else if (mJobs.isEmpty()) {
+ break;
+ }
}
while (!mJobs.isEmpty()) {
Job job;
@@ -161,14 +172,16 @@
// reset it here
Thread.currentThread().interrupt();
} else {
- LOG.warn("Lost job " + job.getJobName(), e);
+ LOG.warn("Lost job " + (null == job.getJobName()
+ ? "<unknown>" : job.getJobName()), e);
continue;
}
}
synchronized (mJobs) {
if (!mJobs.offer(job)) {
- LOG.error("Lost job " + job.getJobName()); // should never
- // happen
+ LOG.error("Lost job " + (null == job.getJobName()
+ ? "<unknown>" : job.getJobName())); // should never
+ // happen
}
}
break;
@@ -176,7 +189,7 @@
try {
TimeUnit.MILLISECONDS.sleep(pollDelayMillis);
} catch (InterruptedException e) {
- interrupted = true;
+ shutdown = true;
continue;
}
} catch (Throwable e) {
@@ -198,8 +211,8 @@
* called. Note that, since submission may be sporatic, this will hang
* if no form of shutdown has been requested.
*/
- public void join() throws InterruptedException {
- mThread.join();
+ public void join(long millis) throws InterruptedException {
+ mThread.join(millis);
}
/**
@@ -207,7 +220,10 @@
* Upstream submitter is assumed dead.
*/
public void abort() {
- graceful = false;
+ synchronized (mJobs) {
+ graceful = false;
+ shutdown = true;
+ }
mThread.interrupt();
}
@@ -216,7 +232,10 @@
* Upstream submitter is assumed dead.
*/
public void shutdown() {
- graceful = true;
+ synchronized (mJobs) {
+ graceful = true;
+ shutdown = true;
+ }
mThread.interrupt();
}
}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobSubmitter.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobSubmitter.java
index 7dd6bf5..ae93d01 100644
--- a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobSubmitter.java
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobSubmitter.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.mapred.gridmix;
import java.io.IOException;
+import java.nio.channels.ClosedByInterruptException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.RejectedExecutionException;
@@ -96,6 +97,10 @@
" (" + job.getJob().getID() + ")");
} catch (IOException e) {
LOG.warn("Failed to submit " + job.getJob().getJobName(), e);
+ if (e.getCause() instanceof ClosedByInterruptException) {
+ throw new InterruptedException("Failed to submit " +
+ job.getJob().getJobName());
+ }
} catch (ClassNotFoundException e) {
LOG.warn("Failed to submit " + job.getJob().getJobName(), e);
}
@@ -144,11 +149,11 @@
* Continue running until all queued jobs have been submitted to the
* cluster.
*/
- public void join() throws InterruptedException {
+ public void join(long millis) throws InterruptedException {
if (!shutdown) {
throw new IllegalStateException("Cannot wait for active submit thread");
}
- sched.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+ sched.awaitTermination(millis, TimeUnit.MILLISECONDS);
}
/**
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ReadRecordFactory.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ReadRecordFactory.java
new file mode 100644
index 0000000..2cb806e
--- /dev/null
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ReadRecordFactory.java
@@ -0,0 +1,85 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+
+/**
+ * For every record consumed, read key + val bytes from the stream provided.
+ */
+class ReadRecordFactory extends RecordFactory {
+
+ /**
+ * Size of internal, scratch buffer to read from internal stream.
+ */
+ public static final String GRIDMIX_READ_BUF_SIZE = "gridmix.read.buffer.size";
+
+ private final byte[] buf;
+ private final InputStream src;
+ private final RecordFactory factory;
+
+ /**
+ * @param targetBytes Expected byte count.
+ * @param targetRecords Expected record count.
+ * @param src Stream to read bytes.
+ * @param conf Used to establish read buffer size. @see #GRIDMIX_READ_BUF_SIZE
+ */
+ public ReadRecordFactory(long targetBytes, long targetRecords,
+ InputStream src, Configuration conf) {
+ this(new AvgRecordFactory(targetBytes, targetRecords, conf), src, conf);
+ }
+
+ /**
+ * @param factory Factory to draw record sizes.
+ * @param src Stream to read bytes.
+ * @param conf Used to establish read buffer size. @see #GRIDMIX_READ_BUF_SIZE
+ */
+ public ReadRecordFactory(RecordFactory factory, InputStream src,
+ Configuration conf) {
+ this.src = src;
+ this.factory = factory;
+ buf = new byte[conf.getInt(GRIDMIX_READ_BUF_SIZE, 64 * 1024)];
+ }
+
+ @Override
+ public boolean next(GridmixKey key, GridmixRecord val) throws IOException {
+ if (!factory.next(key, val)) {
+ return false;
+ }
+ for (int len = (null == key ? 0 : key.getSize()) + val.getSize();
+ len > 0; len -= buf.length) {
+ IOUtils.readFully(src, buf, 0, Math.min(buf.length, len));
+ }
+ return true;
+ }
+
+ @Override
+ public float getProgress() throws IOException {
+ return factory.getProgress();
+ }
+
+ @Override
+ public void close() throws IOException {
+ IOUtils.cleanup(null, src);
+ factory.close();
+ }
+}
diff --git a/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/RecordFactory.java b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/RecordFactory.java
new file mode 100644
index 0000000..7abcf78
--- /dev/null
+++ b/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/RecordFactory.java
@@ -0,0 +1,40 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Interface for producing records as inputs and outputs to tasks.
+ */
+abstract class RecordFactory implements Closeable {
+
+ /**
+ * Transform the given record or perform some operation.
+ * @return true if the record should be emitted.
+ */
+ public abstract boolean next(GridmixKey key, GridmixRecord val)
+ throws IOException;
+
+ /**
+ * Estimate of exhausted record capacity.
+ */
+ public abstract float getProgress() throws IOException;
+
+}
diff --git a/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/DebugJobFactory.java b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/DebugJobFactory.java
index bb0e476..8ba3b10 100644
--- a/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/DebugJobFactory.java
+++ b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/DebugJobFactory.java
@@ -19,6 +19,7 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@@ -27,7 +28,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.TaskType;
@@ -79,21 +79,58 @@
public void close() { }
}
+ static double[] getDistr(Random r, double mindist, int size) {
+ assert 0.0 <= mindist && mindist <= 1.0;
+ final double min = mindist / size;
+ final double rem = 1.0 - min * size;
+ final double[] tmp = new double[size];
+ for (int i = 0; i < tmp.length - 1; ++i) {
+ tmp[i] = r.nextDouble() * rem;
+ }
+ tmp[tmp.length - 1] = rem;
+ Arrays.sort(tmp);
+
+ final double[] ret = new double[size];
+ ret[0] = tmp[0] + min;
+ for (int i = 1; i < size; ++i) {
+ ret[i] = tmp[i] - tmp[i-1] + min;
+ }
+ return ret;
+ }
+
/**
* Generate random task data for a synthetic job.
*/
static class MockJob implements JobStory {
- public static final String MIN_BYTES_IN = "gridmix.test.min.bytes.in";
- public static final String VAR_BYTES_IN = "gridmix.test.var.bytes.in";
- public static final String MIN_BYTES_OUT = "gridmix.test.min.bytes.out";
- public static final String VAR_BYTES_OUT = "gridmix.test.var.bytes.out";
+ static final int MIN_REC = 1 << 14;
+ static final int MIN_BYTES = 1 << 20;
+ static final int VAR_REC = 1 << 14;
+ static final int VAR_BYTES = 4 << 20;
+ static final int MAX_MAP = 5;
+ static final int MAX_RED = 3;
- public static final String MIN_REC_SIZE = "gridmix.test.min.rec.bytes";
- public static final String VAR_REC_SIZE = "gridmix.test.var.rec.bytes";
-
- public static final String MAX_MAPS = "gridmix.test.max.maps";
- public static final String MAX_REDS = "gridmix.test.max.reduces";
+ static void initDist(Random r, double min, int[] recs, long[] bytes,
+ long tot_recs, long tot_bytes) {
+ final double[] recs_dist = getDistr(r, min, recs.length);
+ final double[] bytes_dist = getDistr(r, min, recs.length);
+ long totalbytes = 0L;
+ int totalrecs = 0;
+ for (int i = 0; i < recs.length; ++i) {
+ recs[i] = (int) Math.round(tot_recs * recs_dist[i]);
+ bytes[i] = Math.round(tot_bytes * bytes_dist[i]);
+ totalrecs += recs[i];
+ totalbytes += bytes[i];
+ }
+ // Add/remove excess
+ recs[0] += totalrecs - tot_recs;
+ bytes[0] += totalbytes - tot_bytes;
+ if (LOG.isInfoEnabled()) {
+ LOG.info("DIST: " + Arrays.toString(recs) + " " +
+ tot_recs + "/" + totalrecs + " " +
+ Arrays.toString(bytes) + " " + tot_bytes + "/" + totalbytes);
+ }
+ }
private static final AtomicInteger seq = new AtomicInteger(0);
// set timestamps in the past
@@ -101,97 +138,65 @@
new AtomicLong(System.currentTimeMillis() -
TimeUnit.MILLISECONDS.convert(60, TimeUnit.DAYS));
+ private final int id;
private final String name;
private final int[] m_recsIn, m_recsOut, r_recsIn, r_recsOut;
private final long[] m_bytesIn, m_bytesOut, r_bytesIn, r_bytesOut;
private final long submitTime;
- public MockJob() {
- this(new Configuration(false));
- }
-
public MockJob(Configuration conf) {
- this(conf.getInt(MIN_BYTES_IN, 1 << 20),
- conf.getInt(VAR_BYTES_IN, 5 << 20),
- conf.getInt(MIN_BYTES_OUT, 1 << 20),
- conf.getInt(VAR_BYTES_OUT, 5 << 20),
- conf.getInt(MIN_REC_SIZE , 100),
- conf.getInt(VAR_REC_SIZE , 1 << 15),
- conf.getInt(MAX_MAPS, 5),
- conf.getInt(MAX_REDS, 3));
- }
-
- public MockJob(int min_bytes_in, int var_bytes_in,
- int min_bytes_out, int var_bytes_out,
- int min_rec_size, int var_rec_size,
- int max_maps, int max_reds) {
final Random r = new Random();
- name = String.format("MOCKJOB%05d", seq.getAndIncrement());
+ final long seed = r.nextLong();
+ r.setSeed(seed);
+ id = seq.getAndIncrement();
+ name = String.format("MOCKJOB%05d", id);
+ LOG.info(name + " (" + seed + ")");
submitTime = timestamp.addAndGet(TimeUnit.MILLISECONDS.convert(
r.nextInt(10), TimeUnit.SECONDS));
- int iMapBTotal = 0, oMapBTotal = 0, iRedBTotal = 0, oRedBTotal = 0;
- int iMapRTotal = 0, oMapRTotal = 0, iRedRTotal = 0, oRedRTotal = 0;
- final int iAvgMapRec = r.nextInt(var_rec_size) + min_rec_size;
- final int oAvgMapRec = r.nextInt(var_rec_size) + min_rec_size;
+ m_recsIn = new int[r.nextInt(MAX_MAP) + 1];
+ m_bytesIn = new long[m_recsIn.length];
+ m_recsOut = new int[m_recsIn.length];
+ m_bytesOut = new long[m_recsIn.length];
- // MAP
- m_bytesIn = new long[r.nextInt(max_maps) + 1];
- m_bytesOut = new long[m_bytesIn.length];
- m_recsIn = new int[m_bytesIn.length];
- m_recsOut = new int[m_bytesIn.length];
- for (int i = 0; i < m_bytesIn.length; ++i) {
- m_bytesIn[i] = r.nextInt(var_bytes_in) + min_bytes_in;
- iMapBTotal += m_bytesIn[i];
- m_recsIn[i] = (int)(m_bytesIn[i] / iAvgMapRec);
- iMapRTotal += m_recsIn[i];
+ r_recsIn = new int[r.nextInt(MAX_RED) + 1];
+ r_bytesIn = new long[r_recsIn.length];
+ r_recsOut = new int[r_recsIn.length];
+ r_bytesOut = new long[r_recsIn.length];
- m_bytesOut[i] = r.nextInt(var_bytes_out) + min_bytes_out;
- oMapBTotal += m_bytesOut[i];
- m_recsOut[i] = (int)(m_bytesOut[i] / oAvgMapRec);
- oMapRTotal += m_recsOut[i];
- }
+ // map input
+ final long map_recs = r.nextInt(VAR_REC) + MIN_REC;
+ final long map_bytes = r.nextInt(VAR_BYTES) + MIN_BYTES;
+ initDist(r, 0.5, m_recsIn, m_bytesIn, map_recs, map_bytes);
- // REDUCE
- r_bytesIn = new long[r.nextInt(max_reds) + 1];
- r_bytesOut = new long[r_bytesIn.length];
- r_recsIn = new int[r_bytesIn.length];
- r_recsOut = new int[r_bytesIn.length];
- iRedBTotal = oMapBTotal;
- iRedRTotal = oMapRTotal;
- for (int j = 0; iRedBTotal > 0; ++j) {
- int i = j % r_bytesIn.length;
- final int inc = r.nextInt(var_bytes_out) + min_bytes_out;
- r_bytesIn[i] += inc;
- iRedBTotal -= inc;
- if (iRedBTotal < 0) {
- r_bytesIn[i] += iRedBTotal;
- iRedBTotal = 0;
- }
- iRedRTotal += r_recsIn[i];
- r_recsIn[i] = (int)(r_bytesIn[i] / oAvgMapRec);
- iRedRTotal -= r_recsIn[i];
- if (iRedRTotal < 0) {
- r_recsIn[i] += iRedRTotal;
- iRedRTotal = 0;
- }
+ // shuffle
+ final long shuffle_recs = r.nextInt(VAR_REC) + MIN_REC;
+ final long shuffle_bytes = r.nextInt(VAR_BYTES) + MIN_BYTES;
+ initDist(r, 0.4, m_recsOut, m_bytesOut, shuffle_recs, shuffle_bytes);
+ initDist(r, 0.8, r_recsIn, r_bytesIn, shuffle_recs, shuffle_bytes);
- r_bytesOut[i] = r.nextInt(var_bytes_in) + min_bytes_in;
- oRedBTotal += r_bytesOut[i];
- r_recsOut[i] = (int)(r_bytesOut[i] / iAvgMapRec);
- oRedRTotal += r_recsOut[i];
- }
- r_recsIn[0] += iRedRTotal;
+ // reduce output
+ final long red_recs = r.nextInt(VAR_REC) + MIN_REC;
+ final long red_bytes = r.nextInt(VAR_BYTES) + MIN_BYTES;
+ initDist(r, 0.4, r_recsOut, r_bytesOut, red_recs, red_bytes);
if (LOG.isDebugEnabled()) {
- iRedRTotal = 0;
- iRedBTotal = 0;
- for (int i = 0; i < r_bytesIn.length; ++i) {
+ int iMapBTotal = 0, oMapBTotal = 0, iRedBTotal = 0, oRedBTotal = 0;
+ int iMapRTotal = 0, oMapRTotal = 0, iRedRTotal = 0, oRedRTotal = 0;
+ for (int i = 0; i < m_recsIn.length; ++i) {
+ iMapRTotal += m_recsIn[i];
+ iMapBTotal += m_bytesIn[i];
+ oMapRTotal += m_recsOut[i];
+ oMapBTotal += m_bytesOut[i];
+ }
+ for (int i = 0; i < r_recsIn.length; ++i) {
iRedRTotal += r_recsIn[i];
iRedBTotal += r_bytesIn[i];
+ oRedRTotal += r_recsOut[i];
+ oRedBTotal += r_bytesOut[i];
}
LOG.debug(String.format("%s: M (%03d) %6d/%10d -> %6d/%10d" +
- " R (%03d) %6d/%10d -> %6d/%10d @%d", name,
+ " R (%03d) %6d/%10d -> %6d/%10d @%d", name,
m_bytesIn.length, iMapRTotal, iMapBTotal, oMapRTotal, oMapBTotal,
r_bytesIn.length, iRedRTotal, iRedBTotal, oRedRTotal, oRedBTotal,
submitTime));
@@ -210,7 +215,7 @@
@Override
public JobID getJobID() {
- return null;
+ return new JobID("job_mock_" + name, id);
}
@Override
diff --git a/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFilePool.java b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFilePool.java
new file mode 100644
index 0000000..4be90c6
--- /dev/null
+++ b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFilePool.java
@@ -0,0 +1,189 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Random;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
+
+public class TestFilePool {
+
+ static final Log LOG = LogFactory.getLog(TestFileQueue.class);
+ static final int NFILES = 26;
+ static final Path base = getBaseDir();
+
+ static Path getBaseDir() {
+ try {
+ final Configuration conf = new Configuration();
+ final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+ return new Path(System.getProperty("test.build.data", "/tmp"),
+ "testFilePool").makeQualified(fs);
+ } catch (IOException e) {
+ fail();
+ }
+ return null;
+ }
+
+ @BeforeClass
+ public static void setup() throws IOException {
+ final Configuration conf = new Configuration();
+ final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+ fs.delete(base, true);
+ final Random r = new Random();
+ final long seed = r.nextLong();
+ r.setSeed(seed);
+ LOG.info("seed: " + seed);
+ fs.mkdirs(base);
+ for (int i = 0; i < NFILES; ++i) {
+ Path file = base;
+ for (double d = 0.6; d > 0.0; d *= 0.8) {
+ if (r.nextDouble() < d) {
+ file = new Path(base, Integer.toString(r.nextInt(3)));
+ continue;
+ }
+ break;
+ }
+ OutputStream out = null;
+ try {
+ out = fs.create(new Path(file, "" + (char)('A' + i)));
+ final byte[] b = new byte[1024];
+ Arrays.fill(b, (byte)('A' + i));
+ for (int len = ((i % 13) + 1) * 1024; len > 0; len -= 1024) {
+ out.write(b);
+ }
+ } finally {
+ if (out != null) {
+ out.close();
+ }
+ }
+ }
+ }
+
+ @AfterClass
+ public static void cleanup() throws IOException {
+ final Configuration conf = new Configuration();
+ final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+ fs.delete(base, true);
+ }
+
+ @Test
+ public void testUnsuitable() throws Exception {
+ try {
+ final Configuration conf = new Configuration();
+ // all files 13k or less
+ conf.setLong(FilePool.GRIDMIX_MIN_FILE, 14 * 1024);
+ final FilePool pool = new FilePool(conf, base);
+ pool.refresh();
+ } catch (IOException e) {
+ return;
+ }
+ fail();
+ }
+
+ @Test
+ public void testPool() throws Exception {
+ final Random r = new Random();
+ final Configuration conf = new Configuration();
+ conf.setLong(FilePool.GRIDMIX_MIN_FILE, 3 * 1024);
+ final FilePool pool = new FilePool(conf, base);
+ pool.refresh();
+ final ArrayList<FileStatus> files = new ArrayList<FileStatus>();
+
+ // ensure 1k, 2k files excluded
+ final int expectedPoolSize = (NFILES / 2 * (NFILES / 2 + 1) - 6) * 1024;
+ assertEquals(expectedPoolSize, pool.getInputFiles(Long.MAX_VALUE, files));
+ assertEquals(NFILES - 4, files.size());
+
+ // exact match
+ files.clear();
+ assertEquals(expectedPoolSize, pool.getInputFiles(expectedPoolSize, files));
+
+ // match random within 12k
+ files.clear();
+ final long rand = r.nextInt(expectedPoolSize);
+ assertTrue("Missed: " + rand,
+ (NFILES / 2) * 1024 > rand - pool.getInputFiles(rand, files));
+
+ // all files
+ conf.setLong(FilePool.GRIDMIX_MIN_FILE, 0);
+ pool.refresh();
+ files.clear();
+ assertEquals((NFILES / 2 * (NFILES / 2 + 1)) * 1024,
+ pool.getInputFiles(Long.MAX_VALUE, files));
+ }
+
+ void checkSplitEq(FileSystem fs, CombineFileSplit split, long bytes)
+ throws Exception {
+ long splitBytes = 0L;
+ HashSet<Path> uniq = new HashSet<Path>();
+ for (int i = 0; i < split.getNumPaths(); ++i) {
+ splitBytes += split.getLength(i);
+ assertTrue(
+ split.getLength(i) <= fs.getFileStatus(split.getPath(i)).getLen());
+ assertFalse(uniq.contains(split.getPath(i)));
+ uniq.add(split.getPath(i));
+ }
+ assertEquals(bytes, splitBytes);
+ }
+
+ @Test
+ public void testStriper() throws Exception {
+ final Random r = new Random();
+ final Configuration conf = new Configuration();
+ final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+ conf.setLong(FilePool.GRIDMIX_MIN_FILE, 3 * 1024);
+ final FilePool pool = new FilePool(conf, base) {
+ @Override
+ public BlockLocation[] locationsFor(FileStatus stat, long start, long len)
+ throws IOException {
+ return new BlockLocation[] { new BlockLocation() };
+ }
+ };
+ pool.refresh();
+
+ final int expectedPoolSize = (NFILES / 2 * (NFILES / 2 + 1) - 6) * 1024;
+ final InputStriper striper = new InputStriper(pool, expectedPoolSize);
+ int last = 0;
+ for (int i = 0; i < expectedPoolSize;
+ last = Math.min(expectedPoolSize - i, r.nextInt(expectedPoolSize))) {
+ checkSplitEq(fs, striper.splitFor(pool, last, 0), last);
+ i += last;
+ }
+ final InputStriper striper2 = new InputStriper(pool, expectedPoolSize);
+ checkSplitEq(fs, striper2.splitFor(pool, expectedPoolSize, 0),
+ expectedPoolSize);
+ }
+
+}
diff --git a/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFileQueue.java b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFileQueue.java
new file mode 100644
index 0000000..a4668ee
--- /dev/null
+++ b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFileQueue.java
@@ -0,0 +1,143 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
+
+public class TestFileQueue {
+
+ static final Log LOG = LogFactory.getLog(TestFileQueue.class);
+ static final int NFILES = 4;
+ static final int BLOCK = 256;
+ static final Path[] paths = new Path[NFILES];
+ static final String[] loc = new String[NFILES];
+ static final long[] start = new long[NFILES];
+ static final long[] len = new long[NFILES];
+
+ @BeforeClass
+ public static void setup() throws IOException {
+ final Configuration conf = new Configuration();
+ final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+ final Path p = new Path(System.getProperty("test.build.data", "/tmp"),
+ "testFileQueue").makeQualified(fs);
+ fs.delete(p, true);
+ final byte[] b = new byte[BLOCK];
+ for (int i = 0; i < NFILES; ++i) {
+ Arrays.fill(b, (byte)('A' + i));
+ paths[i] = new Path(p, "" + (char)('A' + i));
+ OutputStream f = null;
+ try {
+ f = fs.create(paths[i]);
+ f.write(b);
+ } finally {
+ if (f != null) {
+ f.close();
+ }
+ }
+ }
+ }
+
+ @AfterClass
+ public static void cleanup() throws IOException {
+ final Configuration conf = new Configuration();
+ final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+ final Path p = new Path(System.getProperty("test.build.data", "/tmp"),
+ "testFileQueue").makeQualified(fs);
+ fs.delete(p, true);
+ }
+
+ static ByteArrayOutputStream fillVerif() throws IOException {
+ final byte[] b = new byte[BLOCK];
+ final ByteArrayOutputStream out = new ByteArrayOutputStream();
+ for (int i = 0; i < NFILES; ++i) {
+ Arrays.fill(b, (byte)('A' + i));
+ out.write(b, 0, (int)len[i]);
+ }
+ return out;
+ }
+
+ @Test
+ public void testRepeat() throws Exception {
+ final Configuration conf = new Configuration();
+ Arrays.fill(loc, "");
+ Arrays.fill(start, 0L);
+ Arrays.fill(len, BLOCK);
+
+ final ByteArrayOutputStream out = fillVerif();
+ final FileQueue q =
+ new FileQueue(new CombineFileSplit(paths, start, len, loc), conf);
+ final byte[] verif = out.toByteArray();
+ final byte[] check = new byte[2 * NFILES * BLOCK];
+ q.read(check, 0, NFILES * BLOCK);
+ assertArrayEquals(verif, Arrays.copyOf(check, NFILES * BLOCK));
+
+ final byte[] verif2 = new byte[2 * NFILES * BLOCK];
+ System.arraycopy(verif, 0, verif2, 0, verif.length);
+ System.arraycopy(verif, 0, verif2, verif.length, verif.length);
+ q.read(check, 0, 2 * NFILES * BLOCK);
+ assertArrayEquals(verif2, check);
+
+ }
+
+ @Test
+ public void testUneven() throws Exception {
+ final Configuration conf = new Configuration();
+ Arrays.fill(loc, "");
+ Arrays.fill(start, 0L);
+ Arrays.fill(len, BLOCK);
+
+ final int B2 = BLOCK / 2;
+ for (int i = 0; i < NFILES; i += 2) {
+ start[i] += B2;
+ len[i] -= B2;
+ }
+ final FileQueue q =
+ new FileQueue(new CombineFileSplit(paths, start, len, loc), conf);
+ final ByteArrayOutputStream out = fillVerif();
+ final byte[] verif = out.toByteArray();
+ final byte[] check = new byte[NFILES / 2 * BLOCK + NFILES / 2 * B2];
+ q.read(check, 0, verif.length);
+ assertArrayEquals(verif, Arrays.copyOf(check, verif.length));
+ q.read(check, 0, verif.length);
+ assertArrayEquals(verif, Arrays.copyOf(check, verif.length));
+ }
+
+ @Test
+ public void testEmpty() throws Exception {
+ final Configuration conf = new Configuration();
+ // verify OK if unused
+ final FileQueue q = new FileQueue(new CombineFileSplit(
+ new Path[0], new long[0], new long[0], new String[0]), conf);
+ }
+
+}
diff --git a/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixRecord.java b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixRecord.java
new file mode 100644
index 0000000..c283e86
--- /dev/null
+++ b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixRecord.java
@@ -0,0 +1,277 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+
+public class TestGridmixRecord {
+ private static final Log LOG = LogFactory.getLog(TestGridmixRecord.class);
+
+ static void lengthTest(GridmixRecord x, GridmixRecord y, int min,
+ int max) throws Exception {
+ final Random r = new Random();
+ final long seed = r.nextLong();
+ r.setSeed(seed);
+ LOG.info("length: " + seed);
+ final DataInputBuffer in = new DataInputBuffer();
+ final DataOutputBuffer out1 = new DataOutputBuffer();
+ final DataOutputBuffer out2 = new DataOutputBuffer();
+ for (int i = min; i < max; ++i) {
+ setSerialize(x, r.nextLong(), i, out1);
+ // check write
+ assertEquals(i, out1.getLength());
+ // write to stream
+ x.write(out2);
+ // check read
+ in.reset(out1.getData(), 0, out1.getLength());
+ y.readFields(in);
+ assertEquals(i, x.getSize());
+ assertEquals(i, y.getSize());
+ }
+ // check stream read
+ in.reset(out2.getData(), 0, out2.getLength());
+ for (int i = min; i < max; ++i) {
+ y.readFields(in);
+ assertEquals(i, y.getSize());
+ }
+ }
+
+ static void randomReplayTest(GridmixRecord x, GridmixRecord y, int min,
+ int max) throws Exception {
+ final Random r = new Random();
+ final long seed = r.nextLong();
+ r.setSeed(seed);
+ LOG.info("randReplay: " + seed);
+ final DataOutputBuffer out1 = new DataOutputBuffer();
+ for (int i = min; i < max; ++i) {
+ final int s = out1.getLength();
+ x.setSeed(r.nextLong());
+ x.setSize(i);
+ x.write(out1);
+ assertEquals(i, out1.getLength() - s);
+ }
+ final DataInputBuffer in = new DataInputBuffer();
+ in.reset(out1.getData(), 0, out1.getLength());
+ final DataOutputBuffer out2 = new DataOutputBuffer();
+ // deserialize written records, write to separate buffer
+ for (int i = min; i < max; ++i) {
+ final int s = in.getPosition();
+ y.readFields(in);
+ assertEquals(i, in.getPosition() - s);
+ y.write(out2);
+ }
+ // verify written contents match
+ assertEquals(out1.getLength(), out2.getLength());
+ // assumes that writes will grow buffer deterministically
+ assertEquals("Bad test", out1.getData().length, out2.getData().length);
+ assertArrayEquals(out1.getData(), out2.getData());
+ }
+
+ static void eqSeedTest(GridmixRecord x, GridmixRecord y, int max)
+ throws Exception {
+ final Random r = new Random();
+ final long s = r.nextLong();
+ r.setSeed(s);
+ LOG.info("eqSeed: " + s);
+ assertEquals(x.fixedBytes(), y.fixedBytes());
+ final int min = x.fixedBytes() + 1;
+ final DataOutputBuffer out1 = new DataOutputBuffer();
+ final DataOutputBuffer out2 = new DataOutputBuffer();
+ for (int i = min; i < max; ++i) {
+ final long seed = r.nextLong();
+ setSerialize(x, seed, i, out1);
+ setSerialize(y, seed, i, out2);
+ assertEquals(x, y);
+ assertEquals(x.hashCode(), y.hashCode());
+
+ // verify written contents match
+ assertEquals(out1.getLength(), out2.getLength());
+ // assumes that writes will grow buffer deterministically
+ assertEquals("Bad test", out1.getData().length, out2.getData().length);
+ assertArrayEquals(out1.getData(), out2.getData());
+ }
+ }
+
+ static void binSortTest(GridmixRecord x, GridmixRecord y, int min,
+ int max, WritableComparator cmp) throws Exception {
+ final Random r = new Random();
+ final long s = r.nextLong();
+ r.setSeed(s);
+ LOG.info("sort: " + s);
+ final DataOutputBuffer out1 = new DataOutputBuffer();
+ final DataOutputBuffer out2 = new DataOutputBuffer();
+ for (int i = min; i < max; ++i) {
+ final long seed1 = r.nextLong();
+ setSerialize(x, seed1, i, out1);
+ assertEquals(0, x.compareSeed(seed1, Math.max(0, i - x.fixedBytes())));
+
+ final long seed2 = r.nextLong();
+ setSerialize(y, seed2, i, out2);
+ assertEquals(0, y.compareSeed(seed2, Math.max(0, i - x.fixedBytes())));
+
+ // for eq sized records, ensure byte cmp where req
+ final int chk = WritableComparator.compareBytes(
+ out1.getData(), 0, out1.getLength(),
+ out2.getData(), 0, out2.getLength());
+ assertEquals(chk, x.compareTo(y));
+ assertEquals(chk, cmp.compare(
+ out1.getData(), 0, out1.getLength(),
+ out2.getData(), 0, out2.getLength()));
+ // write second copy, compare eq
+ final int s1 = out1.getLength();
+ x.write(out1);
+ assertEquals(0, cmp.compare(out1.getData(), 0, s1,
+ out1.getData(), s1, out1.getLength() - s1));
+ final int s2 = out2.getLength();
+ y.write(out2);
+ assertEquals(0, cmp.compare(out2.getData(), 0, s2,
+ out2.getData(), s2, out2.getLength() - s2));
+ assertEquals(chk, cmp.compare(out1.getData(), 0, s1,
+ out2.getData(), s2, out2.getLength() - s2));
+ }
+ }
+
+ static void checkSpec(GridmixKey a, GridmixKey b) throws Exception {
+ final Random r = new Random();
+ final long s = r.nextLong();
+ r.setSeed(s);
+ LOG.info("spec: " + s);
+ final DataInputBuffer in = new DataInputBuffer();
+ final DataOutputBuffer out = new DataOutputBuffer();
+ a.setType(GridmixKey.REDUCE_SPEC);
+ b.setType(GridmixKey.REDUCE_SPEC);
+ for (int i = 0; i < 100; ++i) {
+ final int in_rec = r.nextInt(Integer.MAX_VALUE);
+ a.setReduceInputRecords(in_rec);
+ final int out_rec = r.nextInt(Integer.MAX_VALUE);
+ a.setReduceOutputRecords(out_rec);
+ final int out_bytes = r.nextInt(Integer.MAX_VALUE);
+ a.setReduceOutputBytes(out_bytes);
+ final int min = WritableUtils.getVIntSize(in_rec)
+ + WritableUtils.getVIntSize(out_rec)
+ + WritableUtils.getVIntSize(out_bytes);
+ assertEquals(min + 2, a.fixedBytes()); // meta + vint min
+ final int size = r.nextInt(1024) + a.fixedBytes() + 1;
+ setSerialize(a, r.nextLong(), size, out);
+ assertEquals(size, out.getLength());
+ assertTrue(a.equals(a));
+ assertEquals(0, a.compareTo(a));
+
+ in.reset(out.getData(), 0, out.getLength());
+
+ b.readFields(in);
+ assertEquals(size, b.getSize());
+ assertEquals(in_rec, b.getReduceInputRecords());
+ assertEquals(out_rec, b.getReduceOutputRecords());
+ assertEquals(out_bytes, b.getReduceOutputBytes());
+ assertTrue(a.equals(b));
+ assertEquals(0, a.compareTo(b));
+ assertEquals(a.hashCode(), b.hashCode());
+ }
+ }
+
+ static void setSerialize(GridmixRecord x, long seed, int size,
+ DataOutputBuffer out) throws IOException {
+ x.setSeed(seed);
+ x.setSize(size);
+ out.reset();
+ x.write(out);
+ }
+
+ @Test
+ public void testKeySpec() throws Exception {
+ final int min = 5;
+ final int max = 300;
+ final GridmixKey a = new GridmixKey(GridmixKey.REDUCE_SPEC, 1, 0L);
+ final GridmixKey b = new GridmixKey(GridmixKey.REDUCE_SPEC, 1, 0L);
+ lengthTest(a, b, min, max);
+ randomReplayTest(a, b, min, max);
+ binSortTest(a, b, min, max, new GridmixKey.Comparator());
+ // 2 fixed GR bytes, 1 type, 3 spec
+ eqSeedTest(a, b, max);
+ checkSpec(a, b);
+ }
+
+ @Test
+ public void testKeyData() throws Exception {
+ final int min = 2;
+ final int max = 300;
+ final GridmixKey a = new GridmixKey(GridmixKey.DATA, 1, 0L);
+ final GridmixKey b = new GridmixKey(GridmixKey.DATA, 1, 0L);
+ lengthTest(a, b, min, max);
+ randomReplayTest(a, b, min, max);
+ binSortTest(a, b, min, max, new GridmixKey.Comparator());
+ // 2 fixed GR bytes, 1 type
+ eqSeedTest(a, b, 300);
+ }
+
+ @Test
+ public void testBaseRecord() throws Exception {
+ final int min = 1;
+ final int max = 300;
+ final GridmixRecord a = new GridmixRecord();
+ final GridmixRecord b = new GridmixRecord();
+ lengthTest(a, b, min, max);
+ randomReplayTest(a, b, min, max);
+ binSortTest(a, b, min, max, new GridmixRecord.Comparator());
+ // 2 fixed GR bytes
+ eqSeedTest(a, b, 300);
+ }
+
+ public static void main(String[] argv) throws Exception {
+ boolean fail = false;
+ final TestGridmixRecord test = new TestGridmixRecord();
+ try { test.testKeySpec(); } catch (Exception e) {
+ fail = true;
+ e.printStackTrace();
+ }
+ try {test.testKeyData(); } catch (Exception e) {
+ fail = true;
+ e.printStackTrace();
+ }
+ try {test.testBaseRecord(); } catch (Exception e) {
+ fail = true;
+ e.printStackTrace();
+ }
+ System.exit(fail ? -1 : 0);
+ }
+
+ static void printDebug(GridmixRecord a, GridmixRecord b) throws IOException {
+ DataOutputBuffer out = new DataOutputBuffer();
+ a.write(out);
+ System.out.println("A " +
+ Arrays.toString(Arrays.copyOf(out.getData(), out.getLength())));
+ out.reset();
+ b.write(out);
+ System.out.println("B " +
+ Arrays.toString(Arrays.copyOf(out.getData(), out.getLength())));
+ }
+
+}
diff --git a/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java
index 73da804..b24ff92 100644
--- a/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java
+++ b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java
@@ -20,9 +20,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Comparator;
import java.util.HashMap;
-import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingQueue;
@@ -35,16 +33,14 @@
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.MiniMRCluster;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.CounterGroup;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.TaskReport;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.tools.rumen.JobStory;
import org.apache.hadoop.tools.rumen.TaskInfo;
-import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import static org.apache.hadoop.mapreduce.TaskCounter.*;
@@ -96,7 +92,7 @@
static class TestMonitor extends JobMonitor {
- static final long SLOPBYTES = 5 * 1024;
+ static final long SLOPBYTES = 1024;
private final int expected;
private final BlockingQueue<Job> retiredJobs;
@@ -138,12 +134,12 @@
final TaskReport[] mReports = job.getTaskReports(TaskType.MAP);
assertEquals("Mismatched map count", nMaps, mReports.length);
check(TaskType.MAP, job, spec, mReports,
- 0, 1, nReds * SLOPBYTES, nReds + 1);
+ 0, 0, SLOPBYTES, nReds);
final TaskReport[] rReports = job.getTaskReports(TaskType.REDUCE);
assertEquals("Mismatched reduce count", nReds, rReports.length);
check(TaskType.REDUCE, job, spec, rReports,
- nMaps * SLOPBYTES, nMaps + 1, 0, 1);
+ nMaps * SLOPBYTES, 2 * nMaps, 0, 0);
}
}
@@ -167,7 +163,8 @@
switch (type) {
case MAP:
runInputBytes[i] = counters.findCounter("FileSystemCounters",
- "HDFS_BYTES_READ").getValue();
+ "HDFS_BYTES_READ").getValue() -
+ counters.findCounter(TaskCounter.SPLIT_RAW_BYTES).getValue();
runInputRecords[i] =
(int)counters.findCounter(MAP_INPUT_RECORDS).getValue();
runOutputBytes[i] =
@@ -176,74 +173,97 @@
(int)counters.findCounter(MAP_OUTPUT_RECORDS).getValue();
specInfo = spec.getTaskInfo(TaskType.MAP, i);
+ specInputRecords[i] = specInfo.getInputRecords();
+ specInputBytes[i] = specInfo.getInputBytes();
+ specOutputRecords[i] = specInfo.getOutputRecords();
+ specOutputBytes[i] = specInfo.getOutputBytes();
+ System.out.printf(type + " SPEC: %9d -> %9d :: %5d -> %5d\n",
+ specInputBytes[i], specOutputBytes[i],
+ specInputRecords[i], specOutputRecords[i]);
+ System.out.printf(type + " RUN: %9d -> %9d :: %5d -> %5d\n",
+ runInputBytes[i], runOutputBytes[i],
+ runInputRecords[i], runOutputRecords[i]);
break;
case REDUCE:
- runInputBytes[i] =
- counters.findCounter(REDUCE_SHUFFLE_BYTES).getValue();
- runInputRecords[i] =
- (int)counters.findCounter(REDUCE_INPUT_RECORDS).getValue();
- runOutputBytes[i] =
- counters.findCounter("FileSystemCounters",
- "HDFS_BYTES_WRITTEN").getValue();
- runOutputRecords[i] =
- (int)counters.findCounter(REDUCE_OUTPUT_RECORDS).getValue();
+ runInputBytes[i] = 0;
+ runInputRecords[i] =
+ (int)counters.findCounter(REDUCE_INPUT_RECORDS).getValue();
+ runOutputBytes[i] =
+ counters.findCounter("FileSystemCounters",
+ "HDFS_BYTES_WRITTEN").getValue();
+ runOutputRecords[i] =
+ (int)counters.findCounter(REDUCE_OUTPUT_RECORDS).getValue();
+
specInfo = spec.getTaskInfo(TaskType.REDUCE, i);
+ // There is no reliable counter for reduce input bytes. The
+ // variable-length encoding of intermediate records and other noise
+ // make this quantity difficult to estimate. The shuffle and spec
+ // input bytes are included in debug output for reference, but are
+ // not checked
+ specInputBytes[i] = 0;
+ specInputRecords[i] = specInfo.getInputRecords();
+ specOutputRecords[i] = specInfo.getOutputRecords();
+ specOutputBytes[i] = specInfo.getOutputBytes();
+ System.out.printf(type + " SPEC: (%9d) -> %9d :: %5d -> %5d\n",
+ specInfo.getInputBytes(), specOutputBytes[i],
+ specInputRecords[i], specOutputRecords[i]);
+ System.out.printf(type + " RUN: (%9d) -> %9d :: %5d -> %5d\n",
+ counters.findCounter(REDUCE_SHUFFLE_BYTES).getValue(),
+ runOutputBytes[i], runInputRecords[i], runOutputRecords[i]);
break;
default:
specInfo = null;
fail("Unexpected type: " + type);
}
- specInputBytes[i] = specInfo.getInputBytes();
- specInputRecords[i] = specInfo.getInputRecords();
- specOutputRecords[i] = specInfo.getOutputRecords();
- specOutputBytes[i] = specInfo.getOutputBytes();
- System.out.printf(type + " SPEC: %9d -> %9d :: %5d -> %5d\n",
- specInputBytes[i], specOutputBytes[i],
- specInputRecords[i], specOutputRecords[i]);
- System.out.printf(type + " RUN: %9d -> %9d :: %5d -> %5d\n",
- runInputBytes[i], runOutputBytes[i],
- runInputRecords[i], runOutputRecords[i]);
}
// Check input bytes
Arrays.sort(specInputBytes);
Arrays.sort(runInputBytes);
for (int i = 0; i < runTasks.length; ++i) {
- assertTrue("Mismatched input bytes " +
+ assertTrue("Mismatched " + type + " input bytes " +
specInputBytes[i] + "/" + runInputBytes[i],
- runInputBytes[i] - specInputBytes[i] <= extraInputBytes);
+ eqPlusMinus(runInputBytes[i], specInputBytes[i], extraInputBytes));
}
// Check input records
Arrays.sort(specInputRecords);
Arrays.sort(runInputRecords);
for (int i = 0; i < runTasks.length; ++i) {
- assertTrue("Mismatched input records " +
+ assertTrue("Mismatched " + type + " input records " +
specInputRecords[i] + "/" + runInputRecords[i],
- runInputRecords[i] - specInputRecords[i] <= extraInputRecords);
+ eqPlusMinus(runInputRecords[i], specInputRecords[i],
+ extraInputRecords));
}
// Check output bytes
Arrays.sort(specOutputBytes);
Arrays.sort(runOutputBytes);
for (int i = 0; i < runTasks.length; ++i) {
- assertTrue("Mismatched output bytes " +
+ assertTrue("Mismatched " + type + " output bytes " +
specOutputBytes[i] + "/" + runOutputBytes[i],
- runOutputBytes[i] - specOutputBytes[i] <= extraOutputBytes);
+ eqPlusMinus(runOutputBytes[i], specOutputBytes[i],
+ extraOutputBytes));
}
// Check output records
Arrays.sort(specOutputRecords);
Arrays.sort(runOutputRecords);
for (int i = 0; i < runTasks.length; ++i) {
- assertTrue("Mismatched output records " +
+ assertTrue("Mismatched " + type + " output records " +
specOutputRecords[i] + "/" + runOutputRecords[i],
- runOutputRecords[i] - specOutputRecords[i] <= extraOutputRecords);
+ eqPlusMinus(runOutputRecords[i], specOutputRecords[i],
+ extraOutputRecords));
}
}
+ private static boolean eqPlusMinus(long a, long b, long x) {
+ final long diff = Math.abs(a - b);
+ return diff <= x;
+ }
+
@Override
protected void onSuccess(Job job) {
retiredJobs.add(job);
@@ -292,6 +312,7 @@
};
DebugGridmix client = new DebugGridmix();
final Configuration conf = mrCluster.createJobConf();
+ //conf.setInt(Gridmix.GRIDMIX_KEY_LEN, 2);
int res = ToolRunner.run(conf, client, argv);
assertEquals("Client exited with nonzero status", 0, res);
client.checkMonitor();
diff --git a/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestRecordFactory.java b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestRecordFactory.java
new file mode 100644
index 0000000..e6fde12
--- /dev/null
+++ b/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestRecordFactory.java
@@ -0,0 +1,79 @@
+
+/**
+ * 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.mapred.gridmix;
+
+import java.util.Random;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+public class TestRecordFactory {
+ private static final Log LOG = LogFactory.getLog(TestRecordFactory.class);
+
+ public static void testFactory(long targetBytes, long targetRecs)
+ throws Exception {
+ final Configuration conf = new Configuration();
+ final GridmixKey key = new GridmixKey();
+ final GridmixRecord val = new GridmixRecord();
+ LOG.info("Target bytes/records: " + targetBytes + "/" + targetRecs);
+ final RecordFactory f = new AvgRecordFactory(targetBytes, targetRecs, conf);
+ targetRecs = targetRecs <= 0 && targetBytes >= 0
+ ? Math.max(1,
+ targetBytes / conf.getInt("gridmix.missing.rec.size", 64 * 1024))
+ : targetRecs;
+
+ long records = 0L;
+ final DataOutputBuffer out = new DataOutputBuffer();
+ while (f.next(key, val)) {
+ ++records;
+ key.write(out);
+ val.write(out);
+ }
+ assertEquals(targetRecs, records);
+ assertEquals(targetBytes, out.getLength());
+ }
+
+ @Test
+ public void testRandom() throws Exception {
+ final Random r = new Random();
+ final long targetBytes = r.nextInt(1 << 20) + 3 * (1 << 14);
+ final long targetRecs = r.nextInt(1 << 14);
+ testFactory(targetBytes, targetRecs);
+ }
+
+ @Test
+ public void testAvg() throws Exception {
+ final Random r = new Random();
+ final long avgsize = r.nextInt(1 << 10) + 1;
+ final long targetRecs = r.nextInt(1 << 14);
+ testFactory(targetRecs * avgsize, targetRecs);
+ }
+
+ @Test
+ public void testZero() throws Exception {
+ final Random r = new Random();
+ final long targetBytes = r.nextInt(1 << 20);
+ testFactory(targetBytes, 0);
+ }
+}
diff --git a/src/contrib/index/ivy.xml b/src/contrib/index/ivy.xml
index 0216ac6..f43fb79 100644
--- a/src/contrib/index/ivy.xml
+++ b/src/contrib/index/ivy.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<ivy-module version="1.0">
<info organisation="org.apache.hadoop" module="${ant.project.name}">
<license name="Apache 2.0"/>
diff --git a/src/contrib/index/ivy/libraries.properties b/src/contrib/index/ivy/libraries.properties
index 3fd130b..841cce3 100644
--- a/src/contrib/index/ivy/libraries.properties
+++ b/src/contrib/index/ivy/libraries.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
#This properties file lists the versions of the various artifacts used by index.
#These are the versions of our dependencies (in alphabetical order)
diff --git a/src/contrib/mrunit/ivy.xml b/src/contrib/mrunit/ivy.xml
index 8fab8b2..82356bf 100644
--- a/src/contrib/mrunit/ivy.xml
+++ b/src/contrib/mrunit/ivy.xml
@@ -33,6 +33,7 @@
<conf name="common" visibility="private"
description="artifacts needed to compile/test the application"/>
+ <conf name="test" visibility="private" extends="runtime"/>
</configurations>
<publications>
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/LoadProbingEvent.java
similarity index 63%
copy from src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
copy to src/contrib/mumak/src/java/org/apache/hadoop/mapred/LoadProbingEvent.java
index ec03890..1a5d116 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
+++ b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/LoadProbingEvent.java
@@ -15,30 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
-package org.apache.hadoop.sqoop.util;
+package org.apache.hadoop.mapred;
/**
- * General error during import process.
- *
- *
+ * {@link LoadProbingEvent} is created by {@link SimulatorJobTracker} when the
+ * {@link SimulatorJobSubmissionPolicy} is STRESS. {@link SimulatorJobClient}
+ * picks up the event, and would check whether the system load is stressed. If
+ * not, it would submit the next job.
*/
-@SuppressWarnings("serial")
-public class ImportError extends Exception {
-
- public ImportError() {
- super("ImportError");
- }
-
- public ImportError(final String message) {
- super(message);
- }
-
- public ImportError(final Throwable cause) {
- super(cause);
- }
-
- public ImportError(final String message, final Throwable cause) {
- super(message, cause);
+public class LoadProbingEvent extends SimulatorEvent {
+ public LoadProbingEvent(SimulatorJobClient jc, long timestamp) {
+ super(jc, timestamp);
}
}
diff --git a/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorEngine.java b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorEngine.java
index 24f9896..74d3fe1 100644
--- a/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorEngine.java
+++ b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorEngine.java
@@ -20,7 +20,9 @@
import java.io.IOException;
import java.io.PrintStream;
import java.util.ArrayList;
+import java.util.Iterator;
import java.util.List;
+import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@@ -29,14 +31,15 @@
import org.apache.hadoop.mapred.SimulatorEvent;
import org.apache.hadoop.mapred.SimulatorEventQueue;
import org.apache.hadoop.mapred.JobCompleteEvent;
-import org.apache.hadoop.mapred.JobStatus;
import org.apache.hadoop.mapred.SimulatorJobClient;
import org.apache.hadoop.mapred.SimulatorJobTracker;
import org.apache.hadoop.mapred.SimulatorTaskTracker;
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.apache.hadoop.net.StaticMapping;
import org.apache.hadoop.tools.rumen.ClusterStory;
+import org.apache.hadoop.tools.rumen.ClusterTopologyReader;
import org.apache.hadoop.tools.rumen.JobStoryProducer;
+import org.apache.hadoop.tools.rumen.LoggedNetworkTopology;
import org.apache.hadoop.tools.rumen.MachineNode;
import org.apache.hadoop.tools.rumen.RackNode;
import org.apache.hadoop.tools.rumen.ZombieCluster;
@@ -78,8 +81,6 @@
for (MachineNode node : clusterStory.getMachines()) {
String hostname = node.getName();
- RackNode rackNode = node.getRackNode();
- StaticMapping.addNodeToRack(hostname, rackNode.getName());
String taskTrackerName = "tracker_" + hostname + ":localhost/127.0.0.1:"
+ port;
port++;
@@ -132,13 +133,23 @@
MachineNode defaultNode = new MachineNode.Builder("default", 2)
.setMapSlots(maxMaps).setReduceSlots(maxReduces).build();
- ZombieCluster cluster = new ZombieCluster(new Path(topologyFile),
- defaultNode, jobConf);
+
+ LoggedNetworkTopology topology = new ClusterTopologyReader(new Path(
+ topologyFile), jobConf).get();
+ // Setting the static mapping before removing numeric IP hosts.
+ setStaticMapping(topology);
+ if (getConf().getBoolean("mumak.topology.filter-numeric-ips", true)) {
+ removeIpHosts(topology);
+ }
+ ZombieCluster cluster = new ZombieCluster(topology, defaultNode);
long firstJobStartTime = now + 60000;
JobStoryProducer jobStoryProducer = new SimulatorJobStoryProducer(
new Path(traceFile), cluster, firstJobStartTime, jobConf);
- jc = new SimulatorJobClient(jt, jobStoryProducer);
+ final SimulatorJobSubmissionPolicy submissionPolicy = SimulatorJobSubmissionPolicy
+ .getPolicy(jobConf);
+
+ jc = new SimulatorJobClient(jt, jobStoryProducer, submissionPolicy);
queue.addAll(jc.init(firstJobStartTime));
// create TTs based on topology.json
@@ -236,4 +247,58 @@
long getCurrentTime() {
return currentTime;
}
+
+ // Due to HDFS-778, a node may appear in job history logs as both numeric
+ // ips and as host names. We remove them from the parsed network topology
+ // before feeding it to ZombieCluster.
+ static void removeIpHosts(LoggedNetworkTopology topology) {
+ for (Iterator<LoggedNetworkTopology> rackIt = topology.getChildren()
+ .iterator(); rackIt.hasNext();) {
+ LoggedNetworkTopology rack = rackIt.next();
+ List<LoggedNetworkTopology> nodes = rack.getChildren();
+ for (Iterator<LoggedNetworkTopology> it = nodes.iterator(); it.hasNext();) {
+ LoggedNetworkTopology node = it.next();
+ if (isIPAddress(node.getName())) {
+ it.remove();
+ }
+ }
+ if (nodes.isEmpty()) {
+ rackIt.remove();
+ }
+ }
+ }
+
+ static Pattern IP_PATTERN;
+
+ static {
+ // 0-255
+ String IPV4BK1 = "(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9]?)";
+ // .b.c.d - where b/c/d are 0-255, and optionally adding two more
+ // backslashes before each period
+ String IPV4BKN = "(?:\\\\?\\." + IPV4BK1 + "){3}";
+ String IPV4_PATTERN = IPV4BK1 + IPV4BKN;
+
+ // first hexadecimal number
+ String IPV6BK1 = "(?:[0-9a-fA-F]{1,4})";
+ // remaining 7 hexadecimal numbers, each preceded with ":".
+ String IPV6BKN = "(?::" + IPV6BK1 + "){7}";
+ String IPV6_PATTERN = IPV6BK1 + IPV6BKN;
+
+ IP_PATTERN = Pattern.compile(
+ "^(?:" + IPV4_PATTERN + "|" + IPV6_PATTERN + ")$");
+ }
+
+
+ static boolean isIPAddress(String hostname) {
+ return IP_PATTERN.matcher(hostname).matches();
+ }
+
+ static void setStaticMapping(LoggedNetworkTopology topology) {
+ for (LoggedNetworkTopology rack : topology.getChildren()) {
+ for (LoggedNetworkTopology node : rack.getChildren()) {
+ StaticMapping.addNodeToRack(node.getName(),
+ new RackNode(rack.getName(), 1).getName());
+ }
+ }
+ }
}
diff --git a/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobClient.java b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobClient.java
index 4902346..695575c 100644
--- a/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobClient.java
+++ b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobClient.java
@@ -18,11 +18,14 @@
package org.apache.hadoop.mapred;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Collections;
-import java.util.LinkedHashSet;
+import java.util.IdentityHashMap;
+import java.util.LinkedHashMap;
import java.util.List;
-import java.util.Set;
+import java.util.Map;
+import org.apache.hadoop.mapreduce.ClusterMetrics;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.JobStatus;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
@@ -33,14 +36,67 @@
* Class that simulates a job client. It's main functionality is to submit jobs
* to the simulation engine, and shutdown the simulation engine if the job
* producer runs out of jobs.
- * TODO: Change System.out.printXX to LOG.xxx.
*/
public class SimulatorJobClient implements SimulatorEventListener {
+ protected static class JobSketchInfo {
+ protected int numMaps;
+ protected int numReduces;
+ JobSketchInfo(int numMaps, int numReduces) {
+ this.numMaps = numMaps;
+ this.numReduces = numReduces;
+ }
+ }
+
private final ClientProtocol jobTracker;
private final JobStoryProducer jobStoryProducer;
- private Set<JobID> runningJobs = new LinkedHashSet<JobID>();
+ private final SimulatorJobSubmissionPolicy submissionPolicy;
+ private static final int LOAD_PROB_INTERVAL_START = 1000;
+ private static final int LOAD_PROB_INTERVAL_MAX = 320000;
+ private int loadProbingInterval = LOAD_PROB_INTERVAL_START;
+
+ /**
+ * The minimum ratio between pending+running map tasks (aka. incomplete map
+ * tasks) and cluster map slot capacity for us to consider the cluster is
+ * overloaded. For running maps, we only count them partially. Namely, a 40%
+ * completed map is counted as 0.6 map tasks in our calculation.
+ */
+ private static final float OVERLAOD_MAPTASK_MAPSLOT_RATIO=2.0f;
+ /**
+ * Keep track of the in-flight load-probing event.
+ */
+ private LoadProbingEvent inFlightLPE = null;
+ /**
+ * We do not have handle to the SimulatorEventQueue, and thus cannot cancel
+ * events directly. Instead, we keep an identity-map (should have been an
+ * identity-set except that JDK does not provide an identity-set) to skip
+ * events that are cancelled.
+ */
+ private Map<LoadProbingEvent, Boolean> cancelledLPE =
+ new IdentityHashMap<LoadProbingEvent, Boolean>();
+
+ private Map<JobID, JobSketchInfo> runningJobs =
+ new LinkedHashMap<JobID, JobSketchInfo>();
private boolean noMoreJobs = false;
-
+ private JobStory nextJob;
+
+ /**
+ * Constructor.
+ *
+ * @param jobTracker
+ * The job tracker where we submit job to. Note that the {@link
+ * SimulatorJobClient} interacts with the JobTracker through the
+ * {@link ClientProtocol}.
+ * @param jobStoryProducer
+ * @param submissionPolicy How should we submit jobs to the JobTracker?
+ */
+ public SimulatorJobClient(ClientProtocol jobTracker,
+ JobStoryProducer jobStoryProducer,
+ SimulatorJobSubmissionPolicy submissionPolicy) {
+ this.jobTracker = jobTracker;
+ this.jobStoryProducer = jobStoryProducer;
+ this.submissionPolicy = submissionPolicy;
+ }
+
/**
* Constructor.
*
@@ -50,64 +106,237 @@
* {@link ClientProtocol}.
* @param jobStoryProducer
*/
- public SimulatorJobClient(ClientProtocol jobTracker, JobStoryProducer jobStoryProducer) {
- this.jobTracker = jobTracker;
- this.jobStoryProducer = jobStoryProducer;
+ public SimulatorJobClient(ClientProtocol jobTracker,
+ JobStoryProducer jobStoryProducer) {
+ this(jobTracker, jobStoryProducer, SimulatorJobSubmissionPolicy.REPLAY);
}
-
+
@Override
public List<SimulatorEvent> init(long when) throws IOException {
JobStory job = jobStoryProducer.getNextJob();
- if (job.getSubmissionTime() != when) {
+ if (submissionPolicy == SimulatorJobSubmissionPolicy.REPLAY
+ && job.getSubmissionTime() != when) {
throw new IOException("Inconsistent submission time for the first job: "
+ when + " != " + job.getSubmissionTime()+".");
}
+
JobSubmissionEvent event = new JobSubmissionEvent(this, when, job);
- return Collections.<SimulatorEvent> singletonList(event);
+ if (submissionPolicy != SimulatorJobSubmissionPolicy.STRESS) {
+ return Collections.<SimulatorEvent> singletonList(event);
+ } else {
+ ArrayList<SimulatorEvent> ret = new ArrayList<SimulatorEvent>(2);
+ ret.add(event);
+ inFlightLPE = new LoadProbingEvent(this, when + loadProbingInterval);
+ ret.add(inFlightLPE);
+ return ret;
+ }
+ }
+
+ /**
+ * Doing exponential back-off probing because load probing could be pretty
+ * expensive if we have many pending jobs.
+ *
+ * @param overloaded Is the job tracker currently overloaded?
+ */
+ private void adjustLoadProbingInterval(boolean overloaded) {
+ if (overloaded) {
+ /**
+ * We should only extend LPE interval when there is no in-flight LPE.
+ */
+ if (inFlightLPE == null) {
+ loadProbingInterval = Math.min(loadProbingInterval * 2,
+ LOAD_PROB_INTERVAL_MAX);
+ }
+ } else {
+ loadProbingInterval = LOAD_PROB_INTERVAL_START;
+ }
}
- @Override
- public List<SimulatorEvent> accept(SimulatorEvent event)
- throws IOException {
- if (event instanceof JobSubmissionEvent) {
- JobSubmissionEvent submitEvent = (JobSubmissionEvent)(event);
-
- // Submit job
- JobStatus status = null;
- try {
- status = submitJob(submitEvent.getJob());
- } catch (InterruptedException e) {
- throw new IOException(e);
- }
- runningJobs.add(status.getJobID());
- System.out.println("Job " + status.getJobID() +
- " is submitted at " + submitEvent.getTimeStamp());
+ /**
+ * We try to use some light-weight mechanism to determine cluster load.
+ * @return Whether, from job client perspective, the cluster is overloaded.
+ */
+ private boolean isOverloaded(long now) throws IOException {
+ try {
+ ClusterMetrics clusterMetrics = jobTracker.getClusterMetrics();
- JobStory nextJob = jobStoryProducer.getNextJob();
- if (nextJob == null) {
- noMoreJobs = true;
- return SimulatorEngine.EMPTY_EVENTS;
+ // If there are more jobs than number of task trackers, we assume the
+ // cluster is overloaded. This is to bound the memory usage of the
+ // simulator job tracker, in situations where we have jobs with small
+ // number of map tasks and large number of reduce tasks.
+ if (runningJobs.size() >= clusterMetrics.getTaskTrackerCount()) {
+ System.out.printf("%d Overloaded is %s: " +
+ "#runningJobs >= taskTrackerCount (%d >= %d)\n",
+ now, Boolean.TRUE.toString(),
+ runningJobs.size(), clusterMetrics.getTaskTrackerCount());
+ return true;
}
-
- return Collections.<SimulatorEvent>singletonList(
- new JobSubmissionEvent(this, nextJob.getSubmissionTime(), nextJob));
- } else if (event instanceof JobCompleteEvent) {
- JobCompleteEvent jobCompleteEvent = (JobCompleteEvent)event;
- JobStatus jobStatus = jobCompleteEvent.getJobStatus();
- System.out.println("Job " + jobStatus.getJobID() +
- " completed at " + jobCompleteEvent.getTimeStamp() +
- " with status: " + jobStatus.getState() +
- " runtime: " +
- (jobCompleteEvent.getTimeStamp() - jobStatus.getStartTime()));
- runningJobs.remove(jobCompleteEvent.getJobStatus().getJobID());
- if (noMoreJobs && runningJobs.isEmpty()) {
- jobCompleteEvent.getEngine().shutdown();
+
+ float incompleteMapTasks = 0; // include pending & running map tasks.
+ for (Map.Entry<JobID, JobSketchInfo> entry : runningJobs.entrySet()) {
+ org.apache.hadoop.mapreduce.JobStatus jobStatus = jobTracker
+ .getJobStatus(entry.getKey());
+ incompleteMapTasks += (1 - Math.min(jobStatus.getMapProgress(), 1.0))
+ * entry.getValue().numMaps;
}
- return SimulatorEngine.EMPTY_EVENTS;
- } else {
- throw new IllegalArgumentException("unknown event type: " + event.getClass());
+
+ boolean overloaded = incompleteMapTasks >
+ OVERLAOD_MAPTASK_MAPSLOT_RATIO * clusterMetrics.getMapSlotCapacity();
+ String relOp = (overloaded) ? ">" : "<=";
+ System.out.printf("%d Overloaded is %s: "
+ + "incompleteMapTasks %s %.1f*mapSlotCapacity (%.1f %s %.1f*%d)\n",
+ now, Boolean.toString(overloaded), relOp, OVERLAOD_MAPTASK_MAPSLOT_RATIO,
+ incompleteMapTasks, relOp, OVERLAOD_MAPTASK_MAPSLOT_RATIO,
+ clusterMetrics.getMapSlotCapacity());
+ return overloaded;
+ } catch (InterruptedException e) {
+ throw new IOException("InterruptedException", e);
}
}
+
+ /**
+ * Handles a simulation event that is either JobSubmissionEvent or
+ * JobCompletionEvent.
+ *
+ * @param event SimulatorEvent to respond to
+ * @return list of events generated in response
+ */
+ @Override
+ public List<SimulatorEvent> accept(SimulatorEvent event) throws IOException {
+ if (event instanceof JobSubmissionEvent) {
+ return processJobSubmissionEvent((JobSubmissionEvent) event);
+ } else if (event instanceof JobCompleteEvent) {
+ return processJobCompleteEvent((JobCompleteEvent) event);
+ } else if (event instanceof LoadProbingEvent) {
+ return processLoadProbingEvent((LoadProbingEvent) event);
+ } else {
+ throw new IllegalArgumentException("unknown event type: "
+ + event.getClass());
+ }
+ }
+
+ /**
+ * Responds to a job submission event by submitting the job to the
+ * job tracker. If serializeJobSubmissions is true, it postpones the
+ * submission until after the previous job finished instead.
+ *
+ * @param submitEvent the submission event to respond to
+ */
+ private List<SimulatorEvent> processJobSubmissionEvent(
+ JobSubmissionEvent submitEvent) throws IOException {
+ // Submit job
+ JobStatus status = null;
+ JobStory story = submitEvent.getJob();
+ try {
+ status = submitJob(story);
+ } catch (InterruptedException e) {
+ throw new IOException(e);
+ }
+ runningJobs.put(status.getJobID(), new JobSketchInfo(story.getNumberMaps(),
+ story.getNumberReduces()));
+ System.out.println("Job " + status.getJobID() + " is submitted at "
+ + submitEvent.getTimeStamp());
+
+ // Find the next job to submit
+ nextJob = jobStoryProducer.getNextJob();
+ if (nextJob == null) {
+ noMoreJobs = true;
+ return SimulatorEngine.EMPTY_EVENTS;
+ } else if (submissionPolicy == SimulatorJobSubmissionPolicy.REPLAY) {
+ // enqueue next submission event
+ return Collections
+ .<SimulatorEvent> singletonList(new JobSubmissionEvent(this, nextJob
+ .getSubmissionTime(), nextJob));
+ } else if (submissionPolicy == SimulatorJobSubmissionPolicy.STRESS) {
+ return checkLoadAndSubmitJob(submitEvent.getTimeStamp());
+ }
+
+ return SimulatorEngine.EMPTY_EVENTS;
+ }
+
+ /**
+ * Handles a job completion event.
+ *
+ * @param jobCompleteEvent the submission event to respond to
+ * @throws IOException
+ */
+ private List<SimulatorEvent> processJobCompleteEvent(
+ JobCompleteEvent jobCompleteEvent) throws IOException {
+ JobStatus jobStatus = jobCompleteEvent.getJobStatus();
+ System.out.println("Job " + jobStatus.getJobID() + " completed at "
+ + jobCompleteEvent.getTimeStamp() + " with status: "
+ + jobStatus.getState() + " runtime: "
+ + (jobCompleteEvent.getTimeStamp() - jobStatus.getStartTime()));
+ runningJobs.remove(jobCompleteEvent.getJobStatus().getJobID());
+ if (noMoreJobs && runningJobs.isEmpty()) {
+ jobCompleteEvent.getEngine().shutdown();
+ }
+
+ if (!noMoreJobs) {
+ if (submissionPolicy == SimulatorJobSubmissionPolicy.SERIAL) {
+ long submissionTime = jobCompleteEvent.getTimeStamp() + 1;
+ JobStory story = new SimulatorJobStory(nextJob, submissionTime);
+ return Collections
+ .<SimulatorEvent> singletonList(new JobSubmissionEvent(this,
+ submissionTime, story));
+ } else if (submissionPolicy == SimulatorJobSubmissionPolicy.STRESS) {
+ return checkLoadAndSubmitJob(jobCompleteEvent.getTimeStamp());
+ }
+ }
+ return SimulatorEngine.EMPTY_EVENTS;
+ }
+
+ /**
+ * Check whether job tracker is overloaded. If not, submit the next job.
+ * Pre-condition: noMoreJobs == false
+ * @return A list of {@link SimulatorEvent}'s as the follow-up actions.
+ */
+ private List<SimulatorEvent> checkLoadAndSubmitJob(long now) throws IOException {
+ List<SimulatorEvent> ret = new ArrayList<SimulatorEvent>(2);
+ boolean overloaded = isOverloaded(now);
+ adjustLoadProbingInterval(overloaded);
+
+ if (inFlightLPE != null && (inFlightLPE.getTimeStamp()>now+loadProbingInterval)) {
+ cancelledLPE.put(inFlightLPE, Boolean.TRUE);
+ inFlightLPE = null;
+ }
+
+ if (inFlightLPE == null) {
+ inFlightLPE = new LoadProbingEvent(this, now + loadProbingInterval);
+ ret.add(inFlightLPE);
+ }
+
+ if (!overloaded) {
+ long submissionTime = now + 1;
+ JobStory story = new SimulatorJobStory(nextJob, submissionTime);
+ ret.add(new JobSubmissionEvent(this, submissionTime, story));
+ }
+
+ return ret;
+ }
+
+ /**
+ * Handles a load probing event. If cluster is not overloaded, submit a new job.
+ *
+ * @param loadProbingEvent the load probing event
+ */
+ private List<SimulatorEvent> processLoadProbingEvent(
+ LoadProbingEvent loadProbingEvent) throws IOException {
+ if (cancelledLPE.containsKey(loadProbingEvent)) {
+ cancelledLPE.remove(loadProbingEvent);
+ return SimulatorEngine.EMPTY_EVENTS;
+ }
+
+ assert(loadProbingEvent == inFlightLPE);
+
+ inFlightLPE = null;
+
+ if (noMoreJobs) {
+ return SimulatorEngine.EMPTY_EVENTS;
+ }
+
+ return checkLoadAndSubmitJob(loadProbingEvent.getTimeStamp());
+ }
@SuppressWarnings("deprecation")
private JobStatus submitJob(JobStory job)
@@ -120,6 +349,6 @@
}
SimulatorJobCache.put(org.apache.hadoop.mapred.JobID.downgrade(jobId), job);
- return jobTracker.submitJob(jobId);
+ return jobTracker.submitJob(jobId, "dummy-path", null);
}
}
diff --git a/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobInProgress.java b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobInProgress.java
index 1dabf27..6c35362 100644
--- a/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobInProgress.java
+++ b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobInProgress.java
@@ -28,11 +28,11 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job.RawSplit;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.TaskID;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
import org.apache.hadoop.metrics.MetricsContext;
import org.apache.hadoop.metrics.MetricsUtil;
import org.apache.hadoop.net.Node;
@@ -48,10 +48,10 @@
// cache
private final JobStory jobStory;
- RawSplit[] splits;
+ TaskSplitMetaInfo[] taskSplitMetaInfo;
@SuppressWarnings("deprecation")
- public SimulatorJobInProgress(JobID jobid, JobTracker jobtracker,
+ public SimulatorJobInProgress(JobID jobid, String jobSubmitDir, JobTracker jobtracker,
JobConf default_conf, JobStory jobStory) {
super();
// jobSetupCleanupNeeded set to false in parent cstr, though
@@ -63,7 +63,7 @@
this.jobtracker = jobtracker;
this.conf = jobStory.getJobConf();
this.priority = conf.getJobPriority();
- Path jobDir = jobtracker.getSystemDirectoryForJob(jobid);
+ Path jobDir = new Path(jobSubmitDir);
this.jobFile = new Path(jobDir, "job.xml");
this.status = new JobStatus(jobid, 0.0f, 0.0f, 0.0f, 0.0f, JobStatus.PREP,
priority, conf.getUser(), conf.getJobName(), jobFile.toString(), url);
@@ -127,16 +127,17 @@
}
final String jobFile = "default";
- splits = getRawSplits(jobStory.getInputSplits());
+ taskSplitMetaInfo = createSplits(jobStory);
if (loggingEnabled) {
LOG.debug("(initTasks@SJIP) Created splits for job = " + jobId
- + " number of splits = " + splits.length);
+ + " number of splits = " + taskSplitMetaInfo.length);
}
- createMapTasks(jobFile, splits);
+ createMapTasks(jobFile, taskSplitMetaInfo);
if (numMapTasks > 0) {
- nonRunningMapCache = createCache(splits, maxLevel);
+ nonRunningMapCache = createCache(taskSplitMetaInfo,
+ maxLevel);
if (loggingEnabled) {
LOG.debug("initTasks:numMaps=" + numMapTasks
+ " Size of nonRunningMapCache=" + nonRunningMapCache.size()
@@ -167,25 +168,25 @@
}
}
- RawSplit[] getRawSplits(InputSplit[] splits) throws IOException {
+
+ TaskSplitMetaInfo[] createSplits(JobStory story) throws IOException {
+ InputSplit[] splits = story.getInputSplits();
if (splits == null || splits.length != numMapTasks) {
throw new IllegalArgumentException("Input split size mismatch: expected="
+ numMapTasks + ", actual=" + ((splits == null) ? -1 : splits.length));
}
- RawSplit rawSplits[] = new RawSplit[splits.length];
- for (int i = 0; i < splits.length; i++) {
+ TaskSplitMetaInfo[] splitMetaInfo =
+ new TaskSplitMetaInfo[story.getNumberMaps()];
+ int i = 0;
+ for (InputSplit split : splits) {
try {
- rawSplits[i] = new RawSplit();
- rawSplits[i].setClassName(splits[i].getClass().getName());
- rawSplits[i].setDataLength(splits[i].getLength());
- rawSplits[i].setLocations(splits[i].getLocations());
+ splitMetaInfo[i++] = new TaskSplitMetaInfo(split,0);
} catch (InterruptedException ie) {
throw new IOException(ie);
}
}
-
- return rawSplits;
+ return splitMetaInfo;
}
/**
@@ -208,7 +209,8 @@
assert (jobid == getJobID());
// Get splits for the TaskAttempt
- RawSplit split = splits[taskAttemptID.getTaskID().getId()];
+ TaskSplitMetaInfo split =
+ taskSplitMetaInfo[taskAttemptID.getTaskID().getId()];
int locality = getClosestLocality(taskTracker, split);
TaskID taskId = taskAttemptID.getTaskID();
@@ -232,7 +234,7 @@
return taskAttemptInfo;
}
- private int getClosestLocality(TaskTracker taskTracker, RawSplit split) {
+ private int getClosestLocality(TaskTracker taskTracker, TaskSplitMetaInfo split) {
int locality = 2;
Node taskTrackerNode = jobtracker
diff --git a/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobSubmissionPolicy.java b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobSubmissionPolicy.java
new file mode 100644
index 0000000..f39c245
--- /dev/null
+++ b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobSubmissionPolicy.java
@@ -0,0 +1,50 @@
+/**
+ * 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.mapred;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Job submission policies. The set of policies is closed and encapsulated in
+ * {@link SimulatorJobSubmissionPolicy}. The handling of submission policies is
+ * embedded in the {@link SimulatorEngine} (through various events).
+ *
+ */
+public enum SimulatorJobSubmissionPolicy {
+ /**
+ * replay the trace by following the job inter-arrival rate faithfully.
+ */
+ REPLAY,
+
+ /**
+ * ignore submission time, keep submitting jobs until the cluster is saturated.
+ */
+ STRESS,
+
+ /**
+ * submitting jobs sequentially.
+ */
+ SERIAL;
+
+ public static final String JOB_SUBMISSION_POLICY = "mumak.job-submission.policy";
+
+ static public SimulatorJobSubmissionPolicy getPolicy(Configuration conf) {
+ String policy = conf.get(JOB_SUBMISSION_POLICY, REPLAY.name());
+ return valueOf(policy.toUpperCase());
+ }
+}
diff --git a/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobTracker.java b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobTracker.java
index f3ac708..9504df7 100644
--- a/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobTracker.java
+++ b/src/contrib/mumak/src/java/org/apache/hadoop/mapred/SimulatorJobTracker.java
@@ -35,10 +35,11 @@
import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
import org.apache.hadoop.mapred.SimulatorJobInProgress;
import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.mapreduce.security.TokenStorage;
/**
* {@link SimulatorJobTracker} extends {@link JobTracker}. It implements the
- * {@link JobSubmissionProtocol} and the {@link InterTrackerProtocol} protocols.
+ * {@link InterTrackerProtocol} protocols.
*/
@SuppressWarnings("deprecation")
public class SimulatorJobTracker extends JobTracker {
@@ -173,7 +174,9 @@
}
@Override
- public synchronized JobStatus submitJob(JobID jobId) throws IOException {
+ public synchronized JobStatus submitJob(
+ JobID jobId, String jobSubmitDir, TokenStorage ts)
+ throws IOException {
boolean loggingEnabled = LOG.isDebugEnabled();
if (loggingEnabled) {
LOG.debug("submitJob for jobname = " + jobId);
@@ -191,7 +194,7 @@
}
validateAndSetClock(jobStory.getSubmissionTime());
- SimulatorJobInProgress job = new SimulatorJobInProgress(jobId, this,
+ SimulatorJobInProgress job = new SimulatorJobInProgress(jobId, jobSubmitDir, this,
this.conf,
jobStory);
return addJob(jobId, job);
diff --git a/src/contrib/mumak/src/test/data/topo-with-numeric-ips.json b/src/contrib/mumak/src/test/data/topo-with-numeric-ips.json
new file mode 100644
index 0000000..c0a05a6
--- /dev/null
+++ b/src/contrib/mumak/src/test/data/topo-with-numeric-ips.json
@@ -0,0 +1,22 @@
+{
+ "name" : "<root>",
+ "children" : [ {
+ "name" : "194\\.6\\.129\\.64",
+ "children" : [ {
+ "name" : "node1817\\.megatron\\.com",
+ "children" : null
+ }, {
+ "name" : "194\\.6\\.129\\.67",
+ "children" : null
+ } ]
+ }, {
+ "name" : "192\\.30\\.63\\.64",
+ "children" : [ {
+ "name" : "192\\.30\\.63\\.69",
+ "children" : null
+ }, {
+ "name" : "192\\.30\\.63\\.81",
+ "children" : null
+ } ]
+ } ]
+}
diff --git a/src/contrib/mumak/src/test/data/topo-without-numeric-ips.json b/src/contrib/mumak/src/test/data/topo-without-numeric-ips.json
new file mode 100644
index 0000000..096b449
--- /dev/null
+++ b/src/contrib/mumak/src/test/data/topo-without-numeric-ips.json
@@ -0,0 +1,10 @@
+{
+ "name" : "<root>",
+ "children" : [ {
+ "name" : "194\\.6\\.129\\.64",
+ "children" : [ {
+ "name" : "node1817\\.megatron\\.com",
+ "children" : null
+ } ]
+ } ]
+}
diff --git a/src/contrib/mumak/src/test/org/apache/hadoop/mapred/MockSimulatorJobTracker.java b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/MockSimulatorJobTracker.java
index 1cb546d..d15bcf6 100644
--- a/src/contrib/mumak/src/test/org/apache/hadoop/mapred/MockSimulatorJobTracker.java
+++ b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/MockSimulatorJobTracker.java
@@ -35,6 +35,7 @@
import org.apache.hadoop.mapreduce.JobStatus;
import org.apache.hadoop.mapreduce.QueueAclsInfo;
import org.apache.hadoop.mapreduce.QueueInfo;
+import org.apache.hadoop.mapreduce.security.TokenStorage;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.TaskReport;
import org.apache.hadoop.mapreduce.TaskTrackerInfo;
@@ -43,6 +44,7 @@
import org.apache.hadoop.tools.rumen.TaskInfo;
import org.apache.hadoop.tools.rumen.MapTaskAttemptInfo;
import org.apache.hadoop.tools.rumen.ReduceTaskAttemptInfo;
+import org.apache.hadoop.mapreduce.split.JobSplit.*;
//
// Mock jobtracker class that check heartbeat() in parameters and
// sends responses based on a prepopulated table
@@ -76,7 +78,8 @@
}
@Override
- public JobStatus submitJob(JobID jobId) throws IOException {
+ public JobStatus submitJob(
+ JobID jobId, String jobSubmitDir, TokenStorage ts) throws IOException {
JobStatus status = new JobStatus(jobId, 0.0f, 0.0f, 0.0f, 0.0f,
JobStatus.State.RUNNING, JobPriority.NORMAL, "", "", "", "");
return status;
@@ -172,8 +175,8 @@
final int numSlotsRequired = 1;
org.apache.hadoop.mapred.TaskAttemptID taskIdOldApi =
org.apache.hadoop.mapred.TaskAttemptID.downgrade(taskId);
- Task task = new MapTask("dummyjobfile", taskIdOldApi, 0, "dummysplitclass",
- null, numSlotsRequired);
+ Task task = new MapTask("dummyjobfile", taskIdOldApi, 0, new TaskSplitIndex(),
+ numSlotsRequired);
// all byte counters are 0
TaskInfo taskInfo = new TaskInfo(0, 0, 0, 0, 0);
MapTaskAttemptInfo taskAttemptInfo =
@@ -302,6 +305,11 @@
public String getSystemDir() {
throw new UnsupportedOperationException();
}
+
+ @Override
+ public String getStagingAreaDir() {
+ throw new UnsupportedOperationException();
+ }
@Override
public String getBuildVersion() {
diff --git a/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestRemoveIpsFromLoggedNetworkTopology.java b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestRemoveIpsFromLoggedNetworkTopology.java
new file mode 100644
index 0000000..1b5ef74
--- /dev/null
+++ b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestRemoveIpsFromLoggedNetworkTopology.java
@@ -0,0 +1,97 @@
+/**
+ * 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.mapred;
+
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.tools.rumen.ClusterTopologyReader;
+import org.apache.hadoop.tools.rumen.DeepInequalityException;
+import org.apache.hadoop.tools.rumen.LoggedNetworkTopology;
+import org.apache.hadoop.tools.rumen.TreePath;
+import org.junit.Test;
+
+public class TestRemoveIpsFromLoggedNetworkTopology {
+
+
+ @Test
+ public void testIsIPAddress() {
+ final String[] positives = {
+ "123.13.42.255", // regular ipv4
+ "123.01.0.255", // padded 0
+ "000.001.002.020", // more padded 0
+ "123\\.13\\.42\\.255", // escaped .
+ "0.0.0.0", // all-zero
+ "255.255.255.255", // all-0xff
+
+ "1080:0:0:0:8:800:200C:417A", // regular ipv6
+ "1080:01:020:3:8:0800:200C:417A", // padded 0
+ "1080:01:002:0003:080:0800:0200:417A", // more padded 0
+ "0:0:0:0:0:0:0:0", // all-zero
+ "ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff", // all-0xff
+ };
+
+ final String[] negatives = {
+ "node.megatron.com", // domain name
+ "13.42.255", // too short
+ "123.13.42.255.10", // too long
+ "123.256.42.255", // too large
+ "123.13.42.255.weird.com", // weird
+ "1080:0:0:0:8:200C:417A", // too short
+ "1080:0:0:0:1:8:800:200C:417A", // too long
+ "1080A:0:0:0:8:800:200C:417A", // too large
+ "1080:0:0:0:8:800:200G:417A", // too large
+ };
+
+ for (String s : positives) {
+ Assert.assertTrue(s, SimulatorEngine.isIPAddress(s));
+ }
+
+ for (String s : negatives) {
+ Assert.assertFalse(s, SimulatorEngine.isIPAddress(s));
+ }
+ }
+
+ @Test
+ public void testIpRemoval() throws IOException {
+ final Configuration conf = new Configuration();
+ final FileSystem lfs = FileSystem.getLocal(conf);
+ final Path rootInputDir = new Path(System.getProperty("src.test.data",
+ "data")).makeQualified(lfs.getUri(), lfs.getWorkingDirectory());
+
+ final LoggedNetworkTopology topoWithIps = new ClusterTopologyReader(new Path(
+ rootInputDir, "topo-with-numeric-ips.json"), conf).get();
+ final LoggedNetworkTopology topoWithoutIps = new ClusterTopologyReader(new Path(
+ rootInputDir, "topo-without-numeric-ips.json"), conf).get();
+ try {
+ topoWithIps.deepCompare(topoWithoutIps, new TreePath(null, "<root>"));
+ Assert.fail("Expecting two topologies to differ");
+ } catch (DeepInequalityException e) {
+ }
+ SimulatorEngine.removeIpHosts(topoWithIps);
+ try {
+ topoWithIps.deepCompare(topoWithoutIps, new TreePath(null, "<root>"));
+ } catch (DeepInequalityException e) {
+ Assert.fail("Expecting two topologies to be equal");
+ }
+ }
+}
diff --git a/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorEndToEnd.java b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorEndToEnd.java
index 31cb895..d6cfb51 100644
--- a/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorEndToEnd.java
+++ b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorEndToEnd.java
@@ -36,10 +36,12 @@
public class TestSimulatorEndToEnd {
public static final Log LOG = LogFactory.getLog(MockSimulatorEngine.class);
+ protected SimulatorJobSubmissionPolicy policy = SimulatorJobSubmissionPolicy.REPLAY;
@Test
public void testMain() throws Exception {
final Configuration conf = new Configuration();
+ conf.set(SimulatorJobSubmissionPolicy.JOB_SUBMISSION_POLICY, policy.name());
final FileSystem lfs = FileSystem.getLocal(conf);
final Path rootInputDir = new Path(
System.getProperty("src.test.data", "data")).makeQualified(lfs);
@@ -55,7 +57,7 @@
MockSimulatorEngine mockMumak = new MockSimulatorEngine(numJobs, nTrackers);
String[] args = { traceFile.toString(), topologyFile.toString() };
- int res = ToolRunner.run(new Configuration(), mockMumak, args);
+ int res = ToolRunner.run(conf, mockMumak, args);
Assert.assertEquals(res, 0);
}
diff --git a/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorJobClient.java b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorJobClient.java
index 0dfaad7..e58c38f 100644
--- a/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorJobClient.java
+++ b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorJobClient.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import java.util.Random;
import org.junit.Assert;
import org.junit.Test;
@@ -119,6 +120,7 @@
private long[] times;
private int index = 0;
private List<MockJobStory> jobs = new ArrayList<MockJobStory>();
+ private Random random = new Random();
public MockJobStoryProducer(long[] times, long relativeStartTime) {
super();
@@ -127,7 +129,7 @@
index = 0;
for (long time: times) {
- jobs.add(new MockJobStory(time - relativeStartTime));
+ jobs.add(new MockJobStory(random, time - relativeStartTime));
}
}
@@ -149,9 +151,11 @@
}
static class MockJobStory implements JobStory {
+ private Random random;
private long submissionTime;
- public MockJobStory(long submissionTime) {
+ public MockJobStory(Random random, long submissionTime) {
+ this.random = random;
this.submissionTime = submissionTime;
}
@@ -183,12 +187,12 @@
@Override
public int getNumberMaps() {
- throw new UnsupportedOperationException();
+ return random.nextInt(10)+1;
}
@Override
public int getNumberReduces() {
- throw new UnsupportedOperationException();
+ return random.nextInt(5);
}
@Override
diff --git a/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorJobTracker.java b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorJobTracker.java
index 26784ff..f1d64e8 100644
--- a/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorJobTracker.java
+++ b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorJobTracker.java
@@ -79,7 +79,7 @@
FakeJobs job = new FakeJobs("job1", 0, numMaps, numReduces);
SimulatorJobCache.put(org.apache.hadoop.mapred.JobID.downgrade(jobId), job);
- jobTracker.submitJob(jobId);
+ jobTracker.submitJob(jobId, "dummy-path", null);
}
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorSerialJobSubmission.java
similarity index 63%
copy from src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
copy to src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorSerialJobSubmission.java
index ec03890..395b934 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
+++ b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorSerialJobSubmission.java
@@ -15,30 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+package org.apache.hadoop.mapred;
-package org.apache.hadoop.sqoop.util;
-
-/**
- * General error during import process.
- *
- *
- */
-@SuppressWarnings("serial")
-public class ImportError extends Exception {
-
- public ImportError() {
- super("ImportError");
- }
-
- public ImportError(final String message) {
- super(message);
- }
-
- public ImportError(final Throwable cause) {
- super(cause);
- }
-
- public ImportError(final String message, final Throwable cause) {
- super(message, cause);
+public class TestSimulatorSerialJobSubmission extends TestSimulatorEndToEnd {
+ public TestSimulatorSerialJobSubmission() {
+ super();
+ policy = SimulatorJobSubmissionPolicy.SERIAL;
}
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorStressJobSubmission.java
similarity index 63%
copy from src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
copy to src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorStressJobSubmission.java
index ec03890..57e5774 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
+++ b/src/contrib/mumak/src/test/org/apache/hadoop/mapred/TestSimulatorStressJobSubmission.java
@@ -15,30 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+package org.apache.hadoop.mapred;
-package org.apache.hadoop.sqoop.util;
-
-/**
- * General error during import process.
- *
- *
- */
-@SuppressWarnings("serial")
-public class ImportError extends Exception {
-
- public ImportError() {
- super("ImportError");
- }
-
- public ImportError(final String message) {
- super(message);
- }
-
- public ImportError(final Throwable cause) {
- super(cause);
- }
-
- public ImportError(final String message, final Throwable cause) {
- super(message, cause);
+public class TestSimulatorStressJobSubmission extends TestSimulatorEndToEnd {
+ public TestSimulatorStressJobSubmission() {
+ super();
+ policy = SimulatorJobSubmissionPolicy.STRESS;
}
}
diff --git a/src/contrib/raid/README b/src/contrib/raid/README
new file mode 100644
index 0000000..1fb53e1
--- /dev/null
+++ b/src/contrib/raid/README
@@ -0,0 +1,176 @@
+# Copyright 2008 The Apache Software Foundation Licensed under the
+# Apache License, Version 2.0 (the "License"); you may not use this
+# file except in compliance with the License. You may obtain a copy
+# of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless
+# required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+# implied. See the License for the specific language governing
+# permissions and limitations under the License.
+
+This package implements a Distributed Raid File System. It is used alongwith
+an instance of the Hadoop Distributed File System (HDFS). It can be used to
+provide better protection against data corruption. It can also be used to
+reduce the total storage requirements of HDFS.
+
+Distributed Raid File System consists of two main software components. The first component
+is the RaidNode, a daemon that creates parity files from specified HDFS files.
+The second component "raidfs" is a software that is layered over a HDFS client and it
+intercepts all calls that an application makes to the HDFS client. If HDFS encounters
+corrupted data while reading a file, the raidfs client detects it; it uses the
+relevant parity blocks to recover the corrupted data (if possible) and returns
+the data to the application. The application is completely transparent to the
+fact that parity data was used to satisfy it's read request.
+
+The primary use of this feature is to save disk space for HDFS files.
+HDFS typically stores data in triplicate.
+The Distributed Raid File System can be configured in such a way that a set of
+data blocks of a file are combined together to form one or more parity blocks.
+This allows one to reduce the replication factor of a HDFS file from 3 to 2
+while keeping the failure probabilty relatively same as before. This typically
+results in saving 25% to 30% of storage space in a HDFS cluster.
+
+--------------------------------------------------------------------------------
+
+BUILDING:
+
+In HADOOP_HOME, run ant package to build Hadoop and its contrib packages.
+
+--------------------------------------------------------------------------------
+
+INSTALLING and CONFIGURING:
+
+The entire code is packaged in the form of a single jar file hadoop-*-raid.jar.
+To use HDFS Raid, you need to put the above mentioned jar file on
+the CLASSPATH. The easiest way is to copy the hadoop-*-raid.jar
+from HADOOP_HOME/build/contrib/raid to HADOOP_HOME/lib. Alternatively
+you can modify HADOOP_CLASSPATH to include this jar, in conf/hadoop-env.sh.
+
+There is a single configuration file named raid.xml that describes the HDFS
+path(s) that you want to raid. A sample of this file can be found in
+sc/contrib/raid/conf/raid.xml. Please edit the entries in this file to list the
+path(s) that you want to raid. Then, edit the hdfs-site.xml file for
+your installation to include a reference to this raid.xml. You can add the
+following to your hdfs-site.xml
+ <property>
+ <name>raid.config.file</name>
+ <value>/mnt/hdfs/DFS/conf/raid.xml</value>
+ <description>This is needed by the RaidNode </description>
+ </property>
+
+Please add an entry to your hdfs-site.xml to enable hdfs clients to use the
+parity bits to recover corrupted data.
+
+ <property>
+ <name>fs.hdfs.impl</name>
+ <value>org.apache.hadoop.dfs.DistributedRaidFileSystem</value>
+ <description>The FileSystem for hdfs: uris.</description>
+ </property>
+
+
+--------------------------------------------------------------------------------
+
+OPTIONAL CONFIGIURATION:
+
+The following properties can be set in hdfs-site.xml to further tune you configuration:
+
+ Specifies the location where parity files are located.
+ <property>
+ <name>hdfs.raid.locations</name>
+ <value>hdfs://newdfs.data:8000/raid</value>
+ <description>The location for parity files. If this is
+ is not defined, then defaults to /raid.
+ </descrition>
+ </property>
+
+ Specify the parity stripe length
+ <property>
+ <name>hdfs.raid.stripeLength</name>
+ <value>10</value>
+ <description>The number of blocks in a file to be combined into
+ a single raid parity block. The default value is 5. The lower
+ the number the greater is the disk space you will save when you
+ enable raid.
+ </description>
+ </property>
+
+ Specify RaidNode to not use a map-reduce cluster for raiding files in parallel.
+ <property>
+ <name>fs.raidnode.local</name>
+ <value>true</value>
+ <description>If you do not want to use your map-reduce cluster to
+ raid files in parallel, then specify "true". By default, this
+ value is false, which means that the RaidNode uses the default
+ map-reduce cluster to generate parity blocks.
+ </description>
+ </property>
+
+
+ Specify the periodicy at which the RaidNode re-calculates (if necessary)
+ the parity blocks
+ <property>
+ <name>raid.policy.rescan.interval</name>
+ <value>5000</value>
+ <description>Specify the periodicity in milliseconds after which
+ all source paths are rescanned and parity blocks recomputed if
+ necessary. By default, this value is 1 hour.
+ </description>
+ </property>
+
+ By default, the DistributedRaidFileSystem assumes that the underlying file
+ system is the DistributedFileSystem. If you want to layer the DistributedRaidFileSystem
+ over some other file system, then define a property named fs.raid.underlyingfs.impl
+ that specifies the name of the underlying class. For example, if you want to layer
+ The DistributedRaidFileSystem over an instance of the NewFileSystem, then
+ <property>
+ <name>fs.raid.underlyingfs.impl</name>
+ <value>org.apche.hadoop.new.NewFileSystem</value>
+ <description>Specify the filesystem that is layered immediately below the
+ DistributedRaidFileSystem. By default, this value is DistributedFileSystem.
+ </description>
+
+
+--------------------------------------------------------------------------------
+
+ADMINISTRATION:
+
+The Distributed Raid File System provides support for administration at runtime without
+any downtime to cluster services. It is possible to add/delete new paths to be raided without
+interrupting any load on the cluster. If you change raid.xml, its contents will be
+reload within seconds and the new contents will take effect immediately.
+
+Designate one machine in your cluster to run the RaidNode software. You can run this daemon
+on any machine irrespective of whether that machine is running any other hadoop daemon or not.
+You can start the RaidNode by running the following on the selected machine:
+nohup $HADOOP_HOME/bin/hadoop org.apache.hadoop.raid.RaidNode >> /xxx/logs/hadoop-root-raidnode-hadoop.xxx.com.log &
+
+Run fsckraid periodically (being developed as part of another JIRA). This valudates parity
+blocsk of a file.
+
+--------------------------------------------------------------------------------
+
+IMPLEMENTATION:
+
+The RaidNode periodically scans all the specified paths in the configuration
+file. For each path, it recursively scans all files that have more than 2 blocks
+and that has not been modified during the last few hours (default is 24 hours).
+It picks the specified number of blocks (as specified by the stripe size),
+from the file, generates a parity block by combining them and
+stores the results as another HDFS file in the specified destination
+directory. There is a one-to-one mapping between a HDFS
+file and its parity file. The RaidNode also periodically finds parity files
+that are orphaned and deletes them.
+
+The Distributed Raid FileSystem is layered over a DistributedFileSystem
+instance intercepts all calls that go into HDFS. HDFS throws a ChecksumException
+or a BlocMissingException when a file read encounters bad data. The layered
+Distributed Raid FileSystem catches these exceptions, locates the corresponding
+parity file, extract the original data from the parity files and feeds the
+extracted data back to the application in a completely transparent way.
+
+The layered Distributed Raid FileSystem does not fix the data-loss that it
+encounters while serving data. It merely make the application transparently
+use the parity blocks to re-create the original data. A command line tool
+"fsckraid" is currently under development that will fix the corrupted files
+by extracting the data from the associated parity files. An adminstrator
+can run "fsckraid" manually as and when needed.
diff --git a/src/contrib/raid/build.xml b/src/contrib/raid/build.xml
new file mode 100644
index 0000000..6fa0a1c
--- /dev/null
+++ b/src/contrib/raid/build.xml
@@ -0,0 +1,64 @@
+<?xml version="1.0"?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!--
+Before you can run these subtargets directly, you need
+to call at top-level: ant deploy-contrib compile-core-test
+-->
+<project name="raid" default="jar">
+
+ <import file="../build-contrib.xml"/>
+
+ <!-- the unit test classpath -->
+ <path id="contrib.classpath.raid">
+ <pathelement location="${hadoop.root}/src/contrib/raid/lib"/>
+ <path refid="contrib-classpath"/>
+ </path>
+
+ <target name="test" depends="compile,compile-test,test-junit" description="Automated Test Framework" if="test.available"/>
+
+ <target name="test-junit" depends="compile,compile-test" if="test.available">
+ <junit showoutput="${test.output}" fork="yes" printsummary="yes" errorProperty="tests.failed"
+ haltonfailure="no" failureProperty="tests.failed">
+
+ <classpath refid="test.classpath"/>
+ <sysproperty key="test.build.data" value="${build.test}/data"/>
+ <sysproperty key="build.test" value="${build.test}"/>
+ <sysproperty key="user.dir" value="${build.test}/data"/>
+ <sysproperty key="fs.default.name" value="${fs.default.name}"/>
+ <sysproperty key="hadoop.test.localoutputfile" value="${hadoop.test.localoutputfile}"/>
+ <sysproperty key="hadoop.log.dir" value="${hadoop.log.dir}"/>
+ <sysproperty key="test.src.dir" value="${test.src.dir}"/>
+ <formatter type="plain" />
+ <batchtest todir="${build.test}" unless="testcase">
+ <fileset dir="${src.test}">
+ <include name="**/Test*.java"/>
+ </fileset>
+ </batchtest>
+ <batchtest todir="${build.test}" if="testcase">
+ <fileset dir="${src.test}">
+ <include name="**/${testcase}.java"/>
+ </fileset>
+ </batchtest>
+ </junit>
+ <fail if="tests.failed">Tests failed!</fail>
+ </target>
+
+</project>
+
diff --git a/src/contrib/raid/conf/raid.xml b/src/contrib/raid/conf/raid.xml
new file mode 100644
index 0000000..3cf2dc6
--- /dev/null
+++ b/src/contrib/raid/conf/raid.xml
@@ -0,0 +1,58 @@
+ <configuration>
+ <srcPath prefix="hdfs://dfs1.xxx.com:8000/user/dhruba/">
+ <policy name = "dhruba">
+ <property>
+ <name>srcReplication</name>
+ <value>3</value>
+ <description> pick files for RAID only if their replication factor is
+ greater than or equal to this value.
+ </description>
+ </property>
+ <property>
+ <name>targetReplication</name>
+ <value>2</value>
+ <description> after RAIDing, decrease the replication factor of a file to
+ this value.
+ </description>
+ </property>
+ <property>
+ <name>metaReplication</name>
+ <value>2</value>
+ <description> the replication factor of the RAID meta file
+ </description>
+ </property>
+ <property>
+ <name>modTimePeriod</name>
+ <value>3600000</value>
+ <description> time (milliseconds) after a file is modified to make it a
+ candidate for RAIDing
+ </description>
+ </property>
+ </policy>
+ </srcPath>
+ <srcPath prefix="hdfs://dfs1.xxx.com:9000/warehouse/table1">
+ <policy name = "table1">
+ <property>
+ <name>targetReplication</name>
+ <value>1</value>
+ <description> after RAIDing, decrease the replication factor of a file to
+ this value.
+ </description>
+ </property>
+ <property>
+ <name>metaReplication</name>
+ <value>2</value>
+ <description> the replication factor of the RAID meta file
+ </description>
+ </property>
+ <property>
+ <name>modTimePeriod</name>
+ <value>3600000</value>
+ <description> time (milliseconds) after a file is modified to make it a
+ candidate for RAIDing
+ </description>
+ </property>
+ </policy>
+ </srcPath>
+ </configuration>
+
diff --git a/src/contrib/raid/ivy.xml b/src/contrib/raid/ivy.xml
new file mode 100644
index 0000000..7510fa5
--- /dev/null
+++ b/src/contrib/raid/ivy.xml
@@ -0,0 +1,122 @@
+<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<ivy-module version="1.0">
+ <info organisation="org.apache.hadoop" module="${ant.project.name}">
+ <license name="Apache 2.0"/>
+ <description>Rumen</description>
+ </info>
+ <configurations defaultconfmapping="default">
+ <!--these match the Maven configurations-->
+ <conf name="default" extends="master,runtime"/>
+ <conf name="master" description="contains the artifact but no dependencies"/>
+ <conf name="runtime" description="runtime but not the artifact" />
+
+ <conf name="common" visibility="private" extends="runtime"
+ description="artifacts needed to compile/test the application"/>
+ <conf name="test" visibility="private" extends="runtime"/>
+ </configurations>
+
+ <publications>
+ <!--get the artifact from our module name-->
+ <artifact conf="master"/>
+ </publications>
+ <dependencies>
+ <dependency org="org.apache.hadoop"
+ name="hadoop-core"
+ rev="${hadoop-core.version}"
+ conf="common->default"/>
+ <dependency org="org.apache.hadoop"
+ name="hadoop-core-test"
+ rev="${hadoop-core.version}"
+ conf="test->default"/>
+ <dependency org="org.apache.hadoop"
+ name="hadoop-hdfs"
+ rev="${hadoop-hdfs.version}"
+ conf="common->default"/>
+ <dependency org="org.apache.hadoop"
+ name="hadoop-hdfs-test"
+ rev="${hadoop-hdfs.version}"
+ conf="test->default"/>
+ <dependency org="commons-logging"
+ name="commons-logging"
+ rev="${commons-logging.version}"
+ conf="common->default"/>
+ <dependency org="log4j"
+ name="log4j"
+ rev="${log4j.version}"
+ conf="common->master"/>
+ <dependency org="junit"
+ name="junit"
+ rev="${junit.version}"
+ conf="common->default"/>
+
+ <!-- necessary for Mini*Clusters -->
+ <dependency org="commons-httpclient"
+ name="commons-httpclient"
+ rev="${commons-httpclient.version}"
+ conf="common->master"/>
+ <dependency org="commons-codec"
+ name="commons-codec"
+ rev="${commons-codec.version}"
+ conf="common->default"/>
+ <dependency org="commons-net"
+ name="commons-net"
+ rev="${commons-net.version}"
+ conf="common->default"/>
+ <dependency org="org.mortbay.jetty"
+ name="jetty"
+ rev="${jetty.version}"
+ conf="common->master"/>
+ <dependency org="org.mortbay.jetty"
+ name="jetty-util"
+ rev="${jetty-util.version}"
+ conf="common->master"/>
+ <dependency org="org.mortbay.jetty"
+ name="jsp-api-2.1"
+ rev="${jetty.version}"
+ conf="common->master"/>
+ <dependency org="org.mortbay.jetty"
+ name="jsp-2.1"
+ rev="${jetty.version}"
+ conf="common->master"/>
+ <dependency org="org.mortbay.jetty"
+ name="servlet-api-2.5"
+ rev="${servlet-api-2.5.version}"
+ conf="common->master"/>
+ <dependency org="commons-cli"
+ name="commons-cli"
+ rev="${commons-cli.version}"
+ conf="common->default"/>
+ <dependency org="org.apache.hadoop"
+ name="avro"
+ rev="${avro.version}"
+ conf="common->default"/>
+ <dependency org="org.codehaus.jackson"
+ name="jackson-mapper-asl"
+ rev="${jackson.version}"
+ conf="common->default"/>
+ <dependency org="org.codehaus.jackson"
+ name="jackson-core-asl"
+ rev="${jackson.version}"
+ conf="common->default"/>
+ <dependency org="com.thoughtworks.paranamer"
+ name="paranamer"
+ rev="${paranamer.version}"
+ conf="common->default"/>
+ </dependencies>
+</ivy-module>
diff --git a/src/contrib/raid/ivy/libraries.properties b/src/contrib/raid/ivy/libraries.properties
new file mode 100644
index 0000000..a9f5ebb
--- /dev/null
+++ b/src/contrib/raid/ivy/libraries.properties
@@ -0,0 +1,18 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#This properties file lists the versions of the various artifacts used by hadoop.
+#It drives ivy and the generation of a maven POM
+#These are the versions of our dependencies (in alphabetical order)
diff --git a/src/contrib/raid/src/java/org/apache/hadoop/hdfs/DistributedRaidFileSystem.java b/src/contrib/raid/src/java/org/apache/hadoop/hdfs/DistributedRaidFileSystem.java
new file mode 100644
index 0000000..b82ad2d
--- /dev/null
+++ b/src/contrib/raid/src/java/org/apache/hadoop/hdfs/DistributedRaidFileSystem.java
@@ -0,0 +1,381 @@
+/**
+ * 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.hdfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.DataInput;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.raid.RaidNode;
+import org.apache.hadoop.hdfs.BlockMissingException;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+
+/**
+ * This is an implementation of the Hadoop RAID Filesystem. This FileSystem
+ * wraps an instance of the DistributedFileSystem.
+ * If a file is corrupted, this FileSystem uses the parity blocks to
+ * regenerate the bad block.
+ */
+
+public class DistributedRaidFileSystem extends FilterFileSystem {
+
+ // these are alternate locations that can be used for read-only access
+ Path[] alternates;
+ Configuration conf;
+ int stripeLength;
+
+ DistributedRaidFileSystem() throws IOException {
+ }
+
+ DistributedRaidFileSystem(FileSystem fs) throws IOException {
+ super(fs);
+ alternates = null;
+ stripeLength = 0;
+ }
+
+ /* Initialize a Raid FileSystem
+ */
+ public void initialize(URI name, Configuration conf) throws IOException {
+ super.initialize(name, conf);
+ this.conf = conf;
+
+ String alt = conf.get("hdfs.raid.locations");
+
+ // If no alternates are specified, then behave absolutely same as
+ // the original file system.
+ if (alt == null || alt.length() == 0) {
+ LOG.info("hdfs.raid.locations not defined. Using defaults...");
+ alt = RaidNode.DEFAULT_RAID_LOCATION;
+ }
+
+ // fs.alternate.filesystem.prefix can be of the form:
+ // "hdfs://host:port/myPrefixPath, file:///localPrefix,hftp://host1:port1/"
+ String[] strs = alt.split(",");
+ if (strs == null || strs.length == 0) {
+ LOG.info("hdfs.raid.locations badly defined. Ignoring...");
+ return;
+ }
+
+ // find stripe length configured
+ stripeLength = conf.getInt("hdfs.raid.stripeLength", RaidNode.DEFAULT_STRIPE_LENGTH);
+ if (stripeLength == 0) {
+ LOG.info("dfs.raid.stripeLength is incorrectly defined to be " +
+ stripeLength + " Ignoring...");
+ return;
+ }
+
+ // create a reference to all underlying alternate path prefix
+ alternates = new Path[strs.length];
+ for (int i = 0; i < strs.length; i++) {
+ alternates[i] = new Path(strs[i].trim());
+ alternates[i] = alternates[i].makeQualified(fs);
+ }
+ }
+
+ /*
+ * Returns the underlying filesystem
+ */
+ public FileSystem getFileSystem() throws IOException {
+ return fs;
+ }
+
+ @Override
+ public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+ ExtFSDataInputStream fd = new ExtFSDataInputStream(conf, this, alternates, f,
+ stripeLength, bufferSize);
+ return fd;
+ }
+
+ public void close() throws IOException {
+ if (fs != null) {
+ try {
+ fs.close();
+ } catch(IOException ie) {
+ //this might already be closed, ignore
+ }
+ }
+ }
+
+ /**
+ * Layered filesystem input stream. This input stream tries reading
+ * from alternate locations if it encoumters read errors in the primary location.
+ */
+ private static class ExtFSDataInputStream extends FSDataInputStream {
+ /**
+ * Create an input stream that wraps all the reads/positions/seeking.
+ */
+ private static class ExtFsInputStream extends FSInputStream {
+
+ //The underlying data input stream that the
+ // underlying filesystem will return.
+ private FSDataInputStream underLyingStream;
+ private byte[] oneBytebuff = new byte[1];
+ private int nextLocation;
+ private DistributedRaidFileSystem lfs;
+ private Path path;
+ private final Path[] alternates;
+ private final int buffersize;
+ private final Configuration conf;
+ private final int stripeLength;
+
+ ExtFsInputStream(Configuration conf, DistributedRaidFileSystem lfs, Path[] alternates,
+ Path path, int stripeLength, int buffersize)
+ throws IOException {
+ this.underLyingStream = lfs.fs.open(path, buffersize);
+ this.path = path;
+ this.nextLocation = 0;
+ this.alternates = alternates;
+ this.buffersize = buffersize;
+ this.conf = conf;
+ this.lfs = lfs;
+ this.stripeLength = stripeLength;
+ }
+
+ @Override
+ public synchronized int available() throws IOException {
+ int value = underLyingStream.available();
+ nextLocation = 0;
+ return value;
+ }
+
+ @Override
+ public synchronized void close() throws IOException {
+ underLyingStream.close();
+ super.close();
+ }
+
+ @Override
+ public void mark(int readLimit) {
+ underLyingStream.mark(readLimit);
+ nextLocation = 0;
+ }
+
+ @Override
+ public void reset() throws IOException {
+ underLyingStream.reset();
+ nextLocation = 0;
+ }
+
+ @Override
+ public synchronized int read() throws IOException {
+ long pos = underLyingStream.getPos();
+ while (true) {
+ try {
+ int value = underLyingStream.read();
+ nextLocation = 0;
+ return value;
+ } catch (BlockMissingException e) {
+ setAlternateLocations(e, pos);
+ } catch (ChecksumException e) {
+ setAlternateLocations(e, pos);
+ }
+ }
+ }
+
+ @Override
+ public synchronized int read(byte[] b) throws IOException {
+ long pos = underLyingStream.getPos();
+ while (true) {
+ try{
+ int value = underLyingStream.read(b);
+ nextLocation = 0;
+ return value;
+ } catch (BlockMissingException e) {
+ setAlternateLocations(e, pos);
+ } catch (ChecksumException e) {
+ setAlternateLocations(e, pos);
+ }
+ }
+ }
+
+ @Override
+ public synchronized int read(byte[] b, int offset, int len)
+ throws IOException {
+ long pos = underLyingStream.getPos();
+ while (true) {
+ try{
+ int value = underLyingStream.read(b, offset, len);
+ nextLocation = 0;
+ return value;
+ } catch (BlockMissingException e) {
+ setAlternateLocations(e, pos);
+ } catch (ChecksumException e) {
+ setAlternateLocations(e, pos);
+ }
+ }
+ }
+
+ @Override
+ public synchronized int read(long position, byte[] b, int offset, int len)
+ throws IOException {
+ long pos = underLyingStream.getPos();
+ while (true) {
+ try {
+ int value = underLyingStream.read(position, b, offset, len);
+ nextLocation = 0;
+ return value;
+ } catch (BlockMissingException e) {
+ setAlternateLocations(e, pos);
+ } catch (ChecksumException e) {
+ setAlternateLocations(e, pos);
+ }
+ }
+ }
+
+ @Override
+ public synchronized long skip(long n) throws IOException {
+ long value = underLyingStream.skip(n);
+ nextLocation = 0;
+ return value;
+ }
+
+ @Override
+ public synchronized long getPos() throws IOException {
+ long value = underLyingStream.getPos();
+ nextLocation = 0;
+ return value;
+ }
+
+ @Override
+ public synchronized void seek(long pos) throws IOException {
+ underLyingStream.seek(pos);
+ nextLocation = 0;
+ }
+
+ @Override
+ public boolean seekToNewSource(long targetPos) throws IOException {
+ boolean value = underLyingStream.seekToNewSource(targetPos);
+ nextLocation = 0;
+ return value;
+ }
+
+ /**
+ * position readable again.
+ */
+ @Override
+ public void readFully(long pos, byte[] b, int offset, int length)
+ throws IOException {
+ long post = underLyingStream.getPos();
+ while (true) {
+ try {
+ underLyingStream.readFully(pos, b, offset, length);
+ nextLocation = 0;
+ } catch (BlockMissingException e) {
+ setAlternateLocations(e, post);
+ } catch (ChecksumException e) {
+ setAlternateLocations(e, pos);
+ }
+ }
+ }
+
+ @Override
+ public void readFully(long pos, byte[] b) throws IOException {
+ long post = underLyingStream.getPos();
+ while (true) {
+ try {
+ underLyingStream.readFully(pos, b);
+ nextLocation = 0;
+ } catch (BlockMissingException e) {
+ setAlternateLocations(e, post);
+ } catch (ChecksumException e) {
+ setAlternateLocations(e, pos);
+ }
+ }
+ }
+
+ /**
+ * Extract good file from RAID
+ * @param curpos curexp the current exception
+ * @param curpos the position of the current operation to be retried
+ * @throws IOException if all alternate locations are exhausted
+ */
+ private void setAlternateLocations(IOException curexp, long curpos)
+ throws IOException {
+ while (alternates != null && nextLocation < alternates.length) {
+ try {
+ int idx = nextLocation++;
+ long corruptOffset = -1;
+ if (curexp instanceof BlockMissingException) {
+ corruptOffset = ((BlockMissingException)curexp).getOffset();
+ } else if (curexp instanceof ChecksumException) {
+ corruptOffset = ((ChecksumException)curexp).getPos();
+ }
+ Path npath = RaidNode.unRaid(conf, path, alternates[idx], stripeLength,
+ corruptOffset);
+ FileSystem fs1 = getUnderlyingFileSystem(conf);
+ fs1.initialize(npath.toUri(), conf);
+ LOG.info("Opening alternate path " + npath + " at offset " + curpos);
+ FSDataInputStream fd = fs1.open(npath, buffersize);
+ fd.seek(curpos);
+ underLyingStream.close();
+ underLyingStream = fd;
+ lfs.fs = fs1;
+ path = npath;
+ return;
+ } catch (Exception e) {
+ LOG.info("Error in using alternate path " + path + ". " + e +
+ " Ignoring...");
+ }
+ }
+ throw curexp;
+ }
+
+ /**
+ * The name of the file system that is immediately below the
+ * DistributedRaidFileSystem. This is specified by the
+ * configuration parameter called fs.raid.underlyingfs.impl.
+ * If this parameter is not specified in the configuration, then
+ * the default class DistributedFileSystem is returned.
+ * @param conf the configuration object
+ * @return the filesystem object immediately below DistributedRaidFileSystem
+ * @throws IOException if all alternate locations are exhausted
+ */
+ private FileSystem getUnderlyingFileSystem(Configuration conf) {
+ Class<?> clazz = conf.getClass("fs.raid.underlyingfs.impl", DistributedFileSystem.class);
+ FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
+ return fs;
+ }
+ }
+
+ /**
+ * constructor for ext input stream.
+ * @param fs the underlying filesystem
+ * @param p the path in the underlying file system
+ * @param buffersize the size of IO
+ * @throws IOException
+ */
+ public ExtFSDataInputStream(Configuration conf, DistributedRaidFileSystem lfs,
+ Path[] alternates, Path p, int stripeLength, int buffersize) throws IOException {
+ super(new ExtFsInputStream(conf, lfs, alternates, p, stripeLength, buffersize));
+ }
+ }
+}
diff --git a/src/contrib/raid/src/java/org/apache/hadoop/raid/ConfigManager.java b/src/contrib/raid/src/java/org/apache/hadoop/raid/ConfigManager.java
new file mode 100644
index 0000000..d07017f
--- /dev/null
+++ b/src/contrib/raid/src/java/org/apache/hadoop/raid/ConfigManager.java
@@ -0,0 +1,358 @@
+/**
+ * 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.raid;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.w3c.dom.Text;
+import org.xml.sax.SAXException;
+
+import org.apache.hadoop.raid.protocol.PolicyInfo;
+import org.apache.hadoop.raid.protocol.PolicyList;
+
+/**
+ * Maintains the configuration xml file that is read into memory.
+ */
+class ConfigManager {
+ public static final Log LOG = LogFactory.getLog(
+ "org.apache.hadoop.raid.ConfigManager");
+
+ /** Time to wait between checks of the config file */
+ public static final long RELOAD_INTERVAL = 10 * 1000;
+
+ /** Time to wait between successive runs of all policies */
+ public static final long RESCAN_INTERVAL = 3600 * 1000;
+
+ /**
+ * Time to wait after the config file has been modified before reloading it
+ * (this is done to prevent loading a file that hasn't been fully written).
+ */
+ public static final long RELOAD_WAIT = 5 * 1000;
+
+ private Configuration conf; // Hadoop configuration
+ private String configFileName; // Path to config XML file
+
+ private long lastReloadAttempt; // Last time we tried to reload the config file
+ private long lastSuccessfulReload; // Last time we successfully reloaded config
+ private boolean lastReloadAttemptFailed = false;
+ private long reloadInterval = RELOAD_INTERVAL;
+ private long periodicity; // time between runs of all policies
+
+ // Reload the configuration
+ private boolean doReload;
+ private Thread reloadThread;
+ private volatile boolean running = false;
+
+ // Collection of all configured policies.
+ Collection<PolicyList> allPolicies = new ArrayList<PolicyList>();
+
+ public ConfigManager(Configuration conf) throws IOException, SAXException,
+ RaidConfigurationException, ClassNotFoundException, ParserConfigurationException {
+ this.conf = conf;
+ this.configFileName = conf.get("raid.config.file");
+ this.doReload = conf.getBoolean("raid.config.reload", true);
+ this.reloadInterval = conf.getLong("raid.config.reload.interval", RELOAD_INTERVAL);
+ this.periodicity = conf.getLong("raid.policy.rescan.interval", RESCAN_INTERVAL);
+ if (configFileName == null) {
+ String msg = "No raid.config.file given in conf - " +
+ "the Hadoop Raid utility cannot run. Aborting....";
+ LOG.warn(msg);
+ throw new IOException(msg);
+ }
+ reloadConfigs();
+ lastSuccessfulReload = RaidNode.now();
+ lastReloadAttempt = RaidNode.now();
+ running = true;
+ }
+
+ /**
+ * Reload config file if it hasn't been loaded in a while
+ * Returns true if the file was reloaded.
+ */
+ public synchronized boolean reloadConfigsIfNecessary() {
+ long time = RaidNode.now();
+ if (time > lastReloadAttempt + reloadInterval) {
+ lastReloadAttempt = time;
+ try {
+ File file = new File(configFileName);
+ long lastModified = file.lastModified();
+ if (lastModified > lastSuccessfulReload &&
+ time > lastModified + RELOAD_WAIT) {
+ reloadConfigs();
+ lastSuccessfulReload = time;
+ lastReloadAttemptFailed = false;
+ return true;
+ }
+ } catch (Exception e) {
+ if (!lastReloadAttemptFailed) {
+ LOG.error("Failed to reload config file - " +
+ "will use existing configuration.", e);
+ }
+ lastReloadAttemptFailed = true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Updates the in-memory data structures from the config file. This file is
+ * expected to be in the following whitespace-separated format:
+ *
+ <configuration>
+ <srcPath prefix="hdfs://hadoop.myhost.com:9000/user/warehouse/u_full/*">
+ <destPath> hdfs://dfsarch.data.facebook.com:9000/archive/</destPath>
+ <policy name = RaidScanWeekly>
+ <property>
+ <name>targetReplication</name>
+ <value>2</value>
+ <description> after RAIDing, decrease the replication factor of the file to
+ this value.
+ </description>
+ </property>
+ <property>
+ <name>metaReplication</name>
+ <value>2</value>
+ <description> the replication factor of the RAID meta file
+ </description>
+ </property>
+ <property>
+ <name>stripeLength</name>
+ <value>10</value>
+ <description> the number of blocks to RAID together
+ </description>
+ </property>
+ </policy>
+ </srcPath>
+ </configuration>
+ *
+ * Blank lines and lines starting with # are ignored.
+ *
+ * @throws IOException if the config file cannot be read.
+ * @throws RaidConfigurationException if configuration entries are invalid.
+ * @throws ClassNotFoundException if user-defined policy classes cannot be loaded
+ * @throws ParserConfigurationException if XML parser is misconfigured.
+ * @throws SAXException if config file is malformed.
+ * @returns A new set of policy categories.
+ */
+ void reloadConfigs() throws IOException, ParserConfigurationException,
+ SAXException, ClassNotFoundException, RaidConfigurationException {
+
+ if (configFileName == null) {
+ return;
+ }
+
+ File file = new File(configFileName);
+ if (!file.exists()) {
+ throw new RaidConfigurationException("Configuration file " + configFileName +
+ " does not exist.");
+ }
+
+ // Create some temporary hashmaps to hold the new allocs, and we only save
+ // them in our fields if we have parsed the entire allocs file successfully.
+ List<PolicyList> all = new ArrayList<PolicyList>();
+ long periodicityValue = periodicity;
+
+
+ // Read and parse the configuration file.
+ // allow include files in configuration file
+ DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory.newInstance();
+ docBuilderFactory.setIgnoringComments(true);
+ docBuilderFactory.setNamespaceAware(true);
+ try {
+ docBuilderFactory.setXIncludeAware(true);
+ } catch (UnsupportedOperationException e) {
+ LOG.error("Failed to set setXIncludeAware(true) for raid parser "
+ + docBuilderFactory + ":" + e, e);
+ }
+ LOG.error("Reloading config file " + file);
+
+ DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
+ Document doc = builder.parse(file);
+ Element root = doc.getDocumentElement();
+ if (!"configuration".equalsIgnoreCase(root.getTagName()))
+ throw new RaidConfigurationException("Bad configuration file: " +
+ "top-level element not <configuration>");
+ NodeList elements = root.getChildNodes();
+
+ // loop through all the configured source paths.
+ for (int i = 0; i < elements.getLength(); i++) {
+ Node node = elements.item(i);
+ if (!(node instanceof Element)) {
+ continue;
+ }
+ Element element = (Element)node;
+ String elementTagName = element.getTagName();
+ if ("srcPath".equalsIgnoreCase(elementTagName)) {
+ String srcPathPrefix = element.getAttribute("prefix");
+
+ if (srcPathPrefix == null || srcPathPrefix.length() == 0) {
+ throw new RaidConfigurationException("Bad configuration file: " +
+ "srcPathPrefix not set.");
+ }
+ PolicyList policyList = new PolicyList();
+ all.add(policyList);
+
+ policyList.setSrcPath(conf, srcPathPrefix);
+
+ // loop through all the policies for this source path
+ NodeList policies = element.getChildNodes();
+ for (int j = 0; j < policies.getLength(); j++) {
+ Node node1 = policies.item(j);
+ if (!(node1 instanceof Element)) {
+ continue;
+ }
+ Element policy = (Element)node1;
+ if (!"policy".equalsIgnoreCase(policy.getTagName())) {
+ throw new RaidConfigurationException("Bad configuration file: " +
+ "Expecting <policy> for srcPath " + srcPathPrefix);
+ }
+ String policyName = policy.getAttribute("name");
+ PolicyInfo pinfo = new PolicyInfo(policyName, conf);
+ pinfo.setSrcPath(srcPathPrefix);
+ policyList.add(pinfo);
+
+ // loop through all the properties of this policy
+ NodeList properties = policy.getChildNodes();
+ for (int k = 0; k < properties.getLength(); k++) {
+ Node node2 = properties.item(k);
+ if (!(node2 instanceof Element)) {
+ continue;
+ }
+ Element property = (Element)node2;
+ String propertyName = property.getTagName();
+ if ("destPath".equalsIgnoreCase(propertyName)) {
+ String text = ((Text)property.getFirstChild()).getData().trim();
+ LOG.info(policyName + ".destPath = " + text);
+ pinfo.setDestinationPath(text);
+ } else if ("description".equalsIgnoreCase(propertyName)) {
+ String text = ((Text)property.getFirstChild()).getData().trim();
+ pinfo.setDescription(text);
+ } else if ("property".equalsIgnoreCase(propertyName)) {
+ NodeList nl = property.getChildNodes();
+ String pname=null,pvalue=null;
+ for (int l = 0; l < nl.getLength(); l++){
+ Node node3 = nl.item(l);
+ if (!(node3 instanceof Element)) {
+ continue;
+ }
+ Element item = (Element) node3;
+ String itemName = item.getTagName();
+ if ("name".equalsIgnoreCase(itemName)){
+ pname = ((Text)item.getFirstChild()).getData().trim();
+ } else if ("value".equalsIgnoreCase(itemName)){
+ pvalue = ((Text)item.getFirstChild()).getData().trim();
+ }
+ }
+ if (pname != null && pvalue != null) {
+ LOG.info(policyName + "." + pname + " = " + pvalue);
+ pinfo.setProperty(pname,pvalue);
+ }
+ } else {
+ LOG.info("Found bad property " + propertyName +
+ " for srcPath" + srcPathPrefix +
+ " policy name " + policyName +
+ ". Ignoring.");
+ }
+ } // done with all properties of this policy
+ } // done with all policies for this srcpath
+ }
+ } // done with all srcPaths
+ setAllPolicies(all);
+ periodicity = periodicityValue;
+ return;
+ }
+
+
+ public synchronized long getPeriodicity() {
+ return periodicity;
+ }
+
+ /**
+ * Get a collection of all policies
+ */
+ public synchronized Collection<PolicyList> getAllPolicies() {
+ return new ArrayList(allPolicies);
+ }
+
+ /**
+ * Set a collection of all policies
+ */
+ protected synchronized void setAllPolicies(Collection<PolicyList> value) {
+ this.allPolicies = value;
+ }
+
+ /**
+ * Start a background thread to reload the config file
+ */
+ void startReload() {
+ if (doReload) {
+ reloadThread = new UpdateThread();
+ reloadThread.start();
+ }
+ }
+
+ /**
+ * Stop the background thread that reload the config file
+ */
+ void stopReload() throws InterruptedException {
+ if (reloadThread != null) {
+ running = false;
+ reloadThread.interrupt();
+ reloadThread.join();
+ reloadThread = null;
+ }
+ }
+
+ /**
+ * A thread which reloads the config file.
+ */
+ private class UpdateThread extends Thread {
+ private UpdateThread() {
+ super("Raid update thread");
+ }
+
+ public void run() {
+ while (running) {
+ try {
+ Thread.sleep(reloadInterval);
+ reloadConfigsIfNecessary();
+ } catch (InterruptedException e) {
+ // do nothing
+ } catch (Exception e) {
+ LOG.error("Failed to reload config file ", e);
+ }
+ }
+ }
+ }
+}
diff --git a/src/contrib/raid/src/java/org/apache/hadoop/raid/DistRaid.java b/src/contrib/raid/src/java/org/apache/hadoop/raid/DistRaid.java
new file mode 100644
index 0000000..17a0265
--- /dev/null
+++ b/src/contrib/raid/src/java/org/apache/hadoop/raid/DistRaid.java
@@ -0,0 +1,353 @@
+/**
+ * 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.raid;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.Date;
+import java.text.SimpleDateFormat;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.SequenceFileRecordReader;
+import org.apache.hadoop.raid.RaidNode.Statistics;
+import org.apache.hadoop.raid.protocol.PolicyInfo;
+import org.apache.hadoop.util.StringUtils;
+
+public class DistRaid {
+
+ protected static final Log LOG = LogFactory.getLog(DistRaid.class);
+
+ static final String NAME = "distRaid";
+ static final String JOB_DIR_LABEL = NAME + ".job.dir";
+ static final String OP_LIST_LABEL = NAME + ".op.list";
+ static final String OP_COUNT_LABEL = NAME + ".op.count";
+ static final int OP_LIST_BLOCK_SIZE = 32 * 1024 * 1024; // block size of control file
+ static final short OP_LIST_REPLICATION = 10; // replication factor of control file
+
+ private static final long OP_PER_MAP = 100;
+ private static final int MAX_MAPS_PER_NODE = 20;
+ private static final int SYNC_FILE_MAX = 10;
+ private static final SimpleDateFormat dateForm = new SimpleDateFormat("yyyy-MM-dd HH:mm");
+ private static String jobName = NAME;
+
+ static enum Counter {
+ FILES_SUCCEEDED, FILES_FAILED, PROCESSED_BLOCKS, PROCESSED_SIZE, META_BLOCKS, META_SIZE
+ }
+
+ protected JobConf jobconf;
+
+ /** {@inheritDoc} */
+ public void setConf(Configuration conf) {
+ if (jobconf != conf) {
+ jobconf = conf instanceof JobConf ? (JobConf) conf : new JobConf(conf);
+ }
+ }
+
+ /** {@inheritDoc} */
+ public JobConf getConf() {
+ return jobconf;
+ }
+
+ public DistRaid(Configuration conf) {
+ setConf(createJobConf(conf));
+ }
+
+ private static final Random RANDOM = new Random();
+
+ protected static String getRandomId() {
+ return Integer.toString(RANDOM.nextInt(Integer.MAX_VALUE), 36);
+ }
+
+ /**
+ *
+ * helper class which holds the policy and paths
+ *
+ */
+ public static class RaidPolicyPathPair {
+ public PolicyInfo policy;
+ public List<FileStatus> srcPaths;
+
+ RaidPolicyPathPair(PolicyInfo policy, List<FileStatus> srcPaths) {
+ this.policy = policy;
+ this.srcPaths = srcPaths;
+ }
+ }
+
+ List<RaidPolicyPathPair> raidPolicyPathPairList = new ArrayList<RaidPolicyPathPair>();
+
+ /** Responsible for generating splits of the src file list. */
+ static class DistRaidInputFormat implements InputFormat<Text, PolicyInfo> {
+ /** Do nothing. */
+ public void validateInput(JobConf job) {
+ }
+
+ /**
+ * Produce splits such that each is no greater than the quotient of the
+ * total size and the number of splits requested.
+ *
+ * @param job
+ * The handle to the JobConf object
+ * @param numSplits
+ * Number of splits requested
+ */
+ public InputSplit[] getSplits(JobConf job, int numSplits)
+ throws IOException {
+ final int srcCount = job.getInt(OP_COUNT_LABEL, -1);
+ final int targetcount = srcCount / numSplits;
+ String srclist = job.get(OP_LIST_LABEL, "");
+ if (srcCount < 0 || "".equals(srclist)) {
+ throw new RuntimeException("Invalid metadata: #files(" + srcCount
+ + ") listuri(" + srclist + ")");
+ }
+ Path srcs = new Path(srclist);
+ FileSystem fs = srcs.getFileSystem(job);
+
+ List<FileSplit> splits = new ArrayList<FileSplit>(numSplits);
+
+ Text key = new Text();
+ PolicyInfo value = new PolicyInfo();
+ SequenceFile.Reader in = null;
+ long prev = 0L;
+ int count = 0; // count src
+ try {
+ for (in = new SequenceFile.Reader(fs, srcs, job); in.next(key, value);) {
+ long curr = in.getPosition();
+ long delta = curr - prev;
+ if (++count > targetcount) {
+ count = 0;
+ splits.add(new FileSplit(srcs, prev, delta, (String[]) null));
+ prev = curr;
+ }
+ }
+ } finally {
+ in.close();
+ }
+ long remaining = fs.getFileStatus(srcs).getLen() - prev;
+ if (remaining != 0) {
+ splits.add(new FileSplit(srcs, prev, remaining, (String[]) null));
+ }
+ LOG.info("jobname= " + jobName + " numSplits=" + numSplits +
+ ", splits.size()=" + splits.size());
+ return splits.toArray(new FileSplit[splits.size()]);
+ }
+
+ /** {@inheritDoc} */
+ public RecordReader<Text, PolicyInfo> getRecordReader(InputSplit split,
+ JobConf job, Reporter reporter) throws IOException {
+ return new SequenceFileRecordReader<Text, PolicyInfo>(job,
+ (FileSplit) split);
+ }
+ }
+
+ /** The mapper for raiding files. */
+ static class DistRaidMapper implements
+ Mapper<Text, PolicyInfo, WritableComparable, Text> {
+ private JobConf jobconf;
+ private boolean ignoreFailures;
+
+ private int failcount = 0;
+ private int succeedcount = 0;
+ private Statistics st = null;
+
+ private String getCountString() {
+ return "Succeeded: " + succeedcount + " Failed: " + failcount;
+ }
+
+ /** {@inheritDoc} */
+ public void configure(JobConf job) {
+ this.jobconf = job;
+ ignoreFailures = false;
+ st = new Statistics();
+ }
+
+ /** Run a FileOperation */
+ public void map(Text key, PolicyInfo policy,
+ OutputCollector<WritableComparable, Text> out, Reporter reporter)
+ throws IOException {
+ try {
+ LOG.info("Raiding file=" + key.toString() + " policy=" + policy);
+ Path p = new Path(key.toString());
+ FileStatus fs = p.getFileSystem(jobconf).getFileStatus(p);
+ st.clear();
+ RaidNode.doRaid(jobconf, policy, fs, st, reporter);
+
+ ++succeedcount;
+
+ reporter.incrCounter(Counter.PROCESSED_BLOCKS, st.numProcessedBlocks);
+ reporter.incrCounter(Counter.PROCESSED_SIZE, st.processedSize);
+ reporter.incrCounter(Counter.META_BLOCKS, st.numMetaBlocks);
+ reporter.incrCounter(Counter.META_SIZE, st.metaSize);
+
+ reporter.incrCounter(Counter.FILES_SUCCEEDED, 1);
+ } catch (IOException e) {
+ ++failcount;
+ reporter.incrCounter(Counter.FILES_FAILED, 1);
+
+ String s = "FAIL: " + policy + ", " + key + " "
+ + StringUtils.stringifyException(e);
+ out.collect(null, new Text(s));
+ LOG.info(s);
+ } finally {
+ reporter.setStatus(getCountString());
+ }
+ }
+
+ /** {@inheritDoc} */
+ public void close() throws IOException {
+ if (failcount == 0 || ignoreFailures) {
+ return;
+ }
+ throw new IOException(getCountString());
+ }
+ }
+
+ /**
+ * create new job conf based on configuration passed.
+ *
+ * @param conf
+ * @return
+ */
+ private static JobConf createJobConf(Configuration conf) {
+ JobConf jobconf = new JobConf(conf, DistRaid.class);
+ jobName = NAME + " " + dateForm.format(new Date(RaidNode.now()));
+ jobconf.setJobName(jobName);
+ jobconf.setMapSpeculativeExecution(false);
+
+ jobconf.setJarByClass(DistRaid.class);
+ jobconf.setInputFormat(DistRaidInputFormat.class);
+ jobconf.setOutputKeyClass(Text.class);
+ jobconf.setOutputValueClass(Text.class);
+
+ jobconf.setMapperClass(DistRaidMapper.class);
+ jobconf.setNumReduceTasks(0);
+ return jobconf;
+ }
+
+ /** Add paths to be raided */
+ public void addRaidPaths(PolicyInfo info, List<FileStatus> paths) {
+ raidPolicyPathPairList.add(new RaidPolicyPathPair(info, paths));
+ }
+
+ /** Calculate how many maps to run. */
+ private static int getMapCount(int srcCount, int numNodes) {
+ int numMaps = (int) (srcCount / OP_PER_MAP);
+ numMaps = Math.min(numMaps, numNodes * MAX_MAPS_PER_NODE);
+ return Math.max(numMaps, 1);
+ }
+
+ /** invokes mapred job do parallel raiding */
+ public void doDistRaid() throws IOException {
+ if (raidPolicyPathPairList.size() == 0) {
+ LOG.info("DistRaid has no paths to raid.");
+ return;
+ }
+ try {
+ if (setup()) {
+ JobClient.runJob(jobconf);
+ }
+ } finally {
+ // delete job directory
+ final String jobdir = jobconf.get(JOB_DIR_LABEL);
+ if (jobdir != null) {
+ final Path jobpath = new Path(jobdir);
+ jobpath.getFileSystem(jobconf).delete(jobpath, true);
+ }
+ }
+ raidPolicyPathPairList.clear();
+ }
+
+ /**
+ * set up input file which has the list of input files.
+ *
+ * @return boolean
+ * @throws IOException
+ */
+ private boolean setup() throws IOException {
+ final String randomId = getRandomId();
+ JobClient jClient = new JobClient(jobconf);
+ Path jobdir = new Path(jClient.getSystemDir(), NAME + "_" + randomId);
+
+ LOG.info(JOB_DIR_LABEL + "=" + jobdir);
+ jobconf.set(JOB_DIR_LABEL, jobdir.toString());
+ Path log = new Path(jobdir, "_logs");
+
+ // The control file should have small size blocks. This helps
+ // in spreading out the load from mappers that will be spawned.
+ jobconf.setInt("dfs.blocks.size", OP_LIST_BLOCK_SIZE);
+
+ FileOutputFormat.setOutputPath(jobconf, log);
+ LOG.info("log=" + log);
+
+ // create operation list
+ FileSystem fs = jobdir.getFileSystem(jobconf);
+ Path opList = new Path(jobdir, "_" + OP_LIST_LABEL);
+ jobconf.set(OP_LIST_LABEL, opList.toString());
+ int opCount = 0, synCount = 0;
+ SequenceFile.Writer opWriter = null;
+ try {
+ opWriter = SequenceFile.createWriter(fs, jobconf, opList, Text.class,
+ PolicyInfo.class, SequenceFile.CompressionType.NONE);
+ for (RaidPolicyPathPair p : raidPolicyPathPairList) {
+ for (FileStatus st : p.srcPaths) {
+ opWriter.append(new Text(st.getPath().toString()), p.policy);
+ opCount++;
+ if (++synCount > SYNC_FILE_MAX) {
+ opWriter.sync();
+ synCount = 0;
+ }
+ }
+ }
+
+ } finally {
+ if (opWriter != null) {
+ opWriter.close();
+ }
+ fs.setReplication(opList, OP_LIST_REPLICATION); // increase replication for control file
+ }
+ raidPolicyPathPairList.clear();
+
+ jobconf.setInt(OP_COUNT_LABEL, opCount);
+ LOG.info("Number of files=" + opCount);
+ jobconf.setNumMapTasks(getMapCount(opCount, new JobClient(jobconf)
+ .getClusterStatus().getTaskTrackers()));
+ LOG.info("jobName= " + jobName + " numMapTasks=" + jobconf.getNumMapTasks());
+ return opCount != 0;
+
+ }
+}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java b/src/contrib/raid/src/java/org/apache/hadoop/raid/RaidConfigurationException.java
similarity index 65%
copy from src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
copy to src/contrib/raid/src/java/org/apache/hadoop/raid/RaidConfigurationException.java
index ec03890..7335999 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
+++ b/src/contrib/raid/src/java/org/apache/hadoop/raid/RaidConfigurationException.java
@@ -16,29 +16,15 @@
* limitations under the License.
*/
-package org.apache.hadoop.sqoop.util;
+package org.apache.hadoop.raid;
/**
- * General error during import process.
- *
- *
+ * Thrown when the config file for {@link CronNode} is malformed.
*/
-@SuppressWarnings("serial")
-public class ImportError extends Exception {
-
- public ImportError() {
- super("ImportError");
- }
-
- public ImportError(final String message) {
+public class RaidConfigurationException extends Exception {
+ private static final long serialVersionUID = 4046516718965587999L;
+
+ public RaidConfigurationException(String message) {
super(message);
}
-
- public ImportError(final Throwable cause) {
- super(cause);
- }
-
- public ImportError(final String message, final Throwable cause) {
- super(message, cause);
- }
}
diff --git a/src/contrib/raid/src/java/org/apache/hadoop/raid/RaidNode.java b/src/contrib/raid/src/java/org/apache/hadoop/raid/RaidNode.java
new file mode 100644
index 0000000..c27a121
--- /dev/null
+++ b/src/contrib/raid/src/java/org/apache/hadoop/raid/RaidNode.java
@@ -0,0 +1,1269 @@
+/**
+ * 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.raid;
+
+import java.io.IOException;
+import java.io.FileNotFoundException;
+import java.util.Collection;
+import java.util.List;
+import java.util.LinkedList;
+import java.util.Iterator;
+import java.util.Arrays;
+import java.util.Random;
+import java.util.Set;
+import java.util.HashSet;
+import java.lang.Thread;
+import java.net.InetSocketAddress;
+import org.xml.sax.SAXException;
+import javax.xml.parsers.ParserConfigurationException;
+
+import org.apache.hadoop.ipc.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.mapred.Reporter;
+
+import org.apache.hadoop.raid.protocol.PolicyInfo;
+import org.apache.hadoop.raid.protocol.PolicyList;
+import org.apache.hadoop.raid.protocol.RaidProtocol;
+import org.apache.hadoop.raid.protocol.RaidProtocol.ReturnStatus;
+
+/**
+ * A {@link RaidNode} that implements
+ */
+public class RaidNode implements RaidProtocol {
+
+ static{
+ Configuration.addDefaultResource("hdfs-default.xml");
+ Configuration.addDefaultResource("hdfs-site.xml");
+ Configuration.addDefaultResource("mapred-default.xml");
+ Configuration.addDefaultResource("mapred-site.xml");
+ }
+
+ public static final Log LOG = LogFactory.getLog( "org.apache.hadoop.raid.RaidNode");
+ public static final long SLEEP_TIME = 10000L; // 10 seconds
+ public static final int DEFAULT_PORT = 60000;
+ public static final int DEFAULT_STRIPE_LENGTH = 5; // default value of stripe length
+ public static final String DEFAULT_RAID_LOCATION = "/raid";
+
+ /** RPC server */
+ private Server server;
+ /** RPC server address */
+ private InetSocketAddress serverAddress = null;
+ /** only used for testing purposes */
+ private boolean stopRequested = false;
+
+ /** Configuration Manager */
+ private ConfigManager configMgr;
+
+ /** hadoop configuration */
+ private Configuration conf;
+
+ protected boolean initialized; // Are we initialized?
+ protected volatile boolean running; // Are we running?
+
+ /** Deamon thread to trigger policies */
+ Daemon triggerThread = null;
+
+ /** Deamon thread to delete obsolete parity files */
+ Daemon purgeThread = null;
+
+ /** Do do distributed raiding */
+ boolean isRaidLocal = false;
+
+ // statistics about RAW hdfs blocks. This counts all replicas of a block.
+ public static class Statistics {
+ long numProcessedBlocks; // total blocks encountered in namespace
+ long processedSize; // disk space occupied by all blocks
+ long remainingSize; // total disk space post RAID
+
+ long numMetaBlocks; // total blocks in metafile
+ long metaSize; // total disk space for meta files
+
+ public void clear() {
+ numProcessedBlocks = 0;
+ processedSize = 0;
+ remainingSize = 0;
+ numMetaBlocks = 0;
+ metaSize = 0;
+ }
+ public String toString() {
+ long save = processedSize - (remainingSize + metaSize);
+ long savep = 0;
+ if (processedSize > 0) {
+ savep = (save * 100)/processedSize;
+ }
+ String msg = " numProcessedBlocks = " + numProcessedBlocks +
+ " processedSize = " + processedSize +
+ " postRaidSize = " + remainingSize +
+ " numMetaBlocks = " + numMetaBlocks +
+ " metaSize = " + metaSize +
+ " %save in raw disk space = " + savep;
+ return msg;
+ }
+ }
+
+ // Startup options
+ static public enum StartupOption{
+ TEST ("-test"),
+ REGULAR ("-regular");
+
+ private String name = null;
+ private StartupOption(String arg) {this.name = arg;}
+ public String getName() {return name;}
+ }
+
+ /**
+ * Start RaidNode.
+ * <p>
+ * The raid-node can be started with one of the following startup options:
+ * <ul>
+ * <li>{@link StartupOption#REGULAR REGULAR} - normal raid node startup</li>
+ * </ul>
+ * The option is passed via configuration field:
+ * <tt>fs.raidnode.startup</tt>
+ *
+ * The conf will be modified to reflect the actual ports on which
+ * the RaidNode is up and running if the user passes the port as
+ * <code>zero</code> in the conf.
+ *
+ * @param conf confirguration
+ * @throws IOException
+ */
+
+ RaidNode(Configuration conf) throws IOException {
+ try {
+ initialize(conf);
+ } catch (IOException e) {
+ this.stop();
+ throw e;
+ } catch (Exception e) {
+ this.stop();
+ throw new IOException(e);
+ }
+ }
+
+ public long getProtocolVersion(String protocol,
+ long clientVersion) throws IOException {
+ if (protocol.equals(RaidProtocol.class.getName())) {
+ return RaidProtocol.versionID;
+ } else {
+ throw new IOException("Unknown protocol to name node: " + protocol);
+ }
+ }
+
+ /**
+ * Wait for service to finish.
+ * (Normally, it runs forever.)
+ */
+ public void join() {
+ try {
+ if (server != null) server.join();
+ if (triggerThread != null) triggerThread.join();
+ if (purgeThread != null) purgeThread.join();
+ } catch (InterruptedException ie) {
+ // do nothing
+ }
+ }
+
+ /**
+ * Stop all RaidNode threads and wait for all to finish.
+ */
+ public void stop() {
+ if (stopRequested) {
+ return;
+ }
+ stopRequested = true;
+ running = false;
+ if (server != null) server.stop();
+ if (triggerThread != null) triggerThread.interrupt();
+ if (purgeThread != null) purgeThread.interrupt();
+ }
+
+ private static InetSocketAddress getAddress(String address) {
+ return NetUtils.createSocketAddr(address);
+ }
+
+ public static InetSocketAddress getAddress(Configuration conf) {
+ String nodeport = conf.get("raid.server.address");
+ if (nodeport == null) {
+ nodeport = "localhost:" + DEFAULT_PORT;
+ }
+ return getAddress(nodeport);
+ }
+
+
+ private void initialize(Configuration conf)
+ throws IOException, SAXException, InterruptedException, RaidConfigurationException,
+ ClassNotFoundException, ParserConfigurationException {
+ this.conf = conf;
+ InetSocketAddress socAddr = RaidNode.getAddress(conf);
+ int handlerCount = conf.getInt("fs.raidnode.handler.count", 10);
+
+ isRaidLocal = conf.getBoolean("fs.raidnode.local", false);
+ // read in the configuration
+ configMgr = new ConfigManager(conf);
+
+ // create rpc server
+ this.server = RPC.getServer(this, socAddr.getHostName(), socAddr.getPort(),
+ handlerCount, false, conf);
+
+ // The rpc-server port can be ephemeral... ensure we have the correct info
+ this.serverAddress = this.server.getListenerAddress();
+ LOG.info("RaidNode up at: " + this.serverAddress);
+
+ initialized = true;
+ running = true;
+ this.server.start(); // start RPC server
+
+ // start the deamon thread to fire polcies appropriately
+ this.triggerThread = new Daemon(new TriggerMonitor());
+ this.triggerThread.start();
+
+ // start the thread that deletes obsolete parity files
+ this.purgeThread = new Daemon(new PurgeMonitor());
+ this.purgeThread.start();
+ }
+
+ /**
+ * Implement RaidProtocol methods
+ */
+
+ /** {@inheritDoc} */
+ public PolicyList[] getAllPolicies() throws IOException {
+ Collection<PolicyList> list = configMgr.getAllPolicies();
+ return list.toArray(new PolicyList[list.size()]);
+ }
+
+ /** {@inheritDoc} */
+ public ReturnStatus recoverFile(String inStr, long corruptOffset) throws IOException {
+
+ LOG.info("Recover File for " + inStr + " for corrupt offset " + corruptOffset);
+ Path inputPath = new Path(inStr);
+ Path srcPath = inputPath.makeQualified(inputPath.getFileSystem(conf));
+ PolicyInfo info = findMatchingPolicy(srcPath);
+ if (info != null) {
+
+ // find stripe length from config
+ int stripeLength = getStripeLength(conf, info);
+
+ // create destination path prefix
+ String destPrefix = getDestinationPath(conf, info);
+ Path destPath = new Path(destPrefix.trim());
+ FileSystem fs = FileSystem.get(destPath.toUri(), conf);
+ destPath = destPath.makeQualified(fs);
+
+ unRaid(conf, srcPath, destPath, stripeLength, corruptOffset);
+ }
+ return ReturnStatus.SUCCESS;
+ }
+
+ /**
+ * Periodically checks to see which policies should be fired.
+ */
+ class TriggerMonitor implements Runnable {
+ /**
+ */
+ public void run() {
+ while (running) {
+ try {
+ doProcess();
+ } catch (Exception e) {
+ LOG.error(StringUtils.stringifyException(e));
+ } finally {
+ LOG.info("Trigger thread continuing to run...");
+ }
+ }
+ }
+
+
+ /**
+ * Keep processing policies.
+ * If the config file has changed, then reload config file and start afresh.
+ */
+ private void doProcess() throws IOException, InterruptedException {
+ PolicyList.CompareByPath lexi = new PolicyList.CompareByPath();
+
+ long prevExec = 0;
+ DistRaid dr = null;
+ while (running) {
+
+ boolean reload = configMgr.reloadConfigsIfNecessary();
+ while(!reload && now() < prevExec + configMgr.getPeriodicity()){
+ Thread.sleep(SLEEP_TIME);
+ reload = configMgr.reloadConfigsIfNecessary();
+ }
+
+ prevExec = now();
+
+ // activate all categories
+ Collection<PolicyList> all = configMgr.getAllPolicies();
+
+ // sort all policies by reverse lexicographical order. This is needed
+ // to make the nearest policy take precedence.
+ PolicyList[] sorted = all.toArray(new PolicyList[all.size()]);
+ Arrays.sort(sorted, lexi);
+
+ if (!isRaidLocal) {
+ dr = new DistRaid(conf);
+ }
+ // paths we have processed so far
+ List<String> processed = new LinkedList<String>();
+
+ for (PolicyList category : sorted) {
+ for (PolicyInfo info: category.getAll()) {
+
+ long modTimePeriod = 0;
+ short srcReplication = 0;
+ String str = info.getProperty("modTimePeriod");
+ if (str != null) {
+ modTimePeriod = Long.parseLong(info.getProperty("modTimePeriod"));
+ }
+ str = info.getProperty("srcReplication");
+ if (str != null) {
+ srcReplication = Short.parseShort(info.getProperty("srcReplication"));
+ }
+
+ LOG.info("Triggering Policy Filter " + info.getName() +
+ " " + info.getSrcPath());
+ List<FileStatus> filteredPaths = null;
+ try {
+ filteredPaths = selectFiles(conf, info.getSrcPath(),
+ getDestinationPath(conf, info),
+ modTimePeriod,
+ srcReplication,
+ prevExec);
+ } catch (Exception e) {
+ LOG.info("Exception while invoking filter on policy " + info.getName() +
+ " srcPath " + info.getSrcPath() +
+ " exception " + StringUtils.stringifyException(e));
+ continue;
+ }
+
+ if (filteredPaths == null || filteredPaths.size() == 0) {
+ LOG.info("No filtered paths for policy " + info.getName());
+ continue;
+ }
+
+ // If any of the filtered path has already been accepted
+ // by a previous policy, then skip it.
+ for (Iterator<FileStatus> iter = filteredPaths.iterator(); iter.hasNext();) {
+ String fs = iter.next().getPath().toString() + "/";
+ for (String p : processed) {
+ if (p.startsWith(fs)) {
+ iter.remove();
+ break;
+ }
+ }
+ }
+
+ // Apply the action on accepted paths
+ LOG.info("Triggering Policy Action " + info.getName());
+ try {
+ if (isRaidLocal){
+ doRaid(conf, info, filteredPaths);
+ }
+ else{
+ //add paths for distributed raiding
+ dr.addRaidPaths(info, filteredPaths);
+ }
+ } catch (Exception e) {
+ LOG.info("Exception while invoking action on policy " + info.getName() +
+ " srcPath " + info.getSrcPath() +
+ " exception " + StringUtils.stringifyException(e));
+ continue;
+ }
+
+ // add these paths to processed paths
+ for (Iterator<FileStatus> iter = filteredPaths.iterator(); iter.hasNext();) {
+ String p = iter.next().getPath().toString() + "/";
+ processed.add(p);
+ }
+ }
+ }
+ processed.clear(); // free up memory references before yielding
+
+ //do the distributed raiding
+ if (!isRaidLocal) {
+ dr.doDistRaid();
+ }
+ }
+ }
+ }
+
+ /**
+ * Returns the policy that matches the specified path.
+ * The method below finds the first policy that matches an input path. Since different
+ * policies with different purposes and destinations might be associated with the same input
+ * path, we should be skeptical about the places using the method and we should try to change
+ * the code to avoid it.
+ */
+ private PolicyInfo findMatchingPolicy(Path inpath) throws IOException {
+ PolicyList.CompareByPath lexi = new PolicyList.CompareByPath();
+ Collection<PolicyList> all = configMgr.getAllPolicies();
+
+ // sort all policies by reverse lexicographical order. This is needed
+ // to make the nearest policy take precedence.
+ PolicyList[] sorted = all.toArray(new PolicyList[all.size()]);
+ Arrays.sort(sorted, lexi);
+
+ // loop through all categories of policies.
+ for (PolicyList category : sorted) {
+ PolicyInfo first = category.getAll().iterator().next();
+ if (first != null) {
+ Path[] srcPaths = first.getSrcPathExpanded(); // input src paths unglobbed
+ if (srcPaths == null) {
+ continue;
+ }
+
+ for (Path src: srcPaths) {
+ if (inpath.toString().startsWith(src.toString())) {
+ // if the srcpath is a prefix of the specified path
+ // we have a match!
+ return first;
+ }
+ }
+ }
+ }
+ return null; // no matching policies
+ }
+
+ /**
+ * Returns a list of pathnames that needs raiding.
+ */
+ private List<FileStatus> selectFiles(Configuration conf, Path p, String destPrefix,
+ long modTimePeriod, short srcReplication, long now) throws IOException {
+
+ List<FileStatus> returnSet = new LinkedList<FileStatus>();
+
+ // expand destination prefix path
+ Path destp = new Path(destPrefix.trim());
+ FileSystem fs = FileSystem.get(destp.toUri(), conf);
+ destp = destp.makeQualified(fs);
+
+ fs = p.getFileSystem(conf);
+ FileStatus[] gpaths = fs.globStatus(p);
+ if (gpaths != null){
+ for (FileStatus onepath: gpaths) {
+ recurse(fs, conf, destp, onepath, returnSet, modTimePeriod, srcReplication, now);
+ }
+ }
+ return returnSet;
+ }
+
+ /**
+ * Pick files that need to be RAIDed.
+ */
+ private void recurse(FileSystem srcFs,
+ Configuration conf,
+ Path destPathPrefix,
+ FileStatus src,
+ List<FileStatus> accept,
+ long modTimePeriod,
+ short srcReplication,
+ long now) throws IOException {
+ Path path = src.getPath();
+ FileStatus[] files = null;
+ try {
+ files = srcFs.listStatus(path);
+ } catch (java.io.FileNotFoundException e) {
+ // ignore error because the file could have been deleted by an user
+ LOG.info("FileNotFound " + path + " " + StringUtils.stringifyException(e));
+ } catch (IOException e) {
+ throw e;
+ }
+
+ // If the modTime of the raid file is later than the modtime of the
+ // src file and the src file has not been modified
+ // recently, then that file is a candidate for RAID.
+
+ if (!src.isDir()) { // a file
+
+ // if the source file has fewer than or equal to 2 blocks, then no need to RAID
+ long blockSize = src.getBlockSize();
+ if (2 * blockSize >= src.getLen()) {
+ return;
+ }
+
+ // If the replication factor of the source file is already below what is
+ // expected by RAID, then ignore it.
+ if (src.getReplication() < srcReplication) {
+ return;
+ }
+
+ // check if destination path already exists. If it does and it's modification time
+ // does not match the modTime of the source file, then recalculate RAID
+ boolean add = false;
+ try {
+ Path outpath = new Path(destPathPrefix, makeRelative(path));
+ FileSystem outFs = outpath.getFileSystem(conf);
+ FileStatus ostat = outFs.getFileStatus(outpath);
+ if (ostat.getModificationTime() != src.getModificationTime() &&
+ src.getModificationTime() + modTimePeriod < now) {
+ add = true;
+ }
+ } catch (java.io.FileNotFoundException e) {
+ add = true; // destination file does not exist
+ }
+
+ if (add) {
+ accept.add(src);
+ }
+ return;
+
+ } else if (files != null) {
+ for (FileStatus one:files) {
+ recurse(srcFs, conf, destPathPrefix, one, accept, modTimePeriod, srcReplication, now);
+ }
+ }
+ }
+
+
+ /**
+ * RAID a list of files.
+ */
+ void doRaid(Configuration conf, PolicyInfo info, List<FileStatus> paths)
+ throws IOException {
+ int targetRepl = Integer.parseInt(info.getProperty("targetReplication"));
+ int metaRepl = Integer.parseInt(info.getProperty("metaReplication"));
+ int stripeLength = getStripeLength(conf, info);
+ String destPrefix = getDestinationPath(conf, info);
+ String simulate = info.getProperty("simulate");
+ boolean doSimulate = simulate == null ? false : Boolean
+ .parseBoolean(simulate);
+
+ Statistics statistics = new Statistics();
+ int count = 0;
+
+ Path p = new Path(destPrefix.trim());
+ FileSystem fs = FileSystem.get(p.toUri(), conf);
+ p = p.makeQualified(fs);
+
+ for (FileStatus s : paths) {
+ doRaid(conf, s, p, statistics, null, doSimulate, targetRepl, metaRepl,
+ stripeLength);
+ if (count % 1000 == 0) {
+ LOG.info("RAID statistics " + statistics.toString());
+ }
+ count++;
+ }
+ LOG.info("RAID statistics " + statistics.toString());
+ }
+
+
+ /**
+ * RAID an individual file
+ */
+
+ static public void doRaid(Configuration conf, PolicyInfo info,
+ FileStatus src, Statistics statistics, Reporter reporter) throws IOException {
+ int targetRepl = Integer.parseInt(info.getProperty("targetReplication"));
+ int metaRepl = Integer.parseInt(info.getProperty("metaReplication"));
+ int stripeLength = getStripeLength(conf, info);
+ String destPrefix = getDestinationPath(conf, info);
+ String simulate = info.getProperty("simulate");
+ boolean doSimulate = simulate == null ? false : Boolean
+ .parseBoolean(simulate);
+
+ int count = 0;
+
+ Path p = new Path(destPrefix.trim());
+ FileSystem fs = FileSystem.get(p.toUri(), conf);
+ p = p.makeQualified(fs);
+
+ doRaid(conf, src, p, statistics, reporter, doSimulate, targetRepl, metaRepl,
+ stripeLength);
+ }
+
+
+ /**
+ * RAID an individual file
+ */
+ static private void doRaid(Configuration conf, FileStatus stat, Path destPath,
+ Statistics statistics, Reporter reporter, boolean doSimulate,
+ int targetRepl, int metaRepl, int stripeLength)
+ throws IOException {
+ Path p = stat.getPath();
+ FileSystem srcFs = p.getFileSystem(conf);
+
+ // extract block locations from File system
+ BlockLocation[] locations = srcFs.getFileBlockLocations(stat, 0, stat.getLen());
+
+ // if the file has fewer than 2 blocks, then nothing to do
+ if (locations.length <= 2) {
+ return;
+ }
+
+ // add up the raw disk space occupied by this file
+ long diskSpace = 0;
+ for (BlockLocation l: locations) {
+ diskSpace += (l.getLength() * stat.getReplication());
+ }
+ statistics.numProcessedBlocks += locations.length;
+ statistics.processedSize += diskSpace;
+
+ // generate parity file
+ generateParityFile(conf, stat, reporter, srcFs, destPath, locations, metaRepl, stripeLength);
+
+ // reduce the replication factor of the source file
+ if (!doSimulate) {
+ if (srcFs.setReplication(p, (short)targetRepl) == false) {
+ LOG.info("Error in reducing relication factor of file " + p + " to " + targetRepl);
+ statistics.remainingSize += diskSpace; // no change in disk space usage
+ return;
+ }
+ }
+
+ diskSpace = 0;
+ for (BlockLocation l: locations) {
+ diskSpace += (l.getLength() * targetRepl);
+ }
+ statistics.remainingSize += diskSpace;
+
+ // the metafile will have this many number of blocks
+ int numMeta = locations.length / stripeLength;
+ if (locations.length % stripeLength != 0) {
+ numMeta++;
+ }
+
+ // we create numMeta for every file. This metablock has metaRepl # replicas.
+ // the last block of the metafile might not be completely filled up, but we
+ // ignore that for now.
+ statistics.numMetaBlocks += (numMeta * metaRepl);
+ statistics.metaSize += (numMeta * metaRepl * stat.getBlockSize());
+ }
+
+ /**
+ * Create the parity file.
+ */
+ static private void generateParityFile(Configuration conf, FileStatus stat,
+ Reporter reporter,
+ FileSystem inFs,
+ Path destPathPrefix, BlockLocation[] locations,
+ int metaRepl, int stripeLength) throws IOException {
+
+ // two buffers for generating parity
+ int bufSize = 5 * 1024 * 1024; // 5 MB
+ byte[] bufs = new byte[bufSize];
+ byte[] xor = new byte[bufSize];
+ byte zero = 0;
+
+ Path inpath = stat.getPath();
+ long blockSize = stat.getBlockSize();
+ long fileSize = stat.getLen();
+
+ // create output tmp path
+ Path outpath = new Path(destPathPrefix, makeRelative(inpath));
+ Path tmppath = new Path(outpath + ".tmp");
+ FileSystem outFs = outpath.getFileSystem(conf);
+
+ // if the parity file is already upto-date, then nothing to do
+ try {
+ FileStatus stmp = outFs.getFileStatus(outpath);
+ if (stmp.getModificationTime() == stat.getModificationTime()) {
+ LOG.info("Parity file for " + inpath + "(" + locations.length + ") is " + outpath +
+ " already upto-date. Nothing more to do.");
+ }
+ } catch (IOException e) {
+ // ignore errors because the raid file might not exist yet.
+ }
+
+ LOG.info("Parity file for " + inpath + "(" + locations.length + ") is " + outpath);
+ FSDataOutputStream out = outFs.create(tmppath,
+ true,
+ conf.getInt("io.file.buffer.size", 64 * 1024),
+ (short)metaRepl,
+ blockSize);
+
+ try {
+
+ // loop once for every stripe length
+ for (int startBlock = 0; startBlock < locations.length;) {
+
+ // report progress to Map-reduce framework
+ if (reporter != null) {
+ reporter.progress();
+ }
+ int blocksLeft = locations.length - startBlock;
+ int stripe = Math.min(stripeLength, blocksLeft);
+ LOG.info(" startBlock " + startBlock + " stripe " + stripe);
+
+ // open a new file descriptor for each block in this stripe.
+ // make each fd point to the beginning of each block in this stripe.
+ FSDataInputStream[] ins = new FSDataInputStream[stripe];
+ for (int i = 0; i < stripe; i++) {
+ ins[i] = inFs.open(inpath, bufSize);
+ ins[i].seek(blockSize * (startBlock + i));
+ }
+
+ generateParity(ins,out,blockSize,bufs,xor, reporter);
+
+ // close input file handles
+ for (int i = 0; i < ins.length; i++) {
+ ins[i].close();
+ }
+
+ // increment startBlock to point to the first block to be processed
+ // in the next iteration
+ startBlock += stripe;
+ }
+ out.close();
+ out = null;
+
+ // rename tmppath to the real parity filename
+ if (!outFs.rename(tmppath, outpath)) {
+ String msg = "Unable to rename tmp file " + tmppath + " to " + outpath;
+ LOG.warn(msg);
+ throw new IOException (msg);
+ }
+ } finally {
+ // remove the tmp file if it still exists
+ outFs.delete(tmppath, false);
+ }
+
+ // set the modification time of the RAID file. This is done so that the modTime of the
+ // RAID file reflects that contents of the source file that it has RAIDed. This should
+ // also work for files that are being appended to. This is necessary because the time on
+ // on the destination namenode may not be synchronised with the timestamp of the
+ // source namenode.
+ outFs.setTimes(outpath, stat.getModificationTime(), -1);
+
+ FileStatus outstat = outFs.getFileStatus(outpath);
+ LOG.info("Source file " + inpath + " of size " + fileSize +
+ " Parity file " + outpath + " of size " + outstat.getLen() +
+ " src mtime " + stat.getModificationTime() +
+ " parity mtime " + outstat.getModificationTime());
+ }
+
+ private static int readInputUntilEnd(FSDataInputStream ins, byte[] bufs, int toRead)
+ throws IOException {
+
+ int tread = 0;
+
+ while (tread < toRead) {
+ int read = ins.read(bufs, tread, toRead - tread);
+ if (read == -1) {
+ return tread;
+ } else {
+ tread += read;
+ }
+ }
+
+ return tread;
+ }
+
+ private static void generateParity(FSDataInputStream[] ins, FSDataOutputStream fout,
+ long parityBlockSize, byte[] bufs, byte[] xor, Reporter reporter) throws IOException {
+
+ int bufSize;
+ if ((bufs == null) || (bufs.length == 0)){
+ bufSize = 5 * 1024 * 1024; // 5 MB
+ bufs = new byte[bufSize];
+ } else {
+ bufSize = bufs.length;
+ }
+ if ((xor == null) || (xor.length != bufs.length)){
+ xor = new byte[bufSize];
+ }
+
+ int xorlen = 0;
+
+ // this loop processes all good blocks in selected stripe
+ long remaining = parityBlockSize;
+
+ while (remaining > 0) {
+ int toRead = (int)Math.min(remaining, bufSize);
+
+ if (ins.length > 0) {
+ xorlen = readInputUntilEnd(ins[0], xor, toRead);
+ }
+
+ // read all remaining blocks and xor them into the buffer
+ for (int i = 1; i < ins.length; i++) {
+
+ // report progress to Map-reduce framework
+ if (reporter != null) {
+ reporter.progress();
+ }
+
+ int actualRead = readInputUntilEnd(ins[i], bufs, toRead);
+
+ int j;
+ int xorlimit = (int) Math.min(xorlen,actualRead);
+ for (j = 0; j < xorlimit; j++) {
+ xor[j] ^= bufs[j];
+ }
+ if ( actualRead > xorlen ){
+ for (; j < actualRead; j++) {
+ xor[j] = bufs[j];
+ }
+ xorlen = actualRead;
+ }
+
+ }
+
+ if (xorlen < toRead) {
+ Arrays.fill(bufs, xorlen, toRead, (byte) 0);
+ }
+
+ // write this to the tmp file
+ fout.write(xor, 0, toRead);
+ remaining -= toRead;
+ }
+
+ }
+
+ /**
+ * Extract a good block from the parity block. This assumes that the corruption
+ * is in the main file and the parity file is always good.
+ */
+ public static Path unRaid(Configuration conf, Path srcPath, Path destPathPrefix,
+ int stripeLength, long corruptOffset) throws IOException {
+
+ // extract block locations, size etc from source file
+ Random rand = new Random();
+ FileSystem srcFs = srcPath.getFileSystem(conf);
+ FileStatus srcStat = srcFs.getFileStatus(srcPath);
+ BlockLocation[] locations = srcFs.getFileBlockLocations(srcStat, 0, srcStat.getLen());
+ long blockSize = srcStat.getBlockSize();
+ long fileSize = srcStat.getLen();
+ int totalBlocks = locations.length;
+
+ // find the stripe number where the corrupted offset lies
+ long snum = corruptOffset / (stripeLength * blockSize);
+ long startOffset = snum * stripeLength * blockSize;
+ long corruptBlockInStripe = (corruptOffset - startOffset)/blockSize;
+ long corruptBlockSize = Math.min(blockSize, fileSize - startOffset);
+
+ LOG.info("Start offset of relevent stripe = " + startOffset +
+ " corruptBlockInStripe " + corruptBlockInStripe);
+
+ // open file descriptors to read all good blocks of the file
+ FSDataInputStream[] instmp = new FSDataInputStream[stripeLength];
+ int numLength = 0;
+ for (int i = 0; i < stripeLength; i++) {
+ if (i == corruptBlockInStripe) {
+ continue; // do not open corrupt block
+ }
+ if (startOffset + i * blockSize >= fileSize) {
+ LOG.info("Stop offset of relevent stripe = " +
+ startOffset + i * blockSize);
+ break;
+ }
+ instmp[numLength] = srcFs.open(srcPath);
+ instmp[numLength].seek(startOffset + i * blockSize);
+ numLength++;
+ }
+
+ // create array of inputstream, allocate one extra slot for
+ // parity file. numLength could be smaller than stripeLength
+ // if we are processing the last partial stripe on a file.
+ numLength += 1;
+ FSDataInputStream[] ins = new FSDataInputStream[numLength];
+ for (int i = 0; i < numLength-1; i++) {
+ ins[i] = instmp[i];
+ }
+ LOG.info("Decompose a total of " + numLength + " blocks.");
+
+ // open and seek to the appropriate offset in parity file.
+ Path parityFile = new Path(destPathPrefix, makeRelative(srcPath));
+ FileSystem parityFs = parityFile.getFileSystem(conf);
+ LOG.info("Parity file for " + srcPath + " is " + parityFile);
+ ins[numLength-1] = parityFs.open(parityFile);
+ ins[numLength-1].seek(snum * blockSize);
+ LOG.info("Parity file " + parityFile +
+ " seeking to relevent block at offset " +
+ ins[numLength-1].getPos());
+
+ // create a temporary filename in the source filesystem
+ // do not overwrite an existing tmp file. Make it fail for now.
+ // We need to generate a unique name for this tmp file later on.
+ Path tmpFile = null;
+ FSDataOutputStream fout = null;
+ int retry = 5;
+ try {
+ tmpFile = new Path("/tmp/dhruba/" + rand.nextInt());
+ fout = parityFs.create(tmpFile, false);
+ } catch (IOException e) {
+ if (retry-- <= 0) {
+ LOG.info("Unable to create temporary file " + tmpFile +
+ " Aborting....");
+ throw e;
+ }
+ LOG.info("Unable to create temporary file " + tmpFile +
+ "Retrying....");
+ }
+ LOG.info("Created recovered block file " + tmpFile);
+
+ // buffers for generating parity bits
+ int bufSize = 5 * 1024 * 1024; // 5 MB
+ byte[] bufs = new byte[bufSize];
+ byte[] xor = new byte[bufSize];
+
+ generateParity(ins,fout,corruptBlockSize,bufs,xor,null);
+
+ // close all files
+ fout.close();
+ for (int i = 0; i < ins.length; i++) {
+ ins[i].close();
+ }
+
+ // Now, reopen the source file and the recovered block file
+ // and copy all relevant data to new file
+ Path recoveredPath = new Path(destPathPrefix, makeRelative(srcPath));
+ recoveredPath = new Path(recoveredPath + ".recovered");
+ LOG.info("Creating recovered file " + recoveredPath);
+
+ FSDataInputStream sin = srcFs.open(srcPath);
+ FSDataOutputStream out = parityFs.create(recoveredPath, false,
+ conf.getInt("io.file.buffer.size", 64 * 1024),
+ srcStat.getReplication(),
+ srcStat.getBlockSize());
+
+ FSDataInputStream bin = parityFs.open(tmpFile);
+ long recoveredSize = 0;
+
+ // copy all the good blocks (upto the corruption)
+ // from source file to output file
+ long remaining = corruptOffset / blockSize * blockSize;
+ while (remaining > 0) {
+ int toRead = (int)Math.min(remaining, bufSize);
+ sin.readFully(bufs, 0, toRead);
+ out.write(bufs, 0, toRead);
+ remaining -= toRead;
+ recoveredSize += toRead;
+ }
+ LOG.info("Copied upto " + recoveredSize + " from src file. ");
+
+ // copy recovered block to output file
+ remaining = corruptBlockSize;
+ while (recoveredSize < fileSize &&
+ remaining > 0) {
+ int toRead = (int)Math.min(remaining, bufSize);
+ bin.readFully(bufs, 0, toRead);
+ out.write(bufs, 0, toRead);
+ remaining -= toRead;
+ recoveredSize += toRead;
+ }
+ LOG.info("Copied upto " + recoveredSize + " from recovered-block file. ");
+
+ // skip bad block in src file
+ if (recoveredSize < fileSize) {
+ sin.seek(sin.getPos() + corruptBlockSize);
+ }
+
+ // copy remaining good data from src file to output file
+ while (recoveredSize < fileSize) {
+ int toRead = (int)Math.min(fileSize - recoveredSize, bufSize);
+ sin.readFully(bufs, 0, toRead);
+ out.write(bufs, 0, toRead);
+ recoveredSize += toRead;
+ }
+ out.close();
+ LOG.info("Completed writing " + recoveredSize + " bytes into " +
+ recoveredPath);
+
+ sin.close();
+ bin.close();
+
+ // delete the temporary block file that was created.
+ parityFs.delete(tmpFile, false);
+ LOG.info("Deleted temporary file " + tmpFile);
+
+ // copy the meta information from source path to the newly created
+ // recovered path
+ copyMetaInformation(parityFs, srcStat, recoveredPath);
+
+ return recoveredPath;
+ }
+
+ /**
+ * Periodically delete orphaned parity files.
+ */
+ class PurgeMonitor implements Runnable {
+ /**
+ */
+ public void run() {
+ while (running) {
+ try {
+ doPurge();
+ } catch (Exception e) {
+ LOG.error(StringUtils.stringifyException(e));
+ } finally {
+ LOG.info("Purge parity files thread continuing to run...");
+ }
+ }
+ }
+
+ /**
+ * Delete orphaned files. The reason this is done by a separate thread
+ * is to not burden the TriggerMonitor with scanning the
+ * destination directories.
+ */
+ private void doPurge() throws IOException, InterruptedException {
+ PolicyList.CompareByPath lexi = new PolicyList.CompareByPath();
+
+ long prevExec = 0;
+ while (running) {
+
+ // The config may be reloaded by the TriggerMonitor.
+ // This thread uses whatever config is currently active.
+ while(now() < prevExec + configMgr.getPeriodicity()){
+ Thread.sleep(SLEEP_TIME);
+ }
+
+ prevExec = now();
+
+ // fetch all categories
+ Collection<PolicyList> all = configMgr.getAllPolicies();
+
+ // sort all policies by reverse lexicographical order. This is
+ // needed to make the nearest policy take precedence.
+ PolicyList[] sorted = all.toArray(new PolicyList[all.size()]);
+ Arrays.sort(sorted, lexi);
+
+ // paths we have processed so far
+ Set<Path> processed = new HashSet<Path>();
+
+ for (PolicyList category : sorted) {
+ for (PolicyInfo info: category.getAll()) {
+
+ try {
+ // expand destination prefix path
+ String destinationPrefix = getDestinationPath(conf, info);
+ Path destp = new Path(destinationPrefix.trim());
+ FileSystem destFs = FileSystem.get(destp.toUri(), conf);
+ destp = destp.makeQualified(destFs);
+ destinationPrefix = destp.toUri().getPath(); // strip host:port
+ destp = new Path(destp, makeRelative(info.getSrcPath()));
+
+ // if this destination path has already been processed as part
+ // of another policy, then nothing more to do
+ if (processed.contains(destp)) {
+ LOG.info("Obsolete parity files for policy " +
+ info.getName() + " has already been procesed.");
+ continue;
+ }
+
+ FileSystem srcFs = info.getSrcPath().getFileSystem(conf);
+ FileStatus stat = null;
+ try {
+ stat = destFs.getFileStatus(destp);
+ } catch (FileNotFoundException e) {
+ // do nothing, leave stat = null;
+ }
+ if (stat != null) {
+ LOG.info("Purging obsolete parity files for policy " +
+ info.getName() + " " + destp);
+ recursePurge(srcFs, destFs, destinationPrefix, stat);
+ }
+
+ // this destination path has already been processed
+ processed.add(destp);
+
+ } catch (Exception e) {
+ LOG.warn("Ignoring Exception while processing policy " +
+ info.getName() + " " +
+ StringUtils.stringifyException(e));
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * The destPrefix is the absolute pathname of the destinationPath
+ * specified in the policy (without the host:port)
+ */
+ private void recursePurge(FileSystem srcFs, FileSystem destFs,
+ String destPrefix, FileStatus dest)
+ throws IOException {
+
+ Path destPath = dest.getPath(); // pathname, no host:port
+ String destStr = destPath.toUri().getPath();
+ LOG.debug("Checking " + destPath + " prefix " + destPrefix);
+
+ // Verify the destPrefix is a prefix of the destPath
+ if (!destStr.startsWith(destPrefix)) {
+ LOG.error("Destination path " + destStr + " should have " +
+ destPrefix + " as its prefix.");
+ return;
+ }
+ String src = destStr.replaceFirst(destPrefix, "");
+
+ // if the source path does not exist, then delete the
+ // destination path as well
+ Path srcPath = new Path(src);
+ if (!srcFs.exists(srcPath)) {
+ boolean done = destFs.delete(destPath, false);
+ if (done) {
+ LOG.info("Purged path " + destPath );
+ } else {
+ LOG.info("Unable to purge path " + destPath );
+ }
+ return;
+ }
+ if (!dest.isDir()) {
+ return;
+ }
+ FileStatus[] files = null;
+ files = destFs.listStatus(destPath);
+ if (files != null) {
+ for (FileStatus one:files) {
+ recursePurge(srcFs, destFs, destPrefix, one);
+ }
+ }
+ }
+ }
+
+ /**
+ * If the config file has an entry for hdfs.raid.locations, then that overrides
+ * destination path specified in the raid policy file
+ */
+ static private String getDestinationPath(Configuration conf, PolicyInfo info) {
+ String locs = conf.get("hdfs.raid.locations");
+ if (locs != null) {
+ return locs;
+ }
+ locs = info.getDestinationPath();
+ if (locs == null) {
+ return DEFAULT_RAID_LOCATION;
+ }
+ return locs;
+ }
+
+ /**
+ * If the config file has an entry for hdfs.raid.stripeLength, then use that
+ * specified in the raid policy file
+ */
+ static private int getStripeLength(Configuration conf, PolicyInfo info)
+ throws IOException {
+ int len = conf.getInt("hdfs.raid.stripeLength", 0);
+ if (len != 0) {
+ return len;
+ }
+ String str = info.getProperty("stripeLength");
+ if (str == null) {
+ String msg = "hdfs.raid.stripeLength is not defined." +
+ " Using a default " + DEFAULT_STRIPE_LENGTH;
+ LOG.info(msg);
+ return DEFAULT_STRIPE_LENGTH;
+ }
+ return Integer.parseInt(str);
+ }
+
+ /**
+ * Copy the file owner, modtime, etc from srcPath to the recovered Path.
+ * It is possiible that we might have to retrieve file persmissions,
+ * quotas, etc too in future.
+ */
+ static private void copyMetaInformation(FileSystem fs, FileStatus stat,
+ Path recoveredPath)
+ throws IOException {
+ fs.setOwner(recoveredPath, stat.getOwner(), stat.getGroup());
+ fs.setPermission(recoveredPath, stat.getPermission());
+ fs.setTimes(recoveredPath, stat.getModificationTime(), stat.getAccessTime());
+ }
+
+ /**
+ * Returns current time.
+ */
+ static long now() {
+ return System.currentTimeMillis();
+ }
+
+ /**
+ * Make an absolute path relative by stripping the leading /
+ */
+ static private Path makeRelative(Path path) {
+ if (!path.isAbsolute()) {
+ return path;
+ }
+ String p = path.toUri().getPath();
+ String relative = p.substring(1, p.length());
+ return new Path(relative);
+ }
+
+ private static void printUsage() {
+ System.err.println("Usage: java RaidNode ");
+ }
+
+ private static StartupOption parseArguments(String args[]) {
+ int argsLen = (args == null) ? 0 : args.length;
+ StartupOption startOpt = StartupOption.REGULAR;
+ for(int i=0; i < argsLen; i++) {
+ String cmd = args[i]; // We have to parse command line args in future.
+ }
+ return startOpt;
+ }
+
+
+ /**
+ * Convert command line options to configuration parameters
+ */
+ private static void setStartupOption(Configuration conf, StartupOption opt) {
+ conf.set("fs.raidnode.startup", opt.toString());
+ }
+
+ /**
+ * Create an instance of the RaidNode
+ */
+ public static RaidNode createRaidNode(String argv[],
+ Configuration conf) throws IOException {
+ if (conf == null) {
+ conf = new Configuration();
+ }
+ StartupOption startOpt = parseArguments(argv);
+ if (startOpt == null) {
+ printUsage();
+ return null;
+ }
+ setStartupOption(conf, startOpt);
+ RaidNode node = new RaidNode(conf);
+ return node;
+ }
+
+
+ /**
+ */
+ public static void main(String argv[]) throws Exception {
+ try {
+ StringUtils.startupShutdownMessage(RaidNode.class, argv, LOG);
+ RaidNode raid = createRaidNode(argv, null);
+ if (raid != null) {
+ raid.join();
+ }
+ } catch (Throwable e) {
+ LOG.error(StringUtils.stringifyException(e));
+ System.exit(-1);
+ }
+ }
+
+
+
+}
diff --git a/src/contrib/raid/src/java/org/apache/hadoop/raid/RaidShell.java b/src/contrib/raid/src/java/org/apache/hadoop/raid/RaidShell.java
new file mode 100644
index 0000000..1f6f239
--- /dev/null
+++ b/src/contrib/raid/src/java/org/apache/hadoop/raid/RaidShell.java
@@ -0,0 +1,299 @@
+/**
+ * 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.raid;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.concurrent.TimeUnit;
+import java.net.InetSocketAddress;
+import javax.security.auth.login.LoginException;
+
+import org.apache.hadoop.ipc.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hadoop.raid.protocol.PolicyInfo;
+import org.apache.hadoop.raid.protocol.PolicyList;
+import org.apache.hadoop.raid.protocol.RaidProtocol;
+import org.apache.hadoop.raid.protocol.RaidProtocol.ReturnStatus;
+
+/**
+ * A {@link RaidShell} that allows browsing configured raid policies.
+ */
+public class RaidShell extends Configured implements Tool {
+ public static final Log LOG = LogFactory.getLog( "org.apache.hadoop.RaidShell");
+ public RaidProtocol raidnode;
+ final RaidProtocol rpcRaidnode;
+ private UnixUserGroupInformation ugi;
+ volatile boolean clientRunning = true;
+ private Configuration conf;
+
+ /**
+ * Start RaidShell.
+ * <p>
+ * The RaidShell connects to the specified RaidNode and performs basic
+ * configuration options.
+ * @throws IOException
+ */
+ public RaidShell() throws IOException {
+ this(new Configuration());
+ }
+
+ /**
+ * The RaidShell connects to the specified RaidNode and performs basic
+ * configuration options.
+ * @param conf The Hadoop configuration
+ * @throws IOException
+ */
+ public RaidShell(Configuration conf) throws IOException {
+ super(conf);
+ try {
+ this.ugi = UnixUserGroupInformation.login(conf, true);
+ } catch (LoginException e) {
+ throw (IOException)(new IOException().initCause(e));
+ }
+
+ this.rpcRaidnode = createRPCRaidnode(RaidNode.getAddress(conf), conf, ugi);
+ this.raidnode = createRaidnode(rpcRaidnode);
+ }
+
+ public static RaidProtocol createRaidnode(Configuration conf) throws IOException {
+ return createRaidnode(RaidNode.getAddress(conf), conf);
+ }
+
+ public static RaidProtocol createRaidnode(InetSocketAddress raidNodeAddr,
+ Configuration conf) throws IOException {
+ try {
+ return createRaidnode(createRPCRaidnode(raidNodeAddr, conf,
+ UnixUserGroupInformation.login(conf, true)));
+ } catch (LoginException e) {
+ throw (IOException)(new IOException().initCause(e));
+ }
+ }
+
+ private static RaidProtocol createRPCRaidnode(InetSocketAddress raidNodeAddr,
+ Configuration conf, UnixUserGroupInformation ugi)
+ throws IOException {
+ LOG.info("RaidShell connecting to " + raidNodeAddr);
+ return (RaidProtocol)RPC.getProxy(RaidProtocol.class,
+ RaidProtocol.versionID, raidNodeAddr, ugi, conf,
+ NetUtils.getSocketFactory(conf, RaidProtocol.class));
+ }
+
+ private static RaidProtocol createRaidnode(RaidProtocol rpcRaidnode)
+ throws IOException {
+ RetryPolicy createPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
+ 5, 5000, TimeUnit.MILLISECONDS);
+
+ Map<Class<? extends Exception>,RetryPolicy> remoteExceptionToPolicyMap =
+ new HashMap<Class<? extends Exception>, RetryPolicy>();
+
+ Map<Class<? extends Exception>,RetryPolicy> exceptionToPolicyMap =
+ new HashMap<Class<? extends Exception>, RetryPolicy>();
+ exceptionToPolicyMap.put(RemoteException.class,
+ RetryPolicies.retryByRemoteException(
+ RetryPolicies.TRY_ONCE_THEN_FAIL, remoteExceptionToPolicyMap));
+ RetryPolicy methodPolicy = RetryPolicies.retryByException(
+ RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+ Map<String,RetryPolicy> methodNameToPolicyMap = new HashMap<String,RetryPolicy>();
+
+ methodNameToPolicyMap.put("create", methodPolicy);
+
+ return (RaidProtocol) RetryProxy.create(RaidProtocol.class,
+ rpcRaidnode, methodNameToPolicyMap);
+ }
+
+ private void checkOpen() throws IOException {
+ if (!clientRunning) {
+ IOException result = new IOException("RaidNode closed");
+ throw result;
+ }
+ }
+
+ /**
+ * Close the connection to the raidNode.
+ */
+ public synchronized void close() throws IOException {
+ if(clientRunning) {
+ clientRunning = false;
+ RPC.stopProxy(rpcRaidnode);
+ }
+ }
+
+ /**
+ * Displays format of commands.
+ */
+ private static void printUsage(String cmd) {
+ String prefix = "Usage: java " + RaidShell.class.getSimpleName();
+ if ("-showConfig".equals(cmd)) {
+ System.err.println("Usage: java RaidShell" +
+ " [-showConfig]");
+ } else if ("-recover".equals(cmd)) {
+ System.err.println("Usage: java CronShell" +
+ " [-recover srcPath1 corruptOffset]");
+ } else {
+ System.err.println("Usage: java RaidShell");
+ System.err.println(" [-showConfig ]");
+ System.err.println(" [-help [cmd]]");
+ System.err.println(" [-recover srcPath1 corruptOffset]");
+ System.err.println();
+ ToolRunner.printGenericCommandUsage(System.err);
+ }
+ }
+
+ /**
+ * run
+ */
+ public int run(String argv[]) throws Exception {
+
+ if (argv.length < 1) {
+ printUsage("");
+ return -1;
+ }
+
+ int exitCode = -1;
+ int i = 0;
+ String cmd = argv[i++];
+ //
+ // verify that we have enough command line parameters
+ //
+ if ("-showConfig".equals(cmd)) {
+ if (argv.length < 1) {
+ printUsage(cmd);
+ return exitCode;
+ }
+ } else if ("-recover".equals(cmd)) {
+ if (argv.length < 3) {
+ printUsage(cmd);
+ return exitCode;
+ }
+ }
+
+ try {
+ if ("-showConfig".equals(cmd)) {
+ exitCode = showConfig(cmd, argv, i);
+ } else if ("-recover".equals(cmd)) {
+ exitCode = recover(cmd, argv, i);
+ } else {
+ exitCode = -1;
+ System.err.println(cmd.substring(1) + ": Unknown command");
+ printUsage("");
+ }
+ } catch (IllegalArgumentException arge) {
+ exitCode = -1;
+ System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
+ printUsage(cmd);
+ } catch (RemoteException e) {
+ //
+ // This is a error returned by raidnode server. Print
+ // out the first line of the error mesage, ignore the stack trace.
+ exitCode = -1;
+ try {
+ String[] content;
+ content = e.getLocalizedMessage().split("\n");
+ System.err.println(cmd.substring(1) + ": " +
+ content[0]);
+ } catch (Exception ex) {
+ System.err.println(cmd.substring(1) + ": " +
+ ex.getLocalizedMessage());
+ }
+ } catch (IOException e) {
+ //
+ // IO exception encountered locally.
+ //
+ exitCode = -1;
+ System.err.println(cmd.substring(1) + ": " +
+ e.getLocalizedMessage());
+ } catch (Exception re) {
+ exitCode = -1;
+ System.err.println(cmd.substring(1) + ": " + re.getLocalizedMessage());
+ } finally {
+ }
+ return exitCode;
+ }
+
+ /**
+ * Apply operation specified by 'cmd' on all parameters
+ * starting from argv[startindex].
+ */
+ private int showConfig(String cmd, String argv[], int startindex) throws IOException {
+ int exitCode = 0;
+ int i = startindex;
+ PolicyList[] all = raidnode.getAllPolicies();
+ for (PolicyList list: all) {
+ for (PolicyInfo p : list.getAll()) {
+ System.out.println(p);
+ }
+ }
+ return exitCode;
+ }
+
+ /**
+ * Recovers the specified path from the parity file
+ */
+ public int recover(String cmd, String argv[], int startindex)
+ throws IOException {
+ int exitCode = 0;
+ String[] paths = new String[argv.length - startindex];
+ for (int i = startindex; i < argv.length; i = i + 2) {
+ String path = argv[i];
+ long corruptOffset = Long.parseLong(argv[i+1]);
+ LOG.info("RaidShell recoverFile for " + path + " corruptOffset " + corruptOffset);
+ raidnode.recoverFile(path, corruptOffset);
+ }
+ return 0;
+ }
+
+ /**
+ * main() has some simple utility methods
+ */
+ public static void main(String argv[]) throws Exception {
+ RaidShell shell = null;
+ try {
+ shell = new RaidShell();
+ } catch (RPC.VersionMismatch v) {
+ System.err.println("Version Mismatch between client and server" +
+ "... command aborted.");
+ System.exit(-1);
+ } catch (IOException e) {
+ System.err.println("Bad connection to RaidNode. command aborted.");
+ System.exit(-1);
+ }
+
+ int res;
+ try {
+ res = ToolRunner.run(shell, argv);
+ } finally {
+ shell.close();
+ }
+ System.exit(res);
+ }
+}
diff --git a/src/contrib/raid/src/java/org/apache/hadoop/raid/protocol/PolicyInfo.java b/src/contrib/raid/src/java/org/apache/hadoop/raid/protocol/PolicyInfo.java
new file mode 100644
index 0000000..442ca7f
--- /dev/null
+++ b/src/contrib/raid/src/java/org/apache/hadoop/raid/protocol/PolicyInfo.java
@@ -0,0 +1,225 @@
+/**
+ * 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.raid.protocol;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.util.Date;
+import java.util.List;
+import java.util.Properties;
+import java.util.Enumeration;
+import java.lang.Math;
+import java.lang.Class;
+import java.text.SimpleDateFormat;
+import java.util.StringTokenizer;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.raid.protocol.RaidProtocol.ReturnStatus;
+
+/**
+ * Maintains information about one policy
+ */
+public class PolicyInfo implements Writable {
+ public static final Log LOG = LogFactory.getLog(
+ "org.apache.hadoop.raid.protocol.PolicyInfo");
+ protected static final SimpleDateFormat dateFormat =
+ new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+ private Path srcPath; // the specified src path
+ private String policyName; // name of policy
+ private String destinationPath; // A destination path for this policy
+ private String description; // A verbose description of this policy
+ private Configuration conf; // Hadoop configuration
+
+ private Properties properties; // Policy-dependent properties
+
+ private ReentrantReadWriteLock plock; // protects policy operations.
+
+ /**
+ * Create the empty object
+ */
+ public PolicyInfo() {
+ this.conf = null;
+ this.policyName = "";
+ this.description = "";
+ this.srcPath = null;
+ this.properties = new Properties();
+ this.plock = new ReentrantReadWriteLock();
+ }
+
+ /**
+ * Create the metadata that describes a policy
+ */
+ public PolicyInfo(String policyName, Configuration conf) {
+ this.conf = conf;
+ this.policyName = policyName;
+ this.description = "";
+ this.srcPath = null;
+ this.properties = new Properties();
+ this.plock = new ReentrantReadWriteLock();
+ }
+
+ /**
+ * Sets the input path on which this policy has to be applied
+ */
+ public void setSrcPath(String in) throws IOException {
+ srcPath = new Path(in);
+ srcPath = srcPath.makeQualified(srcPath.getFileSystem(conf));
+ }
+
+ /**
+ * Set the destination path of this policy.
+ */
+ public void setDestinationPath(String des) {
+ this.destinationPath = des;
+ }
+
+ /**
+ * Set the description of this policy.
+ */
+ public void setDescription(String des) {
+ this.description = des;
+ }
+
+ /**
+ * Sets an internal property.
+ * @param name property name.
+ * @param value property value.
+ */
+ public void setProperty(String name, String value) {
+ properties.setProperty(name, value);
+ }
+
+ /**
+ * Returns the value of an internal property.
+ * @param name property name.
+ */
+ public String getProperty(String name) {
+ return properties.getProperty(name);
+ }
+
+ /**
+ * Get the name of this policy.
+ */
+ public String getName() {
+ return this.policyName;
+ }
+
+ /**
+ * Get the destination path of this policy.
+ */
+ public String getDestinationPath() {
+ return this.destinationPath;
+ }
+
+ /**
+ * Get the srcPath
+ */
+ public Path getSrcPath() throws IOException {
+ return srcPath;
+ }
+
+ /**
+ * Get the expanded (unglobbed) forms of the srcPaths
+ */
+ public Path[] getSrcPathExpanded() throws IOException {
+ FileSystem fs = srcPath.getFileSystem(conf);
+
+ // globbing on srcPath
+ FileStatus[] gpaths = fs.globStatus(srcPath);
+ if (gpaths == null) {
+ return null;
+ }
+ Path[] values = new Path[gpaths.length];
+ for (int i = 0; i < gpaths.length; i++) {
+ Path p = gpaths[i].getPath();
+ values[i] = p.makeQualified(fs);
+ }
+ return values;
+ }
+
+ /**
+ * Convert this policy into a printable form
+ */
+ public String toString() {
+ StringBuffer buff = new StringBuffer();
+ buff.append("Policy Name:\t" + policyName + " --------------------\n");
+ buff.append("Source Path:\t" + srcPath + "\n");
+ buff.append("Dest Path:\t" + destinationPath + "\n");
+ for (Enumeration<?> e = properties.propertyNames(); e.hasMoreElements();) {
+ String name = (String) e.nextElement();
+ buff.append( name + ":\t" + properties.getProperty(name) + "\n");
+ }
+ if (description.length() > 0) {
+ int len = Math.min(description.length(), 80);
+ String sub = description.substring(0, len).trim();
+ sub = sub.replaceAll("\n", " ");
+ buff.append("Description:\t" + sub + "...\n");
+ }
+ return buff.toString();
+ }
+
+ //////////////////////////////////////////////////
+ // Writable
+ //////////////////////////////////////////////////
+ static { // register a ctor
+ WritableFactories.setFactory
+ (PolicyInfo.class,
+ new WritableFactory() {
+ public Writable newInstance() { return new PolicyInfo(); }
+ });
+ }
+
+ public void write(DataOutput out) throws IOException {
+ Text.writeString(out, srcPath.toString());
+ Text.writeString(out, policyName);
+ Text.writeString(out, destinationPath);
+ Text.writeString(out, description);
+ out.writeInt(properties.size());
+ for (Enumeration<?> e = properties.propertyNames(); e.hasMoreElements();) {
+ String name = (String) e.nextElement();
+ Text.writeString(out, name);
+ Text.writeString(out, properties.getProperty(name));
+ }
+ }
+
+ public void readFields(DataInput in) throws IOException {
+ this.srcPath = new Path(Text.readString(in));
+ this.policyName = Text.readString(in);
+ this.destinationPath = Text.readString(in);
+ this.description = Text.readString(in);
+ for (int n = in.readInt(); n>0; n--) {
+ String name = Text.readString(in);
+ String value = Text.readString(in);
+ properties.setProperty(name,value);
+ }
+ }
+}
diff --git a/src/contrib/raid/src/java/org/apache/hadoop/raid/protocol/PolicyList.java b/src/contrib/raid/src/java/org/apache/hadoop/raid/protocol/PolicyList.java
new file mode 100644
index 0000000..187ed25
--- /dev/null
+++ b/src/contrib/raid/src/java/org/apache/hadoop/raid/protocol/PolicyList.java
@@ -0,0 +1,119 @@
+/**
+ * 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.raid.protocol;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.util.Collection;
+import java.util.List;
+import java.util.LinkedList;
+import java.util.Comparator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Maintains informatiom about all policies that belong to a category.
+ * These policies have to be applied one-at-a-time and cannot be run
+ * simultaneously.
+ */
+public class PolicyList implements Writable {
+ public static final Log LOG = LogFactory.getLog(
+ "org.apache.hadoop.raid.protocol.PolicyList");
+
+ private List<PolicyInfo> category; // list of policies
+ private Path srcPath;
+
+ /**
+ * Create a new category of policies.
+ */
+ public PolicyList() {
+ this.category = new LinkedList<PolicyInfo>();
+ this.srcPath = null;
+ }
+
+ /**
+ * Add a new policy to this category.
+ */
+ public void add(PolicyInfo info) {
+ category.add(info);
+ }
+
+ public void setSrcPath(Configuration conf, String src) throws IOException {
+ srcPath = new Path(src);
+ srcPath = srcPath.makeQualified(srcPath.getFileSystem(conf));
+ }
+
+ public Path getSrcPath() {
+ return srcPath;
+ }
+
+ /**
+ * Returns the policies in this category
+ */
+ public Collection<PolicyInfo> getAll() {
+ return category;
+ }
+
+ /**
+ * Sort Categries based on their srcPath. reverse lexicographical order.
+ */
+ public static class CompareByPath implements Comparator<PolicyList> {
+ public CompareByPath() throws IOException {
+ }
+ public int compare(PolicyList l1, PolicyList l2) {
+ return 0 - l1.getSrcPath().compareTo(l2.getSrcPath());
+ }
+ }
+
+
+ //////////////////////////////////////////////////
+ // Writable
+ //////////////////////////////////////////////////
+ static { // register a ctor
+ WritableFactories.setFactory
+ (PolicyList.class,
+ new WritableFactory() {
+ public Writable newInstance() { return new PolicyList(); }
+ });
+ }
+
+ public void write(DataOutput out) throws IOException {
+ out.writeInt(category.size());
+ for (PolicyInfo p : category) {
+ p.write(out);
+ }
+ }
+
+ public void readFields(DataInput in) throws IOException {
+ int count = in.readInt();
+ for (int i = 0; i < count; i++) {
+ PolicyInfo p = new PolicyInfo();
+ p.readFields(in);
+ add(p);
+ }
+ }
+}
diff --git a/src/contrib/raid/src/java/org/apache/hadoop/raid/protocol/RaidProtocol.java b/src/contrib/raid/src/java/org/apache/hadoop/raid/protocol/RaidProtocol.java
new file mode 100644
index 0000000..74433c8
--- /dev/null
+++ b/src/contrib/raid/src/java/org/apache/hadoop/raid/protocol/RaidProtocol.java
@@ -0,0 +1,73 @@
+/**
+ * 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.raid.protocol;
+
+import java.util.Collection;
+import java.io.IOException;
+
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.fs.Path;
+
+/**********************************************************************
+ * RaidProtocol is used by user code
+ * {@link org.apache.hadoop.raid.RaidShell} class to communicate
+ * with the RaidNode. User code can manipulate the configured policies.
+ *
+ **********************************************************************/
+public interface RaidProtocol extends VersionedProtocol {
+
+ /**
+ * Compared to the previous version the following changes have been introduced:
+ * Only the latest change is reflected.
+ * 1: new protocol introduced
+ */
+ public static final long versionID = 1L;
+
+ /**
+ * A set of codes returned by RPC calls.
+ */
+ public enum ReturnStatus {
+ SUCCESS ((int)0x01),
+ FAILURE ((int)0x02),
+ RETRY ((int)0x03);
+ private int code;
+
+ private ReturnStatus(int code) {
+ this.code = code;
+ }
+ int getReturnStatus() {return code;}
+ }
+
+ /**
+ * Get a listing of all configured policies
+ * @throws IOException
+ * return all categories of configured policies
+ */
+ public PolicyList[] getAllPolicies() throws IOException;
+
+ /**
+ * Unraid the specified input path. This is called when the specified file
+ * is corrupted. This call will move the specified file to file.old
+ * and then recover it from the RAID subsystem.
+ *
+ * @param inputPath The absolute pathname of the file to be recovered.
+ * @param corruptOffset The offset that has the corruption
+ */
+ public ReturnStatus recoverFile(String inputPath, long corruptOffset) throws IOException;
+
+}
diff --git a/src/contrib/raid/src/test/org/apache/hadoop/hdfs/TestRaidDfs.java b/src/contrib/raid/src/test/org/apache/hadoop/hdfs/TestRaidDfs.java
new file mode 100644
index 0000000..4254e1c
--- /dev/null
+++ b/src/contrib/raid/src/test/org/apache/hadoop/hdfs/TestRaidDfs.java
@@ -0,0 +1,290 @@
+/**
+ * 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.hdfs;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.GregorianCalendar;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.zip.CRC32;
+
+import junit.framework.TestCase;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.DistributedRaidFileSystem;
+import org.apache.hadoop.raid.RaidNode;
+
+public class TestRaidDfs extends TestCase {
+ final static String TEST_DIR = new File(System.getProperty("test.build.data",
+ "build/contrib/raid/test/data")).getAbsolutePath();
+ final static String CONFIG_FILE = new File(TEST_DIR,
+ "test-raid.xml").getAbsolutePath();
+ final static long RELOAD_INTERVAL = 1000;
+ final static Log LOG = LogFactory.getLog("org.apache.hadoop.raid.TestRaidNode");
+ final Random rand = new Random();
+ final static int NUM_DATANODES = 3;
+
+ Configuration conf;
+ String namenode = null;
+ String hftp = null;
+ MiniDFSCluster dfs = null;
+ FileSystem fileSys = null;
+ RaidNode cnode = null;
+ String jobTrackerName = null;
+
+ private void mySetup() throws Exception {
+
+ new File(TEST_DIR).mkdirs(); // Make sure data directory exists
+ conf = new Configuration();
+
+ conf.set("raid.config.file", CONFIG_FILE);
+ conf.setBoolean("raid.config.reload", true);
+ conf.setLong("raid.config.reload.interval", RELOAD_INTERVAL);
+
+ // scan all policies once every 5 second
+ conf.setLong("raid.policy.rescan.interval", 5000);
+
+ // do not use map-reduce cluster for Raiding
+ conf.setBoolean("fs.raidnode.local", true);
+ conf.setInt("hdfs.raid.stripeLength", 3);
+ conf.set("hdfs.raid.locations", "/destraid");
+
+ dfs = new MiniDFSCluster(conf, NUM_DATANODES, true, null);
+ dfs.waitActive();
+ fileSys = dfs.getFileSystem();
+ namenode = fileSys.getUri().toString();
+ hftp = "hftp://localhost.localdomain:" + dfs.getNameNodePort();
+
+ FileSystem.setDefaultUri(conf, namenode);
+
+ FileWriter fileWriter = new FileWriter(CONFIG_FILE);
+ fileWriter.write("<?xml version=\"1.0\"?>\n");
+ String str = "<configuration> " +
+ "<srcPath prefix=\"/user/dhruba/raidtest\"> " +
+ "<policy name = \"RaidTest1\"> " +
+ "<destPath> /destraid</destPath> " +
+ "<property> " +
+ "<name>targetReplication</name> " +
+ "<value>1</value> " +
+ "<description>after RAIDing, decrease the replication factor of a file to this value." +
+ "</description> " +
+ "</property> " +
+ "<property> " +
+ "<name>metaReplication</name> " +
+ "<value>1</value> " +
+ "<description> replication factor of parity file" +
+ "</description> " +
+ "</property> " +
+ "<property> " +
+ "<name>modTimePeriod</name> " +
+ "<value>2000</value> " +
+ "<description> time (milliseconds) after a file is modified to make it " +
+ "a candidate for RAIDing " +
+ "</description> " +
+ "</property> " +
+ "</policy>" +
+ "</srcPath>" +
+ "</configuration>";
+ fileWriter.write(str);
+ fileWriter.close();
+ }
+
+ private void myTearDown() throws Exception {
+ if (cnode != null) { cnode.stop(); cnode.join(); }
+ if (dfs != null) { dfs.shutdown(); }
+ }
+
+ /**
+ * Test DFS Raid
+ */
+ public void testRaidDfs() throws Exception {
+ LOG.info("Test testRaidDfs started.");
+ long blockSize = 8192L;
+ int stripeLength = 3;
+ mySetup();
+ Path file1 = new Path("/user/dhruba/raidtest/file1");
+ Path destPath = new Path("/destraid/user/dhruba/raidtest");
+ long crc1 = createOldFile(fileSys, file1, 1, 7, blockSize);
+ LOG.info("Test testPathFilter created test files");
+
+ // create an instance of the RaidNode
+ cnode = RaidNode.createRaidNode(null, conf);
+
+ try {
+ FileStatus[] listPaths = null;
+
+ // wait till file is raided
+ while (listPaths == null || listPaths.length != 1) {
+ LOG.info("Test testPathFilter waiting for files to be raided.");
+ try {
+ listPaths = fileSys.listStatus(destPath);
+ } catch (FileNotFoundException e) {
+ //ignore
+ }
+ Thread.sleep(1000); // keep waiting
+ }
+ assertEquals(listPaths.length, 1); // all files raided
+ LOG.info("Files raided so far : " + listPaths[0].getPath());
+
+ // extract block locations from File system. Wait till file is closed.
+ LocatedBlocks locations = null;
+ DistributedFileSystem dfs = (DistributedFileSystem) fileSys;
+ while (true) {
+ locations = dfs.getClient().getNamenode().getBlockLocations(file1.toString(),
+ 0, listPaths[0].getLen());
+ if (!locations.isUnderConstruction()) {
+ break;
+ }
+ Thread.sleep(1000);
+ }
+
+ // filter all filesystem calls from client
+ Configuration clientConf = new Configuration(conf);
+ clientConf.set("fs.hdfs.impl", "org.apache.hadoop.dfs.DistributedRaidFileSystem");
+ DistributedRaidFileSystem raidfs = new DistributedRaidFileSystem(dfs);
+ raidfs.initialize(dfs.getUri(), clientConf);
+
+ // corrupt first block of file
+ LOG.info("Corrupt first block of file");
+ corruptBlock(file1, locations.get(0).getBlock());
+ validateFile(raidfs, file1, file1, crc1);
+
+ } catch (Exception e) {
+ LOG.info("testPathFilter Exception " + e + StringUtils.stringifyException(e));
+ throw e;
+ } finally {
+ myTearDown();
+ }
+ LOG.info("Test testPathFilter completed.");
+ }
+
+ //
+ // creates a file and populate it with random data. Returns its crc.
+ //
+ private long createOldFile(FileSystem fileSys, Path name, int repl, int numBlocks, long blocksize)
+ throws IOException {
+ CRC32 crc = new CRC32();
+ FSDataOutputStream stm = fileSys.create(name, true,
+ fileSys.getConf().getInt("io.file.buffer.size", 4096),
+ (short)repl, blocksize);
+ // fill random data into file
+ final byte[] b = new byte[(int)blocksize];
+ for (int i = 0; i < numBlocks; i++) {
+ rand.nextBytes(b);
+ stm.write(b);
+ crc.update(b);
+ }
+ stm.close();
+ return crc.getValue();
+ }
+
+ //
+ // validates that file matches the crc.
+ //
+ private void validateFile(FileSystem fileSys, Path name1, Path name2, long crc)
+ throws IOException {
+
+ FileStatus stat1 = fileSys.getFileStatus(name1);
+ FileStatus stat2 = fileSys.getFileStatus(name2);
+ assertTrue(" Length of file " + name1 + " is " + stat1.getLen() +
+ " is different from length of file " + name1 + " " + stat2.getLen(),
+ stat1.getLen() == stat2.getLen());
+
+ CRC32 newcrc = new CRC32();
+ FSDataInputStream stm = fileSys.open(name2);
+ final byte[] b = new byte[4192];
+ int num = 0;
+ while (num >= 0) {
+ num = stm.read(b);
+ if (num < 0) {
+ break;
+ }
+ newcrc.update(b, 0, num);
+ }
+ stm.close();
+ LOG.info(" Newcrc " + newcrc.getValue() + " old crc " + crc);
+ if (newcrc.getValue() != crc) {
+ fail("CRC mismatch of files " + name1 + " with file " + name2);
+ }
+ }
+
+ /*
+ * The Data directories for a datanode
+ */
+ private File[] getDataNodeDirs(int i) throws IOException {
+ File base_dir = new File(System.getProperty("test.build.data"), "dfs/");
+ File data_dir = new File(base_dir, "data");
+ File dir1 = new File(data_dir, "data"+(2*i+1));
+ File dir2 = new File(data_dir, "data"+(2*i+2));
+ if (dir1.isDirectory() && dir2.isDirectory()) {
+ File[] dir = new File[2];
+ dir[0] = new File(dir1, "current");
+ dir[1] = new File(dir2, "current");
+ return dir;
+ }
+ return new File[0];
+ }
+
+ //
+ // Corrupt specified block of file
+ //
+ void corruptBlock(Path file, Block blockNum) throws IOException {
+ long id = blockNum.getBlockId();
+
+ // Now deliberately remove/truncate data blocks from the block.
+ //
+ for (int i = 0; i < NUM_DATANODES; i++) {
+ File[] dirs = getDataNodeDirs(i);
+
+ for (int j = 0; j < dirs.length; j++) {
+ File[] blocks = dirs[j].listFiles();
+ assertTrue("Blocks do not exist in data-dir", (blocks != null) && (blocks.length >= 0));
+ for (int idx = 0; idx < blocks.length; idx++) {
+ if (blocks[idx].getName().startsWith("blk_" + id) &&
+ !blocks[idx].getName().endsWith(".meta")) {
+ blocks[idx].delete();
+ LOG.info("Deleted block " + blocks[idx]);
+ }
+ }
+ }
+ }
+ }
+
+}
diff --git a/src/contrib/raid/src/test/org/apache/hadoop/raid/TestRaidNode.java b/src/contrib/raid/src/test/org/apache/hadoop/raid/TestRaidNode.java
new file mode 100644
index 0000000..62b4b1c
--- /dev/null
+++ b/src/contrib/raid/src/test/org/apache/hadoop/raid/TestRaidNode.java
@@ -0,0 +1,494 @@
+/**
+ * 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.raid;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.GregorianCalendar;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.zip.CRC32;
+
+import junit.framework.TestCase;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.raid.protocol.PolicyInfo;
+import org.apache.hadoop.raid.protocol.PolicyList;
+
+/**
+ * Test the generation of parity blocks for files with different block
+ * sizes. Also test that a data block can be regenerated from a raid stripe
+ * using the parity block
+ */
+public class TestRaidNode extends TestCase {
+ final static String TEST_DIR = new File(System.getProperty("test.build.data",
+ "build/contrib/raid/test/data")).getAbsolutePath();
+ final static String CONFIG_FILE = new File(TEST_DIR,
+ "test-raid.xml").getAbsolutePath();
+ final static long RELOAD_INTERVAL = 1000;
+ final static Log LOG = LogFactory.getLog("org.apache.hadoop.raid.TestRaidNode");
+ final static Random rand = new Random();
+
+ Configuration conf;
+ String namenode = null;
+ MiniDFSCluster dfs = null;
+ FileSystem fileSys = null;
+
+ /**
+ * create mapreduce and dfs clusters
+ */
+ private void createClusters(boolean local) throws Exception {
+
+ new File(TEST_DIR).mkdirs(); // Make sure data directory exists
+ conf = new Configuration();
+ conf.set("raid.config.file", CONFIG_FILE);
+ conf.setBoolean("raid.config.reload", true);
+ conf.setLong("raid.config.reload.interval", RELOAD_INTERVAL);
+ conf.setBoolean("dfs.permissions.enabled", true);
+
+ // scan all policies once every 5 second
+ conf.setLong("raid.policy.rescan.interval", 5000);
+
+ // the RaidNode does the raiding inline (instead of submitting to map/reduce)
+ conf.setBoolean("fs.raidnode.local", local);
+
+ // create a dfs and map-reduce cluster
+ final int taskTrackers = 4;
+ final int jobTrackerPort = 60050;
+
+ dfs = new MiniDFSCluster(conf, 3, true, null);
+ dfs.waitActive();
+ fileSys = dfs.getFileSystem();
+ namenode = fileSys.getUri().toString();
+ }
+
+ /**
+ * create raid.xml file for RaidNode
+ */
+ private void mySetup(String path, short srcReplication, long targetReplication,
+ long metaReplication, long stripeLength) throws Exception {
+ FileWriter fileWriter = new FileWriter(CONFIG_FILE);
+ fileWriter.write("<?xml version=\"1.0\"?>\n");
+ String str = "<configuration> " +
+ "<srcPath prefix=\"" + path + "\"> " +
+ "<policy name = \"RaidTest1\"> " +
+ "<destPath> /destraid</destPath> " +
+ "<property> " +
+ "<name>srcReplication</name> " +
+ "<value>" + srcReplication + "</value> " +
+ "<description> pick only files whole replFactor is greater than or equal to " +
+ "</description> " +
+ "</property> " +
+ "<property> " +
+ "<name>targetReplication</name> " +
+ "<value>" + targetReplication + "</value> " +
+ "<description>after RAIDing, decrease the replication factor of a file to this value." +
+ "</description> " +
+ "</property> " +
+ "<property> " +
+ "<name>metaReplication</name> " +
+ "<value>" + metaReplication + "</value> " +
+ "<description> replication factor of parity file" +
+ "</description> " +
+ "</property> " +
+ "<property> " +
+ "<name>stripeLength</name> " +
+ "<value>" + stripeLength + "</value> " +
+ "<description> the max number of blocks in a file to RAID together " +
+ "</description> " +
+ "</property> " +
+ "<property> " +
+ "<name>modTimePeriod</name> " +
+ "<value>2000</value> " +
+ "<description> time (milliseconds) after a file is modified to make it " +
+ "a candidate for RAIDing " +
+ "</description> " +
+ "</property> " +
+ "</policy>" +
+ "</srcPath>" +
+ "</configuration>";
+ fileWriter.write(str);
+ fileWriter.close();
+
+
+ }
+
+ /**
+ * stop clusters created earlier
+ */
+ private void stopClusters() throws Exception {
+ if (dfs != null) { dfs.shutdown(); }
+ }
+
+ /**
+ * Test to run a filter
+ */
+ public void testPathFilter() throws Exception {
+ LOG.info("Test testPathFilter started.");
+
+ long blockSizes [] = {1024L};
+ long stripeLengths [] = {1, 2, 5, 6, 10, 11, 12};
+ long targetReplication = 1;
+ long metaReplication = 1;
+ int numBlock = 11;
+ int iter = 0;
+
+ try {
+ createClusters(true);
+ for (long blockSize : blockSizes) {
+ for (long stripeLength : stripeLengths) {
+ doTestPathFilter(iter, targetReplication, metaReplication,
+ stripeLength, blockSize, numBlock);
+ iter++;
+ }
+ }
+ doCheckPolicy();
+ } finally {
+ stopClusters();
+ }
+ LOG.info("Test testPathFilter completed.");
+ }
+
+ /**
+ * Test to run a filter
+ */
+ private void doTestPathFilter(int iter, long targetReplication,
+ long metaReplication, long stripeLength,
+ long blockSize, int numBlock) throws Exception {
+ LOG.info("doTestPathFilter started---------------------------:" + " iter " + iter +
+ " blockSize=" + blockSize + " stripeLength=" + stripeLength);
+ mySetup("/user/dhruba/raidtest", (short)1, targetReplication, metaReplication, stripeLength);
+ RaidShell shell = null;
+ Path dir = new Path("/user/dhruba/raidtest/");
+ Path file1 = new Path(dir + "/file" + iter);
+ RaidNode cnode = null;
+ try {
+ Path recover1 = new Path("/destraid/" + file1 + ".recovered");
+ Path destPath = new Path("/destraid/user/dhruba/raidtest");
+ fileSys.delete(dir, true);
+ fileSys.delete(destPath, true);
+ long crc1 = createOldFile(fileSys, file1, 1, numBlock, blockSize);
+ LOG.info("doTestPathFilter created test files for iteration " + iter);
+
+ // create an instance of the RaidNode
+ cnode = RaidNode.createRaidNode(null, conf);
+ int times = 10;
+
+ while (times-- > 0) {
+ try {
+ shell = new RaidShell(conf);
+ } catch (Exception e) {
+ LOG.info("doTestPathFilter unable to connect to " + RaidNode.getAddress(conf) +
+ " retrying....");
+ Thread.sleep(1000);
+ continue;
+ }
+ break;
+ }
+ LOG.info("doTestPathFilter created RaidShell.");
+ FileStatus[] listPaths = null;
+
+ // wait till file is raided
+ while (true) {
+ try {
+ listPaths = fileSys.listStatus(destPath);
+ int count = 0;
+ if (listPaths != null && listPaths.length == 1) {
+ for (FileStatus s : listPaths) {
+ LOG.info("doTestPathFilter found path " + s.getPath());
+ if (!s.getPath().toString().endsWith(".tmp")) {
+ count++;
+ }
+ }
+ }
+ if (count > 0) {
+ break;
+ }
+ } catch (FileNotFoundException e) {
+ //ignore
+ }
+ LOG.info("doTestPathFilter waiting for files to be raided. Found " +
+ (listPaths == null ? "none" : listPaths.length));
+ Thread.sleep(1000); // keep waiting
+ }
+ // assertEquals(listPaths.length, 1); // all files raided
+ LOG.info("doTestPathFilter all files found in Raid.");
+
+ // check for error at beginning of file
+ if (numBlock >= 1) {
+ LOG.info("Check error at beginning of file.");
+ simulateError(shell, fileSys, file1, recover1, crc1, 0);
+ }
+
+ // check for error at the beginning of second block
+ if (numBlock >= 2) {
+ LOG.info("Check error at beginning of second block.");
+ simulateError(shell, fileSys, file1, recover1, crc1, blockSize + 1);
+ }
+
+ // check for error at the middle of third block
+ if (numBlock >= 3) {
+ LOG.info("Check error at middle of third block.");
+ simulateError(shell, fileSys, file1, recover1, crc1,
+ 2 * blockSize + 10);
+ }
+
+ // check for error at the middle of second stripe
+ if (numBlock >= stripeLength + 1) {
+ LOG.info("Check error at middle of second stripe.");
+ simulateError(shell, fileSys, file1, recover1, crc1,
+ stripeLength * blockSize + 100);
+ }
+
+ } catch (Exception e) {
+ LOG.info("doTestPathFilter Exception " + e +
+ StringUtils.stringifyException(e));
+ throw e;
+ } finally {
+ shell.close();
+ if (cnode != null) { cnode.stop(); cnode.join(); }
+ LOG.info("doTestPathFilter delete file " + file1);
+ fileSys.delete(file1, true);
+ }
+ LOG.info("doTestPathFilter completed:" + " blockSize=" + blockSize +
+ " stripeLength=" + stripeLength);
+ }
+
+ // Check that raid occurs only on files that have a replication factor
+ // greater than or equal to the specified value
+ private void doCheckPolicy() throws Exception {
+ LOG.info("doCheckPolicy started---------------------------:");
+ short srcReplication = 3;
+ long targetReplication = 2;
+ long metaReplication = 1;
+ long stripeLength = 2;
+ long blockSize = 1024;
+ int numBlock = 3;
+ mySetup("/user/dhruba/policytest", srcReplication, targetReplication, metaReplication, stripeLength);
+ RaidShell shell = null;
+ Path dir = new Path("/user/dhruba/policytest/");
+ Path file1 = new Path(dir + "/file1");
+ Path file2 = new Path(dir + "/file2");
+ RaidNode cnode = null;
+ try {
+ Path destPath = new Path("/destraid/user/dhruba/policytest");
+ fileSys.delete(dir, true);
+ fileSys.delete(destPath, true);
+
+ // create a file that has a replciation factor of 2. The Raid code should not
+ // pick this up.
+ long crc1 = createOldFile(fileSys, file1, 2, numBlock, blockSize);
+ LOG.info("doCheckPolicy created test files.");
+
+ // create an instance of the RaidNode
+ cnode = RaidNode.createRaidNode(null, conf);
+ int times = 10;
+
+ while (times-- > 0) {
+ try {
+ shell = new RaidShell(conf);
+ } catch (Exception e) {
+ LOG.info("doCheckPolicy unable to connect to " + RaidNode.getAddress(conf) +
+ " retrying....");
+ Thread.sleep(1000);
+ continue;
+ }
+ break;
+ }
+ LOG.info("doCheckPolicy created RaidShell.");
+
+ // this file should be picked up RaidNode
+ long crc2 = createOldFile(fileSys, file2, 3, numBlock, blockSize);
+ FileStatus[] listPaths = null;
+
+ // wait till file is raided
+ while (true) {
+ Thread.sleep(20000L); // waiting
+ listPaths = fileSys.listStatus(destPath);
+ int count = 0;
+ if (listPaths != null && listPaths.length == 1) {
+ for (FileStatus s : listPaths) {
+ LOG.info("doCheckPolicy found path " + s.getPath());
+ if (!s.getPath().toString().endsWith(".tmp")) {
+ count++;
+ }
+ }
+ }
+ if (count > 0) {
+ break;
+ }
+ LOG.info("doCheckPolicy waiting for files to be raided. Found " +
+ (listPaths == null ? "none" : listPaths.length));
+ }
+ assertEquals(listPaths.length, 1);
+ LOG.info("doCheckPolicy all files found in Raid.");
+ } catch (Exception e) {
+ LOG.info("doCheckPolicy Exception " + e +
+ StringUtils.stringifyException(e));
+ throw e;
+ } finally {
+ shell.close();
+ if (cnode != null) { cnode.stop(); cnode.join(); }
+ LOG.info("doTestPathFilter delete file " + file1);
+ fileSys.delete(file1, false);
+ }
+ LOG.info("doCheckPolicy completed:");
+ }
+
+ /**
+ * Test dist Raid
+ */
+ public void testDistRaid() throws Exception {
+ LOG.info("Test testDistRaid started.");
+ long blockSize = 1024L;
+ long targetReplication = 2;
+ long metaReplication = 2;
+ long stripeLength = 3;
+ short srcReplication = 1;
+
+ try {
+ createClusters(false);
+ mySetup("/user/dhruba/raidtest", srcReplication, targetReplication, metaReplication, stripeLength);
+ LOG.info("Test testDistRaid created test files");
+
+ Path dir = new Path("/user/dhruba/raidtest/");
+ Path destPath = new Path("/destraid/user/dhruba/raidtest");
+ fileSys.delete(dir, true);
+ fileSys.delete(destPath, true);
+
+ ConfigManager configMgr = new ConfigManager(conf);
+ configMgr.reloadConfigsIfNecessary();
+ LOG.info(" testDistRaid ConfigFile Loaded");
+
+ // activate all categories
+ Collection<PolicyList> all = configMgr.getAllPolicies();
+ PolicyList[] sorted = all.toArray(new PolicyList[all.size()]);
+ Iterator<PolicyInfo> pi = sorted[0].getAll().iterator();
+ PolicyInfo p = pi.next();
+ List<FileStatus> ll = new ArrayList<FileStatus>();
+
+ for(int i = 0 ; i < 10; i++){
+ Path file = new Path("/user/dhruba/raidtest/file"+i);
+ createOldFile(fileSys, file, 1, 7, blockSize);
+ FileStatus st = fileSys.getFileStatus(file);
+ ll.add(st);
+ }
+
+ DistRaid dr = new DistRaid(conf);
+ dr.addRaidPaths(p, ll);
+ dr.doDistRaid();
+ LOG.info("Test testDistRaid successful.");
+
+ } catch (Exception e) {
+ LOG.info("testDistRaid Exception " + e + StringUtils.stringifyException(e));
+ throw e;
+ } finally {
+ stopClusters();
+ }
+ LOG.info("Test testDistRaid completed.");
+ }
+
+ //
+ // simulate a corruption at specified offset and verify that eveyrthing is good
+ //
+ void simulateError(RaidShell shell, FileSystem fileSys, Path file1, Path recover1,
+ long crc, long corruptOffset) throws IOException {
+ // recover the file assuming that we encountered a corruption at offset 0
+ String[] args = new String[3];
+ args[0] = "recover";
+ args[1] = file1.toString();
+ args[2] = Long.toString(corruptOffset);
+ shell.recover(args[0], args, 1);
+
+ // compare that the recovered file is identical to the original one
+ LOG.info("Comparing file " + file1 + " with recovered file " + recover1);
+ validateFile(fileSys, file1, recover1, crc);
+ fileSys.delete(recover1, false);
+ }
+
+ //
+ // creates a file and populate it with random data. Returns its crc.
+ //
+ static long createOldFile(FileSystem fileSys, Path name, int repl, int numBlocks, long blocksize)
+ throws IOException {
+ CRC32 crc = new CRC32();
+ FSDataOutputStream stm = fileSys.create(name, true,
+ fileSys.getConf().getInt("io.file.buffer.size", 4096),
+ (short)repl, blocksize);
+ // fill random data into file
+ byte[] b = new byte[(int)blocksize];
+ for (int i = 0; i < numBlocks; i++) {
+ if (i == (numBlocks-1)) {
+ b = new byte[(int)blocksize/2];
+ }
+ rand.nextBytes(b);
+ stm.write(b);
+ crc.update(b);
+ }
+
+ stm.close();
+ return crc.getValue();
+ }
+
+ //
+ // validates that file matches the crc.
+ //
+ private void validateFile(FileSystem fileSys, Path name1, Path name2, long crc)
+ throws IOException {
+
+ FileStatus stat1 = fileSys.getFileStatus(name1);
+ FileStatus stat2 = fileSys.getFileStatus(name2);
+ assertTrue(" Length of file " + name1 + " is " + stat1.getLen() +
+ " is different from length of file " + name1 + " " + stat2.getLen(),
+ stat1.getLen() == stat2.getLen());
+
+ CRC32 newcrc = new CRC32();
+ FSDataInputStream stm = fileSys.open(name2);
+ final byte[] b = new byte[4192];
+ int num = 0;
+ while (num >= 0) {
+ num = stm.read(b);
+ if (num < 0) {
+ break;
+ }
+ newcrc.update(b, 0, num);
+ }
+ stm.close();
+ if (newcrc.getValue() != crc) {
+ fail("CRC mismatch of files " + name1 + " with file " + name2);
+ }
+ }
+}
diff --git a/src/contrib/raid/src/test/org/apache/hadoop/raid/TestRaidPurge.java b/src/contrib/raid/src/test/org/apache/hadoop/raid/TestRaidPurge.java
new file mode 100644
index 0000000..91b77df
--- /dev/null
+++ b/src/contrib/raid/src/test/org/apache/hadoop/raid/TestRaidPurge.java
@@ -0,0 +1,315 @@
+/**
+ * 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.raid;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.GregorianCalendar;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.zip.CRC32;
+
+import junit.framework.TestCase;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.log4j.Level;
+
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.raid.protocol.PolicyInfo;
+import org.apache.hadoop.raid.protocol.PolicyList;
+
+/**
+ * If a file gets deleted, then verify that the parity file gets deleted too.
+ */
+public class TestRaidPurge extends TestCase {
+ final static String TEST_DIR = new File(System.getProperty("test.build.data",
+ "build/contrib/raid/test/data")).getAbsolutePath();
+ final static String CONFIG_FILE = new File(TEST_DIR,
+ "test-raid.xml").getAbsolutePath();
+ final static long RELOAD_INTERVAL = 1000;
+ final static Log LOG = LogFactory.getLog("org.apache.hadoop.raid.TestRaidNode");
+
+ {
+ ((Log4JLogger)RaidNode.LOG).getLogger().setLevel(Level.ALL);
+ }
+
+
+ Configuration conf;
+ String namenode = null;
+ MiniDFSCluster dfs = null;
+ FileSystem fileSys = null;
+
+ /**
+ * create mapreduce and dfs clusters
+ */
+ private void createClusters(boolean local) throws Exception {
+
+ new File(TEST_DIR).mkdirs(); // Make sure data directory exists
+ conf = new Configuration();
+ conf.set("raid.config.file", CONFIG_FILE);
+ conf.setBoolean("raid.config.reload", true);
+ conf.setLong("raid.config.reload.interval", RELOAD_INTERVAL);
+
+ // scan all policies once every 5 second
+ conf.setLong("raid.policy.rescan.interval", 5000);
+
+ // make all deletions not go through Trash
+ conf.set("fs.shell.delete.classname", "org.apache.hadoop.dfs.DFSClient");
+
+ // the RaidNode does the raiding inline (instead of submitting to map/reduce)
+ conf.setBoolean("fs.raidnode.local", local);
+
+ // create a dfs and map-reduce cluster
+ final int taskTrackers = 4;
+ final int jobTrackerPort = 60050;
+
+ dfs = new MiniDFSCluster(conf, 3, true, null);
+ dfs.waitActive();
+ fileSys = dfs.getFileSystem();
+ namenode = fileSys.getUri().toString();
+ }
+
+ /**
+ * create raid.xml file for RaidNode
+ */
+ private void mySetup(long targetReplication,
+ long metaReplication, long stripeLength) throws Exception {
+ FileWriter fileWriter = new FileWriter(CONFIG_FILE);
+ fileWriter.write("<?xml version=\"1.0\"?>\n");
+ String str = "<configuration> " +
+ "<srcPath prefix=\"/user/dhruba/raidtest\"> " +
+ "<policy name = \"RaidTest1\"> " +
+ "<destPath> /destraid</destPath> " +
+ "<property> " +
+ "<name>targetReplication</name> " +
+ "<value>" + targetReplication + "</value> " +
+ "<description>after RAIDing, decrease the replication factor of a file to this value." +
+ "</description> " +
+ "</property> " +
+ "<property> " +
+ "<name>metaReplication</name> " +
+ "<value>" + metaReplication + "</value> " +
+ "<description> replication factor of parity file" +
+ "</description> " +
+ "</property> " +
+ "<property> " +
+ "<name>stripeLength</name> " +
+ "<value>" + stripeLength + "</value> " +
+ "<description> the max number of blocks in a file to RAID together " +
+ "</description> " +
+ "</property> " +
+ "<property> " +
+ "<name>modTimePeriod</name> " +
+ "<value>2000</value> " +
+ "<description> time (milliseconds) after a file is modified to make it " +
+ "a candidate for RAIDing " +
+ "</description> " +
+ "</property> " +
+ "</policy>" +
+ "</srcPath>" +
+ "</configuration>";
+ fileWriter.write(str);
+ fileWriter.close();
+ }
+
+ /**
+ * stop clusters created earlier
+ */
+ private void stopClusters() throws Exception {
+ if (dfs != null) { dfs.shutdown(); }
+ }
+
+ /**
+ * Test that parity files that do not have an associated master file
+ * get deleted.
+ */
+ public void testPurge() throws Exception {
+ LOG.info("Test testPurge started.");
+
+ long blockSizes [] = {1024L};
+ long stripeLengths [] = {5};
+ long targetReplication = 1;
+ long metaReplication = 1;
+ int numBlock = 9;
+ int iter = 0;
+
+ createClusters(true);
+ try {
+ for (long blockSize : blockSizes) {
+ for (long stripeLength : stripeLengths) {
+ doTestPurge(iter, targetReplication, metaReplication,
+ stripeLength, blockSize, numBlock);
+ iter++;
+ }
+ }
+ } finally {
+ stopClusters();
+ }
+ LOG.info("Test testPurge completed.");
+ }
+
+ /**
+ * Create parity file, delete original file and then validate that
+ * parity file is automatically deleted.
+ */
+ private void doTestPurge(int iter, long targetReplication,
+ long metaReplication, long stripeLength,
+ long blockSize, int numBlock) throws Exception {
+ LOG.info("doTestPurge started---------------------------:" + " iter " + iter +
+ " blockSize=" + blockSize + " stripeLength=" + stripeLength);
+ mySetup(targetReplication, metaReplication, stripeLength);
+ RaidShell shell = null;
+ Path dir = new Path("/user/dhruba/raidtest/");
+ Path file1 = new Path(dir + "/file" + iter);
+ RaidNode cnode = null;
+ try {
+ Path userDir = new Path("/destraid/user/dhruba");
+ Path recover1 = new Path("/destraid/" + file1 + ".recovered");
+ Path destPath = new Path("/destraid/user/dhruba/raidtest");
+ fileSys.delete(dir, true);
+ fileSys.delete(destPath, true);
+ long crc1 = TestRaidNode.createOldFile(fileSys, file1, 1, numBlock, blockSize);
+ LOG.info("doTestPurge created test files for iteration " + iter);
+
+ // create an instance of the RaidNode
+ cnode = RaidNode.createRaidNode(null, conf);
+ int times = 10;
+
+ while (times-- > 0) {
+ try {
+ shell = new RaidShell(conf);
+ } catch (Exception e) {
+ LOG.info("doTestPurge unable to connect to " + RaidNode.getAddress(conf) +
+ " retrying....");
+ Thread.sleep(1000);
+ continue;
+ }
+ break;
+ }
+ LOG.info("doTestPurge created RaidShell.");
+ FileStatus[] listPaths = null;
+
+ // wait till file is raided
+ while (true) {
+ try {
+ listPaths = fileSys.listStatus(destPath);
+ int count = 0;
+ if (listPaths != null && listPaths.length == 1) {
+ for (FileStatus s : listPaths) {
+ LOG.info("doTestPurge found path " + s.getPath());
+ if (!s.getPath().toString().endsWith(".tmp")) {
+ count++;
+ }
+ }
+ }
+ if (count > 0) {
+ break;
+ }
+ } catch (FileNotFoundException e) {
+ //ignore
+ }
+ LOG.info("doTestPurge waiting for files to be raided. Found " +
+ (listPaths == null ? "none" : listPaths.length));
+ Thread.sleep(1000); // keep waiting
+ }
+ // assertEquals(listPaths.length, 1); // all files raided
+ LOG.info("doTestPurge all files found in Raid.");
+
+ // delete original file
+ assertTrue("Unable to delete original file " + file1 ,
+ fileSys.delete(file1, true));
+ LOG.info("deleted file " + file1);
+
+ // wait till parity file is automatically deleted
+ while (true) {
+ listPaths = fileSys.listStatus(destPath);
+ int count = 0;
+ if (listPaths != null && listPaths.length == 1) {
+ for (FileStatus s : listPaths) {
+ LOG.info("doTestPurge found path " + s.getPath());
+ if (!s.getPath().toString().endsWith(".tmp")) {
+ count++;
+ }
+ }
+ }
+ if (count == 0) {
+ break;
+ }
+ LOG.info("doTestPurge waiting for parity files to be removed. Found " +
+ (listPaths == null ? "none" : listPaths.length));
+ Thread.sleep(1000); // keep waiting
+ }
+
+ // verify that if we delete the directory itself, then the correspoding
+ // directory in the parity space is deleted too.
+ assertTrue("The directory " + userDir + " should have one entry",
+ fileSys.listStatus(userDir).length == 1);
+ assertTrue("Unable to delete original directory " + dir,
+ fileSys.delete(dir, true));
+ LOG.info("deleted dir " + dir);
+
+ // wait till parity directory is automatically deleted
+ while (true) {
+ listPaths = fileSys.listStatus(userDir);
+ int count = 0;
+ if (listPaths != null) {
+ for (FileStatus s : listPaths) {
+ LOG.info("doTestPurge found path " + s.getPath());
+ count++;
+ }
+ }
+ if (count == 0) {
+ break;
+ }
+ LOG.info("doTestPurge waiting for parity dir to be removed. Found " +
+ (listPaths == null ? "none" : listPaths.length));
+ Thread.sleep(1000); // keep waiting
+ }
+
+
+ } catch (Exception e) {
+ LOG.info("doTestPurge Exception " + e +
+ StringUtils.stringifyException(e));
+ throw e;
+ } finally {
+ shell.close();
+ if (cnode != null) { cnode.stop(); cnode.join(); }
+ LOG.info("doTestPurge delete file " + file1);
+ fileSys.delete(file1, true);
+ }
+ LOG.info("doTestPurge completed:" + " blockSize=" + blockSize +
+ " stripeLength=" + stripeLength);
+ }
+}
diff --git a/src/contrib/sqoop/build.xml b/src/contrib/sqoop/build.xml
index b5717dd..28463ca 100644
--- a/src/contrib/sqoop/build.xml
+++ b/src/contrib/sqoop/build.xml
@@ -33,7 +33,8 @@
<!-- Override with our own version so we can enforce build dependencies -->
<!-- on compile-mapred-test for MiniMRCluster, and MRUnit. -->
<!-- ================================================================== -->
- <target name="compile-test" depends="compile-examples" if="test.available">
+ <target name="compile-test" depends="compile-examples, ivy-retrieve-test"
+ if="test.available">
<echo message="Compiling ${name} dependencies" />
<!-- need top-level compile-mapred-test for MiniMRCluster -->
<subant target="compile-mapred-test">
@@ -132,8 +133,6 @@
<classpath>
<path refid="test.classpath"/>
<path refid="contrib-classpath"/>
- <!-- tools.jar from Sun JDK also required to invoke javac. -->
- <pathelement path="${env.JAVA_HOME}/lib/tools.jar" />
<!-- need thirdparty JDBC drivers for thirdparty tests -->
<fileset dir="${sqoop.thirdparty.lib.dir}"
includes="*.jar" />
diff --git a/src/contrib/sqoop/doc/Sqoop-manpage.txt b/src/contrib/sqoop/doc/Sqoop-manpage.txt
index d4a3d66..b98d3d6 100644
--- a/src/contrib/sqoop/doc/Sqoop-manpage.txt
+++ b/src/contrib/sqoop/doc/Sqoop-manpage.txt
@@ -113,6 +113,12 @@
When using direct mode, write to multiple files of
approximately _size_ bytes each.
+Export control options
+~~~~~~~~~~~~~~~~~~~~~~
+
+--export-dir (dir)::
+ Export from an HDFS path into a table (set with
+ --table)
Output line formatting options
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
@@ -145,6 +151,14 @@
--package-name (package)::
Puts auto-generated classes in the named Java package
+Library loading options
+~~~~~~~~~~~~~~~~~~~~~~~
+--jar-file (file)::
+ Disable code generation; use specified jar
+
+--class-name (name)::
+ The class within the jar that represents the table to import/export
+
Additional commands
~~~~~~~~~~~~~~~~~~~
diff --git a/src/contrib/sqoop/doc/SqoopUserGuide.txt b/src/contrib/sqoop/doc/SqoopUserGuide.txt
index f1bf350..e777774 100644
--- a/src/contrib/sqoop/doc/SqoopUserGuide.txt
+++ b/src/contrib/sqoop/doc/SqoopUserGuide.txt
@@ -59,6 +59,8 @@
include::hive.txt[]
+include::export.txt[]
+
include::supported-dbs.txt[]
include::api-reference.txt[]
diff --git a/src/contrib/sqoop/doc/api-reference.txt b/src/contrib/sqoop/doc/api-reference.txt
index 6fdfd12..2b22d0a 100644
--- a/src/contrib/sqoop/doc/api-reference.txt
+++ b/src/contrib/sqoop/doc/api-reference.txt
@@ -81,14 +81,14 @@
rewrite the majority of +getColNames()+.
+ConnManager+ implementations receive a lot of their configuration data from a
-Sqoop-specific class, +ImportOptions+. While +ImportOptions+ does not currently
-contain many setter methods, clients should not assume +ImportOptions+ are
-immutable. More setter methods may be added in the future. +ImportOptions+ does
+Sqoop-specific class, +SqoopOptions+. While +SqoopOptions+ does not currently
+contain many setter methods, clients should not assume +SqoopOptions+ are
+immutable. More setter methods may be added in the future. +SqoopOptions+ does
not directly store specific per-manager options. Instead, it contains a
reference to the +Configuration+ returned by +Tool.getConf()+ after parsing
command-line arguments with the +GenericOptionsParser+. This allows extension
arguments via "+-D any.specific.param=any.value+" without requiring any layering
-of options parsing or modification of +ImportOptions+.
+of options parsing or modification of +SqoopOptions+.
All existing +ConnManager+ implementations are stateless. Thus, the system which
instantiates +ConnManagers+ may implement multiple instances of the same
@@ -102,7 +102,7 @@
+ManagerFactory+ implementation should be provided with the new ConnManager.
+ManagerFactory+ has a single method of note, named +accept()+. This method will
determine whether it can instantiate a +ConnManager+ for the user's
-+ImportOptions+. If so, it returns the +ConnManager+ instance. Otherwise, it
++SqoopOptions+. If so, it returns the +ConnManager+ instance. Otherwise, it
returns +null+.
The +ManagerFactory+ implementations used are governed by the
@@ -110,7 +110,7 @@
libraries can install the 3rd-party library containing a new +ManagerFactory+
and +ConnManager+(s), and configure sqoop-site.xml to use the new
+ManagerFactory+. The +DefaultManagerFactory+ principly discriminates between
-databases by parsing the connect string stored in +ImportOptions+.
+databases by parsing the connect string stored in +SqoopOptions+.
Extension authors may make use of classes in the +org.apache.hadoop.sqoop.io+,
+mapred+, +mapreduce+, and +util+ packages to facilitate their implementations.
@@ -124,7 +124,7 @@
This section describes the internal architecture of Sqoop.
The Sqoop program is driven by the +org.apache.hadoop.sqoop.Sqoop+ main class.
-A limited number of additional classes are in the same package; +ImportOptions+
+A limited number of additional classes are in the same package; +SqoopOptions+
(described earlier) and +ConnFactory+ (which manipulates +ManagerFactory+
instances).
@@ -135,11 +135,11 @@
+org.apache.hadoop.sqoop.Sqoop+ is the main class and implements _Tool_. A new
instance is launched with +ToolRunner+. It parses its arguments using the
-+ImportOptions+ class. Within the +ImportOptions+, an +ImportAction+ will be
++SqoopOptions+ class. Within the +SqoopOptions+, an +ImportAction+ will be
chosen by the user. This may be import all tables, import one specific table,
execute a SQL statement, or others.
-A +ConnManager+ is then instantiated based on the data in the +ImportOptions+.
+A +ConnManager+ is then instantiated based on the data in the +SqoopOptions+.
The +ConnFactory+ is used to get a +ConnManager+ from a +ManagerFactory+; the
mechanics of this were described in an earlier section.
@@ -161,7 +161,7 @@
extended with additional parameters in the future, which optionally further
direct the import operation. Similarly, the +exportTable()+ method receives an
argument of type +ExportJobContext+. These classes contain the name of the table
-to import/export, a reference to the +ImportOptions+ object, and other related
+to import/export, a reference to the +SqoopOptions+ object, and other related
data.
Subpackages
@@ -207,8 +207,8 @@
importers.
* +Executor+ launches external processes and connects these to stream handlers
generated by an AsyncSink (see more detail below).
-* +ExportError+ is thrown by +ConnManagers+ when exports fail.
-* +ImportError+ is thrown by +ConnManagers+ when imports fail.
+* +ExportException+ is thrown by +ConnManagers+ when exports fail.
+* +ImportException+ is thrown by +ConnManagers+ when imports fail.
* +JdbcUrl+ handles parsing of connect strings, which are URL-like but not
specification-conforming. (In particular, JDBC connect strings may have
+multi:part:scheme://+ components.)
diff --git a/src/contrib/sqoop/doc/export.txt b/src/contrib/sqoop/doc/export.txt
new file mode 100644
index 0000000..84f286b
--- /dev/null
+++ b/src/contrib/sqoop/doc/export.txt
@@ -0,0 +1,58 @@
+
+////
+ 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.
+////
+
+
+Exporting to a Database
+-----------------------
+
+In addition to importing database tables into HDFS, Sqoop can also
+work in "reverse," reading the contents of a file or directory in
+HDFS, interpreting the data as database rows, and inserting them
+into a specified database table.
+
+To run an export, invoke Sqoop with the +--export-dir+ and
++--table+ options. e.g.:
+
+----
+$ sqoop --connect jdbc:mysql://db.example.com/foo --table bar \
+ --export-dir /results/bar_data
+----
+
+This will take the files in +/results/bar_data+ and inject their
+contents in to the +bar+ table in the +foo+ database on +db.example.com+.
+The target table must already exist in the database. Sqoop will perform
+a set of +INSERT INTO+ operations, without regard for existing content. If
+Sqoop attempts to insert rows which violate constraints in the database
+(e.g., a particular primary key value already exists), then the export
+will fail.
+
+As in import mode, Sqoop will auto-generate an interoperability class
+to use with the particular table in question. This will be used to parse
+the records in HDFS files before loading their contents into the database.
+You must specify the same delimiters (e.g., with +--fields-terminated-by+,
+etc.) as are used in the files to export in order to parse the data
+correctly. If your data is stored in SequenceFiles (created with an import
+in the +--as-sequencefile+ format), then you do not need to specify
+delimiters.
+
+If you have an existing auto-generated jar and class that you intend to use
+with Sqoop, you can specify these with the +--jar-file+ and +--class-name+
+parameters. Providing these options will disable autogeneration of a new
+class based on the target table.
+
+
diff --git a/src/contrib/sqoop/ivy.xml b/src/contrib/sqoop/ivy.xml
index 0a8c5d7..a50e5a7 100644
--- a/src/contrib/sqoop/ivy.xml
+++ b/src/contrib/sqoop/ivy.xml
@@ -41,20 +41,18 @@
</publications>
<dependencies>
<dependency org="org.slf4j" name="slf4j-api" rev="${slf4j-api.version}" conf="common->master"/>
- <dependency org="org.slf4j" name="slf4j-log4j12" rev="${slf4j-log4j12.version}" conf="common->master"/>
<dependency org="org.apache.hadoop" name="hadoop-core" rev="${hadoop-core.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-core-test" rev="${hadoop-core.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-hdfs" rev="${hadoop-hdfs.version}" conf="common->default"/>
- <dependency org="org.apache.hadoop" name="hadoop-hdfs-test" rev="${hadoop-hdfs.version}" conf="common->default"/>
+ <dependency org="org.apache.hadoop" name="hadoop-hdfs-test" rev="${hadoop-hdfs.version}" conf="test->default"/>
<dependency org="commons-logging" name="commons-logging" rev="${commons-logging.version}" conf="common->default"/>
<dependency org="log4j" name="log4j" rev="${log4j.version}" conf="common->master"/>
- <dependency org="org.mortbay.jetty" name="servlet-api-2.5" rev="${servlet-api-2.5.version}" conf="common->default"/>
<dependency org="junit" name="junit" rev="${junit.version}" conf="common->default"/>
<dependency org="commons-httpclient" name="commons-httpclient" rev="${commons-httpclient.version}" conf="common->default"/>
<dependency org="commons-cli" name="commons-cli" rev="${commons-cli.version}" conf="common->default"/>
<dependency org="hsqldb" name="hsqldb" rev="${hsqldb.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="avro" rev="${avro.version}" conf="common->default"/>
- <dependency org="javax.servlet" name="servlet-api" rev="${servlet-api.version}" conf="common->master"/>
+ <dependency org="org.mortbay.jetty" name="servlet-api-2.5" rev="${servlet-api-2.5.version}" conf="common->master"/>
<dependency org="org.mortbay.jetty" name="jetty" rev="${jetty.version}" conf="common->master"/>
<dependency org="commons-io" name="commons-io" rev="${commons-io.version}" conf="common->default"/>
<dependency org="org.mortbay.jetty" name="jetty-util" rev="${jetty-util.version}" conf="common->master"/>
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ConnFactory.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ConnFactory.java
index 7d3f4a8..90683dc 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ConnFactory.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ConnFactory.java
@@ -87,7 +87,7 @@
* @return a ConnManager instance for the appropriate database
* @throws IOException if it cannot find a ConnManager for this schema
*/
- public ConnManager getManager(ImportOptions opts) throws IOException {
+ public ConnManager getManager(SqoopOptions opts) throws IOException {
// Try all the available manager factories.
for (ManagerFactory factory : factories) {
LOG.debug("Trying ManagerFactory: " + factory.getClass().getName());
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/Sqoop.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/Sqoop.java
index 6507c1c..6ddd37d 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/Sqoop.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/Sqoop.java
@@ -19,6 +19,8 @@
package org.apache.hadoop.sqoop;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -29,15 +31,17 @@
import org.apache.hadoop.sqoop.hive.HiveImport;
import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.manager.ExportJobContext;
import org.apache.hadoop.sqoop.manager.ImportJobContext;
import org.apache.hadoop.sqoop.orm.ClassWriter;
import org.apache.hadoop.sqoop.orm.CompilationManager;
-import org.apache.hadoop.sqoop.util.ImportError;
+import org.apache.hadoop.sqoop.util.ExportException;
+import org.apache.hadoop.sqoop.util.ImportException;
/**
* Main entry-point for Sqoop
* Usage: hadoop jar (this_jar_name) org.apache.hadoop.sqoop.Sqoop (options)
- * See the ImportOptions class for options.
+ * See the SqoopOptions class for options.
*/
public class Sqoop extends Configured implements Tool {
@@ -53,41 +57,58 @@
Configuration.addDefaultResource("sqoop-site.xml");
}
- private ImportOptions options;
+ private SqoopOptions options;
private ConnManager manager;
private HiveImport hiveImport;
+ private List<String> generatedJarFiles;
public Sqoop() {
+ generatedJarFiles = new ArrayList<String>();
}
- public ImportOptions getOptions() {
+ public SqoopOptions getOptions() {
return options;
}
/**
+ * @return a list of jar files generated as part of this im/export process
+ */
+ public List<String> getGeneratedJarFiles() {
+ ArrayList<String> out = new ArrayList<String>(generatedJarFiles);
+ return out;
+ }
+
+ /**
* Generate the .class and .jar files
* @return the filename of the emitted jar file.
* @throws IOException
*/
private String generateORM(String tableName) throws IOException {
+ String existingJar = options.getExistingJarName();
+ if (existingJar != null) {
+ // The user has pre-specified a jar and class to use. Don't generate.
+ LOG.info("Using existing jar: " + existingJar);
+ return existingJar;
+ }
+
LOG.info("Beginning code generation");
CompilationManager compileMgr = new CompilationManager(options);
ClassWriter classWriter = new ClassWriter(options, manager, tableName, compileMgr);
classWriter.generate();
compileMgr.compile();
compileMgr.jar();
- return compileMgr.getJarFilename();
+ String jarFile = compileMgr.getJarFilename();
+ this.generatedJarFiles.add(jarFile);
+ return jarFile;
}
- private void importTable(String tableName) throws IOException, ImportError {
+ private void importTable(String tableName) throws IOException, ImportException {
String jarFile = null;
// Generate the ORM code for the tables.
- // TODO(aaron): Allow this to be bypassed if the user has already generated code,
- // or if they're using a non-MapReduce import method (e.g., mysqldump).
jarFile = generateORM(tableName);
- if (options.getAction() == ImportOptions.ControlAction.FullImport) {
+ if (options.getAction() == SqoopOptions.ControlAction.FullImport) {
// Proceed onward to do the import.
ImportJobContext context = new ImportJobContext(tableName, jarFile, options);
manager.importTable(context);
@@ -99,17 +120,26 @@
}
}
+ private void exportTable(String tableName) throws ExportException, IOException {
+ String jarFile = null;
+
+ // Generate the ORM code for the tables.
+ jarFile = generateORM(tableName);
+
+ ExportJobContext context = new ExportJobContext(tableName, jarFile, options);
+ manager.exportTable(context);
+ }
/**
* Actual main entry-point for the program
*/
public int run(String [] args) {
- options = new ImportOptions();
+ options = new SqoopOptions();
options.setConf(getConf());
try {
options.parse(args);
options.validate();
- } catch (ImportOptions.InvalidOptionsException e) {
+ } catch (SqoopOptions.InvalidOptionsException e) {
// display the error msg
System.err.println(e.getMessage());
return 1; // exit on exception here
@@ -131,8 +161,8 @@
hiveImport = new HiveImport(options, manager, getConf());
}
- ImportOptions.ControlAction action = options.getAction();
- if (action == ImportOptions.ControlAction.ListTables) {
+ SqoopOptions.ControlAction action = options.getAction();
+ if (action == SqoopOptions.ControlAction.ListTables) {
String [] tables = manager.listTables();
if (null == tables) {
System.err.println("Could not retrieve tables list from server");
@@ -143,7 +173,7 @@
System.out.println(tbl);
}
}
- } else if (action == ImportOptions.ControlAction.ListDatabases) {
+ } else if (action == SqoopOptions.ControlAction.ListDatabases) {
String [] databases = manager.listDatabases();
if (null == databases) {
System.err.println("Could not retrieve database list from server");
@@ -154,10 +184,29 @@
System.out.println(db);
}
}
- } else if (action == ImportOptions.ControlAction.DebugExec) {
+ } else if (action == SqoopOptions.ControlAction.DebugExec) {
// just run a SQL statement for debugging purposes.
manager.execAndPrint(options.getDebugSqlCmd());
return 0;
+ } else if (action == SqoopOptions.ControlAction.Export) {
+ // Export a table.
+ try {
+ exportTable(options.getTableName());
+ } catch (IOException ioe) {
+ LOG.error("Encountered IOException running export job: " + ioe.toString());
+ if (System.getProperty(SQOOP_RETHROW_PROPERTY) != null) {
+ throw new RuntimeException(ioe);
+ } else {
+ return 1;
+ }
+ } catch (ExportException ee) {
+ LOG.error("Error during export: " + ee.toString());
+ if (System.getProperty(SQOOP_RETHROW_PROPERTY) != null) {
+ throw new RuntimeException(ee);
+ } else {
+ return 1;
+ }
+ }
} else {
// This is either FullImport or GenerateOnly.
@@ -184,7 +233,7 @@
} else {
return 1;
}
- } catch (ImportError ie) {
+ } catch (ImportException ie) {
LOG.error("Error during import: " + ie.toString());
if (System.getProperty(SQOOP_RETHROW_PROPERTY) != null) {
throw new RuntimeException(ie);
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ImportOptions.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/SqoopOptions.java
similarity index 90%
rename from src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ImportOptions.java
rename to src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/SqoopOptions.java
index 6eacbc7..0db319e 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ImportOptions.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/SqoopOptions.java
@@ -35,9 +35,9 @@
/**
* Command-line arguments used by Sqoop
*/
-public class ImportOptions {
+public class SqoopOptions {
- public static final Log LOG = LogFactory.getLog(ImportOptions.class.getName());
+ public static final Log LOG = LogFactory.getLog(SqoopOptions.class.getName());
/**
* Thrown when invalid cmdline options are given
@@ -66,7 +66,8 @@
ListTables, // list available tables and exit.
GenerateOnly, // generate ORM code but do not import.
FullImport, // generate code (as needed) and import.
- DebugExec // just execute a single sql command and print its results.
+ DebugExec, // just execute a single sql command and print its results.
+ Export // export a table from HDFS to a database.
}
// selects in-HDFS destination file format
@@ -101,10 +102,15 @@
private boolean hiveImport;
private String packageName; // package to prepend to auto-named classes.
private String className; // package+class to apply to individual table import.
+ // also used as an *input* class with existingJarFile.
+ private String existingJarFile; // Name of a jar containing existing table definition
+ // class to use.
private int numMappers;
private boolean useCompression;
private long directSplitSize; // In direct mode, open a new stream every X bytes.
+ private String exportDir; // HDFS path to read from when performing an export
+
private char inputFieldDelim;
private char inputRecordDelim;
private char inputEnclosedBy;
@@ -127,17 +133,17 @@
private String [] extraArgs;
- public ImportOptions() {
+ public SqoopOptions() {
initDefaults();
}
/**
- * Alternate ImportOptions interface used mostly for unit testing
+ * Alternate SqoopOptions interface used mostly for unit testing
* @param connect JDBC connect string to use
* @param database Database to read
* @param table Table to read
*/
- public ImportOptions(final String connect, final String table) {
+ public SqoopOptions(final String connect, final String table) {
initDefaults();
this.connectString = connect;
@@ -188,6 +194,8 @@
this.hiveHome = props.getProperty("hive.home", this.hiveHome);
this.className = props.getProperty("java.classname", this.className);
this.packageName = props.getProperty("java.packagename", this.packageName);
+ this.existingJarFile = props.getProperty("java.jar.file", this.existingJarFile);
+ this.exportDir = props.getProperty("export.dir", this.exportDir);
this.direct = getBooleanProperty(props, "direct.import", this.direct);
this.hiveImport = getBooleanProperty(props, "hive.import", this.hiveImport);
@@ -303,6 +311,10 @@
System.out.println("--direct-split-size (n) Split the input stream every 'n' bytes");
System.out.println(" when importing in direct mode.");
System.out.println("");
+ System.out.println("Export options:");
+ System.out.println("--export-dir (dir) Export from an HDFS path into a table");
+ System.out.println(" (set with --table)");
+ System.out.println("");
System.out.println("Output line formatting options:");
System.out.println("--fields-terminated-by (char) Sets the field separator character");
System.out.println("--lines-terminated-by (char) Sets the end-of-line character");
@@ -327,6 +339,11 @@
System.out.println("--class-name (name) When generating one class, use this name.");
System.out.println(" This overrides --package-name.");
System.out.println("");
+ System.out.println("Library loading options:");
+ System.out.println("--jar-file (file) Disable code generation; use specified jar");
+ System.out.println("--class-name (name) The class within the jar that represents");
+ System.out.println(" the table to import/export");
+ System.out.println("");
System.out.println("Additional commands:");
System.out.println("--list-tables List tables in database and exit");
System.out.println("--list-databases List all databases available and exit");
@@ -457,6 +474,9 @@
this.action = ControlAction.ListTables;
} else if (args[i].equals("--all-tables")) {
this.allTables = true;
+ } else if (args[i].equals("--export-dir")) {
+ this.exportDir = args[++i];
+ this.action = ControlAction.Export;
} else if (args[i].equals("--local")) {
// TODO(aaron): Remove this after suitable deprecation time period.
LOG.warn("--local is deprecated; use --direct instead.");
@@ -486,21 +506,21 @@
String numMappersStr = args[++i];
this.numMappers = Integer.valueOf(numMappersStr);
} else if (args[i].equals("--fields-terminated-by")) {
- this.outputFieldDelim = ImportOptions.toChar(args[++i]);
+ this.outputFieldDelim = SqoopOptions.toChar(args[++i]);
this.areDelimsManuallySet = true;
} else if (args[i].equals("--lines-terminated-by")) {
- this.outputRecordDelim = ImportOptions.toChar(args[++i]);
+ this.outputRecordDelim = SqoopOptions.toChar(args[++i]);
this.areDelimsManuallySet = true;
} else if (args[i].equals("--optionally-enclosed-by")) {
- this.outputEnclosedBy = ImportOptions.toChar(args[++i]);
+ this.outputEnclosedBy = SqoopOptions.toChar(args[++i]);
this.outputMustBeEnclosed = false;
this.areDelimsManuallySet = true;
} else if (args[i].equals("--enclosed-by")) {
- this.outputEnclosedBy = ImportOptions.toChar(args[++i]);
+ this.outputEnclosedBy = SqoopOptions.toChar(args[++i]);
this.outputMustBeEnclosed = true;
this.areDelimsManuallySet = true;
} else if (args[i].equals("--escaped-by")) {
- this.outputEscapedBy = ImportOptions.toChar(args[++i]);
+ this.outputEscapedBy = SqoopOptions.toChar(args[++i]);
this.areDelimsManuallySet = true;
} else if (args[i].equals("--mysql-delimiters")) {
this.outputFieldDelim = ',';
@@ -510,17 +530,17 @@
this.outputMustBeEnclosed = false;
this.areDelimsManuallySet = true;
} else if (args[i].equals("--input-fields-terminated-by")) {
- this.inputFieldDelim = ImportOptions.toChar(args[++i]);
+ this.inputFieldDelim = SqoopOptions.toChar(args[++i]);
} else if (args[i].equals("--input-lines-terminated-by")) {
- this.inputRecordDelim = ImportOptions.toChar(args[++i]);
+ this.inputRecordDelim = SqoopOptions.toChar(args[++i]);
} else if (args[i].equals("--input-optionally-enclosed-by")) {
- this.inputEnclosedBy = ImportOptions.toChar(args[++i]);
+ this.inputEnclosedBy = SqoopOptions.toChar(args[++i]);
this.inputMustBeEnclosed = false;
} else if (args[i].equals("--input-enclosed-by")) {
- this.inputEnclosedBy = ImportOptions.toChar(args[++i]);
+ this.inputEnclosedBy = SqoopOptions.toChar(args[++i]);
this.inputMustBeEnclosed = true;
} else if (args[i].equals("--input-escaped-by")) {
- this.inputEscapedBy = ImportOptions.toChar(args[++i]);
+ this.inputEscapedBy = SqoopOptions.toChar(args[++i]);
} else if (args[i].equals("--outdir")) {
this.codeOutputDir = args[++i];
} else if (args[i].equals("--as-sequencefile")) {
@@ -539,6 +559,8 @@
this.useCompression = true;
} else if (args[i].equals("--direct-split-size")) {
this.directSplitSize = Long.parseLong(args[++i]);
+ } else if (args[i].equals("--jar-file")) {
+ this.existingJarFile = args[++i];
} else if (args[i].equals("--list-databases")) {
this.action = ControlAction.ListDatabases;
} else if (args[i].equals("--generate-only")) {
@@ -606,6 +628,15 @@
&& this.tableName == null) {
throw new InvalidOptionsException(
"One of --table or --all-tables is required for import." + HELP_STR);
+ } else if (this.action == ControlAction.Export && this.allTables) {
+ throw new InvalidOptionsException("You cannot export with --all-tables." + HELP_STR);
+ } else if (this.action == ControlAction.Export && this.tableName == null) {
+ throw new InvalidOptionsException("Export requires a --table argument." + HELP_STR);
+ } else if (this.existingJarFile != null && this.className == null) {
+ throw new InvalidOptionsException("Jar specified with --jar-file, but no "
+ + "class specified with --class-name." + HELP_STR);
+ } else if (this.existingJarFile != null && this.action == ControlAction.GenerateOnly) {
+ throw new InvalidOptionsException("Cannot generate code using existing jar." + HELP_STR);
}
if (this.hiveImport) {
@@ -644,10 +675,22 @@
return connectString;
}
+ public void setConnectString(String connectStr) {
+ this.connectString = connectStr;
+ }
+
public String getTableName() {
return tableName;
}
+ public String getExportDir() {
+ return exportDir;
+ }
+
+ public String getExistingJarName() {
+ return existingJarFile;
+ }
+
public String[] getColumns() {
if (null == columns) {
return null;
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/hive/HiveImport.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/hive/HiveImport.java
index 7e478f2..f5a709b 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/hive/HiveImport.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/hive/HiveImport.java
@@ -32,7 +32,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.manager.ConnManager;
import org.apache.hadoop.sqoop.util.Executor;
import org.apache.hadoop.sqoop.util.LoggingAsyncSink;
@@ -46,11 +46,11 @@
public static final Log LOG = LogFactory.getLog(HiveImport.class.getName());
- private ImportOptions options;
+ private SqoopOptions options;
private ConnManager connManager;
private Configuration configuration;
- public HiveImport(final ImportOptions opts, final ConnManager connMgr, final Configuration conf) {
+ public HiveImport(final SqoopOptions opts, final ConnManager connMgr, final Configuration conf) {
this.options = opts;
this.connManager = connMgr;
this.configuration = conf;
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/hive/TableDefWriter.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/hive/TableDefWriter.java
index f8ab7e2..fa06c8c 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/hive/TableDefWriter.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/hive/TableDefWriter.java
@@ -22,9 +22,8 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.manager.ConnManager;
-import org.apache.hadoop.sqoop.hive.HiveTypes;
import java.io.File;
import java.io.IOException;
@@ -47,7 +46,7 @@
public static final Log LOG = LogFactory.getLog(TableDefWriter.class.getName());
- private ImportOptions options;
+ private SqoopOptions options;
private ConnManager connManager;
private Configuration configuration;
private String tableName;
@@ -62,7 +61,7 @@
* @param withComments if true, then tables will be created with a
* timestamp comment.
*/
- public TableDefWriter(final ImportOptions opts, final ConnManager connMgr,
+ public TableDefWriter(final SqoopOptions opts, final ConnManager connMgr,
final String table, final Configuration config, final boolean withComments) {
this.options = opts;
this.connManager = connMgr;
@@ -95,9 +94,9 @@
first = false;
Integer colType = columnTypes.get(col);
- String hiveColType = HiveTypes.toHiveType(colType);
+ String hiveColType = connManager.toHiveType(colType);
if (null == hiveColType) {
- throw new IOException("Hive does not support the SQL type for column " + col);
+ throw new IOException("Hive does not support the SQL type for column " + col);
}
sb.append(col + " " + hiveColType);
@@ -115,10 +114,10 @@
sb.append("COMMENT 'Imported by sqoop on " + curDateStr + "' ");
}
- sb.append("ROW FORMAT DELIMITED FIELDS TERMINATED BY '\\0");
- sb.append(Integer.toOctalString((int) options.getOutputFieldDelim()));
- sb.append("' LINES TERMINATED BY '\\0");
- sb.append(Integer.toOctalString((int) options.getOutputRecordDelim()));
+ sb.append("ROW FORMAT DELIMITED FIELDS TERMINATED BY '");
+ sb.append(getHiveOctalCharCode((int) options.getOutputFieldDelim()));
+ sb.append("' LINES TERMINATED BY '");
+ sb.append(getHiveOctalCharCode((int) options.getOutputRecordDelim()));
sb.append("' STORED AS TEXTFILE");
LOG.debug("Create statement: " + sb.toString());
@@ -170,5 +169,28 @@
LOG.debug("Load statement: " + sb.toString());
return sb.toString();
}
+
+ /**
+ * Return a string identifying the character to use as a delimiter
+ * in Hive, in octal representation.
+ * Hive can specify delimiter characters in the form '\ooo' where
+ * ooo is a three-digit octal number between 000 and 177. Values
+ * may not be truncated ('\12' is wrong; '\012' is ok) nor may they
+ * be zero-prefixed (e.g., '\0177' is wrong).
+ *
+ * @param charNum the character to use as a delimiter
+ * @return a string of the form "\ooo" where ooo is an octal number
+ * in [000, 177].
+ * @throws IllegalArgumentException if charNum >> 0177.
+ */
+ static String getHiveOctalCharCode(int charNum)
+ throws IllegalArgumentException {
+ if (charNum > 0177) {
+ throw new IllegalArgumentException(
+ "Character " + charNum + " is an out-of-range delimiter");
+ }
+
+ return String.format("\\%03o", charNum);
+ }
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/lib/FieldFormatter.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/lib/FieldFormatter.java
index 95d4010..a1b6742 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/lib/FieldFormatter.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/lib/FieldFormatter.java
@@ -57,6 +57,10 @@
boolean escapingLegal = (null != escape && escape.length() > 0 && !escape.equals("\000"));
String withEscapes;
+ if (null == str) {
+ return null;
+ }
+
if (escapingLegal) {
// escaping is legal. Escape any instances of the escape char itself
withEscapes = str.replace(escape, escape + escape);
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ConnManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ConnManager.java
index df72788..493bb81 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ConnManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ConnManager.java
@@ -26,7 +26,8 @@
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.sqoop.util.ImportError;
+import org.apache.hadoop.sqoop.util.ExportException;
+import org.apache.hadoop.sqoop.util.ImportException;
/**
* Abstract interface that manages connections to a database.
@@ -56,6 +57,20 @@
public abstract String getPrimaryKey(String tableName);
/**
+ * Return java type for SQL type
+ * @param sqlType sql type
+ * @return java type
+ */
+ public abstract String toJavaType(int sqlType);
+
+ /**
+ * Return hive type for SQL type
+ * @param sqlType sql type
+ * @return hive type
+ */
+ public abstract String toHiveType(int sqlType);
+
+ /**
* Return an unordered mapping from colname to sqltype for
* all columns in a table.
*
@@ -91,11 +106,43 @@
* Perform an import of a table from the database into HDFS
*/
public abstract void importTable(ImportJobContext context)
- throws IOException, ImportError;
+ throws IOException, ImportException;
+
+ /**
+ * When using a column name in a generated SQL query, how (if at all)
+ * should we escape that column name? e.g., a column named "table"
+ * may need to be quoted with backtiks: "`table`".
+ *
+ * @param colName the column name as provided by the user, etc.
+ * @return how the column name should be rendered in the sql text.
+ */
+ public String escapeColName(String colName) {
+ return colName;
+ }
+
+ /**
+ * When using a table name in a generated SQL query, how (if at all)
+ * should we escape that column name? e.g., a table named "table"
+ * may need to be quoted with backtiks: "`table`".
+ *
+ * @param tableName the table name as provided by the user, etc.
+ * @return how the table name should be rendered in the sql text.
+ */
+ public String escapeTableName(String tableName) {
+ return tableName;
+ }
/**
* Perform any shutdown operations on the connection.
*/
public abstract void close() throws SQLException;
+
+ /**
+ * Export data stored in HDFS into a table in a database
+ */
+ public void exportTable(ExportJobContext context)
+ throws IOException, ExportException {
+ throw new ExportException("This database does not support exports");
+ }
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/DefaultManagerFactory.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/DefaultManagerFactory.java
index 2ef12e6..e6a7a2e 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/DefaultManagerFactory.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/DefaultManagerFactory.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.sqoop.manager;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -31,7 +31,7 @@
public static final Log LOG = LogFactory.getLog(DefaultManagerFactory.class.getName());
- public ConnManager accept(ImportOptions options) {
+ public ConnManager accept(SqoopOptions options) {
String manualDriver = options.getDriverClassName();
if (manualDriver != null) {
// User has manually specified JDBC implementation with --driver.
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/DirectPostgresqlManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/DirectPostgresqlManager.java
index a3f9454..7c80f70 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/DirectPostgresqlManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/DirectPostgresqlManager.java
@@ -34,14 +34,14 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.io.SplittableBufferedWriter;
import org.apache.hadoop.sqoop.util.AsyncSink;
import org.apache.hadoop.sqoop.util.DirectImportUtils;
import org.apache.hadoop.sqoop.util.ErrorableAsyncSink;
import org.apache.hadoop.sqoop.util.ErrorableThread;
import org.apache.hadoop.sqoop.util.Executor;
-import org.apache.hadoop.sqoop.util.ImportError;
+import org.apache.hadoop.sqoop.util.ImportException;
import org.apache.hadoop.sqoop.util.JdbcUrl;
import org.apache.hadoop.sqoop.util.LoggingAsyncSink;
import org.apache.hadoop.sqoop.util.PerfCounters;
@@ -53,7 +53,7 @@
public class DirectPostgresqlManager extends PostgresqlManager {
public static final Log LOG = LogFactory.getLog(DirectPostgresqlManager.class.getName());
- public DirectPostgresqlManager(final ImportOptions opts) {
+ public DirectPostgresqlManager(final SqoopOptions opts) {
// Inform superclass that we're overriding import method via alt. constructor.
super(opts, true);
}
@@ -66,9 +66,9 @@
static class PostgresqlAsyncSink extends ErrorableAsyncSink {
private final SplittableBufferedWriter writer;
private final PerfCounters counters;
- private final ImportOptions options;
+ private final SqoopOptions options;
- PostgresqlAsyncSink(final SplittableBufferedWriter w, final ImportOptions opts,
+ PostgresqlAsyncSink(final SplittableBufferedWriter w, final SqoopOptions opts,
final PerfCounters ctrs) {
this.writer = w;
this.options = opts;
@@ -85,11 +85,11 @@
private final SplittableBufferedWriter writer;
private final InputStream stream;
- private final ImportOptions options;
+ private final SqoopOptions options;
private final PerfCounters counters;
PostgresqlStreamThread(final InputStream is, final SplittableBufferedWriter w,
- final ImportOptions opts, final PerfCounters ctrs) {
+ final SqoopOptions opts, final PerfCounters ctrs) {
this.stream = is;
this.writer = w;
this.options = opts;
@@ -278,15 +278,15 @@
* via COPY FILE TO STDOUT.
*/
public void importTable(ImportJobContext context)
- throws IOException, ImportError {
+ throws IOException, ImportException {
String tableName = context.getTableName();
String jarFile = context.getJarFile();
- ImportOptions options = context.getOptions();
+ SqoopOptions options = context.getOptions();
LOG.info("Beginning psql fast path import");
- if (options.getFileLayout() != ImportOptions.FileLayout.TextFile) {
+ if (options.getFileLayout() != SqoopOptions.FileLayout.TextFile) {
// TODO(aaron): Support SequenceFile-based load-in
LOG.warn("File import layout" + options.getFileLayout()
+ " is not supported by");
@@ -323,7 +323,7 @@
int port = JdbcUrl.getPort(connectString);
if (null == databaseName) {
- throw new ImportError("Could not determine database name");
+ throw new ImportException("Could not determine database name");
}
LOG.info("Performing import of table " + tableName + " from database " + databaseName);
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ExportJobContext.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ExportJobContext.java
new file mode 100644
index 0000000..cb32a51
--- /dev/null
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ExportJobContext.java
@@ -0,0 +1,56 @@
+/**
+ * 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.sqoop.manager;
+
+import org.apache.hadoop.sqoop.SqoopOptions;
+
+/**
+ * A set of parameters describing an export operation; this is passed to
+ * ConnManager.exportTable() as its argument.
+ */
+public class ExportJobContext {
+
+ private String tableName;
+ private String jarFile;
+ private SqoopOptions options;
+
+ public ExportJobContext(final String table, final String jar, final SqoopOptions opts) {
+ this.tableName = table;
+ this.jarFile = jar;
+ this.options = opts;
+ }
+
+ /** @return the name of the table to export. */
+ public String getTableName() {
+ return tableName;
+ }
+
+ /** @return the name of the jar file containing the user's compiled
+ * ORM classes to use during the export.
+ */
+ public String getJarFile() {
+ return jarFile;
+ }
+
+ /** @return the SqoopOptions configured by the user */
+ public SqoopOptions getOptions() {
+ return options;
+ }
+}
+
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/GenericJdbcManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/GenericJdbcManager.java
index 78b629e..104ce67 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/GenericJdbcManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/GenericJdbcManager.java
@@ -24,7 +24,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
/**
* Database manager that is connects to a generic JDBC-compliant
@@ -38,7 +38,7 @@
private String jdbcDriverClass;
private Connection connection;
- public GenericJdbcManager(final String driverClass, final ImportOptions opts) {
+ public GenericJdbcManager(final String driverClass, final SqoopOptions opts) {
super(opts);
this.jdbcDriverClass = driverClass;
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/HsqldbManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/HsqldbManager.java
index 25e52d8..210dfb0 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/HsqldbManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/HsqldbManager.java
@@ -21,7 +21,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
/**
* Manages connections to hsqldb databases.
@@ -38,7 +38,7 @@
// "PUBLIC";
private static final String HSQL_SCHEMA_NAME = "PUBLIC";
- public HsqldbManager(final ImportOptions opts) {
+ public HsqldbManager(final SqoopOptions opts) {
super(DRIVER_CLASS, opts);
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ImportJobContext.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ImportJobContext.java
index 81f2b9f..06c2022 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ImportJobContext.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ImportJobContext.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.sqoop.manager;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
/**
* A set of parameters describing an import operation; this is passed to
@@ -28,9 +28,9 @@
private String tableName;
private String jarFile;
- private ImportOptions options;
+ private SqoopOptions options;
- public ImportJobContext(final String table, final String jar, final ImportOptions opts) {
+ public ImportJobContext(final String table, final String jar, final SqoopOptions opts) {
this.tableName = table;
this.jarFile = jar;
this.options = opts;
@@ -48,8 +48,8 @@
return jarFile;
}
- /** @return the ImportOptions configured by the user */
- public ImportOptions getOptions() {
+ /** @return the SqoopOptions configured by the user */
+ public SqoopOptions getOptions() {
return options;
}
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/LocalMySQLManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/LocalMySQLManager.java
index 263f3eb..0180c06 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/LocalMySQLManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/LocalMySQLManager.java
@@ -34,7 +34,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.io.SplittableBufferedWriter;
import org.apache.hadoop.sqoop.lib.FieldFormatter;
import org.apache.hadoop.sqoop.lib.RecordParser;
@@ -42,7 +42,7 @@
import org.apache.hadoop.sqoop.util.DirectImportUtils;
import org.apache.hadoop.sqoop.util.ErrorableAsyncSink;
import org.apache.hadoop.sqoop.util.ErrorableThread;
-import org.apache.hadoop.sqoop.util.ImportError;
+import org.apache.hadoop.sqoop.util.ImportException;
import org.apache.hadoop.sqoop.util.JdbcUrl;
import org.apache.hadoop.sqoop.util.LoggingAsyncSink;
import org.apache.hadoop.sqoop.util.PerfCounters;
@@ -154,11 +154,11 @@
*/
static class ReparsingAsyncSink extends ErrorableAsyncSink {
private final SplittableBufferedWriter writer;
- private final ImportOptions options;
+ private final SqoopOptions options;
private final PerfCounters counters;
ReparsingAsyncSink(final SplittableBufferedWriter w,
- final ImportOptions opts, final PerfCounters ctrs) {
+ final SqoopOptions opts, final PerfCounters ctrs) {
this.writer = w;
this.options = opts;
this.counters = ctrs;
@@ -174,12 +174,12 @@
ReparsingStreamThread.class.getName());
private final SplittableBufferedWriter writer;
- private final ImportOptions options;
+ private final SqoopOptions options;
private final InputStream stream;
private final PerfCounters counters;
ReparsingStreamThread(final InputStream is,
- final SplittableBufferedWriter w, final ImportOptions opts,
+ final SplittableBufferedWriter w, final SqoopOptions opts,
final PerfCounters ctrs) {
this.writer = w;
this.options = opts;
@@ -291,7 +291,7 @@
}
- public LocalMySQLManager(final ImportOptions options) {
+ public LocalMySQLManager(final SqoopOptions options) {
super(options, false);
}
@@ -343,15 +343,15 @@
* the database and upload the files directly to HDFS.
*/
public void importTable(ImportJobContext context)
- throws IOException, ImportError {
+ throws IOException, ImportException {
String tableName = context.getTableName();
String jarFile = context.getJarFile();
- ImportOptions options = context.getOptions();
+ SqoopOptions options = context.getOptions();
LOG.info("Beginning mysqldump fast path import");
- if (options.getFileLayout() != ImportOptions.FileLayout.TextFile) {
+ if (options.getFileLayout() != SqoopOptions.FileLayout.TextFile) {
// TODO(aaron): Support SequenceFile-based load-in
LOG.warn("File import layout " + options.getFileLayout()
+ " is not supported by");
@@ -370,7 +370,7 @@
int port = JdbcUrl.getPort(connectString);
if (null == databaseName) {
- throw new ImportError("Could not determine database name");
+ throw new ImportException("Could not determine database name");
}
LOG.info("Performing import of table " + tableName + " from database " + databaseName);
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ManagerFactory.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ManagerFactory.java
index 5a90661..4bcd756 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ManagerFactory.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ManagerFactory.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.sqoop.manager;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
/**
* Interface for factory classes for ConnManager implementations.
@@ -28,6 +28,6 @@
* one such call returns a non-null ConnManager instance.
*/
public abstract class ManagerFactory {
- public abstract ConnManager accept(ImportOptions options);
+ public abstract ConnManager accept(SqoopOptions options);
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java
index 494a95b..c9290be 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java
@@ -19,6 +19,8 @@
package org.apache.hadoop.sqoop.manager;
import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
@@ -28,8 +30,8 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.sqoop.ImportOptions;
-import org.apache.hadoop.sqoop.util.ImportError;
+import org.apache.hadoop.sqoop.SqoopOptions;
+import org.apache.hadoop.sqoop.util.ImportException;
/**
* Manages connections to MySQL databases
@@ -44,11 +46,11 @@
// set to true after we warn the user that we can use direct fastpath.
private static boolean warningPrinted = false;
- public MySQLManager(final ImportOptions opts) {
+ public MySQLManager(final SqoopOptions opts) {
super(DRIVER_CLASS, opts);
}
- protected MySQLManager(final ImportOptions opts, boolean ignored) {
+ protected MySQLManager(final SqoopOptions opts, boolean ignored) {
// constructor used by subclasses to avoid the --direct warning.
super(DRIVER_CLASS, opts);
}
@@ -56,7 +58,7 @@
@Override
protected String getColNamesQuery(String tableName) {
// Use mysql-specific hints and LIMIT to return fast
- return "SELECT t.* FROM " + tableName + " AS t LIMIT 1";
+ return "SELECT t.* FROM " + escapeTableName(tableName) + " AS t LIMIT 1";
}
@Override
@@ -93,7 +95,7 @@
@Override
public void importTable(ImportJobContext context)
- throws IOException, ImportError {
+ throws IOException, ImportException {
// Check that we're not doing a MapReduce from localhost. If we are, point
// out that we could use mysqldump.
@@ -113,11 +115,63 @@
}
}
+ checkDateTimeBehavior(context);
+
// Then run the normal importTable() method.
super.importTable(context);
}
/**
+ * MySQL allows TIMESTAMP fields to have the value '0000-00-00 00:00:00',
+ * which causes errors in import. If the user has not set the
+ * zeroDateTimeBehavior property already, we set it for them to coerce
+ * the type to null.
+ */
+ private void checkDateTimeBehavior(ImportJobContext context) {
+ final String zeroBehaviorStr = "zeroDateTimeBehavior";
+ final String convertToNull = "=convertToNull";
+
+ String connectStr = context.getOptions().getConnectString();
+ if (connectStr.indexOf("jdbc:") != 0) {
+ // This connect string doesn't have the prefix we expect.
+ // We can't parse the rest of it here.
+ return;
+ }
+
+ // This starts with 'jdbc:mysql://' ... let's remove the 'jdbc:'
+ // prefix so that java.net.URI can parse the rest of the line.
+ String uriComponent = connectStr.substring(5);
+ try {
+ URI uri = new URI(uriComponent);
+ String query = uri.getQuery(); // get the part after a '?'
+
+ // If they haven't set the zeroBehavior option, set it to
+ // squash-null for them.
+ if (null == query) {
+ connectStr = connectStr + "?" + zeroBehaviorStr + convertToNull;
+ LOG.info("Setting zero DATETIME behavior to convertToNull (mysql)");
+ } else if (query.length() == 0) {
+ connectStr = connectStr + zeroBehaviorStr + convertToNull;
+ LOG.info("Setting zero DATETIME behavior to convertToNull (mysql)");
+ } else if (query.indexOf(zeroBehaviorStr) == -1) {
+ if (!connectStr.endsWith("&")) {
+ connectStr = connectStr + "&";
+ }
+ connectStr = connectStr + zeroBehaviorStr + convertToNull;
+ LOG.info("Setting zero DATETIME behavior to convertToNull (mysql)");
+ }
+
+ LOG.debug("Rewriting connect string to " + connectStr);
+ context.getOptions().setConnectString(connectStr);
+ } catch (URISyntaxException use) {
+ // Just ignore this. If we can't parse the URI, don't attempt
+ // to add any extra flags to it.
+ LOG.debug("mysql: Couldn't parse connect str in checkDateTimeBehavior: "
+ + use);
+ }
+ }
+
+ /**
* Executes an arbitrary SQL statement. Sets mysql-specific parameter
* to ensure the entire table is not buffered in RAM before reading
* any rows. A consequence of this is that every ResultSet returned
@@ -141,5 +195,35 @@
LOG.info("Executing SQL statement: " + stmt);
return statement.executeQuery();
}
+
+ /**
+ * When using a column name in a generated SQL query, how (if at all)
+ * should we escape that column name? e.g., a column named "table"
+ * may need to be quoted with backtiks: "`table`".
+ *
+ * @param colName the column name as provided by the user, etc.
+ * @return how the column name should be rendered in the sql text.
+ */
+ public String escapeColName(String colName) {
+ if (null == colName) {
+ return null;
+ }
+ return "`" + colName + "`";
+ }
+
+ /**
+ * When using a table name in a generated SQL query, how (if at all)
+ * should we escape that column name? e.g., a table named "table"
+ * may need to be quoted with backtiks: "`table`".
+ *
+ * @param tableName the table name as provided by the user, etc.
+ * @return how the table name should be rendered in the sql text.
+ */
+ public String escapeTableName(String tableName) {
+ if (null == tableName) {
+ return null;
+ }
+ return "`" + tableName + "`";
+ }
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/OracleManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/OracleManager.java
index 7b0a63f..869057b 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/OracleManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/OracleManager.java
@@ -23,15 +23,18 @@
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
+import java.sql.Types;
import java.util.ArrayList;
+import java.util.TimeZone;
+import java.lang.reflect.Method;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.mapred.ImportJob;
-import org.apache.hadoop.sqoop.util.ImportError;
+import org.apache.hadoop.sqoop.util.ImportException;
/**
* Manages connections to Oracle databases.
@@ -44,13 +47,13 @@
// driver class to ensure is loaded when making db connection.
private static final String DRIVER_CLASS = "oracle.jdbc.OracleDriver";
- public OracleManager(final ImportOptions opts) {
+ public OracleManager(final SqoopOptions opts) {
super(DRIVER_CLASS, opts);
}
protected String getColNamesQuery(String tableName) {
// SqlManager uses "tableName AS t" which doesn't work in Oracle.
- return "SELECT t.* FROM " + tableName + " t";
+ return "SELECT t.* FROM " + escapeTableName(tableName) + " t";
}
/**
@@ -83,19 +86,61 @@
// We only use this for metadata queries. Loosest semantics are okay.
connection.setTransactionIsolation(Connection.TRANSACTION_READ_COMMITTED);
+ // Setting session time zone
+ setSessionTimeZone(connection);
+
return connection;
}
/**
+ * Set session time zone
+ * @param conn Connection object
+ * @throws SQLException instance
+ */
+ private void setSessionTimeZone(Connection conn) throws SQLException {
+ // need to use reflection to call the method setSessionTimeZone on the OracleConnection class
+ // because oracle specific java libraries are not accessible in this context
+ Method method;
+ try {
+ method = conn.getClass().getMethod(
+ "setSessionTimeZone", new Class [] {String.class});
+ } catch (Exception ex) {
+ LOG.error("Could not find method setSessionTimeZone in " + conn.getClass().getName(), ex);
+ // rethrow SQLException
+ throw new SQLException(ex);
+ }
+
+ // Need to set the time zone in order for Java
+ // to correctly access the column "TIMESTAMP WITH LOCAL TIME ZONE"
+ String clientTimeZone = TimeZone.getDefault().getID();
+ try {
+ method.setAccessible(true);
+ method.invoke(conn, clientTimeZone);
+ LOG.info("Time zone has been set");
+ } catch (Exception ex) {
+ LOG.warn("Time zone " + clientTimeZone +
+ " could not be set on oracle database.");
+ LOG.info("Setting default time zone: UTC");
+ try {
+ method.invoke(conn, "UTC");
+ } catch (Exception ex2) {
+ LOG.error("Could not set time zone for oracle connection", ex2);
+ // rethrow SQLException
+ throw new SQLException(ex);
+ }
+ }
+ }
+
+ /**
* This importTable() implementation continues to use the older DBInputFormat
* because DataDrivenDBInputFormat does not currently work with Oracle.
*/
public void importTable(ImportJobContext context)
- throws IOException, ImportError {
+ throws IOException, ImportException {
String tableName = context.getTableName();
String jarFile = context.getJarFile();
- ImportOptions options = context.getOptions();
+ SqoopOptions options = context.getOptions();
ImportJob importer = new ImportJob(options);
String splitCol = options.getSplitByCol();
if (null == splitCol) {
@@ -105,11 +150,123 @@
if (null == splitCol) {
// Can't infer a primary key.
- throw new ImportError("No primary key could be found for table " + tableName
+ throw new ImportException("No primary key could be found for table " + tableName
+ ". Please specify one with --split-by.");
}
importer.runImport(tableName, jarFile, splitCol, options.getConf());
}
+
+ /**
+ * Resolve a database-specific type to the Java type that should contain it.
+ * @param sqlType
+ * @return the name of a Java type to hold the sql datatype, or null if none.
+ */
+ public String toJavaType(int sqlType) {
+ String defaultJavaType = super.toJavaType(sqlType);
+ return (defaultJavaType == null) ? dbToJavaType(sqlType) : defaultJavaType;
+ }
+
+ /**
+ * Attempt to map sql type to java type
+ * @param sqlType sql type
+ * @return java type
+ */
+ private String dbToJavaType(int sqlType) {
+ // load class oracle.jdbc.OracleTypes
+ // need to use reflection because oracle specific libraries
+ // are not accessible in this context
+ Class typeClass = getTypeClass("oracle.jdbc.OracleTypes");
+
+ // check if it is TIMESTAMPTZ
+ int dbType = getDatabaseType(typeClass, "TIMESTAMPTZ");
+ if (sqlType == dbType) {
+ return "java.sql.Timestamp";
+ }
+
+ // check if it is TIMESTAMPLTZ
+ dbType = getDatabaseType(typeClass, "TIMESTAMPLTZ");
+ if (sqlType == dbType) {
+ return "java.sql.Timestamp";
+ }
+
+ // return null if no java type was found for sqlType
+ return null;
+ }
+
+ /**
+ * Attempt to map sql type to hive type
+ * @param sqlType sql data type
+ * @return hive data type
+ */
+ public String toHiveType(int sqlType) {
+ String defaultHiveType = super.toHiveType(sqlType);
+ return (defaultHiveType == null) ? dbToHiveType(sqlType) : defaultHiveType;
+ }
+
+ /**
+ * Resolve a database-specific type to Hive type
+ * @param sqlType sql type
+ * @return hive type
+ */
+ private String dbToHiveType(int sqlType) {
+ // load class oracle.jdbc.OracleTypes
+ // need to use reflection because oracle specific libraries
+ // are not accessible in this context
+ Class typeClass = getTypeClass("oracle.jdbc.OracleTypes");
+
+ // check if it is TIMESTAMPTZ
+ int dbType = getDatabaseType(typeClass, "TIMESTAMPTZ");
+ if (sqlType == dbType) {
+ return "STRING";
+ }
+
+ // check if it is TIMESTAMPLTZ
+ dbType = getDatabaseType(typeClass, "TIMESTAMPLTZ");
+ if (sqlType == dbType) {
+ return "STRING";
+ }
+
+ // return null if no hive type was found for sqlType
+ return null;
+ }
+
+ /**
+ * Get database type
+ * @param clazz oracle class representing sql types
+ * @param fieldName field name
+ * @return value of database type constant
+ */
+ private int getDatabaseType(Class clazz, String fieldName) {
+ // need to use reflection to extract constant values
+ // because the database specific java libraries are not accessible in this context
+ int value = -1;
+ try {
+ java.lang.reflect.Field field = clazz.getDeclaredField(fieldName);
+ value = field.getInt(null);
+ } catch (NoSuchFieldException ex) {
+ LOG.error("Could not retrieve value for field " + fieldName, ex);
+ } catch (IllegalAccessException ex) {
+ LOG.error("Could not retrieve value for field " + fieldName, ex);
+ }
+ return value;
+ }
+
+ /**
+ * Load class by name
+ * @param className class name
+ * @return class instance
+ */
+ private Class getTypeClass(String className) {
+ // need to use reflection to load class
+ // because the database specific java libraries are not accessible in this context
+ Class typeClass = null;
+ try {
+ typeClass = Class.forName(className);
+ } catch (ClassNotFoundException ex) {
+ LOG.error("Could not load class " + className, ex);
+ }
+ return typeClass;
+ }
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/PostgresqlManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/PostgresqlManager.java
index f815153..283d741 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/PostgresqlManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/PostgresqlManager.java
@@ -28,8 +28,8 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.sqoop.ImportOptions;
-import org.apache.hadoop.sqoop.util.ImportError;
+import org.apache.hadoop.sqoop.SqoopOptions;
+import org.apache.hadoop.sqoop.util.ImportException;
/**
* Manages connections to Postgresql databases
@@ -46,11 +46,11 @@
// set to true after we warn the user that we can use direct fastpath.
private static boolean warningPrinted = false;
- public PostgresqlManager(final ImportOptions opts) {
+ public PostgresqlManager(final SqoopOptions opts) {
super(DRIVER_CLASS, opts);
}
- protected PostgresqlManager(final ImportOptions opts, boolean ignored) {
+ protected PostgresqlManager(final SqoopOptions opts, boolean ignored) {
// constructor used by subclasses to avoid the --direct warning.
super(DRIVER_CLASS, opts);
}
@@ -67,12 +67,12 @@
@Override
protected String getColNamesQuery(String tableName) {
// Use LIMIT to return fast
- return "SELECT t.* FROM " + tableName + " AS t LIMIT 1";
+ return "SELECT t.* FROM " + escapeTableName(tableName) + " AS t LIMIT 1";
}
@Override
public void importTable(ImportJobContext context)
- throws IOException, ImportError {
+ throws IOException, ImportException {
// The user probably should have requested --direct to invoke pg_dump.
// Display a warning informing them of this fact.
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java
index 1ffaa5e..42846d9 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java
@@ -18,9 +18,12 @@
package org.apache.hadoop.sqoop.manager;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
+import org.apache.hadoop.sqoop.hive.HiveTypes;
import org.apache.hadoop.sqoop.mapreduce.DataDrivenImportJob;
-import org.apache.hadoop.sqoop.util.ImportError;
+import org.apache.hadoop.sqoop.mapreduce.ExportJob;
+import org.apache.hadoop.sqoop.util.ExportException;
+import org.apache.hadoop.sqoop.util.ImportException;
import org.apache.hadoop.sqoop.util.ResultSetPrinter;
import java.io.IOException;
@@ -49,14 +52,14 @@
public static final Log LOG = LogFactory.getLog(SqlManager.class.getName());
- protected ImportOptions options;
+ protected SqoopOptions options;
/**
* Constructs the SqlManager
* @param opts
* @param specificMgr
*/
- public SqlManager(final ImportOptions opts) {
+ public SqlManager(final SqoopOptions opts) {
this.options = opts;
}
@@ -65,7 +68,8 @@
* be tuned per-database, but the main extraction loop is still inheritable.
*/
protected String getColNamesQuery(String tableName) {
- return "SELECT t.* FROM " + tableName + " AS t";
+ // adding where clause to prevent loading a big table
+ return "SELECT t.* FROM " + escapeTableName(tableName) + " AS t WHERE 1=0";
}
@Override
@@ -167,13 +171,13 @@
if (!first) {
sb.append(", ");
}
- sb.append(col);
+ sb.append(escapeColName(col));
first = false;
}
sb.append(" FROM ");
- sb.append(tableName);
+ sb.append(escapeTableName(tableName));
sb.append(" AS "); // needed for hsqldb; doesn't hurt anyone else.
- sb.append(tableName);
+ sb.append(escapeTableName(tableName));
return execute(sb.toString());
}
@@ -261,10 +265,10 @@
* via DataDrivenImportJob to read the table with DataDrivenDBInputFormat.
*/
public void importTable(ImportJobContext context)
- throws IOException, ImportError {
+ throws IOException, ImportException {
String tableName = context.getTableName();
String jarFile = context.getJarFile();
- ImportOptions options = context.getOptions();
+ SqoopOptions options = context.getOptions();
DataDrivenImportJob importer = new DataDrivenImportJob(options);
String splitCol = options.getSplitByCol();
if (null == splitCol) {
@@ -274,7 +278,7 @@
if (null == splitCol) {
// Can't infer a primary key.
- throw new ImportError("No primary key could be found for table " + tableName
+ throw new ImportException("No primary key could be found for table " + tableName
+ ". Please specify one with --split-by.");
}
@@ -305,7 +309,7 @@
* @param sqlType
* @return the name of a Java type to hold the sql datatype, or null if none.
*/
- public static String toJavaType(int sqlType) {
+ public String toJavaType(int sqlType) {
// mappings from http://java.sun.com/j2se/1.3/docs/guide/jdbc/getstart/mapping.html
if (sqlType == Types.INTEGER) {
return "Integer";
@@ -344,10 +348,19 @@
} else {
// TODO(aaron): Support BINARY, VARBINARY, LONGVARBINARY, DISTINCT, CLOB, BLOB, ARRAY,
// STRUCT, REF, JAVA_OBJECT.
+ // return database specific java data type
return null;
}
}
+ /**
+ * Resolve a database-specific type to Hive data type
+ * @param sqlType sql type
+ * @return hive type
+ */
+ public String toHiveType(int sqlType) {
+ return HiveTypes.toHiveType(sqlType);
+ }
public void close() throws SQLException {
}
@@ -424,4 +437,13 @@
return connection;
}
+
+ /**
+ * Export data stored in HDFS into a table in a database
+ */
+ public void exportTable(ExportJobContext context)
+ throws IOException, ExportException {
+ ExportJob exportJob = new ExportJob(context);
+ exportJob.runExport();
+ }
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapred/ImportJob.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapred/ImportJob.java
index d770531..d7167bb 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapred/ImportJob.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapred/ImportJob.java
@@ -42,7 +42,7 @@
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.sqoop.ConnFactory;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.manager.ConnManager;
import org.apache.hadoop.sqoop.orm.TableClassName;
import org.apache.hadoop.sqoop.util.ClassLoaderStack;
@@ -55,9 +55,9 @@
public static final Log LOG = LogFactory.getLog(ImportJob.class.getName());
- private ImportOptions options;
+ private SqoopOptions options;
- public ImportJob(final ImportOptions opts) {
+ public ImportJob(final SqoopOptions opts) {
this.options = opts;
}
@@ -100,7 +100,7 @@
outputPath = new Path(tableName);
}
- if (options.getFileLayout() == ImportOptions.FileLayout.TextFile) {
+ if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) {
job.setOutputFormat(RawKeyTextOutputFormat.class);
job.setMapperClass(TextImportMapper.class);
job.setOutputKeyClass(Text.class);
@@ -109,7 +109,7 @@
FileOutputFormat.setCompressOutput(job, true);
FileOutputFormat.setOutputCompressorClass(job, GzipCodec.class);
}
- } else if (options.getFileLayout() == ImportOptions.FileLayout.SequenceFile) {
+ } else if (options.getFileLayout() == SqoopOptions.FileLayout.SequenceFile) {
job.setOutputFormat(SequenceFileOutputFormat.class);
if (options.shouldUseCompression()) {
SequenceFileOutputFormat.setCompressOutput(job, true);
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/DataDrivenImportJob.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/DataDrivenImportJob.java
index b9f6d31..f61f42c 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/DataDrivenImportJob.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/DataDrivenImportJob.java
@@ -39,7 +39,7 @@
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
import org.apache.hadoop.sqoop.ConnFactory;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.manager.ConnManager;
import org.apache.hadoop.sqoop.orm.TableClassName;
import org.apache.hadoop.sqoop.util.ClassLoaderStack;
@@ -53,9 +53,9 @@
public static final Log LOG = LogFactory.getLog(DataDrivenImportJob.class.getName());
- private ImportOptions options;
+ private SqoopOptions options;
- public DataDrivenImportJob(final ImportOptions opts) {
+ public DataDrivenImportJob(final SqoopOptions opts) {
this.options = opts;
}
@@ -74,7 +74,8 @@
String tableClassName = new TableClassName(options).getClassForTable(tableName);
- boolean isLocal = "local".equals(conf.get("mapred.job.tracker"));
+ boolean isLocal = "local".equals(conf.get("mapreduce.jobtracker.address"))
+ || "local".equals(conf.get("mapred.job.tracker"));
ClassLoader prevClassLoader = null;
if (isLocal) {
// If we're using the LocalJobRunner, then instead of using the compiled jar file
@@ -100,7 +101,7 @@
outputPath = new Path(tableName);
}
- if (options.getFileLayout() == ImportOptions.FileLayout.TextFile) {
+ if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) {
job.setOutputFormatClass(RawKeyTextOutputFormat.class);
job.setMapperClass(TextImportMapper.class);
job.setOutputKeyClass(Text.class);
@@ -109,7 +110,7 @@
FileOutputFormat.setCompressOutput(job, true);
FileOutputFormat.setOutputCompressorClass(job, GzipCodec.class);
}
- } else if (options.getFileLayout() == ImportOptions.FileLayout.SequenceFile) {
+ } else if (options.getFileLayout() == SqoopOptions.FileLayout.SequenceFile) {
job.setOutputFormatClass(SequenceFileOutputFormat.class);
job.setMapperClass(AutoProgressMapper.class);
if (options.shouldUseCompression()) {
@@ -123,7 +124,7 @@
int numMapTasks = options.getNumMappers();
if (numMapTasks < 1) {
- numMapTasks = ImportOptions.DEFAULT_NUM_MAPPERS;
+ numMapTasks = SqoopOptions.DEFAULT_NUM_MAPPERS;
LOG.warn("Invalid mapper count; using " + numMapTasks + " mappers.");
}
job.getConfiguration().setInt("mapred.map.tasks", numMapTasks);
@@ -148,14 +149,23 @@
colNames = mgr.getColumnNames(tableName);
}
+ String [] sqlColNames = null;
+ if (null != colNames) {
+ sqlColNames = new String[colNames.length];
+ for (int i = 0; i < colNames.length; i++) {
+ sqlColNames[i] = mgr.escapeColName(colNames[i]);
+ }
+ }
+
// It's ok if the where clause is null in DBInputFormat.setInput.
String whereClause = options.getWhereClause();
// We can't set the class properly in here, because we may not have the
// jar loaded in this JVM. So we start by calling setInput() with DBWritable,
// and then overriding the string manually.
- DataDrivenDBInputFormat.setInput(job, DBWritable.class, tableName, whereClause,
- splitByCol, colNames);
+ DataDrivenDBInputFormat.setInput(job, DBWritable.class,
+ mgr.escapeTableName(tableName), whereClause,
+ mgr.escapeColName(splitByCol), sqlColNames);
job.getConfiguration().set(DBConfiguration.INPUT_CLASS_PROPERTY, tableClassName);
PerfCounters counters = new PerfCounters();
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/ExportJob.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/ExportJob.java
new file mode 100644
index 0000000..7f45655
--- /dev/null
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/ExportJob.java
@@ -0,0 +1,207 @@
+/**
+ * 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.sqoop.mapreduce;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.db.DBConfiguration;
+import org.apache.hadoop.mapreduce.lib.db.DBOutputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+
+import org.apache.hadoop.sqoop.ConnFactory;
+import org.apache.hadoop.sqoop.SqoopOptions;
+import org.apache.hadoop.sqoop.lib.SqoopRecord;
+import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.manager.ExportJobContext;
+import org.apache.hadoop.sqoop.orm.TableClassName;
+import org.apache.hadoop.sqoop.util.ClassLoaderStack;
+
+/**
+ * Actually runs a jdbc export job using the ORM files generated by the sqoop.orm package.
+ * Uses DBOutputFormat
+ */
+public class ExportJob {
+
+ public static final Log LOG = LogFactory.getLog(ExportJob.class.getName());
+
+ public static final String SQOOP_EXPORT_TABLE_CLASS_KEY = "sqoop.export.table.class";
+
+ private ExportJobContext context;
+
+ public ExportJob(final ExportJobContext ctxt) {
+ this.context = ctxt;
+ }
+
+ /**
+ * @return true if p is a SequenceFile, or a directory containing
+ * SequenceFiles.
+ */
+ private boolean isSequenceFiles(Path p) throws IOException {
+ Configuration conf = context.getOptions().getConf();
+ FileSystem fs = p.getFileSystem(conf);
+
+ try {
+ FileStatus stat = fs.getFileStatus(p);
+
+ if (null == stat) {
+ // Couldn't get the item.
+ LOG.warn("Input path " + p + " does not exist");
+ return false;
+ }
+
+ if (stat.isDir()) {
+ FileStatus [] subitems = fs.listStatus(p);
+ if (subitems == null || subitems.length == 0) {
+ LOG.warn("Input path " + p + " contains no files");
+ return false; // empty dir.
+ }
+
+ // Pick a random child entry to examine instead.
+ stat = subitems[0];
+ }
+
+ if (null == stat) {
+ LOG.warn("null FileStatus object in isSequenceFiles(); assuming false.");
+ return false;
+ }
+
+ Path target = stat.getPath();
+ // Test target's header to see if it contains magic numbers indicating it's
+ // a SequenceFile.
+ byte [] header = new byte[3];
+ FSDataInputStream is = null;
+ try {
+ is = fs.open(target);
+ is.readFully(header);
+ } catch (IOException ioe) {
+ // Error reading header or EOF; assume not a SequenceFile.
+ LOG.warn("IOException checking SequenceFile header: " + ioe);
+ return false;
+ } finally {
+ try {
+ if (null != is) {
+ is.close();
+ }
+ } catch (IOException ioe) {
+ // ignore; closing.
+ LOG.warn("IOException closing input stream: " + ioe + "; ignoring.");
+ }
+ }
+
+ // Return true (isSequenceFile) iff the magic number sticks.
+ return header[0] == 'S' && header[1] == 'E' && header[2] == 'Q';
+ } catch (FileNotFoundException fnfe) {
+ LOG.warn("Input path " + p + " does not exist");
+ return false; // doesn't exist!
+ }
+ }
+
+ /**
+ * Run an export job to dump a table from HDFS to a database
+ */
+ public void runExport() throws IOException {
+
+ SqoopOptions options = context.getOptions();
+ Configuration conf = options.getConf();
+ String tableName = context.getTableName();
+ String tableClassName = new TableClassName(options).getClassForTable(tableName);
+ String ormJarFile = context.getJarFile();
+
+ LOG.info("Beginning export of " + tableName);
+
+ boolean isLocal = "local".equals(conf.get("mapreduce.jobtracker.address"))
+ || "local".equals(conf.get("mapred.job.tracker"));
+ ClassLoader prevClassLoader = null;
+ if (isLocal) {
+ // If we're using the LocalJobRunner, then instead of using the compiled jar file
+ // as the job source, we're running in the current thread. Push on another classloader
+ // that loads from that jar in addition to everything currently on the classpath.
+ prevClassLoader = ClassLoaderStack.addJarFile(ormJarFile, tableClassName);
+ }
+
+ try {
+ Job job = new Job(conf);
+
+ // Set the external jar to use for the job.
+ job.getConfiguration().set("mapred.jar", ormJarFile);
+
+ Path inputPath = new Path(context.getOptions().getExportDir());
+ inputPath = inputPath.makeQualified(FileSystem.get(conf));
+
+ if (isSequenceFiles(inputPath)) {
+ job.setInputFormatClass(SequenceFileInputFormat.class);
+ job.setMapperClass(SequenceFileExportMapper.class);
+ } else {
+ job.setInputFormatClass(TextInputFormat.class);
+ job.setMapperClass(TextExportMapper.class);
+ }
+
+ FileInputFormat.addInputPath(job, inputPath);
+ job.setNumReduceTasks(0);
+
+ ConnManager mgr = new ConnFactory(conf).getManager(options);
+ String username = options.getUsername();
+ if (null == username || username.length() == 0) {
+ DBConfiguration.configureDB(job.getConfiguration(), mgr.getDriverClass(),
+ options.getConnectString());
+ } else {
+ DBConfiguration.configureDB(job.getConfiguration(), mgr.getDriverClass(),
+ options.getConnectString(), username, options.getPassword());
+ }
+
+ String [] colNames = options.getColumns();
+ if (null == colNames) {
+ colNames = mgr.getColumnNames(tableName);
+ }
+ DBOutputFormat.setOutput(job, tableName, colNames);
+
+ job.setOutputFormatClass(DBOutputFormat.class);
+ job.getConfiguration().set(SQOOP_EXPORT_TABLE_CLASS_KEY, tableClassName);
+ job.setMapOutputKeyClass(SqoopRecord.class);
+ job.setMapOutputValueClass(NullWritable.class);
+
+ try {
+ job.waitForCompletion(false);
+ } catch (InterruptedException ie) {
+ throw new IOException(ie);
+ } catch (ClassNotFoundException cnfe) {
+ throw new IOException(cnfe);
+ }
+ } finally {
+ if (isLocal && null != prevClassLoader) {
+ // unload the special classloader for this jar.
+ ClassLoaderStack.setCurrentClassLoader(prevClassLoader);
+ }
+ }
+ }
+}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/SequenceFileExportMapper.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/SequenceFileExportMapper.java
new file mode 100644
index 0000000..15a9dd4
--- /dev/null
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/SequenceFileExportMapper.java
@@ -0,0 +1,43 @@
+/**
+ * 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.sqoop.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+
+import org.apache.hadoop.sqoop.lib.SqoopRecord;
+
+/**
+ * Reads a SqoopRecord from the SequenceFile in which it's packed and emits
+ * that DBWritable to the DBOutputFormat for writeback to the database.
+ */
+public class SequenceFileExportMapper
+ extends AutoProgressMapper<LongWritable, SqoopRecord, SqoopRecord, NullWritable> {
+
+ public SequenceFileExportMapper() {
+ }
+
+ public void map(LongWritable key, SqoopRecord val, Context context)
+ throws IOException, InterruptedException {
+ context.write(val, NullWritable.get());
+ }
+}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/TextExportMapper.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/TextExportMapper.java
new file mode 100644
index 0000000..cb1d002
--- /dev/null
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapreduce/TextExportMapper.java
@@ -0,0 +1,82 @@
+/**
+ * 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.sqoop.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.lib.db.DBWritable;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import org.apache.hadoop.sqoop.lib.RecordParser;
+import org.apache.hadoop.sqoop.lib.SqoopRecord;
+
+/**
+ * Converts an input record from a string representation to a parsed Sqoop record
+ * and emits that DBWritable to the DBOutputFormat for writeback to the database.
+ */
+public class TextExportMapper
+ extends AutoProgressMapper<LongWritable, Text, SqoopRecord, NullWritable> {
+
+ private SqoopRecord recordImpl;
+
+ public TextExportMapper() {
+ }
+
+ protected void setup(Context context) throws IOException, InterruptedException {
+ super.setup(context);
+
+ Configuration conf = context.getConfiguration();
+
+ // Instantiate a copy of the user's class to hold and parse the record.
+ String recordClassName = conf.get(ExportJob.SQOOP_EXPORT_TABLE_CLASS_KEY);
+ if (null == recordClassName) {
+ throw new IOException("Export table class name ("
+ + ExportJob.SQOOP_EXPORT_TABLE_CLASS_KEY
+ + ") is not set!");
+ }
+
+ try {
+ Class cls = Class.forName(recordClassName, true,
+ Thread.currentThread().getContextClassLoader());
+ recordImpl = (SqoopRecord) ReflectionUtils.newInstance(cls, conf);
+ } catch (ClassNotFoundException cnfe) {
+ throw new IOException(cnfe);
+ }
+
+ if (null == recordImpl) {
+ throw new IOException("Could not instantiate object of type " + recordClassName);
+ }
+ }
+
+
+ public void map(LongWritable key, Text val, Context context)
+ throws IOException, InterruptedException {
+ try {
+ recordImpl.parse(val);
+ context.write(recordImpl, NullWritable.get());
+ } catch (RecordParser.ParseError pe) {
+ throw new IOException("Could not parse record: " + val, pe);
+ }
+ }
+}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java
index 9c6f8fd..611e0b0 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.sqoop.orm;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.manager.ConnManager;
import org.apache.hadoop.sqoop.manager.SqlManager;
import org.apache.hadoop.sqoop.lib.BigDecimalSerializer;
@@ -33,6 +33,7 @@
import java.io.OutputStream;
import java.io.OutputStreamWriter;
import java.io.Writer;
+import java.util.HashSet;
import java.util.Map;
import org.apache.commons.logging.Log;
@@ -40,14 +41,57 @@
/**
* Creates an ORM class to represent a table from a database
- *
- *
- *
*/
public class ClassWriter {
public static final Log LOG = LogFactory.getLog(ClassWriter.class.getName());
+ // The following are keywords and cannot be used for class, method, or field
+ // names.
+ public static final HashSet<String> JAVA_RESERVED_WORDS;
+
+ static {
+ JAVA_RESERVED_WORDS = new HashSet<String>();
+
+ JAVA_RESERVED_WORDS.add("abstract");
+ JAVA_RESERVED_WORDS.add("else");
+ JAVA_RESERVED_WORDS.add("int");
+ JAVA_RESERVED_WORDS.add("strictfp");
+ JAVA_RESERVED_WORDS.add("assert");
+ JAVA_RESERVED_WORDS.add("enum");
+ JAVA_RESERVED_WORDS.add("interface");
+ JAVA_RESERVED_WORDS.add("super");
+ JAVA_RESERVED_WORDS.add("boolean");
+ JAVA_RESERVED_WORDS.add("extends");
+ JAVA_RESERVED_WORDS.add("long");
+ JAVA_RESERVED_WORDS.add("switch");
+ JAVA_RESERVED_WORDS.add("break");
+ JAVA_RESERVED_WORDS.add("false");
+ JAVA_RESERVED_WORDS.add("native");
+ JAVA_RESERVED_WORDS.add("synchronized");
+ JAVA_RESERVED_WORDS.add("byte");
+ JAVA_RESERVED_WORDS.add("final");
+ JAVA_RESERVED_WORDS.add("new");
+ JAVA_RESERVED_WORDS.add("this");
+ JAVA_RESERVED_WORDS.add("case");
+ JAVA_RESERVED_WORDS.add("finally");
+ JAVA_RESERVED_WORDS.add("null");
+ JAVA_RESERVED_WORDS.add("throw");
+ JAVA_RESERVED_WORDS.add("catch");
+ JAVA_RESERVED_WORDS.add("float");
+ JAVA_RESERVED_WORDS.add("package");
+ JAVA_RESERVED_WORDS.add("throws");
+ JAVA_RESERVED_WORDS.add("char");
+ JAVA_RESERVED_WORDS.add("for");
+ JAVA_RESERVED_WORDS.add("private");
+ JAVA_RESERVED_WORDS.add("transient");
+ JAVA_RESERVED_WORDS.add("class");
+ JAVA_RESERVED_WORDS.add("goto");
+ JAVA_RESERVED_WORDS.add("protected");
+ JAVA_RESERVED_WORDS.add("true");
+ JAVA_RESERVED_WORDS.add("const");
+ }
+
/**
* This version number is injected into all generated Java classes to denote
* which version of the ClassWriter's output format was used to generate the
@@ -57,7 +101,7 @@
*/
public static final int CLASS_WRITER_VERSION = 2;
- private ImportOptions options;
+ private SqoopOptions options;
private ConnManager connManager;
private String tableName;
private CompilationManager compileManager;
@@ -68,7 +112,7 @@
* @param connMgr the connection manager used to describe the table.
* @param table the name of the table to read.
*/
- public ClassWriter(final ImportOptions opts, final ConnManager connMgr,
+ public ClassWriter(final SqoopOptions opts, final ConnManager connMgr,
final String table, final CompilationManager compMgr) {
this.options = opts;
this.connManager = connMgr;
@@ -76,6 +120,87 @@
this.compileManager = compMgr;
}
+ /**
+ * Given some character that can't be in an identifier,
+ * try to map it to a string that can.
+ *
+ * @param c a character that can't be in a Java identifier
+ * @return a string of characters that can, or null if there's
+ * no good translation.
+ */
+ static String getIdentifierStrForChar(char c) {
+ if (Character.isJavaIdentifierPart(c)) {
+ return "" + c;
+ } else if (Character.isWhitespace(c)) {
+ // Eliminate whitespace.
+ return null;
+ } else {
+ // All other characters map to underscore.
+ return "_";
+ }
+ }
+
+ /**
+ * @param word a word to test.
+ * @return true if 'word' is reserved the in Java language.
+ */
+ private static boolean isReservedWord(String word) {
+ return JAVA_RESERVED_WORDS.contains(word);
+ }
+
+ /**
+ * Coerce a candidate name for an identifier into one which will
+ * definitely compile.
+ *
+ * Ensures that the returned identifier matches [A-Za-z_][A-Za-z0-9_]*
+ * and is not a reserved word.
+ *
+ * @param candidate A string we want to use as an identifier
+ * @return A string naming an identifier which compiles and is
+ * similar to the candidate.
+ */
+ public static String toIdentifier(String candidate) {
+ StringBuilder sb = new StringBuilder();
+ boolean first = true;
+ for (char c : candidate.toCharArray()) {
+ if (Character.isJavaIdentifierStart(c) && first) {
+ // Ok for this to be the first character of the identifier.
+ sb.append(c);
+ first = false;
+ } else if (Character.isJavaIdentifierPart(c) && !first) {
+ // Ok for this character to be in the output identifier.
+ sb.append(c);
+ } else {
+ // We have a character in the original that can't be
+ // part of this identifier we're building.
+ // If it's just not allowed to be the first char, add a leading '_'.
+ // If we have a reasonable translation (e.g., '-' -> '_'), do that.
+ // Otherwise, drop it.
+ if (first && Character.isJavaIdentifierPart(c)
+ && !Character.isJavaIdentifierStart(c)) {
+ sb.append("_");
+ sb.append(c);
+ first = false;
+ } else {
+ // Try to map this to a different character or string.
+ // If we can't just give up.
+ String translated = getIdentifierStrForChar(c);
+ if (null != translated) {
+ sb.append(translated);
+ first = false;
+ }
+ }
+ }
+ }
+
+ String output = sb.toString();
+ if (isReservedWord(output)) {
+ // e.g., 'class' -> '_class';
+ return "_" + output;
+ }
+
+ return output;
+ }
/**
* @param javaType
@@ -251,7 +376,7 @@
for (String col : colNames) {
int sqlType = columnTypes.get(col);
- String javaType = SqlManager.toJavaType(sqlType);
+ String javaType = connManager.toJavaType(sqlType);
if (null == javaType) {
LOG.error("Cannot resolve SQL type " + sqlType);
continue;
@@ -281,7 +406,7 @@
fieldNum++;
int sqlType = columnTypes.get(col);
- String javaType = SqlManager.toJavaType(sqlType);
+ String javaType = connManager.toJavaType(sqlType);
if (null == javaType) {
LOG.error("No Java type for SQL type " + sqlType);
continue;
@@ -318,7 +443,7 @@
fieldNum++;
int sqlType = columnTypes.get(col);
- String javaType = SqlManager.toJavaType(sqlType);
+ String javaType = connManager.toJavaType(sqlType);
if (null == javaType) {
LOG.error("No Java type for SQL type " + sqlType);
continue;
@@ -351,7 +476,7 @@
for (String col : colNames) {
int sqlType = columnTypes.get(col);
- String javaType = SqlManager.toJavaType(sqlType);
+ String javaType = connManager.toJavaType(sqlType);
if (null == javaType) {
LOG.error("No Java type for SQL type " + sqlType);
continue;
@@ -407,7 +532,7 @@
boolean first = true;
for (String col : colNames) {
int sqlType = columnTypes.get(col);
- String javaType = SqlManager.toJavaType(sqlType);
+ String javaType = connManager.toJavaType(sqlType);
if (null == javaType) {
LOG.error("No Java type for SQL type " + sqlType);
continue;
@@ -472,7 +597,7 @@
private void parseColumn(String colName, int colType, StringBuilder sb) {
// assume that we have __it and __cur_str vars, based on __loadFromFields() code.
sb.append(" __cur_str = __it.next();\n");
- String javaType = SqlManager.toJavaType(colType);
+ String javaType = connManager.toJavaType(colType);
parseNullVal(colName, sb);
if (javaType.equals("String")) {
@@ -565,7 +690,7 @@
for (String col : colNames) {
int sqlType = columnTypes.get(col);
- String javaType = SqlManager.toJavaType(sqlType);
+ String javaType = connManager.toJavaType(sqlType);
if (null == javaType) {
LOG.error("No Java type for SQL type " + sqlType);
continue;
@@ -593,8 +718,21 @@
colNames = connManager.getColumnNames(tableName);
}
+ // Translate all the column names into names that are safe to
+ // use as identifiers.
+ String [] cleanedColNames = new String[colNames.length];
+ for (int i = 0; i < colNames.length; i++) {
+ String col = colNames[i];
+ String identifier = toIdentifier(col);
+ cleanedColNames[i] = identifier;
+
+ // make sure the col->type mapping holds for the
+ // new identifier name, too.
+ columnTypes.put(identifier, columnTypes.get(col));
+ }
+
// Generate the Java code
- StringBuilder sb = generateClassForColumns(columnTypes, colNames);
+ StringBuilder sb = generateClassForColumns(columnTypes, cleanedColNames);
// Write this out to a file.
String codeOutDir = options.getCodeOutputDir();
@@ -605,16 +743,18 @@
String sourceFilename = className.replace('.', File.separatorChar) + ".java";
String filename = codeOutDir + sourceFilename;
- LOG.debug("Writing source file: " + filename);
- LOG.debug("Table name: " + tableName);
- StringBuilder sbColTypes = new StringBuilder();
- for (String col : colNames) {
- Integer colType = columnTypes.get(col);
- sbColTypes.append(col + ":" + colType + ", ");
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Writing source file: " + filename);
+ LOG.debug("Table name: " + tableName);
+ StringBuilder sbColTypes = new StringBuilder();
+ for (String col : colNames) {
+ Integer colType = columnTypes.get(col);
+ sbColTypes.append(col + ":" + colType + ", ");
+ }
+ String colTypeStr = sbColTypes.toString();
+ LOG.debug("Columns: " + colTypeStr);
+ LOG.debug("sourceFilename is " + sourceFilename);
}
- String colTypeStr = sbColTypes.toString();
- LOG.debug("Columns: " + colTypeStr);
- LOG.debug("sourceFilename is " + sourceFilename);
compileManager.addSourceFile(sourceFilename);
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/CompilationManager.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/CompilationManager.java
index c8fd152..99630e6 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/CompilationManager.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/CompilationManager.java
@@ -31,12 +31,17 @@
import java.util.jar.JarOutputStream;
import java.util.zip.ZipEntry;
+import javax.tools.JavaCompiler;
+import javax.tools.JavaFileObject;
+import javax.tools.StandardJavaFileManager;
+import javax.tools.ToolProvider;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.util.FileListing;
/**
@@ -53,10 +58,10 @@
public static final Log LOG = LogFactory.getLog(CompilationManager.class.getName());
- private ImportOptions options;
+ private SqoopOptions options;
private List<String> sources;
- public CompilationManager(final ImportOptions opts) {
+ public CompilationManager(final SqoopOptions opts) {
options = opts;
sources = new ArrayList<String>();
}
@@ -109,9 +114,14 @@
// ensure that the jar output dir exists.
String jarOutDir = options.getJarOutputDir();
- boolean mkdirSuccess = new File(jarOutDir).mkdirs();
- if (!mkdirSuccess) {
- LOG.debug("Warning: Could not make directories for " + jarOutDir);
+ File jarOutDirObj = new File(jarOutDir);
+ if (!jarOutDirObj.exists()) {
+ boolean mkdirSuccess = jarOutDirObj.mkdirs();
+ if (!mkdirSuccess) {
+ LOG.debug("Warning: Could not make directories for " + jarOutDir);
+ }
+ } else if (LOG.isDebugEnabled()) {
+ LOG.debug("Found existing " + jarOutDir);
}
// find hadoop-*-core.jar for classpath.
@@ -141,8 +151,12 @@
String curClasspath = System.getProperty("java.class.path");
+ String srcOutDir = new File(options.getCodeOutputDir()).getAbsolutePath();
+ if (!srcOutDir.endsWith(File.separator)) {
+ srcOutDir = srcOutDir + File.separator;
+ }
+
args.add("-sourcepath");
- String srcOutDir = options.getCodeOutputDir();
args.add(srcOutDir);
args.add("-d");
@@ -151,21 +165,36 @@
args.add("-classpath");
args.add(curClasspath + File.pathSeparator + coreJar + sqoopJar);
- // add all the source files
+ JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
+ StandardJavaFileManager fileManager =
+ compiler.getStandardFileManager(null, null, null);
+
+ ArrayList<String> srcFileNames = new ArrayList<String>();
for (String srcfile : sources) {
- args.add(srcOutDir + srcfile);
+ srcFileNames.add(srcOutDir + srcfile);
+ LOG.debug("Adding source file: " + srcOutDir + srcfile);
}
- StringBuilder sb = new StringBuilder();
- for (String arg : args) {
- sb.append(arg + " ");
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Invoking javac with args:");
+ for (String arg : args) {
+ LOG.debug(" " + arg);
+ }
}
- // NOTE(aaron): Usage is at http://java.sun.com/j2se/1.5.0/docs/tooldocs/solaris/javac.html
- LOG.debug("Invoking javac with args: " + sb.toString());
- int javacRet = com.sun.tools.javac.Main.compile(args.toArray(new String[0]));
- if (javacRet != 0) {
- throw new IOException("javac exited with status " + javacRet);
+ Iterable<? extends JavaFileObject> srcFileObjs =
+ fileManager.getJavaFileObjectsFromStrings(srcFileNames);
+ JavaCompiler.CompilationTask task = compiler.getTask(
+ null, // Write to stderr
+ fileManager,
+ null, // No special diagnostic handling
+ args,
+ null, // Compile all classes in the source compilation units
+ srcFileObjs);
+
+ boolean result = task.call();
+ if (!result) {
+ throw new IOException("Error returned by javac");
}
}
@@ -209,8 +238,6 @@
// read the file into a buffer, and write it to the jar file.
for (File entry : dirEntries) {
if (!entry.isDirectory()) {
- LOG.debug("Considering entry: " + entry);
-
// chomp off the portion of the full path that is shared
// with the base directory where class files were put;
// we only record the subdir parts in the zip entry.
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/TableClassName.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/TableClassName.java
index 5d78934..2d766f7 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/TableClassName.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/TableClassName.java
@@ -18,23 +18,23 @@
package org.apache.hadoop.sqoop.orm;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
/**
* Reconciles the table name being imported with the class naming information
- * specified in ImportOptions to determine the actual package and class name
+ * specified in SqoopOptions to determine the actual package and class name
* to use for a table.
*/
public class TableClassName {
public static final Log LOG = LogFactory.getLog(TableClassName.class.getName());
- private final ImportOptions options;
+ private final SqoopOptions options;
- public TableClassName(final ImportOptions opts) {
+ public TableClassName(final SqoopOptions opts) {
if (null == opts) {
throw new NullPointerException("Cannot instantiate a TableClassName on null options.");
} else {
@@ -90,7 +90,8 @@
}
// no specific class; no specific package.
- return tableName;
+ // Just make sure it's a legal identifier.
+ return ClassWriter.toIdentifier(tableName);
}
/**
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/DirectImportUtils.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/DirectImportUtils.java
index 9ba4783..5c13063 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/DirectImportUtils.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/DirectImportUtils.java
@@ -29,7 +29,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.io.SplittingOutputStream;
import org.apache.hadoop.sqoop.io.SplittableBufferedWriter;
import org.apache.hadoop.util.Shell;
@@ -70,7 +70,7 @@
* returned stream.
*/
public static SplittableBufferedWriter createHdfsSink(Configuration conf,
- ImportOptions options, String tableName) throws IOException {
+ SqoopOptions options, String tableName) throws IOException {
FileSystem fs = FileSystem.get(conf);
String warehouseDir = options.getWarehouseDir();
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ExportException.java
similarity index 75%
copy from src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
copy to src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ExportException.java
index ec03890..0043544 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ExportException.java
@@ -19,26 +19,24 @@
package org.apache.hadoop.sqoop.util;
/**
- * General error during import process.
- *
- *
+ * General error during export process.
*/
@SuppressWarnings("serial")
-public class ImportError extends Exception {
+public class ExportException extends Exception {
- public ImportError() {
- super("ImportError");
+ public ExportException() {
+ super("ExportException");
}
- public ImportError(final String message) {
+ public ExportException(final String message) {
super(message);
}
- public ImportError(final Throwable cause) {
+ public ExportException(final Throwable cause) {
super(cause);
}
- public ImportError(final String message, final Throwable cause) {
+ public ExportException(final String message, final Throwable cause) {
super(message, cause);
}
}
diff --git a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportException.java
similarity index 78%
rename from src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
rename to src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportException.java
index ec03890..b2ab6e0 100644
--- a/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
+++ b/src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportException.java
@@ -24,21 +24,21 @@
*
*/
@SuppressWarnings("serial")
-public class ImportError extends Exception {
+public class ImportException extends Exception {
- public ImportError() {
- super("ImportError");
+ public ImportException() {
+ super("ImportException");
}
- public ImportError(final String message) {
+ public ImportException(final String message) {
super(message);
}
- public ImportError(final Throwable cause) {
+ public ImportException(final Throwable cause) {
super(cause);
}
- public ImportError(final String message, final Throwable cause) {
+ public ImportException(final String message, final Throwable cause) {
super(message, cause);
}
}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/SmokeTests.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/SmokeTests.java
index e3c351d..5f1059a 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/SmokeTests.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/SmokeTests.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.sqoop;
import org.apache.hadoop.sqoop.hive.TestHiveImport;
+import org.apache.hadoop.sqoop.hive.TestTableDefWriter;
import org.apache.hadoop.sqoop.io.TestSplittableBufferedWriter;
import org.apache.hadoop.sqoop.lib.TestFieldFormatter;
import org.apache.hadoop.sqoop.lib.TestRecordParser;
@@ -47,6 +48,7 @@
suite.addTestSuite(TestSqlManager.class);
suite.addTestSuite(TestClassWriter.class);
suite.addTestSuite(TestColumnTypes.class);
+ suite.addTestSuite(TestExport.class);
suite.addTestSuite(TestMultiCols.class);
suite.addTestSuite(TestMultiMaps.class);
suite.addTestSuite(TestSplitBy.class);
@@ -54,10 +56,11 @@
suite.addTestSuite(TestHiveImport.class);
suite.addTestSuite(TestRecordParser.class);
suite.addTestSuite(TestFieldFormatter.class);
- suite.addTestSuite(TestImportOptions.class);
+ suite.addTestSuite(TestSqoopOptions.class);
suite.addTestSuite(TestParseMethods.class);
suite.addTestSuite(TestConnFactory.class);
suite.addTestSuite(TestSplittableBufferedWriter.class);
+ suite.addTestSuite(TestTableDefWriter.class);
suite.addTest(MapreduceTests.suite());
return suite;
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestAllTables.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestAllTables.java
index 8e31a54..31c5167 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestAllTables.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestAllTables.java
@@ -58,6 +58,8 @@
args.add(HsqldbTestServer.getUrl());
args.add("--num-mappers");
args.add("1");
+ args.add("--escaped-by");
+ args.add("\\");
return args.toArray(new String[0]);
}
@@ -86,9 +88,18 @@
// create two tables.
this.expectedStrings.add("A winner");
this.expectedStrings.add("is you!");
+ this.expectedStrings.add(null);
+ int i = 0;
for (String expectedStr: this.expectedStrings) {
- this.createTableForColType("VARCHAR(32) PRIMARY KEY", "'" + expectedStr + "'");
+ String wrappedStr = null;
+ if (expectedStr != null) {
+ wrappedStr = "'" + expectedStr + "'";
+ }
+
+ String [] types = { "INT NOT NULL PRIMARY KEY", "VARCHAR(32)" };
+ String [] vals = { Integer.toString(i++) , wrappedStr };
+ this.createTableWithColTypes(types, vals);
this.tableNames.add(this.getTableName());
this.removeTableDir();
incrementTableNum();
@@ -100,13 +111,15 @@
runImport(argv);
Path warehousePath = new Path(this.getWarehouseDir());
+ int i = 0;
for (String tableName : this.tableNames) {
Path tablePath = new Path(warehousePath, tableName);
Path filePath = new Path(tablePath, "part-m-00000");
// dequeue the expected value for this table. This
// list has the same order as the tableNames list.
- String expectedVal = this.expectedStrings.get(0);
+ String expectedVal = Integer.toString(i++) + ","
+ + this.expectedStrings.get(0);
this.expectedStrings.remove(0);
BufferedReader reader = new BufferedReader(
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestConnFactory.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestConnFactory.java
index c4da32d..6af7ba3 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestConnFactory.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestConnFactory.java
@@ -41,7 +41,7 @@
conf.set(ConnFactory.FACTORY_CLASS_NAMES_KEY, AlwaysDummyFactory.class.getName());
ConnFactory factory = new ConnFactory(conf);
- ConnManager manager = factory.getManager(new ImportOptions());
+ ConnManager manager = factory.getManager(new SqoopOptions());
assertNotNull("No manager returned", manager);
assertTrue("Expected a DummyManager", manager instanceof DummyManager);
}
@@ -52,7 +52,7 @@
ConnFactory factory = new ConnFactory(conf);
try {
- ConnManager manager = factory.getManager(new ImportOptions());
+ ConnManager manager = factory.getManager(new SqoopOptions());
fail("factory.getManager() expected to throw IOException");
} catch (IOException ioe) {
// Expected this. Test passes.
@@ -69,7 +69,7 @@
conf.set(ConnFactory.FACTORY_CLASS_NAMES_KEY, classNames);
ConnFactory factory = new ConnFactory(conf);
- ConnManager manager = factory.getManager(new ImportOptions());
+ ConnManager manager = factory.getManager(new SqoopOptions());
assertNotNull("No manager returned", manager);
assertTrue("Expected a DummyManager", manager instanceof DummyManager);
}
@@ -77,14 +77,14 @@
////// mock classes used for test cases above //////
public static class AlwaysDummyFactory extends ManagerFactory {
- public ConnManager accept(ImportOptions opts) {
+ public ConnManager accept(SqoopOptions opts) {
// Always return a new DummyManager
return new DummyManager();
}
}
public static class EmptyFactory extends ManagerFactory {
- public ConnManager accept(ImportOptions opts) {
+ public ConnManager accept(SqoopOptions opts) {
// Never instantiate a proper ConnManager;
return null;
}
@@ -113,6 +113,24 @@
return null;
}
+ /**
+ * Default implementation
+ * @param sqlType sql data type
+ * @return java data type
+ */
+ public String toJavaType(int sqlType) {
+ return null;
+ }
+
+ /**
+ * Default implementation
+ * @param sqlType sql data type
+ * @return hive data type
+ */
+ public String toHiveType(int sqlType) {
+ return null;
+ }
+
public Map<String, Integer> getColumnTypes(String tableName) {
return null;
}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestExport.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestExport.java
new file mode 100644
index 0000000..14c8550
--- /dev/null
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestExport.java
@@ -0,0 +1,539 @@
+/**
+ * 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.sqoop;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import org.apache.hadoop.sqoop.lib.RecordParser;
+import org.apache.hadoop.sqoop.lib.SqoopRecord;
+import org.apache.hadoop.sqoop.testutil.ExportJobTestCase;
+import org.apache.hadoop.sqoop.util.ClassLoaderStack;
+
+import org.junit.Before;
+
+/**
+ * Test that we can export data from HDFS into databases.
+ */
+public class TestExport extends ExportJobTestCase {
+
+ @Before
+ public void setUp() {
+ // start the server
+ super.setUp();
+
+ // throw away any existing data that might be in the database.
+ try {
+ this.getTestServer().dropExistingSchema();
+ } catch (SQLException sqlE) {
+ fail(sqlE.toString());
+ }
+ }
+
+ private String getRecordLine(int recordNum, ColumnGenerator... extraCols) {
+ String idStr = Integer.toString(recordNum);
+ StringBuilder sb = new StringBuilder();
+
+ sb.append(idStr);
+ sb.append("\t");
+ sb.append(getMsgPrefix());
+ sb.append(idStr);
+ for (ColumnGenerator gen : extraCols) {
+ sb.append("\t");
+ sb.append(gen.getExportText(recordNum));
+ }
+ sb.append("\n");
+
+ return sb.toString();
+ }
+
+ /** When generating data for export tests, each column is generated
+ according to a ColumnGenerator. Methods exist for determining
+ what to put into text strings in the files to export, as well
+ as what the string representation of the column as returned by
+ the database should look like.
+ */
+ interface ColumnGenerator {
+ /** for a row with id rowNum, what should we write into that
+ line of the text file to export?
+ */
+ public String getExportText(int rowNum);
+
+ /** for a row with id rowNum, what should the database return
+ for the given column's value?
+ */
+ public String getVerifyText(int rowNum);
+
+ /** Return the column type to put in the CREATE TABLE statement */
+ public String getType();
+ }
+
+ /**
+ * Create a data file that gets exported to the db
+ * @param fileNum the number of the file (for multi-file export)
+ * @param numRecords how many records to write to the file.
+ * @param gzip is true if the file should be gzipped.
+ */
+ private void createTextFile(int fileNum, int numRecords, boolean gzip,
+ ColumnGenerator... extraCols) throws IOException {
+ int startId = fileNum * numRecords;
+
+ String ext = ".txt";
+ if (gzip) {
+ ext = ext + ".gz";
+ }
+ Path tablePath = getTablePath();
+ Path filePath = new Path(tablePath, "part" + fileNum + ext);
+
+ Configuration conf = new Configuration();
+ conf.set("fs.default.name", "file:///");
+ FileSystem fs = FileSystem.get(conf);
+ fs.mkdirs(tablePath);
+ OutputStream os = fs.create(filePath);
+ if (gzip) {
+ CompressionCodecFactory ccf = new CompressionCodecFactory(conf);
+ CompressionCodec codec = ccf.getCodec(filePath);
+ os = codec.createOutputStream(os);
+ }
+ BufferedWriter w = new BufferedWriter(new OutputStreamWriter(os));
+ for (int i = 0; i < numRecords; i++) {
+ w.write(getRecordLine(startId + i, extraCols));
+ }
+ w.close();
+ os.close();
+
+ if (gzip) {
+ verifyCompressedFile(filePath, numRecords);
+ }
+ }
+
+ private void verifyCompressedFile(Path f, int expectedNumLines) throws IOException {
+ Configuration conf = new Configuration();
+ conf.set("fs.default.name", "file:///");
+ FileSystem fs = FileSystem.get(conf);
+ InputStream is = fs.open(f);
+ CompressionCodecFactory ccf = new CompressionCodecFactory(conf);
+ CompressionCodec codec = ccf.getCodec(f);
+ LOG.info("gzip check codec is " + codec);
+ Decompressor decompressor = CodecPool.getDecompressor(codec);
+ if (null == decompressor) {
+ LOG.info("Verifying gzip sanity with null decompressor");
+ } else {
+ LOG.info("Verifying gzip sanity with decompressor: " + decompressor.toString());
+ }
+ is = codec.createInputStream(is, decompressor);
+ BufferedReader r = new BufferedReader(new InputStreamReader(is));
+ int numLines = 0;
+ while (true) {
+ String ln = r.readLine();
+ if (ln == null) {
+ break;
+ }
+ numLines++;
+ }
+
+ r.close();
+ assertEquals("Did not read back correct number of lines",
+ expectedNumLines, numLines);
+ LOG.info("gzip sanity check returned " + numLines + " lines; ok.");
+ }
+
+ /**
+ * Create a data file in SequenceFile format that gets exported to the db
+ * @param fileNum the number of the file (for multi-file export).
+ * @param numRecords how many records to write to the file.
+ * @param className the table class name to instantiate and populate
+ * for each record.
+ */
+ private void createSequenceFile(int fileNum, int numRecords, String className)
+ throws IOException {
+
+ try {
+ // Instantiate the value record object via reflection.
+ Class cls = Class.forName(className, true,
+ Thread.currentThread().getContextClassLoader());
+ SqoopRecord record = (SqoopRecord) ReflectionUtils.newInstance(cls, new Configuration());
+
+ // Create the SequenceFile.
+ Configuration conf = new Configuration();
+ conf.set("fs.default.name", "file:///");
+ FileSystem fs = FileSystem.get(conf);
+ Path tablePath = getTablePath();
+ Path filePath = new Path(tablePath, "part" + fileNum);
+ fs.mkdirs(tablePath);
+ SequenceFile.Writer w =
+ SequenceFile.createWriter(fs, conf, filePath, LongWritable.class, cls);
+
+ // Now write the data.
+ int startId = fileNum * numRecords;
+ for (int i = 0; i < numRecords; i++) {
+ record.parse(getRecordLine(startId + i));
+ w.append(new LongWritable(startId + i), record);
+ }
+
+ w.close();
+ } catch (ClassNotFoundException cnfe) {
+ throw new IOException(cnfe);
+ } catch (RecordParser.ParseError pe) {
+ throw new IOException(pe);
+ }
+ }
+
+ /** Return the column name for a column index.
+ * Each table contains two columns named 'id' and 'msg', and then an
+ * arbitrary number of additional columns defined by ColumnGenerators.
+ * These columns are referenced by idx 0, 1, 2...
+ * @param idx the index of the ColumnGenerator in the array passed to
+ * createTable().
+ * @return the name of the column
+ */
+ protected String forIdx(int idx) {
+ return "col" + idx;
+ }
+
+ /** Create the table definition to export to, removing any prior table.
+ By specifying ColumnGenerator arguments, you can add extra columns
+ to the table of arbitrary type.
+ */
+ public void createTable(ColumnGenerator... extraColumns) throws SQLException {
+ Connection conn = getTestServer().getConnection();
+ PreparedStatement statement = conn.prepareStatement(
+ "DROP TABLE " + getTableName() + " IF EXISTS",
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ statement.executeUpdate();
+ conn.commit();
+ statement.close();
+
+ StringBuilder sb = new StringBuilder();
+ sb.append("CREATE TABLE ");
+ sb.append(getTableName());
+ sb.append(" (id INT NOT NULL PRIMARY KEY, msg VARCHAR(64)");
+ int colNum = 0;
+ for (ColumnGenerator gen : extraColumns) {
+ sb.append(", " + forIdx(colNum++) + " " + gen.getType());
+ }
+ sb.append(")");
+
+ statement = conn.prepareStatement(sb.toString(),
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ statement.executeUpdate();
+ conn.commit();
+ statement.close();
+ }
+
+ /** Removing an existing table directory from the filesystem */
+ private void removeTablePath() throws IOException {
+ Configuration conf = new Configuration();
+ conf.set("fs.default.name", "file:///");
+ FileSystem fs = FileSystem.get(conf);
+ fs.delete(getTablePath(), true);
+ }
+
+ /** Verify that on a given row, a column has a given value.
+ * @param id the id column specifying the row to test.
+ */
+ private void assertColValForRowId(int id, String colName, String expectedVal)
+ throws SQLException {
+ Connection conn = getTestServer().getConnection();
+ LOG.info("Verifying column " + colName + " has value " + expectedVal);
+
+ PreparedStatement statement = conn.prepareStatement(
+ "SELECT " + colName + " FROM " + getTableName() + " WHERE id = " + id,
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ ResultSet rs = statement.executeQuery();
+ rs.next();
+
+ String actualVal = rs.getString(1);
+ rs.close();
+ statement.close();
+
+ assertEquals("Got unexpected column value", expectedVal, actualVal);
+ }
+
+ /** Verify that for the max and min values of the 'id' column, the values
+ for a given column meet the expected values.
+ */
+ private void assertColMinAndMax(String colName, ColumnGenerator generator)
+ throws SQLException {
+ int minId = getMinRowId();
+ int maxId = getMaxRowId();
+
+ LOG.info("Checking min/max for column " + colName + " with type " + generator.getType());
+
+ String expectedMin = generator.getVerifyText(minId);
+ String expectedMax = generator.getVerifyText(maxId);
+
+ assertColValForRowId(minId, colName, expectedMin);
+ assertColValForRowId(maxId, colName, expectedMax);
+ }
+
+ /** Export 10 rows, make sure they load in correctly */
+ public void testTextExport() throws IOException, SQLException {
+
+ final int TOTAL_RECORDS = 10;
+
+ createTextFile(0, TOTAL_RECORDS, false);
+ createTable();
+ runExport(getArgv(true));
+ verifyExport(TOTAL_RECORDS);
+ }
+
+ /** Export 10 rows from gzipped text files. */
+ public void testGzipExport() throws IOException, SQLException {
+
+ LOG.info("Beginning gzip export test");
+
+ final int TOTAL_RECORDS = 10;
+
+ createTextFile(0, TOTAL_RECORDS, true);
+ createTable();
+ runExport(getArgv(true));
+ verifyExport(TOTAL_RECORDS);
+ LOG.info("Complete gzip export test");
+ }
+
+ /** Run 2 mappers, make sure all records load in correctly */
+ public void testMultiMapTextExport() throws IOException, SQLException {
+
+ final int RECORDS_PER_MAP = 10;
+ final int NUM_FILES = 2;
+
+ for (int f = 0; f < NUM_FILES; f++) {
+ createTextFile(f, RECORDS_PER_MAP, false);
+ }
+
+ createTable();
+ runExport(getArgv(true));
+ verifyExport(RECORDS_PER_MAP * NUM_FILES);
+ }
+
+
+ /** Export some rows from a SequenceFile, make sure they import correctly */
+ public void testSequenceFileExport() throws IOException, SQLException {
+
+ final int TOTAL_RECORDS = 10;
+
+ // First, generate class and jar files that represent the table we're exporting to.
+ LOG.info("Creating initial schema for SeqFile test");
+ createTable();
+ LOG.info("Generating code...");
+ List<String> generatedJars = runExport(getArgv(true, "--generate-only"));
+
+ // Now, wipe the created table so we can export on top of it again.
+ LOG.info("Resetting schema and data...");
+ createTable();
+
+ // Wipe the directory we use when creating files to export to ensure
+ // it's ready for new SequenceFiles.
+ removeTablePath();
+
+ assertNotNull(generatedJars);
+ assertEquals("Expected 1 generated jar file", 1, generatedJars.size());
+ String jarFileName = generatedJars.get(0);
+ // Sqoop generates jars named "foo.jar"; by default, this should contain a
+ // class named 'foo'. Extract the class name.
+ Path jarPath = new Path(jarFileName);
+ String jarBaseName = jarPath.getName();
+ assertTrue(jarBaseName.endsWith(".jar"));
+ assertTrue(jarBaseName.length() > ".jar".length());
+ String className = jarBaseName.substring(0, jarBaseName.length() - ".jar".length());
+
+ LOG.info("Using jar filename: " + jarFileName);
+ LOG.info("Using class name: " + className);
+
+ ClassLoader prevClassLoader = null;
+
+ try {
+ if (null != jarFileName) {
+ prevClassLoader = ClassLoaderStack.addJarFile(jarFileName, className);
+ }
+
+ // Now use this class and jar name to create a sequence file.
+ LOG.info("Writing data to SequenceFiles");
+ createSequenceFile(0, TOTAL_RECORDS, className);
+
+ // Now run and verify the export.
+ LOG.info("Exporting SequenceFile-based data");
+ runExport(getArgv(true, "--class-name", className, "--jar-file", jarFileName));
+ verifyExport(TOTAL_RECORDS);
+ } finally {
+ if (null != prevClassLoader) {
+ ClassLoaderStack.setCurrentClassLoader(prevClassLoader);
+ }
+ }
+ }
+
+ public void testIntCol() throws IOException, SQLException {
+ final int TOTAL_RECORDS = 10;
+
+ // generate a column equivalent to rownum.
+ ColumnGenerator gen = new ColumnGenerator() {
+ public String getExportText(int rowNum) {
+ return "" + rowNum;
+ }
+ public String getVerifyText(int rowNum) {
+ return "" + rowNum;
+ }
+ public String getType() {
+ return "INTEGER";
+ }
+ };
+
+ createTextFile(0, TOTAL_RECORDS, false, gen);
+ createTable(gen);
+ runExport(getArgv(true));
+ verifyExport(TOTAL_RECORDS);
+ assertColMinAndMax(forIdx(0), gen);
+ }
+
+ public void testBigIntCol() throws IOException, SQLException {
+ final int TOTAL_RECORDS = 10;
+
+ // generate a column that won't fit in a normal int.
+ ColumnGenerator gen = new ColumnGenerator() {
+ public String getExportText(int rowNum) {
+ long val = (long) rowNum * 1000000000;
+ return "" + val;
+ }
+ public String getVerifyText(int rowNum) {
+ long val = (long) rowNum * 1000000000;
+ return "" + val;
+ }
+ public String getType() {
+ return "BIGINT";
+ }
+ };
+
+ createTextFile(0, TOTAL_RECORDS, false, gen);
+ createTable(gen);
+ runExport(getArgv(true));
+ verifyExport(TOTAL_RECORDS);
+ assertColMinAndMax(forIdx(0), gen);
+ }
+
+ private String pad(int n) {
+ if (n <= 9) {
+ return "0" + n;
+ } else {
+ return String.valueOf(n);
+ }
+ }
+
+ public void testDatesAndTimes() throws IOException, SQLException {
+ final int TOTAL_RECORDS = 10;
+
+ ColumnGenerator genDate = new ColumnGenerator() {
+ public String getExportText(int rowNum) {
+ int day = rowNum + 1;
+ return "2009-10-" + day;
+ }
+ public String getVerifyText(int rowNum) {
+ int day = rowNum + 1;
+ return "2009-10-" + pad(day);
+ }
+ public String getType() {
+ return "DATE";
+ }
+ };
+
+ ColumnGenerator genTime = new ColumnGenerator() {
+ public String getExportText(int rowNum) {
+ return "10:01:" + rowNum;
+ }
+ public String getVerifyText(int rowNum) {
+ return "10:01:" + pad(rowNum);
+ }
+ public String getType() {
+ return "TIME";
+ }
+ };
+
+ createTextFile(0, TOTAL_RECORDS, false, genDate, genTime);
+ createTable(genDate, genTime);
+ runExport(getArgv(true));
+ verifyExport(TOTAL_RECORDS);
+ assertColMinAndMax(forIdx(0), genDate);
+ assertColMinAndMax(forIdx(1), genTime);
+ }
+
+ public void testNumericTypes() throws IOException, SQLException {
+ final int TOTAL_RECORDS = 10;
+
+ // Check floating point values
+ ColumnGenerator genFloat = new ColumnGenerator() {
+ public String getExportText(int rowNum) {
+ double v = 3.141 * (double) rowNum;
+ return "" + v;
+ }
+ public String getVerifyText(int rowNum) {
+ double v = 3.141 * (double) rowNum;
+ return "" + v;
+ }
+ public String getType() {
+ return "FLOAT";
+ }
+ };
+
+ // Check precise decimal placement. The first of ten
+ // rows will be 2.7181; the last of ten rows will be
+ // 2.71810.
+ ColumnGenerator genNumeric = new ColumnGenerator() {
+ public String getExportText(int rowNum) {
+ int digit = rowNum + 1;
+ return "2.718" + digit;
+ }
+ public String getVerifyText(int rowNum) {
+ int digit = rowNum + 1;
+ return "2.718" + digit;
+ }
+ public String getType() {
+ return "NUMERIC";
+ }
+ };
+
+ createTextFile(0, TOTAL_RECORDS, false, genFloat, genNumeric);
+ createTable(genFloat, genNumeric);
+ runExport(getArgv(true));
+ verifyExport(TOTAL_RECORDS);
+ assertColMinAndMax(forIdx(0), genFloat);
+ assertColMinAndMax(forIdx(1), genNumeric);
+ }
+}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestImportOptions.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestImportOptions.java
deleted file mode 100644
index f99874c..0000000
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestImportOptions.java
+++ /dev/null
@@ -1,228 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.sqoop;
-
-import junit.framework.TestCase;
-
-
-/**
- * Test aspects of the ImportOptions class
- */
-public class TestImportOptions extends TestCase {
-
- // tests for the toChar() parser
- public void testNormalChar() throws ImportOptions.InvalidOptionsException {
- assertEquals('a', ImportOptions.toChar("a"));
- }
-
- public void testEmptyString() throws ImportOptions.InvalidOptionsException {
- try {
- ImportOptions.toChar("");
- fail("Expected exception");
- } catch (ImportOptions.InvalidOptionsException ioe) {
- // expect this.
- }
- }
-
- public void testNullString() throws ImportOptions.InvalidOptionsException {
- try {
- ImportOptions.toChar(null);
- fail("Expected exception");
- } catch (ImportOptions.InvalidOptionsException ioe) {
- // expect this.
- }
- }
-
- public void testTooLong() throws ImportOptions.InvalidOptionsException {
- // Should just use the first character and log a warning.
- assertEquals('x', ImportOptions.toChar("xyz"));
- }
-
- public void testHexChar1() throws ImportOptions.InvalidOptionsException {
- assertEquals(0xF, ImportOptions.toChar("\\0xf"));
- }
-
- public void testHexChar2() throws ImportOptions.InvalidOptionsException {
- assertEquals(0xF, ImportOptions.toChar("\\0xF"));
- }
-
- public void testHexChar3() throws ImportOptions.InvalidOptionsException {
- assertEquals(0xF0, ImportOptions.toChar("\\0xf0"));
- }
-
- public void testHexChar4() throws ImportOptions.InvalidOptionsException {
- assertEquals(0xF0, ImportOptions.toChar("\\0Xf0"));
- }
-
- public void testEscapeChar1() throws ImportOptions.InvalidOptionsException {
- assertEquals('\n', ImportOptions.toChar("\\n"));
- }
-
- public void testEscapeChar2() throws ImportOptions.InvalidOptionsException {
- assertEquals('\\', ImportOptions.toChar("\\\\"));
- }
-
- public void testEscapeChar3() throws ImportOptions.InvalidOptionsException {
- assertEquals('\\', ImportOptions.toChar("\\"));
- }
-
- public void testUnknownEscape1() throws ImportOptions.InvalidOptionsException {
- try {
- ImportOptions.toChar("\\Q");
- fail("Expected exception");
- } catch (ImportOptions.InvalidOptionsException ioe) {
- // expect this.
- }
- }
-
- public void testUnknownEscape2() throws ImportOptions.InvalidOptionsException {
- try {
- ImportOptions.toChar("\\nn");
- fail("Expected exception");
- } catch (ImportOptions.InvalidOptionsException ioe) {
- // expect this.
- }
- }
-
- public void testEscapeNul1() throws ImportOptions.InvalidOptionsException {
- assertEquals('\000', ImportOptions.toChar("\\0"));
- }
-
- public void testEscapeNul2() throws ImportOptions.InvalidOptionsException {
- assertEquals('\000', ImportOptions.toChar("\\00"));
- }
-
- public void testEscapeNul3() throws ImportOptions.InvalidOptionsException {
- assertEquals('\000', ImportOptions.toChar("\\0000"));
- }
-
- public void testEscapeNul4() throws ImportOptions.InvalidOptionsException {
- assertEquals('\000', ImportOptions.toChar("\\0x0"));
- }
-
- public void testOctalChar1() throws ImportOptions.InvalidOptionsException {
- assertEquals(04, ImportOptions.toChar("\\04"));
- }
-
- public void testOctalChar2() throws ImportOptions.InvalidOptionsException {
- assertEquals(045, ImportOptions.toChar("\\045"));
- }
-
- public void testErrOctalChar() throws ImportOptions.InvalidOptionsException {
- try {
- ImportOptions.toChar("\\095");
- fail("Expected exception");
- } catch (NumberFormatException nfe) {
- // expected.
- }
- }
-
- public void testErrHexChar() throws ImportOptions.InvalidOptionsException {
- try {
- ImportOptions.toChar("\\0x9K5");
- fail("Expected exception");
- } catch (NumberFormatException nfe) {
- // expected.
- }
- }
-
- // test that setting output delimiters also sets input delimiters
- public void testDelimitersInherit() throws ImportOptions.InvalidOptionsException {
- String [] args = {
- "--fields-terminated-by",
- "|"
- };
-
- ImportOptions opts = new ImportOptions();
- opts.parse(args);
- assertEquals('|', opts.getInputFieldDelim());
- assertEquals('|', opts.getOutputFieldDelim());
- }
-
- // test that setting output delimiters and setting input delims separately works
- public void testDelimOverride1() throws ImportOptions.InvalidOptionsException {
- String [] args = {
- "--fields-terminated-by",
- "|",
- "--input-fields-terminated-by",
- "*"
- };
-
- ImportOptions opts = new ImportOptions();
- opts.parse(args);
- assertEquals('*', opts.getInputFieldDelim());
- assertEquals('|', opts.getOutputFieldDelim());
- }
-
- // test that the order in which delims are specified doesn't matter
- public void testDelimOverride2() throws ImportOptions.InvalidOptionsException {
- String [] args = {
- "--input-fields-terminated-by",
- "*",
- "--fields-terminated-by",
- "|"
- };
-
- ImportOptions opts = new ImportOptions();
- opts.parse(args);
- assertEquals('*', opts.getInputFieldDelim());
- assertEquals('|', opts.getOutputFieldDelim());
- }
-
- public void testBadNumMappers1() {
- String [] args = {
- "--num-mappers",
- "x"
- };
-
- try {
- ImportOptions opts = new ImportOptions();
- opts.parse(args);
- fail("Expected InvalidOptionsException");
- } catch (ImportOptions.InvalidOptionsException ioe) {
- // expected.
- }
- }
-
- public void testBadNumMappers2() {
- String [] args = {
- "-m",
- "x"
- };
-
- try {
- ImportOptions opts = new ImportOptions();
- opts.parse(args);
- fail("Expected InvalidOptionsException");
- } catch (ImportOptions.InvalidOptionsException ioe) {
- // expected.
- }
- }
-
- public void testGoodNumMappers() throws ImportOptions.InvalidOptionsException {
- String [] args = {
- "-m",
- "4"
- };
-
- ImportOptions opts = new ImportOptions();
- opts.parse(args);
- assertEquals(4, opts.getNumMappers());
- }
-}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestMultiMaps.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestMultiMaps.java
index b557869..a5f5936 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestMultiMaps.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestMultiMaps.java
@@ -31,7 +31,7 @@
import org.apache.hadoop.mapred.Utils;
import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.sqoop.ImportOptions.InvalidOptionsException;
+import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException;
import org.apache.hadoop.sqoop.orm.CompilationManager;
import org.apache.hadoop.sqoop.testutil.CommonArgs;
import org.apache.hadoop.sqoop.testutil.HsqldbTestServer;
@@ -119,7 +119,7 @@
String [] argv = getArgv(true, columns, splitByCol);
runImport(argv);
try {
- ImportOptions opts = new ImportOptions();
+ SqoopOptions opts = new SqoopOptions();
opts.parse(getArgv(false, columns, splitByCol));
CompilationManager compileMgr = new CompilationManager(opts);
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestSplitBy.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestSplitBy.java
index 35fb879..842d59d 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestSplitBy.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestSplitBy.java
@@ -26,7 +26,7 @@
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.sqoop.ImportOptions.InvalidOptionsException;
+import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException;
import org.apache.hadoop.sqoop.orm.CompilationManager;
import org.apache.hadoop.sqoop.testutil.CommonArgs;
import org.apache.hadoop.sqoop.testutil.HsqldbTestServer;
@@ -98,7 +98,7 @@
String [] argv = getArgv(true, columns, splitByCol);
runImport(argv);
try {
- ImportOptions opts = new ImportOptions();
+ SqoopOptions opts = new SqoopOptions();
opts.parse(getArgv(false, columns, splitByCol));
CompilationManager compileMgr = new CompilationManager(opts);
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestSqoopOptions.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestSqoopOptions.java
new file mode 100644
index 0000000..4f42455
--- /dev/null
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestSqoopOptions.java
@@ -0,0 +1,228 @@
+/**
+ * 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.sqoop;
+
+import junit.framework.TestCase;
+
+
+/**
+ * Test aspects of the SqoopOptions class
+ */
+public class TestSqoopOptions extends TestCase {
+
+ // tests for the toChar() parser
+ public void testNormalChar() throws SqoopOptions.InvalidOptionsException {
+ assertEquals('a', SqoopOptions.toChar("a"));
+ }
+
+ public void testEmptyString() throws SqoopOptions.InvalidOptionsException {
+ try {
+ SqoopOptions.toChar("");
+ fail("Expected exception");
+ } catch (SqoopOptions.InvalidOptionsException ioe) {
+ // expect this.
+ }
+ }
+
+ public void testNullString() throws SqoopOptions.InvalidOptionsException {
+ try {
+ SqoopOptions.toChar(null);
+ fail("Expected exception");
+ } catch (SqoopOptions.InvalidOptionsException ioe) {
+ // expect this.
+ }
+ }
+
+ public void testTooLong() throws SqoopOptions.InvalidOptionsException {
+ // Should just use the first character and log a warning.
+ assertEquals('x', SqoopOptions.toChar("xyz"));
+ }
+
+ public void testHexChar1() throws SqoopOptions.InvalidOptionsException {
+ assertEquals(0xF, SqoopOptions.toChar("\\0xf"));
+ }
+
+ public void testHexChar2() throws SqoopOptions.InvalidOptionsException {
+ assertEquals(0xF, SqoopOptions.toChar("\\0xF"));
+ }
+
+ public void testHexChar3() throws SqoopOptions.InvalidOptionsException {
+ assertEquals(0xF0, SqoopOptions.toChar("\\0xf0"));
+ }
+
+ public void testHexChar4() throws SqoopOptions.InvalidOptionsException {
+ assertEquals(0xF0, SqoopOptions.toChar("\\0Xf0"));
+ }
+
+ public void testEscapeChar1() throws SqoopOptions.InvalidOptionsException {
+ assertEquals('\n', SqoopOptions.toChar("\\n"));
+ }
+
+ public void testEscapeChar2() throws SqoopOptions.InvalidOptionsException {
+ assertEquals('\\', SqoopOptions.toChar("\\\\"));
+ }
+
+ public void testEscapeChar3() throws SqoopOptions.InvalidOptionsException {
+ assertEquals('\\', SqoopOptions.toChar("\\"));
+ }
+
+ public void testUnknownEscape1() throws SqoopOptions.InvalidOptionsException {
+ try {
+ SqoopOptions.toChar("\\Q");
+ fail("Expected exception");
+ } catch (SqoopOptions.InvalidOptionsException ioe) {
+ // expect this.
+ }
+ }
+
+ public void testUnknownEscape2() throws SqoopOptions.InvalidOptionsException {
+ try {
+ SqoopOptions.toChar("\\nn");
+ fail("Expected exception");
+ } catch (SqoopOptions.InvalidOptionsException ioe) {
+ // expect this.
+ }
+ }
+
+ public void testEscapeNul1() throws SqoopOptions.InvalidOptionsException {
+ assertEquals('\000', SqoopOptions.toChar("\\0"));
+ }
+
+ public void testEscapeNul2() throws SqoopOptions.InvalidOptionsException {
+ assertEquals('\000', SqoopOptions.toChar("\\00"));
+ }
+
+ public void testEscapeNul3() throws SqoopOptions.InvalidOptionsException {
+ assertEquals('\000', SqoopOptions.toChar("\\0000"));
+ }
+
+ public void testEscapeNul4() throws SqoopOptions.InvalidOptionsException {
+ assertEquals('\000', SqoopOptions.toChar("\\0x0"));
+ }
+
+ public void testOctalChar1() throws SqoopOptions.InvalidOptionsException {
+ assertEquals(04, SqoopOptions.toChar("\\04"));
+ }
+
+ public void testOctalChar2() throws SqoopOptions.InvalidOptionsException {
+ assertEquals(045, SqoopOptions.toChar("\\045"));
+ }
+
+ public void testErrOctalChar() throws SqoopOptions.InvalidOptionsException {
+ try {
+ SqoopOptions.toChar("\\095");
+ fail("Expected exception");
+ } catch (NumberFormatException nfe) {
+ // expected.
+ }
+ }
+
+ public void testErrHexChar() throws SqoopOptions.InvalidOptionsException {
+ try {
+ SqoopOptions.toChar("\\0x9K5");
+ fail("Expected exception");
+ } catch (NumberFormatException nfe) {
+ // expected.
+ }
+ }
+
+ // test that setting output delimiters also sets input delimiters
+ public void testDelimitersInherit() throws SqoopOptions.InvalidOptionsException {
+ String [] args = {
+ "--fields-terminated-by",
+ "|"
+ };
+
+ SqoopOptions opts = new SqoopOptions();
+ opts.parse(args);
+ assertEquals('|', opts.getInputFieldDelim());
+ assertEquals('|', opts.getOutputFieldDelim());
+ }
+
+ // test that setting output delimiters and setting input delims separately works
+ public void testDelimOverride1() throws SqoopOptions.InvalidOptionsException {
+ String [] args = {
+ "--fields-terminated-by",
+ "|",
+ "--input-fields-terminated-by",
+ "*"
+ };
+
+ SqoopOptions opts = new SqoopOptions();
+ opts.parse(args);
+ assertEquals('*', opts.getInputFieldDelim());
+ assertEquals('|', opts.getOutputFieldDelim());
+ }
+
+ // test that the order in which delims are specified doesn't matter
+ public void testDelimOverride2() throws SqoopOptions.InvalidOptionsException {
+ String [] args = {
+ "--input-fields-terminated-by",
+ "*",
+ "--fields-terminated-by",
+ "|"
+ };
+
+ SqoopOptions opts = new SqoopOptions();
+ opts.parse(args);
+ assertEquals('*', opts.getInputFieldDelim());
+ assertEquals('|', opts.getOutputFieldDelim());
+ }
+
+ public void testBadNumMappers1() {
+ String [] args = {
+ "--num-mappers",
+ "x"
+ };
+
+ try {
+ SqoopOptions opts = new SqoopOptions();
+ opts.parse(args);
+ fail("Expected InvalidOptionsException");
+ } catch (SqoopOptions.InvalidOptionsException ioe) {
+ // expected.
+ }
+ }
+
+ public void testBadNumMappers2() {
+ String [] args = {
+ "-m",
+ "x"
+ };
+
+ try {
+ SqoopOptions opts = new SqoopOptions();
+ opts.parse(args);
+ fail("Expected InvalidOptionsException");
+ } catch (SqoopOptions.InvalidOptionsException ioe) {
+ // expected.
+ }
+ }
+
+ public void testGoodNumMappers() throws SqoopOptions.InvalidOptionsException {
+ String [] args = {
+ "-m",
+ "4"
+ };
+
+ SqoopOptions opts = new SqoopOptions();
+ opts.parse(args);
+ assertEquals(4, opts.getNumMappers());
+ }
+}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestWhere.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestWhere.java
index 8c0a2d4..8f5d700 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestWhere.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestWhere.java
@@ -26,7 +26,7 @@
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.sqoop.ImportOptions.InvalidOptionsException;
+import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException;
import org.apache.hadoop.sqoop.orm.CompilationManager;
import org.apache.hadoop.sqoop.testutil.CommonArgs;
import org.apache.hadoop.sqoop.testutil.HsqldbTestServer;
@@ -103,7 +103,7 @@
String [] argv = getArgv(true, columns, whereClause);
runImport(argv);
try {
- ImportOptions opts = new ImportOptions();
+ SqoopOptions opts = new SqoopOptions();
opts.parse(getArgv(false, columns, whereClause));
CompilationManager compileMgr = new CompilationManager(opts);
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/hive/TestHiveImport.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/hive/TestHiveImport.java
index 11d5854..2467c1e 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/hive/TestHiveImport.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/hive/TestHiveImport.java
@@ -27,7 +27,7 @@
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.testutil.CommonArgs;
import org.apache.hadoop.sqoop.testutil.HsqldbTestServer;
import org.apache.hadoop.sqoop.testutil.ImportJobTestCase;
@@ -71,11 +71,11 @@
return args.toArray(new String[0]);
}
- private ImportOptions getImportOptions(String [] extraArgs) {
- ImportOptions opts = new ImportOptions();
+ private SqoopOptions getSqoopOptions(String [] extraArgs) {
+ SqoopOptions opts = new SqoopOptions();
try {
opts.parse(getArgv(false, extraArgs));
- } catch (ImportOptions.InvalidOptionsException ioe) {
+ } catch (SqoopOptions.InvalidOptionsException ioe) {
fail("Invalid options: " + ioe.toString());
}
@@ -91,7 +91,7 @@
// set up our mock hive shell to compare our generated script
// against the correct expected one.
- ImportOptions options = getImportOptions(extraArgs);
+ SqoopOptions options = getSqoopOptions(extraArgs);
String hiveHome = options.getHiveHome();
assertNotNull("hive.home was not set", hiveHome);
Path testDataPath = new Path(new Path(hiveHome), "scripts/" + verificationScript);
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/hive/TestTableDefWriter.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/hive/TestTableDefWriter.java
new file mode 100644
index 0000000..e8aed69
--- /dev/null
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/hive/TestTableDefWriter.java
@@ -0,0 +1,55 @@
+/**
+ * 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.sqoop.hive;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+/**
+ * Test Hive DDL statement generation.
+ */
+public class TestTableDefWriter extends TestCase {
+
+ public static final Log LOG = LogFactory.getLog(TestHiveImport.class.getName());
+
+
+ // Test getHiveOctalCharCode and expect an IllegalArgumentException.
+ private void expectExceptionInCharCode(int charCode) {
+ try {
+ TableDefWriter.getHiveOctalCharCode(charCode);
+ fail("Expected IllegalArgumentException");
+ } catch (IllegalArgumentException iae) {
+ // Expected; ok.
+ }
+ }
+
+ public void testHiveOctalCharCode() {
+ assertEquals("\\000", TableDefWriter.getHiveOctalCharCode(0));
+ assertEquals("\\001", TableDefWriter.getHiveOctalCharCode(1));
+ assertEquals("\\012", TableDefWriter.getHiveOctalCharCode((int) '\n'));
+ assertEquals("\\177", TableDefWriter.getHiveOctalCharCode(0177));
+
+ expectExceptionInCharCode(4096);
+ expectExceptionInCharCode(0200);
+ expectExceptionInCharCode(254);
+ }
+}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/lib/TestFieldFormatter.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/lib/TestFieldFormatter.java
index c29e80f..661a9ac 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/lib/TestFieldFormatter.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/lib/TestFieldFormatter.java
@@ -37,6 +37,10 @@
public void testNullArgs() {
String result = FieldFormatter.escapeAndEnclose("", null, null, null, false);
assertEquals("", result);
+
+ char [] encloseFor = { '\"' };
+ assertNull(FieldFormatter.escapeAndEnclose(null, "\\", "\"", encloseFor,
+ false));
}
public void testBasicStr() {
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/LocalMySQLTest.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/LocalMySQLTest.java
index 28d9916..0bf694c 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/LocalMySQLTest.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/LocalMySQLTest.java
@@ -37,7 +37,8 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
+import org.apache.hadoop.sqoop.testutil.CommonArgs;
import org.apache.hadoop.sqoop.testutil.ImportJobTestCase;
import org.apache.hadoop.sqoop.util.FileListing;
@@ -77,7 +78,7 @@
@Before
public void setUp() {
- ImportOptions options = new ImportOptions(CONNECT_STRING, TABLE_NAME);
+ SqoopOptions options = new SqoopOptions(CONNECT_STRING, TABLE_NAME);
options.setUsername(getCurrentUser());
manager = new LocalMySQLManager(options);
@@ -181,19 +182,21 @@
}
}
- private String [] getArgv(boolean mysqlOutputDelims, String... extraArgs) {
+ private String [] getArgv(boolean mysqlOutputDelims, boolean isDirect,
+ String tableName, String... extraArgs) {
ArrayList<String> args = new ArrayList<String>();
- args.add("-D");
- args.add("fs.default.name=file:///");
+ CommonArgs.addHadoopFlags(args);
args.add("--table");
- args.add(TABLE_NAME);
+ args.add(tableName);
args.add("--warehouse-dir");
args.add(getWarehouseDir());
args.add("--connect");
args.add(CONNECT_STRING);
- args.add("--direct");
+ if (isDirect) {
+ args.add("--direct");
+ }
args.add("--username");
args.add(getCurrentUser());
args.add("--where");
@@ -214,12 +217,19 @@
return args.toArray(new String[0]);
}
- private void doLocalBulkImport(boolean mysqlOutputDelims,
- String [] expectedResults, String [] extraArgs) throws IOException {
+ private void doImport(boolean mysqlOutputDelims, boolean isDirect,
+ String tableName, String [] expectedResults, String [] extraArgs)
+ throws IOException {
Path warehousePath = new Path(this.getWarehouseDir());
- Path tablePath = new Path(warehousePath, TABLE_NAME);
- Path filePath = new Path(tablePath, "data-00000");
+ Path tablePath = new Path(warehousePath, tableName);
+
+ Path filePath;
+ if (isDirect) {
+ filePath = new Path(tablePath, "data-00000");
+ } else {
+ filePath = new Path(tablePath, "part-m-00000");
+ }
File tableFile = new File(tablePath.toString());
if (tableFile.exists() && tableFile.isDirectory()) {
@@ -227,7 +237,7 @@
FileListing.recursiveDeleteDir(tableFile);
}
- String [] argv = getArgv(mysqlOutputDelims, extraArgs);
+ String [] argv = getArgv(mysqlOutputDelims, isDirect, tableName, extraArgs);
try {
runImport(argv);
} catch (IOException ioe) {
@@ -237,7 +247,7 @@
}
File f = new File(filePath.toString());
- assertTrue("Could not find imported data file", f.exists());
+ assertTrue("Could not find imported data file: " + f, f.exists());
BufferedReader r = null;
try {
// Read through the file and make sure it's all there.
@@ -262,7 +272,7 @@
"3,Fred,2009-01-23,15,marketing"
};
- doLocalBulkImport(false, expectedResults, null);
+ doImport(false, true, TABLE_NAME, expectedResults, null);
}
@Test
@@ -275,7 +285,7 @@
String [] extraArgs = { "-", "--lock-tables" };
- doLocalBulkImport(false, expectedResults, extraArgs);
+ doImport(false, true, TABLE_NAME, expectedResults, extraArgs);
}
@Test
@@ -286,6 +296,110 @@
"3,'Fred','2009-01-23',15,'marketing'"
};
- doLocalBulkImport(true, expectedResults, null);
+ doImport(true, true, TABLE_NAME, expectedResults, null);
+ }
+
+ @Test
+ public void testMysqlJdbcImport() throws IOException {
+ String [] expectedResults = {
+ "2,Bob,2009-04-20,400.0,sales",
+ "3,Fred,2009-01-23,15.0,marketing"
+ };
+
+ doImport(false, false, TABLE_NAME, expectedResults, null);
+ }
+
+ @Test
+ public void testJdbcEscapedTableName() throws Exception {
+ // Test a JDBC-based import of a table whose name is
+ // a reserved sql keyword (and is thus `quoted`)
+ final String reservedTableName = "TABLE";
+ SqoopOptions options = new SqoopOptions(CONNECT_STRING,
+ reservedTableName);
+ options.setUsername(getCurrentUser());
+ ConnManager mgr = new MySQLManager(options);
+
+ Connection connection = null;
+ Statement st = null;
+
+ try {
+ connection = mgr.getConnection();
+ connection.setAutoCommit(false);
+ st = connection.createStatement();
+
+ // create the database table and populate it with data.
+ st.executeUpdate("DROP TABLE IF EXISTS `" + reservedTableName + "`");
+ st.executeUpdate("CREATE TABLE `" + reservedTableName + "` ("
+ + "id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
+ + "name VARCHAR(24) NOT NULL, "
+ + "start_date DATE, "
+ + "salary FLOAT, "
+ + "dept VARCHAR(32))");
+
+ st.executeUpdate("INSERT INTO `" + reservedTableName + "` VALUES("
+ + "2,'Aaron','2009-05-14',1000000.00,'engineering')");
+ connection.commit();
+ } finally {
+ if (null != st) {
+ st.close();
+ }
+
+ if (null != connection) {
+ connection.close();
+ }
+ }
+
+ String [] expectedResults = {
+ "2,Aaron,2009-05-14,1000000.0,engineering"
+ };
+
+ doImport(false, false, reservedTableName, expectedResults, null);
+ }
+
+ @Test
+ public void testJdbcEscapedColumnName() throws Exception {
+ // Test a JDBC-based import of a table with a column whose name is
+ // a reserved sql keyword (and is thus `quoted`)
+ final String tableName = "mysql_escaped_col_table";
+ SqoopOptions options = new SqoopOptions(CONNECT_STRING,
+ tableName);
+ options.setUsername(getCurrentUser());
+ ConnManager mgr = new MySQLManager(options);
+
+ Connection connection = null;
+ Statement st = null;
+
+ try {
+ connection = mgr.getConnection();
+ connection.setAutoCommit(false);
+ st = connection.createStatement();
+
+ // create the database table and populate it with data.
+ st.executeUpdate("DROP TABLE IF EXISTS " + tableName);
+ st.executeUpdate("CREATE TABLE " + tableName + " ("
+ + "id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
+ + "`table` VARCHAR(24) NOT NULL, "
+ + "`CREATE` DATE, "
+ + "salary FLOAT, "
+ + "dept VARCHAR(32))");
+
+ st.executeUpdate("INSERT INTO " + tableName + " VALUES("
+ + "2,'Aaron','2009-05-14',1000000.00,'engineering')");
+ connection.commit();
+ } finally {
+ if (null != st) {
+ st.close();
+ }
+
+ if (null != connection) {
+ connection.close();
+ }
+ }
+
+ String [] expectedResults = {
+ "2,Aaron,2009-05-14,1000000.0,engineering"
+ };
+
+ doImport(false, false, tableName, expectedResults, null);
}
}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/MySQLAuthTest.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/MySQLAuthTest.java
index aeb14ee..7b73d22 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/MySQLAuthTest.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/MySQLAuthTest.java
@@ -39,7 +39,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.testutil.CommonArgs;
import org.apache.hadoop.sqoop.testutil.ImportJobTestCase;
@@ -79,7 +79,7 @@
@Before
public void setUp() {
- ImportOptions options = new ImportOptions(AUTH_CONNECT_STRING, AUTH_TABLE_NAME);
+ SqoopOptions options = new SqoopOptions(AUTH_CONNECT_STRING, AUTH_TABLE_NAME);
options.setUsername(AUTH_TEST_USER);
options.setPassword(AUTH_TEST_PASS);
@@ -131,7 +131,8 @@
}
}
- private String [] getArgv(boolean includeHadoopFlags) {
+ private String [] getArgv(boolean includeHadoopFlags,
+ boolean useDirect, String connectString, String tableName) {
ArrayList<String> args = new ArrayList<String>();
if (includeHadoopFlags) {
@@ -139,12 +140,14 @@
}
args.add("--table");
- args.add(AUTH_TABLE_NAME);
+ args.add(tableName);
args.add("--warehouse-dir");
args.add(getWarehouseDir());
args.add("--connect");
- args.add(AUTH_CONNECT_STRING);
- args.add("--direct");
+ args.add(connectString);
+ if (useDirect) {
+ args.add("--direct");
+ }
args.add("--username");
args.add(AUTH_TEST_USER);
args.add("--password");
@@ -162,7 +165,7 @@
*/
@Test
public void testAuthAccess() {
- String [] argv = getArgv(true);
+ String [] argv = getArgv(true, true, AUTH_CONNECT_STRING, AUTH_TABLE_NAME);
try {
runImport(argv);
} catch (IOException ioe) {
@@ -190,4 +193,115 @@
IOUtils.closeStream(r);
}
}
+
+ @Test
+ public void testZeroTimestamp() throws IOException, SQLException {
+ // MySQL timestamps can hold values whose range causes problems
+ // for java.sql.Timestamp. The MySQLManager adds settings to the
+ // connect string which configure the driver's handling of
+ // zero-valued timestamps. Check that all of these modifications
+ // to the connect string are successful.
+
+ try {
+ // A connect string with a null 'query' component.
+ doZeroTimestampTest(0, true, AUTH_CONNECT_STRING);
+
+ // A connect string with a zero-length query component.
+ doZeroTimestampTest(1, true, AUTH_CONNECT_STRING + "?");
+
+ // A connect string with another argument
+ doZeroTimestampTest(2, true, AUTH_CONNECT_STRING + "?connectTimeout=0");
+ doZeroTimestampTest(3, true, AUTH_CONNECT_STRING + "?connectTimeout=0&");
+
+ // A connect string with the zero-timestamp behavior already
+ // configured.
+ doZeroTimestampTest(4, true, AUTH_CONNECT_STRING
+ + "?zeroDateTimeBehavior=convertToNull");
+
+ // And finally, behavior already configured in such a way as to
+ // cause the timestamp import to fail.
+ doZeroTimestampTest(5, false, AUTH_CONNECT_STRING
+ + "?zeroDateTimeBehavior=exception");
+ } finally {
+ // Clean up our mess on the way out.
+ dropTimestampTables();
+ }
+ }
+
+ private void dropTimestampTables() throws SQLException {
+ SqoopOptions options = new SqoopOptions(AUTH_CONNECT_STRING, null);
+ options.setUsername(AUTH_TEST_USER);
+ options.setPassword(AUTH_TEST_PASS);
+
+ manager = new LocalMySQLManager(options);
+
+ Connection connection = null;
+ Statement st = null;
+
+ connection = manager.getConnection();
+ connection.setAutoCommit(false);
+ st = connection.createStatement();
+
+ st.executeUpdate("DROP TABLE IF EXISTS mysqlTimestampTable0");
+ st.executeUpdate("DROP TABLE IF EXISTS mysqlTimestampTable1");
+ st.executeUpdate("DROP TABLE IF EXISTS mysqlTimestampTable2");
+ st.executeUpdate("DROP TABLE IF EXISTS mysqlTimestampTable3");
+ st.executeUpdate("DROP TABLE IF EXISTS mysqlTimestampTable4");
+ st.executeUpdate("DROP TABLE IF EXISTS mysqlTimestampTable5");
+ connection.commit();
+ st.close();
+ connection.close();
+ }
+
+ public void doZeroTimestampTest(int testNum, boolean expectSuccess,
+ String connectString) throws IOException, SQLException {
+
+ final String tableName = "mysqlTimestampTable" + Integer.toString(testNum);
+
+ // Create a table containing a full-zeros timestamp.
+ SqoopOptions options = new SqoopOptions(connectString, tableName);
+ options.setUsername(AUTH_TEST_USER);
+ options.setPassword(AUTH_TEST_PASS);
+
+ manager = new LocalMySQLManager(options);
+
+ Connection connection = null;
+ Statement st = null;
+
+ connection = manager.getConnection();
+ connection.setAutoCommit(false);
+ st = connection.createStatement();
+
+ // create the database table and populate it with data.
+ st.executeUpdate("DROP TABLE IF EXISTS " + tableName);
+ st.executeUpdate("CREATE TABLE " + tableName + " ("
+ + "id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
+ + "ts TIMESTAMP NOT NULL)");
+
+ st.executeUpdate("INSERT INTO " + tableName + " VALUES("
+ + "NULL,'0000-00-00 00:00:00.0')");
+ connection.commit();
+ st.close();
+ connection.close();
+
+ // Run the import.
+ String [] argv = getArgv(true, false, connectString, tableName);
+ runImport(argv);
+
+ // Make sure the result file is there.
+ Path warehousePath = new Path(this.getWarehouseDir());
+ Path tablePath = new Path(warehousePath, tableName);
+ Path filePath = new Path(tablePath, "part-m-00000");
+
+ File f = new File(filePath.toString());
+ if (expectSuccess) {
+ assertTrue("Could not find imported data file", f.exists());
+ BufferedReader r = new BufferedReader(new InputStreamReader(
+ new FileInputStream(f)));
+ assertEquals("1,null", r.readLine());
+ IOUtils.closeStream(r);
+ } else {
+ assertFalse("Imported data when expected failure", f.exists());
+ }
+ }
}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/OracleManagerTest.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/OracleManagerTest.java
index 6e58316..40771d9 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/OracleManagerTest.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/OracleManagerTest.java
@@ -27,7 +27,13 @@
import java.sql.Connection;
import java.sql.SQLException;
import java.sql.Statement;
+import java.util.Date;
+import java.util.Calendar;
+import java.util.TimeZone;
import java.util.ArrayList;
+import java.text.ParseException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
import junit.framework.TestCase;
@@ -39,7 +45,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.testutil.CommonArgs;
import org.apache.hadoop.sqoop.testutil.ImportJobTestCase;
import org.apache.hadoop.sqoop.util.FileListing;
@@ -78,7 +84,7 @@
@Before
public void setUp() {
- ImportOptions options = new ImportOptions(CONNECT_STRING, TABLE_NAME);
+ SqoopOptions options = new SqoopOptions(CONNECT_STRING, TABLE_NAME);
options.setUsername(ORACLE_USER_NAME);
options.setPassword(ORACLE_USER_PASS);
@@ -101,14 +107,16 @@
+ "start_date DATE, "
+ "salary FLOAT, "
+ "dept VARCHAR2(32), "
+ + "timestamp_tz TIMESTAMP WITH TIME ZONE, "
+ + "timestamp_ltz TIMESTAMP WITH LOCAL TIME ZONE, "
+ "PRIMARY KEY (id))");
st.executeUpdate("INSERT INTO " + TABLE_NAME + " VALUES("
- + "1,'Aaron',to_date('2009-05-14','yyyy-mm-dd'),1000000.00,'engineering')");
+ + "1,'Aaron',to_date('2009-05-14','yyyy-mm-dd'),1000000.00,'engineering','29-DEC-09 12.00.00.000000000 PM','29-DEC-09 12.00.00.000000000 PM')");
st.executeUpdate("INSERT INTO " + TABLE_NAME + " VALUES("
- + "2,'Bob',to_date('2009-04-20','yyyy-mm-dd'),400.00,'sales')");
+ + "2,'Bob',to_date('2009-04-20','yyyy-mm-dd'),400.00,'sales','30-DEC-09 12.00.00.000000000 PM','30-DEC-09 12.00.00.000000000 PM')");
st.executeUpdate("INSERT INTO " + TABLE_NAME + " VALUES("
- + "3,'Fred',to_date('2009-01-23','yyyy-mm-dd'),15.00,'marketing')");
+ + "3,'Fred',to_date('2009-01-23','yyyy-mm-dd'),15.00,'marketing','31-DEC-09 12.00.00.000000000 PM','31-DEC-09 12.00.00.000000000 PM')");
connection.commit();
} catch (SQLException sqlE) {
LOG.error("Encountered SQL Exception: " + sqlE);
@@ -180,7 +188,7 @@
ioe.printStackTrace();
fail(ioe.toString());
}
-
+
File f = new File(filePath.toString());
assertTrue("Could not find imported data file", f.exists());
BufferedReader r = null;
@@ -188,7 +196,7 @@
// Read through the file and make sure it's all there.
r = new BufferedReader(new InputStreamReader(new FileInputStream(f)));
for (String expectedLine : expectedResults) {
- assertEquals(expectedLine, r.readLine());
+ compareRecords(expectedLine, r.readLine());
}
} catch (IOException ioe) {
LOG.error("Got IOException verifying results: " + ioe.toString());
@@ -208,11 +216,80 @@
// a strict DATE type. Thus we include HH:MM:SS.mmmmm below.
// See http://www.oracle.com/technology/tech/java/sqlj_jdbc/htdocs/jdbc_faq.html#08_01
String [] expectedResults = {
- "1,Aaron,2009-05-14 00:00:00.0,1000000,engineering",
- "2,Bob,2009-04-20 00:00:00.0,400,sales",
- "3,Fred,2009-01-23 00:00:00.0,15,marketing"
+ "1,Aaron,2009-05-14 00:00:00.0,1000000,engineering,2009-12-29 12:00:00.0,2009-12-29 12:00:00.0",
+ "2,Bob,2009-04-20 00:00:00.0,400,sales,2009-12-30 12:00:00.0,2009-12-30 12:00:00.0",
+ "3,Fred,2009-01-23 00:00:00.0,15,marketing,2009-12-31 12:00:00.0,2009-12-31 12:00:00.0"
};
runOracleTest(expectedResults);
}
+
+ /**
+ * Compare two lines
+ * @param expectedLine expected line
+ * @param receivedLine received line
+ * @throws IOException exception during lines comparison
+ */
+ private void compareRecords(String expectedLine, String receivedLine) throws IOException {
+ // handle null case
+ if (expectedLine == null || receivedLine == null) {
+ return;
+ }
+
+ // check if lines are equal
+ if (expectedLine.equals(receivedLine)) {
+ return;
+ }
+
+ // check if size is the same
+ String [] expectedValues = expectedLine.split(",");
+ String [] receivedValues = receivedLine.split(",");
+ if (expectedValues.length != 7 || receivedValues.length != 7) {
+ LOG.error("Number of expected fields did not match number of received fields");
+ throw new IOException("Number of expected fields did not match number of received fields");
+ }
+
+ // check first 5 values
+ boolean mismatch = false;
+ for (int i = 0; !mismatch && i < 5; i++) {
+ mismatch = !expectedValues[i].equals(receivedValues[i]);
+ }
+ if (mismatch) {
+ throw new IOException("Expected:<" + expectedLine + "> but was:<" + receivedLine + ">");
+ }
+
+ Date expectedDate = null;
+ Date receivedDate = null;
+ DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S");
+ int offset = TimeZone.getDefault().getOffset(System.currentTimeMillis()) / 3600000;
+ for (int i = 5; i < 7; i++) {
+ // parse expected timestamp
+ try {
+ expectedDate = df.parse(expectedValues[i]);
+ } catch (ParseException ex) {
+ LOG.error("Could not parse expected timestamp: " + expectedValues[i]);
+ throw new IOException("Could not parse expected timestamp: " + expectedValues[i]);
+ }
+
+ // parse received timestamp
+ try {
+ receivedDate = df.parse(receivedValues[i]);
+ } catch (ParseException ex) {
+ LOG.error("Could not parse received timestamp: " + receivedValues[i]);
+ throw new IOException("Could not parse received timestamp: " + receivedValues[i]);
+ }
+
+ // compare two timestamps considering timezone offset
+ Calendar expectedCal = Calendar.getInstance();
+ expectedCal.setTime(expectedDate);
+ expectedCal.add(Calendar.HOUR, offset);
+
+ Calendar receivedCal = Calendar.getInstance();
+ receivedCal.setTime(receivedDate);
+
+ if (!expectedCal.equals(receivedCal)) {
+ throw new IOException("Expected:<" + expectedLine + "> but was:<" + receivedLine + ">, while timezone offset is: " + offset);
+ }
+ }
+ }
}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/PostgresqlTest.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/PostgresqlTest.java
index dde921f..d4a066f 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/PostgresqlTest.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/PostgresqlTest.java
@@ -35,7 +35,8 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
+import org.apache.hadoop.sqoop.testutil.CommonArgs;
import org.apache.hadoop.sqoop.testutil.ImportJobTestCase;
import org.apache.hadoop.sqoop.util.FileListing;
@@ -84,7 +85,7 @@
public void setUp() {
LOG.debug("Setting up another postgresql test...");
- ImportOptions options = new ImportOptions(CONNECT_STRING, TABLE_NAME);
+ SqoopOptions options = new SqoopOptions(CONNECT_STRING, TABLE_NAME);
options.setUsername(DATABASE_USER);
ConnManager manager = null;
@@ -150,8 +151,7 @@
private String [] getArgv(boolean isDirect) {
ArrayList<String> args = new ArrayList<String>();
- args.add("-D");
- args.add("fs.default.name=file:///");
+ CommonArgs.addHadoopFlags(args);
args.add("--table");
args.add(TABLE_NAME);
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java
index 6e86ff8..863de48 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java
@@ -21,6 +21,8 @@
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
+import java.sql.Connection;
+import java.sql.Statement;
import java.sql.SQLException;
import java.util.Enumeration;
import java.util.jar.JarEntry;
@@ -34,8 +36,8 @@
import org.junit.Before;
import org.junit.Test;
-import org.apache.hadoop.sqoop.ImportOptions;
-import org.apache.hadoop.sqoop.ImportOptions.InvalidOptionsException;
+import org.apache.hadoop.sqoop.SqoopOptions;
+import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException;
import org.apache.hadoop.sqoop.manager.ConnManager;
import org.apache.hadoop.sqoop.testutil.DirUtil;
import org.apache.hadoop.sqoop.testutil.HsqldbTestServer;
@@ -53,7 +55,7 @@
// instance variables populated during setUp, used during tests
private HsqldbTestServer testServer;
private ConnManager manager;
- private ImportOptions options;
+ private SqoopOptions options;
@Before
public void setUp() {
@@ -71,7 +73,7 @@
}
manager = testServer.getManager();
- options = testServer.getImportOptions();
+ options = testServer.getSqoopOptions();
// sanity check: make sure we're in a tmp dir before we blow anything away.
assertTrue("Test generates code in non-tmp dir!",
@@ -267,5 +269,50 @@
runGenerationTest(argv, OVERRIDE_PACKAGE_NAME + "." + HsqldbTestServer.getTableName());
}
+
+
+ // Test the SQL identifier -> Java identifier conversion.
+ @Test
+ public void testIdentifierConversion() {
+ assertNull(ClassWriter.getIdentifierStrForChar(' '));
+ assertNull(ClassWriter.getIdentifierStrForChar('\t'));
+ assertNull(ClassWriter.getIdentifierStrForChar('\r'));
+ assertNull(ClassWriter.getIdentifierStrForChar('\n'));
+ assertEquals("x", ClassWriter.getIdentifierStrForChar('x'));
+ assertEquals("_", ClassWriter.getIdentifierStrForChar('-'));
+ assertEquals("_", ClassWriter.getIdentifierStrForChar('_'));
+
+ assertEquals("foo", ClassWriter.toIdentifier("foo"));
+ assertEquals("_class", ClassWriter.toIdentifier("class"));
+ assertEquals("_class", ClassWriter.toIdentifier("cla ss"));
+ assertEquals("_int", ClassWriter.toIdentifier("int"));
+ assertEquals("thisismanywords", ClassWriter.toIdentifier("this is many words"));
+ assertEquals("_9isLegalInSql", ClassWriter.toIdentifier("9isLegalInSql"));
+ assertEquals("___", ClassWriter.toIdentifier("___"));
+ }
+
+ @Test
+ public void testWeirdColumnNames() throws SQLException {
+ // Recreate the table with column names that aren't legal Java identifiers.
+ String tableName = HsqldbTestServer.getTableName();
+ Connection connection = testServer.getConnection();
+ Statement st = connection.createStatement();
+ st.executeUpdate("DROP TABLE " + tableName + " IF EXISTS");
+ st.executeUpdate("CREATE TABLE " + tableName + " (class INT, \"9field\" INT)");
+ st.executeUpdate("INSERT INTO " + tableName + " VALUES(42, 41)");
+ connection.commit();
+ connection.close();
+
+ String [] argv = {
+ "--bindir",
+ JAR_GEN_DIR,
+ "--outdir",
+ CODE_GEN_DIR,
+ "--package-name",
+ OVERRIDE_PACKAGE_NAME
+ };
+
+ runGenerationTest(argv, OVERRIDE_PACKAGE_NAME + "." + HsqldbTestServer.getTableName());
+ }
}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestParseMethods.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestParseMethods.java
index c499d4a..71dfb96 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestParseMethods.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestParseMethods.java
@@ -34,8 +34,8 @@
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.sqoop.ImportOptions;
-import org.apache.hadoop.sqoop.ImportOptions.InvalidOptionsException;
+import org.apache.hadoop.sqoop.SqoopOptions;
+import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException;
import org.apache.hadoop.sqoop.mapred.RawKeyTextOutputFormat;
import org.apache.hadoop.sqoop.orm.CompilationManager;
import org.apache.hadoop.sqoop.testutil.CommonArgs;
@@ -99,7 +99,7 @@
encloseRequired);
runImport(argv);
try {
- ImportOptions opts = new ImportOptions();
+ SqoopOptions opts = new SqoopOptions();
String tableClassName = getTableName();
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/BaseSqoopTestCase.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/BaseSqoopTestCase.java
new file mode 100644
index 0000000..f170df9
--- /dev/null
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/BaseSqoopTestCase.java
@@ -0,0 +1,286 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.testutil;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.BasicConfigurator;
+import org.junit.After;
+import org.junit.Before;
+
+import org.apache.hadoop.sqoop.manager.ConnManager;
+
+import junit.framework.TestCase;
+
+/**
+ * Class that implements common methods required for tests
+ */
+public class BaseSqoopTestCase extends TestCase {
+
+ public static final Log LOG = LogFactory.getLog(BaseSqoopTestCase.class.getName());
+
+ /** Base directory for all temporary data */
+ public static final String TEMP_BASE_DIR;
+
+ /** Where to import table data to in the local filesystem for testing */
+ public static final String LOCAL_WAREHOUSE_DIR;
+
+ // Initializer for the above
+ static {
+ String tmpDir = System.getProperty("test.build.data", "/tmp/");
+ if (!tmpDir.endsWith(File.separator)) {
+ tmpDir = tmpDir + File.separator;
+ }
+
+ TEMP_BASE_DIR = tmpDir;
+ LOCAL_WAREHOUSE_DIR = TEMP_BASE_DIR + "sqoop/warehouse";
+ }
+
+ // Used if a test manually sets the table name to be used.
+ private String curTableName;
+
+ protected void setCurTableName(String curName) {
+ this.curTableName = curName;
+ }
+
+ /**
+ * Because of how classloading works, we don't actually want to name
+ * all the tables the same thing -- they'll actually just use the same
+ * implementation of the Java class that was classloaded before. So we
+ * use this counter to uniquify table names.
+ */
+ private static int tableNum = 0;
+
+ /** When creating sequentially-identified tables, what prefix should
+ * be applied to these tables?
+ */
+ protected String getTablePrefix() {
+ return "SQOOP_TABLE_";
+ }
+
+ protected String getTableName() {
+ if (null != curTableName) {
+ return curTableName;
+ } else {
+ return getTablePrefix() + Integer.toString(tableNum);
+ }
+ }
+
+ protected String getWarehouseDir() {
+ return LOCAL_WAREHOUSE_DIR;
+ }
+
+ private String [] colNames;
+ protected String [] getColNames() {
+ return colNames;
+ }
+
+ protected HsqldbTestServer getTestServer() {
+ return testServer;
+ }
+
+ protected ConnManager getManager() {
+ return manager;
+ }
+
+ // instance variables populated during setUp, used during tests
+ private HsqldbTestServer testServer;
+ private ConnManager manager;
+
+ private static boolean isLog4jConfigured = false;
+
+ protected void incrementTableNum() {
+ tableNum++;
+ }
+
+ @Before
+ public void setUp() {
+
+ incrementTableNum();
+
+ if (!isLog4jConfigured) {
+ BasicConfigurator.configure();
+ isLog4jConfigured = true;
+ LOG.info("Configured log4j with console appender.");
+ }
+
+ testServer = new HsqldbTestServer();
+ try {
+ testServer.resetServer();
+ } catch (SQLException sqlE) {
+ LOG.error("Got SQLException: " + sqlE.toString());
+ fail("Got SQLException: " + sqlE.toString());
+ } catch (ClassNotFoundException cnfe) {
+ LOG.error("Could not find class for db driver: " + cnfe.toString());
+ fail("Could not find class for db driver: " + cnfe.toString());
+ }
+
+ manager = testServer.getManager();
+ }
+
+ @After
+ public void tearDown() {
+ setCurTableName(null); // clear user-override table name.
+
+ try {
+ if (null != manager) {
+ manager.close();
+ }
+ } catch (SQLException sqlE) {
+ LOG.error("Got SQLException: " + sqlE.toString());
+ fail("Got SQLException: " + sqlE.toString());
+ }
+
+ }
+
+ static final String BASE_COL_NAME = "DATA_COL";
+
+ /**
+ * Create a table with a set of columns and add a row of values.
+ * @param colTypes the types of the columns to make
+ * @param vals the SQL text for each value to insert
+ */
+ protected void createTableWithColTypes(String [] colTypes, String [] vals) {
+ Connection conn = null;
+ try {
+ conn = getTestServer().getConnection();
+ PreparedStatement statement = conn.prepareStatement(
+ "DROP TABLE " + getTableName() + " IF EXISTS",
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ statement.executeUpdate();
+ statement.close();
+
+ String columnDefStr = "";
+ String columnListStr = "";
+ String valueListStr = "";
+
+ String [] myColNames = new String[colTypes.length];
+
+ for (int i = 0; i < colTypes.length; i++) {
+ String colName = BASE_COL_NAME + Integer.toString(i);
+ columnDefStr += colName + " " + colTypes[i];
+ columnListStr += colName;
+ valueListStr += vals[i];
+ myColNames[i] = colName;
+ if (i < colTypes.length - 1) {
+ columnDefStr += ", ";
+ columnListStr += ", ";
+ valueListStr += ", ";
+ }
+ }
+
+ statement = conn.prepareStatement(
+ "CREATE TABLE " + getTableName() + "(" + columnDefStr + ")",
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ statement.executeUpdate();
+ statement.close();
+
+ statement = conn.prepareStatement(
+ "INSERT INTO " + getTableName() + "(" + columnListStr + ")"
+ + " VALUES(" + valueListStr + ")",
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ statement.executeUpdate();
+ statement.close();
+ conn.commit();
+ this.colNames = myColNames;
+ } catch (SQLException sqlException) {
+ fail("Could not create table: " + sqlException.toString());
+ } finally {
+ if (null != conn) {
+ try {
+ conn.close();
+ } catch (SQLException sqlE) {
+ LOG.warn("Got SQLException during close: " + sqlE.toString());
+ }
+ }
+ }
+ }
+
+ /**
+ * Create a table with a single column and put a data element in it.
+ * @param colType the type of the column to create
+ * @param val the value to insert (reformatted as a string)
+ */
+ protected void createTableForColType(String colType, String val) {
+ String [] types = { colType };
+ String [] vals = { val };
+
+ createTableWithColTypes(types, vals);
+ }
+
+ protected Path getTablePath() {
+ Path warehousePath = new Path(getWarehouseDir());
+ Path tablePath = new Path(warehousePath, getTableName());
+ return tablePath;
+ }
+
+ protected Path getDataFilePath() {
+ return new Path(getTablePath(), "part-m-00000");
+ }
+
+ protected void removeTableDir() {
+ File tableDirFile = new File(getTablePath().toString());
+ if (tableDirFile.exists()) {
+ // Remove the director where the table will be imported to,
+ // prior to running the MapReduce job.
+ if (!DirUtil.deleteDir(tableDirFile)) {
+ LOG.warn("Could not delete table directory: " + tableDirFile.getAbsolutePath());
+ }
+ }
+ }
+
+ /**
+ * verify that the single-column single-row result can be read back from the db.
+ */
+ protected void verifyReadback(int colNum, String expectedVal) {
+ ResultSet results = null;
+ try {
+ results = getManager().readTable(getTableName(), getColNames());
+ assertNotNull("Null results from readTable()!", results);
+ assertTrue("Expected at least one row returned", results.next());
+ String resultVal = results.getString(colNum);
+ if (null != expectedVal) {
+ assertNotNull("Expected non-null result value", resultVal);
+ }
+
+ assertEquals("Error reading inserted value back from db", expectedVal, resultVal);
+ assertFalse("Expected at most one row returned", results.next());
+ } catch (SQLException sqlE) {
+ fail("Got SQLException: " + sqlE.toString());
+ } finally {
+ if (null != results) {
+ try {
+ results.close();
+ } catch (SQLException sqlE) {
+ fail("Got SQLException in resultset.close(): " + sqlE.toString());
+ }
+ }
+ }
+ }
+}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/ExportJobTestCase.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/ExportJobTestCase.java
new file mode 100644
index 0000000..0a21f80
--- /dev/null
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/ExportJobTestCase.java
@@ -0,0 +1,196 @@
+/**
+ * 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.sqoop.testutil;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.hadoop.sqoop.Sqoop;
+
+/**
+ * Class that implements common methods required for tests which export data
+ * from HDFS to databases, to verify correct export
+ */
+public class ExportJobTestCase extends BaseSqoopTestCase {
+
+ public static final Log LOG = LogFactory.getLog(ExportJobTestCase.class.getName());
+
+ protected String getTablePrefix() {
+ return "EXPORT_TABLE_";
+ }
+
+ /**
+ * Create the argv to pass to Sqoop
+ * @param includeHadoopFlags if true, then include -D various.settings=values
+ * @return the argv as an array of strings.
+ */
+ protected String [] getArgv(boolean includeHadoopFlags, String... additionalArgv) {
+ ArrayList<String> args = new ArrayList<String>();
+
+ if (includeHadoopFlags) {
+ CommonArgs.addHadoopFlags(args);
+ }
+
+ args.add("--table");
+ args.add(getTableName());
+ args.add("--export-dir");
+ args.add(getTablePath().toString());
+ args.add("--connect");
+ args.add(HsqldbTestServer.getUrl());
+ args.add("--fields-terminated-by");
+ args.add("\\t");
+ args.add("--lines-terminated-by");
+ args.add("\\n");
+
+
+ if (null != additionalArgv) {
+ for (String arg : additionalArgv) {
+ args.add(arg);
+ }
+ }
+
+ return args.toArray(new String[0]);
+ }
+
+ /** When exporting text columns, what should the text contain? */
+ protected String getMsgPrefix() {
+ return "textfield";
+ }
+
+
+ /** @return the minimum 'id' value in the table */
+ protected int getMinRowId() throws SQLException {
+ Connection conn = getTestServer().getConnection();
+ PreparedStatement statement = conn.prepareStatement(
+ "SELECT MIN(id) FROM " + getTableName(),
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ ResultSet rs = statement.executeQuery();
+ rs.next();
+ int minVal = rs.getInt(1);
+ rs.close();
+ statement.close();
+
+ return minVal;
+ }
+
+ /** @return the maximum 'id' value in the table */
+ protected int getMaxRowId() throws SQLException {
+ Connection conn = getTestServer().getConnection();
+ PreparedStatement statement = conn.prepareStatement(
+ "SELECT MAX(id) FROM " + getTableName(),
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ ResultSet rs = statement.executeQuery();
+ rs.next();
+ int maxVal = rs.getInt(1);
+ rs.close();
+ statement.close();
+
+ return maxVal;
+ }
+
+ /**
+ * Check that we got back the expected row set
+ * @param expectedNumRecords The number of records we expected to load
+ * into the database.
+ */
+ protected void verifyExport(int expectedNumRecords) throws IOException, SQLException {
+ Connection conn = getTestServer().getConnection();
+
+ LOG.info("Verifying export: " + getTableName());
+ // Check that we got back the correct number of records.
+ PreparedStatement statement = conn.prepareStatement(
+ "SELECT COUNT(*) FROM " + getTableName(),
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ ResultSet rs = statement.executeQuery();
+ rs.next();
+ int actualNumRecords = rs.getInt(1);
+ rs.close();
+ statement.close();
+
+ assertEquals("Got back unexpected row count", expectedNumRecords,
+ actualNumRecords);
+
+ // Check that we start with row 0.
+ int minVal = getMinRowId();
+ assertEquals("Minimum row was not zero", 0, minVal);
+
+ // Check that the last row we loaded is numRows - 1
+ int maxVal = getMaxRowId();
+ assertEquals("Maximum row had invalid id", expectedNumRecords - 1, maxVal);
+
+ // Check that the string values associated with these points match up.
+ statement = conn.prepareStatement("SELECT msg FROM " + getTableName()
+ + " WHERE id = " + minVal,
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ rs = statement.executeQuery();
+ rs.next();
+ String minMsg = rs.getString(1);
+ rs.close();
+ statement.close();
+
+ assertEquals("Invalid msg field for min value", getMsgPrefix() + minVal, minMsg);
+
+ statement = conn.prepareStatement("SELECT msg FROM " + getTableName()
+ + " WHERE id = " + maxVal,
+ ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+ rs = statement.executeQuery();
+ rs.next();
+ String maxMsg = rs.getString(1);
+ rs.close();
+ statement.close();
+
+ assertEquals("Invalid msg field for min value", getMsgPrefix() + maxVal, maxMsg);
+ }
+
+ /**
+ * Run a MapReduce-based export (using the argv provided to control execution).
+ * @return the generated jar filename
+ */
+ protected List<String> runExport(String [] argv) throws IOException {
+ // run the tool through the normal entry-point.
+ int ret;
+ List<String> generatedJars = null;
+ try {
+ Sqoop exporter = new Sqoop();
+ ret = ToolRunner.run(exporter, argv);
+ generatedJars = exporter.getGeneratedJarFiles();
+ } catch (Exception e) {
+ LOG.error("Got exception running Sqoop: " + e.toString());
+ e.printStackTrace();
+ ret = 1;
+ }
+
+ // expect a successful return.
+ if (0 != ret) {
+ throw new IOException("Failure during job; return status " + ret);
+ }
+
+ return generatedJars;
+ }
+
+}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/HsqldbTestServer.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/HsqldbTestServer.java
index b1197fc..71d0882 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/HsqldbTestServer.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/HsqldbTestServer.java
@@ -29,7 +29,7 @@
import org.apache.commons.logging.LogFactory;
import org.hsqldb.Server;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.manager.ConnManager;
import org.apache.hadoop.sqoop.manager.HsqldbManager;
@@ -226,13 +226,13 @@
populateData();
}
- public ImportOptions getImportOptions() {
- return new ImportOptions(HsqldbTestServer.getUrl(),
+ public SqoopOptions getSqoopOptions() {
+ return new SqoopOptions(HsqldbTestServer.getUrl(),
HsqldbTestServer.getTableName());
}
public ConnManager getManager() {
- return new HsqldbManager(getImportOptions());
+ return new HsqldbManager(getSqoopOptions());
}
diff --git a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java
index 090cf3a..28f1710 100644
--- a/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java
+++ b/src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java
@@ -20,250 +20,29 @@
import java.io.File;
import java.io.IOException;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
import java.util.ArrayList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.ToolRunner;
-import org.apache.log4j.BasicConfigurator;
-import org.junit.After;
-import org.junit.Before;
-import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.SqoopOptions;
import org.apache.hadoop.sqoop.Sqoop;
-import org.apache.hadoop.sqoop.ImportOptions.InvalidOptionsException;
-import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException;
import org.apache.hadoop.sqoop.orm.CompilationManager;
import org.apache.hadoop.sqoop.util.ClassLoaderStack;
-import junit.framework.TestCase;
-
/**
* Class that implements common methods required for tests which import data
* from SQL into HDFS and verify correct import.
*/
-public class ImportJobTestCase extends TestCase {
+public class ImportJobTestCase extends BaseSqoopTestCase {
public static final Log LOG = LogFactory.getLog(ImportJobTestCase.class.getName());
- /** Base directory for all temporary data */
- public static final String TEMP_BASE_DIR;
-
- /** Where to import table data to in the local filesystem for testing */
- public static final String LOCAL_WAREHOUSE_DIR;
-
- // Initializer for the above
- static {
- String tmpDir = System.getProperty("test.build.data", "/tmp/");
- if (!tmpDir.endsWith(File.separator)) {
- tmpDir = tmpDir + File.separator;
- }
-
- TEMP_BASE_DIR = tmpDir;
- LOCAL_WAREHOUSE_DIR = TEMP_BASE_DIR + "sqoop/warehouse";
- }
-
- // Used if a test manually sets the table name to be used.
- private String curTableName;
-
- protected void setCurTableName(String curName) {
- this.curTableName = curName;
- }
-
- /**
- * Because of how classloading works, we don't actually want to name
- * all the tables the same thing -- they'll actually just use the same
- * implementation of the Java class that was classloaded before. So we
- * use this counter to uniquify table names.
- */
- private static int tableNum = 0;
-
- /** the name of a table that we'll populate with items for each test. */
- static final String TABLE_NAME = "IMPORT_TABLE_";
-
- protected String getTableName() {
- if (null != curTableName) {
- return curTableName;
- } else {
- return TABLE_NAME + Integer.toString(tableNum);
- }
- }
-
- protected String getWarehouseDir() {
- return LOCAL_WAREHOUSE_DIR;
- }
-
- private String [] colNames;
- protected String [] getColNames() {
- return colNames;
- }
-
- protected HsqldbTestServer getTestServer() {
- return testServer;
- }
-
- protected ConnManager getManager() {
- return manager;
- }
-
- // instance variables populated during setUp, used during tests
- private HsqldbTestServer testServer;
- private ConnManager manager;
-
- private static boolean isLog4jConfigured = false;
-
- protected void incrementTableNum() {
- tableNum++;
- }
-
- @Before
- public void setUp() {
-
- incrementTableNum();
-
- if (!isLog4jConfigured) {
- BasicConfigurator.configure();
- isLog4jConfigured = true;
- LOG.info("Configured log4j with console appender.");
- }
-
- testServer = new HsqldbTestServer();
- try {
- testServer.resetServer();
- } catch (SQLException sqlE) {
- LOG.error("Got SQLException: " + sqlE.toString());
- fail("Got SQLException: " + sqlE.toString());
- } catch (ClassNotFoundException cnfe) {
- LOG.error("Could not find class for db driver: " + cnfe.toString());
- fail("Could not find class for db driver: " + cnfe.toString());
- }
-
- manager = testServer.getManager();
- }
-
- @After
- public void tearDown() {
- setCurTableName(null); // clear user-override table name.
-
- try {
- if (null != manager) {
- manager.close();
- }
- } catch (SQLException sqlE) {
- LOG.error("Got SQLException: " + sqlE.toString());
- fail("Got SQLException: " + sqlE.toString());
- }
-
- }
-
- static final String BASE_COL_NAME = "DATA_COL";
-
- /**
- * Create a table with a set of columns and add a row of values.
- * @param colTypes the types of the columns to make
- * @param vals the SQL text for each value to insert
- */
- protected void createTableWithColTypes(String [] colTypes, String [] vals) {
- Connection conn = null;
- try {
- conn = getTestServer().getConnection();
- PreparedStatement statement = conn.prepareStatement(
- "DROP TABLE " + getTableName() + " IF EXISTS",
- ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
- statement.executeUpdate();
- statement.close();
-
- String columnDefStr = "";
- String columnListStr = "";
- String valueListStr = "";
-
- String [] myColNames = new String[colTypes.length];
-
- for (int i = 0; i < colTypes.length; i++) {
- String colName = BASE_COL_NAME + Integer.toString(i);
- columnDefStr += colName + " " + colTypes[i];
- columnListStr += colName;
- valueListStr += vals[i];
- myColNames[i] = colName;
- if (i < colTypes.length - 1) {
- columnDefStr += ", ";
- columnListStr += ", ";
- valueListStr += ", ";
- }
- }
-
- statement = conn.prepareStatement(
- "CREATE TABLE " + getTableName() + "(" + columnDefStr + ")",
- ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
- statement.executeUpdate();
- statement.close();
-
- statement = conn.prepareStatement(
- "INSERT INTO " + getTableName() + "(" + columnListStr + ")"
- + " VALUES(" + valueListStr + ")",
- ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
- statement.executeUpdate();
- statement.close();
- conn.commit();
- this.colNames = myColNames;
- } catch (SQLException sqlException) {
- fail("Could not create table: " + sqlException.toString());
- } finally {
- if (null != conn) {
- try {
- conn.close();
- } catch (SQLException sqlE) {
- LOG.warn("Got SQLException during close: " + sqlE.toString());
- }
- }
- }
- }
-
- /**
- * Create a table with a single column and put a data element in it.
- * @param colType the type of the column to create
- * @param val the value to insert (reformatted as a string)
- */
- protected void createTableForColType(String colType, String val) {
- String [] types = { colType };
- String [] vals = { val };
-
- createTableWithColTypes(types, vals);
- }
-
- /**
- * verify that the single-column single-row result can be read back from the db.
- *
- */
- protected void verifyReadback(int colNum, String expectedVal) {
- ResultSet results = null;
- try {
- results = getManager().readTable(getTableName(), getColNames());
- assertNotNull("Null results from readTable()!", results);
- assertTrue("Expected at least one row returned", results.next());
- String resultVal = results.getString(colNum);
- if (null != expectedVal) {
- assertNotNull("Expected non-null result value", resultVal);
- }
-
- assertEquals("Error reading inserted value back from db", expectedVal, resultVal);
- assertFalse("Expected at most one row returned", results.next());
- } catch (SQLException sqlE) {
- fail("Got SQLException: " + sqlE.toString());
- } finally {
- if (null != results) {
- try {
- results.close();
- } catch (SQLException sqlE) {
- fail("Got SQLException in resultset.close(): " + sqlE.toString());
- }
- }
- }
+ protected String getTablePrefix() {
+ return "IMPORT_TABLE_";
}
/**
@@ -306,27 +85,6 @@
return args.toArray(new String[0]);
}
- protected Path getTablePath() {
- Path warehousePath = new Path(getWarehouseDir());
- Path tablePath = new Path(warehousePath, getTableName());
- return tablePath;
- }
-
- protected Path getDataFilePath() {
- return new Path(getTablePath(), "part-m-00000");
- }
-
- protected void removeTableDir() {
- File tableDirFile = new File(getTablePath().toString());
- if (tableDirFile.exists()) {
- // Remove the director where the table will be imported to,
- // prior to running the MapReduce job.
- if (!DirUtil.deleteDir(tableDirFile)) {
- LOG.warn("Could not delete table directory: " + tableDirFile.getAbsolutePath());
- }
- }
- }
-
/**
* Do a MapReduce-based import of the table and verify that the results
* were imported as expected. (tests readFields(ResultSet) and toString())
@@ -353,7 +111,7 @@
// expect a successful return.
assertEquals("Failure during job", 0, ret);
- ImportOptions opts = new ImportOptions();
+ SqoopOptions opts = new SqoopOptions();
try {
opts.parse(getArgv(false, importCols));
} catch (InvalidOptionsException ioe) {
diff --git a/src/contrib/sqoop/testdata/hive/scripts/customDelimImport.q b/src/contrib/sqoop/testdata/hive/scripts/customDelimImport.q
index a64166e..1e7defb 100644
--- a/src/contrib/sqoop/testdata/hive/scripts/customDelimImport.q
+++ b/src/contrib/sqoop/testdata/hive/scripts/customDelimImport.q
@@ -1,2 +1,2 @@
-CREATE TABLE CUSTOM_DELIM_IMPORT ( DATA_COL0 STRING, DATA_COL1 INT, DATA_COL2 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\054' LINES TERMINATED BY '\0174' STORED AS TEXTFILE;
+CREATE TABLE CUSTOM_DELIM_IMPORT ( DATA_COL0 STRING, DATA_COL1 INT, DATA_COL2 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\054' LINES TERMINATED BY '\174' STORED AS TEXTFILE;
LOAD DATA INPATH 'file:BASEPATH/sqoop/warehouse/CUSTOM_DELIM_IMPORT' INTO TABLE CUSTOM_DELIM_IMPORT;
diff --git a/src/contrib/sqoop/testdata/hive/scripts/dateImport.q b/src/contrib/sqoop/testdata/hive/scripts/dateImport.q
index fc74ff3..223f67a 100644
--- a/src/contrib/sqoop/testdata/hive/scripts/dateImport.q
+++ b/src/contrib/sqoop/testdata/hive/scripts/dateImport.q
@@ -1,2 +1,2 @@
-CREATE TABLE DATE_HIVE_IMPORT ( DATA_COL0 STRING, DATA_COL1 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\01' LINES TERMINATED BY '\012' STORED AS TEXTFILE;
+CREATE TABLE DATE_HIVE_IMPORT ( DATA_COL0 STRING, DATA_COL1 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\001' LINES TERMINATED BY '\012' STORED AS TEXTFILE;
LOAD DATA INPATH 'file:BASEPATH/sqoop/warehouse/DATE_HIVE_IMPORT' INTO TABLE DATE_HIVE_IMPORT;
diff --git a/src/contrib/sqoop/testdata/hive/scripts/failingImport.q b/src/contrib/sqoop/testdata/hive/scripts/failingImport.q
index fc74ff3..223f67a 100644
--- a/src/contrib/sqoop/testdata/hive/scripts/failingImport.q
+++ b/src/contrib/sqoop/testdata/hive/scripts/failingImport.q
@@ -1,2 +1,2 @@
-CREATE TABLE DATE_HIVE_IMPORT ( DATA_COL0 STRING, DATA_COL1 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\01' LINES TERMINATED BY '\012' STORED AS TEXTFILE;
+CREATE TABLE DATE_HIVE_IMPORT ( DATA_COL0 STRING, DATA_COL1 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\001' LINES TERMINATED BY '\012' STORED AS TEXTFILE;
LOAD DATA INPATH 'file:BASEPATH/sqoop/warehouse/DATE_HIVE_IMPORT' INTO TABLE DATE_HIVE_IMPORT;
diff --git a/src/contrib/sqoop/testdata/hive/scripts/normalImport.q b/src/contrib/sqoop/testdata/hive/scripts/normalImport.q
index 328e37c..76e690c 100644
--- a/src/contrib/sqoop/testdata/hive/scripts/normalImport.q
+++ b/src/contrib/sqoop/testdata/hive/scripts/normalImport.q
@@ -1,2 +1,2 @@
-CREATE TABLE NORMAL_HIVE_IMPORT ( DATA_COL0 STRING, DATA_COL1 INT, DATA_COL2 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\01' LINES TERMINATED BY '\012' STORED AS TEXTFILE;
+CREATE TABLE NORMAL_HIVE_IMPORT ( DATA_COL0 STRING, DATA_COL1 INT, DATA_COL2 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\001' LINES TERMINATED BY '\012' STORED AS TEXTFILE;
LOAD DATA INPATH 'file:BASEPATH/sqoop/warehouse/NORMAL_HIVE_IMPORT' INTO TABLE NORMAL_HIVE_IMPORT;
diff --git a/src/contrib/sqoop/testdata/hive/scripts/numericImport.q b/src/contrib/sqoop/testdata/hive/scripts/numericImport.q
index bb6c154..263b295 100644
--- a/src/contrib/sqoop/testdata/hive/scripts/numericImport.q
+++ b/src/contrib/sqoop/testdata/hive/scripts/numericImport.q
@@ -1,2 +1,2 @@
-CREATE TABLE NUMERIC_HIVE_IMPORT ( DATA_COL0 DOUBLE, DATA_COL1 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\01' LINES TERMINATED BY '\012' STORED AS TEXTFILE;
+CREATE TABLE NUMERIC_HIVE_IMPORT ( DATA_COL0 DOUBLE, DATA_COL1 STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\001' LINES TERMINATED BY '\012' STORED AS TEXTFILE;
LOAD DATA INPATH 'file:BASEPATH/sqoop/warehouse/NUMERIC_HIVE_IMPORT' INTO TABLE NUMERIC_HIVE_IMPORT;
diff --git a/src/contrib/streaming/ivy.xml b/src/contrib/streaming/ivy.xml
index 39ffbd8..ac7f6ec 100644
--- a/src/contrib/streaming/ivy.xml
+++ b/src/contrib/streaming/ivy.xml
@@ -1,4 +1,20 @@
<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
<ivy-module version="1.0">
<info organisation="org.apache.hadoop" module="${ant.project.name}">
<license name="Apache 2.0"/>
@@ -27,11 +43,11 @@
<dependency org="org.apache.hadoop" name="hadoop-core"
rev="${hadoop-core.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-core-test"
- rev="${hadoop-core.version}" conf="common->default"/>
+ rev="${hadoop-core.version}" conf="test->default"/>
<dependency org="org.apache.hadoop" name="hadoop-hdfs"
rev="${hadoop-hdfs.version}" conf="common->default"/>
<dependency org="org.apache.hadoop" name="hadoop-hdfs-test"
- rev="${hadoop-hdfs.version}" conf="common->default"/>
+ rev="${hadoop-hdfs.version}" conf="test->default"/>
<dependency org="commons-cli" name="commons-cli"
rev="${commons-cli.version}" conf="common->default"/>
<dependency org="commons-logging" name="commons-logging"
@@ -56,7 +72,5 @@
rev="${avro.version}" conf="common->default"/>
<dependency org="org.slf4j" name="slf4j-api"
rev="${slf4j-api.version}" conf="common->master"/>
- <dependency org="org.slf4j" name="slf4j-log4j12"
- rev="${slf4j-log4j12.version}" conf="common->master"/>
</dependencies>
</ivy-module>
diff --git a/src/contrib/streaming/ivy/libraries.properties b/src/contrib/streaming/ivy/libraries.properties
index a470b37..8a80dd8 100644
--- a/src/contrib/streaming/ivy/libraries.properties
+++ b/src/contrib/streaming/ivy/libraries.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
#This properties file lists the versions of the various artifacts used by streaming.
#It drives ivy and the generation of a maven POM
diff --git a/src/contrib/streaming/src/java/org/apache/hadoop/streaming/AutoInputFormat.java b/src/contrib/streaming/src/java/org/apache/hadoop/streaming/AutoInputFormat.java
index 04b27e4..10cf614 100644
--- a/src/contrib/streaming/src/java/org/apache/hadoop/streaming/AutoInputFormat.java
+++ b/src/contrib/streaming/src/java/org/apache/hadoop/streaming/AutoInputFormat.java
@@ -52,7 +52,7 @@
public RecordReader getRecordReader(InputSplit split, JobConf job,
Reporter reporter) throws IOException {
FileSplit fileSplit = (FileSplit) split;
- FileSystem fs = FileSystem.get(job);
+ FileSystem fs = FileSystem.get(fileSplit.getPath().toUri(), job);
FSDataInputStream is = fs.open(fileSplit.getPath());
byte[] header = new byte[3];
RecordReader reader = null;
diff --git a/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java b/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
index f3544a0..27323a2 100644
--- a/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
+++ b/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
@@ -25,10 +25,9 @@
import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Iterator;
import java.util.List;
-import java.util.ListIterator;
import java.util.Map;
+import java.util.regex.Pattern;
import java.util.TreeMap;
import java.util.TreeSet;
@@ -123,13 +122,13 @@
postProcessArgs();
setJobConf();
- return submitAndMonitorJob();
}catch (IllegalArgumentException ex) {
//ignore, since log will already be printed
// print the log in debug mode.
LOG.debug("Error in streaming job", ex);
return 1;
}
+ return submitAndMonitorJob();
}
/**
@@ -191,9 +190,8 @@
}
msg("addTaskEnvironment=" + addTaskEnvironment_);
- Iterator it = packageFiles_.iterator();
- while (it.hasNext()) {
- File f = new File((String) it.next());
+ for (final String packageFile : packageFiles_) {
+ File f = new File(packageFile);
if (f.isFile()) {
shippedCanonFiles_.add(f.getCanonicalPath());
}
@@ -262,37 +260,44 @@
inputSpecs_.add(input);
}
}
- output_ = (String) cmdLine.getOptionValue("output");
+ output_ = cmdLine.getOptionValue("output");
- mapCmd_ = (String)cmdLine.getOptionValue("mapper");
- comCmd_ = (String)cmdLine.getOptionValue("combiner");
- redCmd_ = (String)cmdLine.getOptionValue("reducer");
+ mapCmd_ = cmdLine.getOptionValue("mapper");
+ comCmd_ = cmdLine.getOptionValue("combiner");
+ redCmd_ = cmdLine.getOptionValue("reducer");
lazyOutput_ = cmdLine.hasOption("lazyOutput");
values = cmdLine.getOptionValues("file");
if (values != null && values.length > 0) {
+ StringBuilder unpackRegex = new StringBuilder(
+ config_.getPattern(JobContext.JAR_UNPACK_PATTERN,
+ JobConf.UNPACK_JAR_PATTERN_DEFAULT).pattern());
for (String file : values) {
packageFiles_.add(file);
+ String fname = new File(file).getName();
+ unpackRegex.append("|(?:").append(Pattern.quote(fname)).append(")");
}
+ config_.setPattern(JobContext.JAR_UNPACK_PATTERN,
+ Pattern.compile(unpackRegex.toString()));
validate(packageFiles_);
}
- String fsName = (String)cmdLine.getOptionValue("dfs");
+ String fsName = cmdLine.getOptionValue("dfs");
if (null != fsName){
LOG.warn("-dfs option is deprecated, please use -fs instead.");
config_.set("fs.default.name", fsName);
}
- additionalConfSpec_ = (String)cmdLine.getOptionValue("additionalconfspec");
- inputFormatSpec_ = (String)cmdLine.getOptionValue("inputformat");
- outputFormatSpec_ = (String)cmdLine.getOptionValue("outputformat");
- numReduceTasksSpec_ = (String)cmdLine.getOptionValue("numReduceTasks");
- partitionerSpec_ = (String)cmdLine.getOptionValue("partitioner");
- inReaderSpec_ = (String)cmdLine.getOptionValue("inputreader");
- mapDebugSpec_ = (String)cmdLine.getOptionValue("mapdebug");
- reduceDebugSpec_ = (String)cmdLine.getOptionValue("reducedebug");
- ioSpec_ = (String)cmdLine.getOptionValue("io");
+ additionalConfSpec_ = cmdLine.getOptionValue("additionalconfspec");
+ inputFormatSpec_ = cmdLine.getOptionValue("inputformat");
+ outputFormatSpec_ = cmdLine.getOptionValue("outputformat");
+ numReduceTasksSpec_ = cmdLine.getOptionValue("numReduceTasks");
+ partitionerSpec_ = cmdLine.getOptionValue("partitioner");
+ inReaderSpec_ = cmdLine.getOptionValue("inputreader");
+ mapDebugSpec_ = cmdLine.getOptionValue("mapdebug");
+ reduceDebugSpec_ = cmdLine.getOptionValue("reducedebug");
+ ioSpec_ = cmdLine.getOptionValue("io");
String[] car = cmdLine.getOptionValues("cacheArchive");
if (null != car && car.length > 0){
@@ -582,7 +587,7 @@
/** @return path to the created Jar file or null if no files are necessary.
*/
protected String packageJobJar() throws IOException {
- ArrayList unjarFiles = new ArrayList();
+ ArrayList<String> unjarFiles = new ArrayList<String>();
// Runtime code: ship same version of code as self (job submitter code)
// usually found in: build/contrib or build/hadoop-<version>-dev-streaming.jar
@@ -884,15 +889,11 @@
protected void listJobConfProperties()
{
msg("==== JobConf properties:");
- Iterator it = jobConf_.iterator();
- TreeMap sorted = new TreeMap();
- while(it.hasNext()) {
- Map.Entry en = (Map.Entry)it.next();
+ TreeMap<String,String> sorted = new TreeMap<String,String>();
+ for (final Map.Entry<String, String> en : jobConf_) {
sorted.put(en.getKey(), en.getValue());
}
- it = sorted.entrySet().iterator();
- while(it.hasNext()) {
- Map.Entry en = (Map.Entry)it.next();
+ for (final Map.Entry<String,String> en: sorted.entrySet()) {
msg(en.getKey() + "=" + en.getValue());
}
msg("====");
@@ -997,11 +998,11 @@
protected JobClient jc_;
// command-line arguments
- protected ArrayList inputSpecs_ = new ArrayList(); // <String>
- protected TreeSet seenPrimary_ = new TreeSet(); // <String>
+ protected ArrayList<String> inputSpecs_ = new ArrayList<String>();
+ protected TreeSet<String> seenPrimary_ = new TreeSet<String>();
protected boolean hasSimpleInputSpecs_;
- protected ArrayList packageFiles_ = new ArrayList(); // <String>
- protected ArrayList shippedCanonFiles_ = new ArrayList(); // <String>
+ protected ArrayList<String> packageFiles_ = new ArrayList<String>();
+ protected ArrayList<String> shippedCanonFiles_ = new ArrayList<String>();
//protected TreeMap<String, String> userJobConfProps_ = new TreeMap<String, String>();
protected String output_;
protected String mapCmd_;
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestAutoInputFormat.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestAutoInputFormat.java
index 97593d3..4ff3f54 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestAutoInputFormat.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestAutoInputFormat.java
@@ -36,9 +36,10 @@
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.streaming.AutoInputFormat;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
-public class TestAutoInputFormat extends TestCase {
+public class TestAutoInputFormat {
private static Configuration conf = new Configuration();
@@ -49,6 +50,7 @@
private static final int SPLITS_COUNT = 2;
@SuppressWarnings( { "unchecked", "deprecation" })
+ @Test
public void testFormat() throws IOException {
JobConf job = new JobConf(conf);
FileSystem fs = FileSystem.getLocal(conf);
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestDumpTypedBytes.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestDumpTypedBytes.java
index b9857f2..c290f50 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestDumpTypedBytes.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestDumpTypedBytes.java
@@ -31,10 +31,12 @@
import org.apache.hadoop.streaming.DumpTypedBytes;
import org.apache.hadoop.typedbytes.TypedBytesInput;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
-public class TestDumpTypedBytes extends TestCase {
+public class TestDumpTypedBytes {
+ @Test
public void testDumping() throws Exception {
Configuration conf = new Configuration();
MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestFileArgs.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestFileArgs.java
new file mode 100644
index 0000000..5a38300
--- /dev/null
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestFileArgs.java
@@ -0,0 +1,101 @@
+/**
+ * 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.streaming;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.util.zip.GZIPOutputStream;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+
+/**
+ * This class tests that the '-file' argument to streaming results
+ * in files being unpacked in the job working directory.
+ */
+public class TestFileArgs extends TestStreaming
+{
+ private MiniDFSCluster dfs = null;
+ private MiniMRCluster mr = null;
+ private FileSystem fileSys = null;
+ private String strJobTracker = null;
+ private String strNamenode = null;
+ private String namenode = null;
+ private Configuration conf = null;
+
+ private static final String EXPECTED_OUTPUT =
+ "job.jar\t\nsidefile\t\ntmp\t\n";
+
+ private static final String LS_PATH = "/bin/ls";
+
+ public TestFileArgs() throws IOException
+ {
+ // Set up mini cluster
+ conf = new Configuration();
+ dfs = new MiniDFSCluster(conf, 1, true, null);
+ fileSys = dfs.getFileSystem();
+ namenode = fileSys.getUri().getAuthority();
+ mr = new MiniMRCluster(1, namenode, 1);
+ strJobTracker = JTConfig.JT_IPC_ADDRESS + "=localhost:" + mr.getJobTrackerPort();
+ strNamenode = "fs.default.name=hdfs://" + namenode;
+
+ FileSystem.setDefaultUri(conf, "hdfs://" + namenode);
+
+ // Set up side file
+ FileSystem localFs = FileSystem.getLocal(conf);
+ DataOutputStream dos = localFs.create(new Path("sidefile"));
+ dos.write("hello world\n".getBytes("UTF-8"));
+ dos.close();
+ }
+
+ @Override
+ protected String getExpectedOutput() {
+ return EXPECTED_OUTPUT;
+ }
+
+ @Override
+ protected Configuration getConf() {
+ return conf;
+ }
+
+ @Override
+ protected String[] genArgs() {
+ return new String[] {
+ "-input", INPUT_FILE.getAbsolutePath(),
+ "-output", OUTPUT_DIR.getAbsolutePath(),
+ "-file", new java.io.File("sidefile").getAbsolutePath(),
+ "-mapper", LS_PATH,
+ "-numReduceTasks", "0",
+ "-jobconf", strNamenode,
+ "-jobconf", strJobTracker,
+ "-jobconf", "stream.tmpdir=" + System.getProperty("test.build.data","/tmp")
+ };
+ }
+
+
+ public static void main(String[]args) throws Exception
+ {
+ new TestFileArgs().testCommandLine();
+ }
+
+}
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestGzipInput.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestGzipInput.java
index 13f5da9..e9c128a 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestGzipInput.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestGzipInput.java
@@ -30,13 +30,13 @@
{
public TestGzipInput() throws IOException {
- INPUT_FILE = new File("input.txt.gz");
+ INPUT_FILE = new File(TEST_DIR, "input.txt.gz");
}
protected void createInput() throws IOException
{
GZIPOutputStream out = new GZIPOutputStream(
- new FileOutputStream(INPUT_FILE.getAbsoluteFile()));
+ new FileOutputStream(INPUT_FILE.getAbsoluteFile()));
out.write(input.getBytes("UTF-8"));
out.close();
}
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestLoadTypedBytes.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestLoadTypedBytes.java
index 96a25fc..df619f9 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestLoadTypedBytes.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestLoadTypedBytes.java
@@ -31,10 +31,12 @@
import org.apache.hadoop.typedbytes.TypedBytesOutput;
import org.apache.hadoop.typedbytes.TypedBytesWritable;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
-public class TestLoadTypedBytes extends TestCase {
+public class TestLoadTypedBytes {
+ @Test
public void testLoading() throws Exception {
Configuration conf = new Configuration();
MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java
index 2eaed29..4e1b2db 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java
@@ -24,9 +24,14 @@
import java.io.DataOutputStream;
import java.io.InputStreamReader;
import java.io.BufferedReader;
+import java.util.Arrays;
import java.util.zip.ZipEntry;
import java.util.jar.JarOutputStream;
import java.util.zip.ZipOutputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
@@ -34,6 +39,10 @@
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.util.StringUtils;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
/**
* This class tests cacheArchive option of streaming
@@ -42,12 +51,14 @@
*/
public class TestMultipleArchiveFiles extends TestStreaming
{
+ private static final Log LOG = LogFactory.getLog(TestMultipleArchiveFiles.class);
private StreamJob job;
- private String INPUT_FILE = "input.txt";
- private String CACHE_ARCHIVE_1 = "cacheArchive1.zip";
+ private String INPUT_DIR = "multiple-archive-files/";
+ private String INPUT_FILE = INPUT_DIR + "input.txt";
+ private String CACHE_ARCHIVE_1 = INPUT_DIR + "cacheArchive1.zip";
private File CACHE_FILE_1 = null;
- private String CACHE_ARCHIVE_2 = "cacheArchive2.zip";
+ private String CACHE_ARCHIVE_2 = INPUT_DIR + "cacheArchive2.zip";
private File CACHE_FILE_2 = null;
private String expectedOutput = null;
private String OUTPUT_DIR = "out";
@@ -59,27 +70,23 @@
private String strNamenode = null;
private String namenode = null;
- public TestMultipleArchiveFiles() throws IOException {
+ public TestMultipleArchiveFiles() throws Exception {
CACHE_FILE_1 = new File("cacheArchive1");
CACHE_FILE_2 = new File("cacheArchive2");
input = "HADOOP";
expectedOutput = "HADOOP\t\nHADOOP\t\n";
- try {
- conf = new Configuration();
- dfs = new MiniDFSCluster(conf, 1, true, null);
- fileSys = dfs.getFileSystem();
- namenode = fileSys.getUri().getAuthority();
- mr = new MiniMRCluster(1, namenode, 3);
- strJobTracker = JTConfig.JT_IPC_ADDRESS + "=localhost:" + mr.getJobTrackerPort();
- strNamenode = "fs.default.name=" + namenode;
- } catch (Exception e) {
- e.printStackTrace();
- }
+ conf = new Configuration();
+ dfs = new MiniDFSCluster(conf, 1, true, null);
+ fileSys = dfs.getFileSystem();
+ namenode = fileSys.getUri().getAuthority();
+ mr = new MiniMRCluster(1, namenode, 3);
+ strJobTracker = JTConfig.JT_IPC_ADDRESS + "=localhost:" + mr.getJobTrackerPort();
+ strNamenode = "fs.default.name=" + namenode;
}
protected void createInput() throws IOException
{
-
+ fileSys.delete(new Path(INPUT_DIR), true);
DataOutputStream dos = fileSys.create(new Path(INPUT_FILE));
String inputFileString = "symlink1/cacheArchive1\nsymlink2/cacheArchive2";
dos.write(inputFileString.getBytes("UTF-8"));
@@ -103,14 +110,9 @@
}
protected String[] genArgs() {
- String cacheArchiveString1 = null;
- String cacheArchiveString2 = null;
- try {
- cacheArchiveString1 = fileSys.getUri().toString()+fileSys.getWorkingDirectory().toString()+"/"+CACHE_ARCHIVE_1+"#symlink1";
- cacheArchiveString2 = fileSys.getUri().toString()+fileSys.getWorkingDirectory().toString()+"/"+CACHE_ARCHIVE_2+"#symlink2";
- } catch (Exception e) {
- e.printStackTrace();
- }
+ String workDir = fileSys.getWorkingDirectory().toString() + "/";
+ String cache1 = workDir + CACHE_ARCHIVE_1 + "#symlink1";
+ String cache2 = workDir + CACHE_ARCHIVE_2 + "#symlink2";
return new String[] {
"-input", INPUT_FILE.toString(),
@@ -118,39 +120,32 @@
"-mapper", "xargs cat",
"-reducer", "cat",
"-jobconf", "mapreduce.job.reduces=1",
- "-cacheArchive", cacheArchiveString1,
- "-cacheArchive", cacheArchiveString2,
+ "-cacheArchive", cache1,
+ "-cacheArchive", cache2,
"-jobconf", strNamenode,
"-jobconf", strJobTracker,
"-jobconf", "stream.tmpdir=" + System.getProperty("test.build.data","/tmp")
};
}
- public void testCommandLine() {
- try {
- createInput();
- job = new StreamJob(genArgs(), true);
- if(job.go() != 0) {
- throw new Exception("Job Failed");
- }
- StringBuffer output = new StringBuffer(256);
- Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
- new Path(OUTPUT_DIR)));
- for (int i = 0; i < fileList.length; i++){
- BufferedReader bread =
- new BufferedReader(new InputStreamReader(fileSys.open(fileList[i])));
- output.append(bread.readLine());
- output.append("\n");
- output.append(bread.readLine());
- output.append("\n");
- }
- assertEquals(expectedOutput, output.toString());
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- CACHE_FILE_1.delete();
- CACHE_FILE_2.delete();
+ //@Test
+ public void testCommandLine() throws Exception {
+ createInput();
+ String args[] = genArgs();
+ LOG.info("Testing streaming command line:\n" +
+ StringUtils.join(" ", Arrays.asList(args)));
+ job = new StreamJob(genArgs(), true);
+ if(job.go() != 0) {
+ throw new Exception("Job Failed");
}
+ StringBuffer output = new StringBuffer(256);
+ Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
+ new Path(OUTPUT_DIR)));
+ for (int i = 0; i < fileList.length; i++){
+ LOG.info("Adding output from file: " + fileList[i]);
+ output.append(StreamUtil.slurpHadoop(fileList[i], fileSys));
+ }
+ assertEquals(expectedOutput, output.toString());
}
public static void main(String[]args) throws Exception
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java
index 8706d8f..a2e59cd 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java
@@ -25,7 +25,8 @@
import java.io.PrintWriter;
import java.io.StringWriter;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -40,7 +41,7 @@
* This test case tests the symlink creation
* utility provided by distributed caching
*/
-public class TestMultipleCachefiles extends TestCase
+public class TestMultipleCachefiles
{
String INPUT_FILE = "/testing-streaming/input.txt";
String OUTPUT_DIR = "/testing-streaming/out";
@@ -59,98 +60,87 @@
{
}
- public void testMultipleCachefiles()
+ @Test
+ public void testMultipleCachefiles() throws Exception
{
- try {
- boolean mayExit = false;
- MiniMRCluster mr = null;
- MiniDFSCluster dfs = null;
- try{
- Configuration conf = new Configuration();
- dfs = new MiniDFSCluster(conf, 1, true, null);
- FileSystem fileSys = dfs.getFileSystem();
- String namenode = fileSys.getUri().toString();
+ boolean mayExit = false;
+ MiniMRCluster mr = null;
+ MiniDFSCluster dfs = null;
+ try{
+ Configuration conf = new Configuration();
+ dfs = new MiniDFSCluster(conf, 1, true, null);
+ FileSystem fileSys = dfs.getFileSystem();
+ String namenode = fileSys.getUri().toString();
- mr = new MiniMRCluster(1, namenode, 3);
- // During tests, the default Configuration will use a local mapred
- // So don't specify -config or -cluster
- String strJobtracker = JTConfig.JT_IPC_ADDRESS + "=localhost:" + mr.getJobTrackerPort();
- String strNamenode = "fs.default.name=" + namenode;
- String argv[] = new String[] {
- "-input", INPUT_FILE,
- "-output", OUTPUT_DIR,
- "-mapper", map,
- "-reducer", reduce,
- //"-verbose",
- //"-jobconf", "stream.debug=set"
- "-jobconf", strNamenode,
- "-jobconf", strJobtracker,
- "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp"),
- "-jobconf",
- JobConf.MAPRED_MAP_TASK_JAVA_OPTS + "=" +
- "-Dcontrib.name=" + System.getProperty("contrib.name") + " " +
- "-Dbuild.test=" + System.getProperty("build.test") + " " +
- conf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS,
- conf.get(JobConf.MAPRED_TASK_JAVA_OPTS, "")),
- "-jobconf",
- JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS + "=" +
- "-Dcontrib.name=" + System.getProperty("contrib.name") + " " +
- "-Dbuild.test=" + System.getProperty("build.test") + " " +
- conf.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS,
- conf.get(JobConf.MAPRED_TASK_JAVA_OPTS, "")),
- "-cacheFile", fileSys.getUri() + CACHE_FILE + "#" + mapString,
- "-cacheFile", fileSys.getUri() + CACHE_FILE_2 + "#" + mapString2
- };
+ mr = new MiniMRCluster(1, namenode, 3);
+ // During tests, the default Configuration will use a local mapred
+ // So don't specify -config or -cluster
+ String strJobtracker = JTConfig.JT_IPC_ADDRESS + "=localhost:" + mr.getJobTrackerPort();
+ String strNamenode = "fs.default.name=" + namenode;
+ String argv[] = new String[] {
+ "-input", INPUT_FILE,
+ "-output", OUTPUT_DIR,
+ "-mapper", map,
+ "-reducer", reduce,
+ //"-verbose",
+ //"-jobconf", "stream.debug=set"
+ "-jobconf", strNamenode,
+ "-jobconf", strJobtracker,
+ "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp"),
+ "-jobconf",
+ JobConf.MAPRED_MAP_TASK_JAVA_OPTS + "=" +
+ "-Dcontrib.name=" + System.getProperty("contrib.name") + " " +
+ "-Dbuild.test=" + System.getProperty("build.test") + " " +
+ conf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS,
+ conf.get(JobConf.MAPRED_TASK_JAVA_OPTS, "")),
+ "-jobconf",
+ JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS + "=" +
+ "-Dcontrib.name=" + System.getProperty("contrib.name") + " " +
+ "-Dbuild.test=" + System.getProperty("build.test") + " " +
+ conf.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS,
+ conf.get(JobConf.MAPRED_TASK_JAVA_OPTS, "")),
+ "-cacheFile", fileSys.getUri() + CACHE_FILE + "#" + mapString,
+ "-cacheFile", fileSys.getUri() + CACHE_FILE_2 + "#" + mapString2
+ };
- fileSys.delete(new Path(OUTPUT_DIR), true);
-
- DataOutputStream file = fileSys.create(new Path(INPUT_FILE));
- file.writeBytes(mapString + "\n");
- file.writeBytes(mapString2 + "\n");
- file.close();
- file = fileSys.create(new Path(CACHE_FILE));
- file.writeBytes(cacheString + "\n");
- file.close();
- file = fileSys.create(new Path(CACHE_FILE_2));
- file.writeBytes(cacheString2 + "\n");
- file.close();
-
- job = new StreamJob(argv, mayExit);
- job.go();
-
- fileSys = dfs.getFileSystem();
- String line = null;
- String line2 = null;
- Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
- new Path(OUTPUT_DIR),
- new Utils.OutputFileUtils
- .OutputFilesFilter()));
- for (int i = 0; i < fileList.length; i++){
- System.out.println(fileList[i].toString());
- BufferedReader bread =
- new BufferedReader(new InputStreamReader(fileSys.open(fileList[i])));
- line = bread.readLine();
- System.out.println(line);
- line2 = bread.readLine();
- System.out.println(line2);
- }
- assertEquals(cacheString + "\t", line);
- assertEquals(cacheString2 + "\t", line2);
- } finally{
- if (dfs != null) { dfs.shutdown(); }
- if (mr != null) { mr.shutdown();}
- }
+ fileSys.delete(new Path(OUTPUT_DIR), true);
- } catch(Exception e) {
- failTrace(e);
- }
- }
+ DataOutputStream file = fileSys.create(new Path(INPUT_FILE));
+ file.writeBytes(mapString + "\n");
+ file.writeBytes(mapString2 + "\n");
+ file.close();
+ file = fileSys.create(new Path(CACHE_FILE));
+ file.writeBytes(cacheString + "\n");
+ file.close();
+ file = fileSys.create(new Path(CACHE_FILE_2));
+ file.writeBytes(cacheString2 + "\n");
+ file.close();
+
+ job = new StreamJob(argv, mayExit);
+ job.go();
- void failTrace(Exception e)
- {
- StringWriter sw = new StringWriter();
- e.printStackTrace(new PrintWriter(sw));
- fail(sw.toString());
+ fileSys = dfs.getFileSystem();
+ String line = null;
+ String line2 = null;
+ Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
+ new Path(OUTPUT_DIR),
+ new Utils.OutputFileUtils
+ .OutputFilesFilter()));
+ for (int i = 0; i < fileList.length; i++){
+ System.out.println(fileList[i].toString());
+ BufferedReader bread =
+ new BufferedReader(new InputStreamReader(fileSys.open(fileList[i])));
+ line = bread.readLine();
+ System.out.println(line);
+ line2 = bread.readLine();
+ System.out.println(line2);
+ }
+ assertEquals(cacheString + "\t", line);
+ assertEquals(cacheString2 + "\t", line2);
+ } finally{
+ if (dfs != null) { dfs.shutdown(); }
+ if (mr != null) { mr.shutdown();}
+ }
}
public static void main(String[]args) throws Exception
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestRawBytesStreaming.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestRawBytesStreaming.java
index d165009..7210a7f 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestRawBytesStreaming.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestRawBytesStreaming.java
@@ -26,9 +26,10 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
-public class TestRawBytesStreaming extends TestCase {
+public class TestRawBytesStreaming {
protected File INPUT_FILE = new File("input.txt");
protected File OUTPUT_DIR = new File("out");
@@ -62,7 +63,8 @@
"-verbose"
};
}
-
+
+ @Test
public void testCommandLine() throws Exception {
try {
try {
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamAggregate.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamAggregate.java
index 2202d9d..2a3e70e 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamAggregate.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamAggregate.java
@@ -18,7 +18,8 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
import java.io.*;
import org.apache.hadoop.fs.FileUtil;
@@ -29,7 +30,7 @@
* It uses Hadoop Aggregate to count the numbers of word occurrences
* in the input.
*/
-public class TestStreamAggregate extends TestCase
+public class TestStreamAggregate
{
protected File INPUT_FILE = new File("stream_aggregate_input.txt");
protected File OUTPUT_DIR = new File("stream_aggregate_out");
@@ -69,8 +70,8 @@
};
}
- public void testCommandLine()
- {
+ @Test
+ public void testCommandLine() throws Exception {
try {
try {
FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
@@ -90,25 +91,12 @@
System.err.println("outEx1=" + outputExpect);
System.err.println(" out1=" + output);
assertEquals(outputExpect, output);
- } catch(Exception e) {
- failTrace(e);
} finally {
- try {
- INPUT_FILE.delete();
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- } catch (IOException e) {
- failTrace(e);
- }
+ INPUT_FILE.delete();
+ FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
}
}
- private void failTrace(Exception e)
- {
- StringWriter sw = new StringWriter();
- e.printStackTrace(new PrintWriter(sw));
- fail(sw.toString());
- }
-
public static void main(String[]args) throws Exception
{
new TestStreaming().testCommandLine();
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamDataProtocol.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamDataProtocol.java
index a869ded..7ae64f7 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamDataProtocol.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamDataProtocol.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
import java.io.*;
import java.util.*;
import org.apache.hadoop.conf.Configuration;
@@ -27,10 +26,13 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
/**
* This class tests hadoopStreaming in MapReduce local mode.
*/
-public class TestStreamDataProtocol extends TestCase
+public class TestStreamDataProtocol
{
// "map" command: grep -E (red|green|blue)
@@ -71,22 +73,23 @@
//"-verbose",
"-jobconf", "stream.map.output.field.separator=.",
"-jobconf", "stream.num.map.output.key.fields=2",
- "-jobconf", "mapreduce.mapreduce.mapreduce.map.output.key.field.separator=.",
+ "-jobconf", "mapreduce.map.output.key.field.separator=.",
"-jobconf", "num.key.fields.for.partition=1",
"-jobconf", "mapreduce.job.reduces=2",
"-jobconf", "mapreduce.task.files.preserve.failedtasks=true",
"-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp")
};
}
-
- public void testCommandLine()
+
+ @Test
+ public void testCommandLine() throws Exception
{
try {
- try {
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- } catch (Exception e) {
- }
+ FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
+ } catch (Exception e) {
+ }
+ try {
createInput();
boolean mayExit = false;
@@ -101,25 +104,12 @@
System.err.println(" out1=" + output);
System.err.println(" equals=" + outputExpect.compareTo(output));
assertEquals(outputExpect, output);
- } catch(Exception e) {
- failTrace(e);
} finally {
- try {
- INPUT_FILE.delete();
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- } catch (IOException e) {
- failTrace(e);
- }
+ INPUT_FILE.delete();
+ FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
}
}
- private void failTrace(Exception e)
- {
- StringWriter sw = new StringWriter();
- e.printStackTrace(new PrintWriter(sw));
- fail(sw.toString());
- }
-
public static void main(String[]args) throws Exception
{
new TestStreamDataProtocol().testCommandLine();
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamJob.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamJob.java
index cf89c5b..7ea92cb 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamJob.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamJob.java
@@ -25,13 +25,15 @@
import org.apache.hadoop.mapred.KeyValueTextInputFormat;
import org.apache.hadoop.mapred.SequenceFileInputFormat;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
/**
* This class tests hadoop Streaming's StreamJob class.
*/
-public class TestStreamJob extends TestCase {
+public class TestStreamJob {
+ @Test
public void testCreateJob() throws IOException {
JobConf job;
ArrayList<String> dummyArgs = new ArrayList<String>();
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamReduceNone.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamReduceNone.java
index e6d99c9..1ba0bbe 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamReduceNone.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamReduceNone.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
import java.io.*;
import java.util.*;
import org.apache.hadoop.conf.Configuration;
@@ -26,13 +25,16 @@
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
+import static org.junit.Assert.*;
+import org.junit.Test;
+
/**
* This class tests hadoopStreaming in MapReduce local mode.
* It tests the case where number of reducers is set to 0.
In this case, the mappers are expected to write out outputs directly.
No reducer/combiner will be activated.
*/
-public class TestStreamReduceNone extends TestCase
+public class TestStreamReduceNone
{
protected File INPUT_FILE = new File("stream_reduce_none_input.txt");
protected File OUTPUT_DIR = new File("stream_reduce_none_out");
@@ -69,8 +71,9 @@
"-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp")
};
}
-
- public void testCommandLine()
+
+ @Test
+ public void testCommandLine() throws Exception
{
String outFileName = "part-00000";
File outFile = null;
@@ -92,25 +95,12 @@
System.err.println("outEx1=" + outputExpect);
System.err.println(" out1=" + output);
assertEquals(outputExpect, output);
- } catch(Exception e) {
- failTrace(e);
} finally {
- try {
- INPUT_FILE.delete();
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- } catch (IOException e) {
- failTrace(e);
- }
+ INPUT_FILE.delete();
+ FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
}
}
- private void failTrace(Exception e)
- {
- StringWriter sw = new StringWriter();
- e.printStackTrace(new PrintWriter(sw));
- fail(sw.toString());
- }
-
public static void main(String[]args) throws Exception
{
new TestStreamReduceNone().testCommandLine();
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamXmlRecordReader.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamXmlRecordReader.java
index 70df7f0..7d29358 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamXmlRecordReader.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamXmlRecordReader.java
@@ -24,6 +24,9 @@
import org.apache.hadoop.fs.FileUtil;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
/**
* This class tests StreamXmlRecordReader
* The test creates an XML file, uses StreamXmlRecordReader and compares
@@ -60,7 +63,8 @@
};
}
- public void testCommandLine() {
+ @Test
+ public void testCommandLine() throws Exception {
try {
try {
FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
@@ -73,8 +77,6 @@
String output = StreamUtil.slurp(outFile);
outFile.delete();
assertEquals(input, output);
- } catch (Exception e) {
- e.printStackTrace();
} finally {
try {
INPUT_FILE.delete();
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java
index 205db7b..4c7bc7a 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java
@@ -28,7 +28,8 @@
import java.util.ArrayList;
import java.util.Arrays;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -51,7 +52,7 @@
* test-unix
* </pre>
*/
-public class TestStreamedMerge extends TestCase {
+public class TestStreamedMerge {
public TestStreamedMerge() throws IOException {
UtilTest utilTest = new UtilTest(getClass().getName());
@@ -104,24 +105,16 @@
return c;
}
- void lsr() {
- try {
- System.out.println("lsr /");
- ToolRunner.run(conf_, new FsShell(), new String[]{ "-lsr", "/" });
- } catch (Exception e) {
- e.printStackTrace();
- }
+ void lsr() throws Exception {
+ System.out.println("lsr /");
+ ToolRunner.run(conf_, new FsShell(), new String[]{ "-lsr", "/" });
}
- void printSampleInput() {
- try {
- System.out.println("cat /input/part-00");
- String content = StreamUtil.slurpHadoop(new Path("/input/part-00"), fs_);
- System.out.println(content);
- System.out.println("cat done.");
- } catch (Exception e) {
- e.printStackTrace();
- }
+ void printSampleInput() throws IOException {
+ System.out.println("cat /input/part-00");
+ String content = StreamUtil.slurpHadoop(new Path("/input/part-00"), fs_);
+ System.out.println(content);
+ System.out.println("cat done.");
}
void callStreaming(String argSideOutput, boolean inputTagged) throws IOException {
@@ -210,7 +203,8 @@
StringBuffer buf_;
}
- public void testMain() throws IOException {
+ @Test
+ public void testMain() throws Exception {
boolean success = false;
String base = new File(".").getAbsolutePath();
System.setProperty("hadoop.log.dir", base + "/logs");
@@ -228,8 +222,6 @@
}
doAllTestJobs();
success = true;
- } catch (IOException io) {
- io.printStackTrace();
} finally {
try {
fs_.close();
@@ -243,14 +235,14 @@
}
}
- void doAllTestJobs() throws IOException
+ void doAllTestJobs() throws Exception
{
goSocketTagged(true, false);
goSocketTagged(false, false);
goSocketTagged(true, true);
}
- void goSocketTagged(boolean socket, boolean inputTagged) throws IOException {
+ void goSocketTagged(boolean socket, boolean inputTagged) throws Exception {
System.out.println("***** goSocketTagged: " + socket + ", " + inputTagged);
String expect = createInputs(inputTagged);
lsr();
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java
index 8337919..0e6dfb9 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java
@@ -18,21 +18,28 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
import java.io.*;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.conf.Configuration;
+
/**
* This class tests hadoopStreaming in MapReduce local mode.
*/
-public class TestStreaming extends TestCase
+public class TestStreaming
{
// "map" command: grep -E (red|green|blue)
// reduce command: uniq
- protected File INPUT_FILE = new File("input.txt");
- protected File OUTPUT_DIR = new File("out");
+ protected File TEST_DIR;
+ protected File INPUT_FILE;
+ protected File OUTPUT_DIR;
protected String input = "roses.are.red\nviolets.are.blue\nbunnies.are.pink\n";
// map behaves like "/usr/bin/tr . \\n"; (split words into lines)
protected String map = StreamUtil.makeJavaCommand(TrApp.class, new String[]{".", "\\n"});
@@ -48,13 +55,20 @@
UtilTest utilTest = new UtilTest(getClass().getName());
utilTest.checkUserDir();
utilTest.redirectIfAntJunit();
+ TEST_DIR = new File(getClass().getName()).getAbsoluteFile();
+ OUTPUT_DIR = new File(TEST_DIR, "out");
+ INPUT_FILE = new File(TEST_DIR, "input.txt");
+ }
+
+ protected String getInputData() {
+ return input;
}
protected void createInput() throws IOException
{
- DataOutputStream out = new DataOutputStream(
- new FileOutputStream(INPUT_FILE.getAbsoluteFile()));
- out.write(input.getBytes("UTF-8"));
+ DataOutputStream out = getFileSystem().create(
+ new Path(INPUT_FILE.getAbsolutePath()));
+ out.write(getInputData().getBytes("UTF-8"));
out.close();
}
@@ -70,36 +84,43 @@
"-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp")
};
}
-
- public void testCommandLine() throws IOException
+
+ protected Configuration getConf() {
+ return new Configuration();
+ }
+
+ protected FileSystem getFileSystem() throws IOException {
+ return FileSystem.get(getConf());
+ }
+
+ protected String getExpectedOutput() {
+ return outputExpect;
+ }
+
+ protected void checkOutput() throws IOException {
+ Path outPath = new Path(OUTPUT_DIR.getAbsolutePath(), "part-00000");
+ FileSystem fs = getFileSystem();
+ String output = StreamUtil.slurpHadoop(outPath, fs);
+ fs.delete(outPath, true);
+ System.err.println("outEx1=" + getExpectedOutput());
+ System.err.println(" out1=" + output);
+ assertEquals(getExpectedOutput(), output);
+ }
+
+ @Test
+ public void testCommandLine() throws Exception
{
- try {
- try {
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- } catch (Exception e) {
- }
+ UtilTest.recursiveDelete(TEST_DIR);
+ assertTrue("Creating " + TEST_DIR, TEST_DIR.mkdirs());
+ createInput();
+ boolean mayExit = false;
- createInput();
- boolean mayExit = false;
-
- // During tests, the default Configuration will use a local mapred
- // So don't specify -config or -cluster
- job = new StreamJob(genArgs(), mayExit);
- job.go();
- File outFile = new File(OUTPUT_DIR, "part-00000").getAbsoluteFile();
- String output = StreamUtil.slurp(outFile);
- outFile.delete();
- System.err.println("outEx1=" + outputExpect);
- System.err.println(" out1=" + output);
- assertEquals(outputExpect, output);
- } finally {
- try {
- INPUT_FILE.delete();
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
+ // During tests, the default Configuration will use a local mapred
+ // So don't specify -config or -cluster
+ job = new StreamJob(genArgs(), mayExit);
+ int ret = job.go();
+ assertEquals(0, ret);
+ checkOutput();
}
public static void main(String[]args) throws Exception
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingCombiner.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingCombiner.java
index 80fd1e8..60d728a 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingCombiner.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingCombiner.java
@@ -22,6 +22,9 @@
import org.apache.hadoop.mapred.Counters;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
public class TestStreamingCombiner extends TestStreaming {
protected String combine = StreamUtil.makeJavaCommand(UniqApp.class, new String[]{""});
@@ -41,7 +44,8 @@
};
}
- public void testCommandLine() throws IOException {
+ @Test
+ public void testCommandLine() throws Exception {
super.testCommandLine();
// validate combiner counters
String counterGrp = "org.apache.hadoop.mapred.Task$Counter";
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingCounters.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingCounters.java
index 9468cff..ca864ba 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingCounters.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingCounters.java
@@ -18,6 +18,9 @@
package org.apache.hadoop.streaming;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
import java.io.File;
import java.io.IOException;
@@ -30,11 +33,11 @@
* This class tests streaming counters in MapReduce local mode.
*/
public class TestStreamingCounters extends TestStreaming {
-
public TestStreamingCounters() throws IOException {
super();
}
+ @Test
public void testCommandLine() throws IOException
{
try {
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingEmptyInpNonemptyOut.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingEmptyInpNonemptyOut.java
index a393f58..0678a87 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingEmptyInpNonemptyOut.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingEmptyInpNonemptyOut.java
@@ -18,7 +18,8 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
+import org.junit.Test;
+
import java.io.*;
import org.apache.hadoop.fs.FileUtil;
@@ -30,7 +31,7 @@
* was hanging forever. Now this issue is solved. Similarly reducer is also
* checked for task completion with empty input and nonempty output.
*/
-public class TestStreamingEmptyInpNonemptyOut extends TestCase
+public class TestStreamingEmptyInpNonemptyOut
{
protected File INPUT_FILE = new File("emptyInputFile.txt");
@@ -74,7 +75,8 @@
"-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp")
};
}
-
+
+ @Test
public void testEmptyInputNonemptyOutput() throws IOException
{
try {
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingExitStatus.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingExitStatus.java
index 8ef4bcf..630df26 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingExitStatus.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingExitStatus.java
@@ -18,7 +18,10 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
+import org.junit.Test;
+import org.junit.Before;
+import static org.junit.Assert.*;
+
import java.io.*;
import java.util.*;
import org.apache.hadoop.conf.Configuration;
@@ -30,10 +33,12 @@
* reducers have non-zero exit status and the
* stream.non.zero.exit.status.is.failure jobconf is set.
*/
-public class TestStreamingExitStatus extends TestCase
+public class TestStreamingExitStatus
{
- protected File INPUT_FILE = new File("input.txt");
- protected File OUTPUT_DIR = new File("out");
+ protected File TEST_DIR =
+ new File("TestStreamingExitStatus").getAbsoluteFile();
+ protected File INPUT_FILE = new File(TEST_DIR, "input.txt");
+ protected File OUTPUT_DIR = new File(TEST_DIR, "out");
protected String failingTask = StreamUtil.makeJavaCommand(FailApp.class, new String[]{"true"});
protected String echoTask = StreamUtil.makeJavaCommand(FailApp.class, new String[]{"false"});
@@ -57,52 +62,48 @@
};
}
+ @Before
public void setUp() throws IOException {
- UtilTest.recursiveDelete(INPUT_FILE);
- UtilTest.recursiveDelete(OUTPUT_DIR);
-
+ UtilTest.recursiveDelete(TEST_DIR);
+ assertTrue(TEST_DIR.mkdirs());
+
FileOutputStream out = new FileOutputStream(INPUT_FILE.getAbsoluteFile());
out.write("hello\n".getBytes());
out.close();
}
- public void runStreamJob(boolean exitStatusIsFailure, boolean failMap) {
- try {
- boolean mayExit = false;
- int returnStatus = 0;
+ public void runStreamJob(boolean exitStatusIsFailure, boolean failMap) throws Exception {
+ boolean mayExit = false;
+ int returnStatus = 0;
- StreamJob job = new StreamJob(genArgs(exitStatusIsFailure, failMap), mayExit);
- returnStatus = job.go();
-
- if (exitStatusIsFailure) {
- assertEquals("Streaming Job failure code expected", /*job not successful:*/1, returnStatus);
- } else {
- assertEquals("Streaming Job expected to succeed", 0, returnStatus);
- }
- } catch (Exception e) {
- failTrace(e);
+ StreamJob job = new StreamJob(genArgs(exitStatusIsFailure, failMap), mayExit);
+ returnStatus = job.go();
+
+ if (exitStatusIsFailure) {
+ assertEquals("Streaming Job failure code expected", /*job not successful:*/1, returnStatus);
+ } else {
+ assertEquals("Streaming Job expected to succeed", 0, returnStatus);
}
}
-
- public void testMapFailOk() {
+
+ @Test
+ public void testMapFailOk() throws Exception {
runStreamJob(false, true);
}
-
- public void testMapFailNotOk() {
+
+ @Test
+ public void testMapFailNotOk() throws Exception {
runStreamJob(true, true);
}
-
- public void testReduceFailOk() {
+
+ @Test
+ public void testReduceFailOk() throws Exception {
runStreamJob(false, false);
}
- public void testReduceFailNotOk() {
+ @Test
+ public void testReduceFailNotOk() throws Exception {
runStreamJob(true, false);
}
- protected void failTrace(Exception e) {
- StringWriter sw = new StringWriter();
- e.printStackTrace(new PrintWriter(sw));
- fail(sw.toString());
- }
}
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java
index 35de83d..2c061c7 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java
@@ -18,7 +18,9 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
import java.io.*;
import java.util.*;
import org.apache.hadoop.conf.Configuration;
@@ -56,6 +58,7 @@
};
}
+ @Test
public void testCommandLine()
{
try {
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java
index 6d20752..3ce9930 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java
@@ -18,16 +18,19 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
import java.io.*;
import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.mapreduce.JobContext;
/**
* This class tests hadoopStreaming in MapReduce local mode.
* This testcase looks at different cases of tab position in input.
*/
-public class TestStreamingKeyValue extends TestCase
+public class TestStreamingKeyValue
{
protected File INPUT_FILE = new File("input.txt");
protected File OUTPUT_DIR = new File("stream_out");
@@ -68,13 +71,14 @@
"-input", INPUT_FILE.getAbsolutePath(),
"-output", OUTPUT_DIR.getAbsolutePath(),
"-mapper", "cat",
- "-jobconf", "mapreduce.task.files.preserve.failedtasks=true",
+ "-jobconf", JobContext.PRESERVE_FAILED_TASK_FILES + "=true",
"-jobconf", "stream.non.zero.exit.is.failure=true",
"-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp")
};
}
- public void testCommandLine()
+ @Test
+ public void testCommandLine() throws Exception
{
String outFileName = "part-00000";
File outFile = null;
@@ -96,25 +100,12 @@
System.err.println("outEx1=" + outputExpect);
System.err.println(" out1=" + output);
assertEquals(outputExpect, output);
- } catch(Exception e) {
- failTrace(e);
} finally {
- try {
- INPUT_FILE.delete();
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- } catch (IOException e) {
- failTrace(e);
- }
+ INPUT_FILE.delete();
+ FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
}
}
- private void failTrace(Exception e)
- {
- StringWriter sw = new StringWriter();
- e.printStackTrace(new PrintWriter(sw));
- fail(sw.toString());
- }
-
public static void main(String[]args) throws Exception
{
new TestStreamingKeyValue().testCommandLine();
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingSeparator.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingSeparator.java
index 7944bd5..825445f 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingSeparator.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingSeparator.java
@@ -18,7 +18,9 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
import java.io.*;
import java.util.*;
import org.apache.hadoop.conf.Configuration;
@@ -29,7 +31,7 @@
/**
* This class tests hadoopStreaming with customized separator in MapReduce local mode.
*/
-public class TestStreamingSeparator extends TestCase
+public class TestStreamingSeparator
{
// "map" command: grep -E (red|green|blue)
@@ -86,7 +88,8 @@
};
}
- public void testCommandLine()
+ @Test
+ public void testCommandLine() throws Exception
{
try {
try {
@@ -107,25 +110,12 @@
System.err.println("outEx1=" + outputExpect);
System.err.println(" out1=" + output);
assertEquals(outputExpect, output);
- } catch(Exception e) {
- failTrace(e);
} finally {
- try {
- INPUT_FILE.delete();
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- } catch (IOException e) {
- failTrace(e);
- }
+ INPUT_FILE.delete();
+ FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
}
}
- private void failTrace(Exception e)
- {
- StringWriter sw = new StringWriter();
- e.printStackTrace(new PrintWriter(sw));
- fail(sw.toString());
- }
-
public static void main(String[]args) throws Exception
{
new TestStreamingSeparator().testCommandLine();
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStatus.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStatus.java
index 67cb288..5ccb4ce 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStatus.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStatus.java
@@ -22,7 +22,8 @@
import java.io.IOException;
import java.io.File;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -32,12 +33,13 @@
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+
/**
* Tests for the ability of a streaming task to set the status
* by writing "reporter:status:" lines to stderr. Uses MiniMR
* since the local jobtracker doesn't track status.
*/
-public class TestStreamingStatus extends TestCase {
+public class TestStreamingStatus {
private static String TEST_ROOT_DIR =
new File(System.getProperty("test.build.data","/tmp"))
.toURI().toString().replace(' ', '+');
@@ -78,6 +80,7 @@
} catch (Exception e) {}
}
+ @Test
public void testStreamingStatus() throws Exception {
MiniMRCluster mr = null;
FileSystem fs = null;
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStderr.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStderr.java
index 29be956..3ee2b39 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStderr.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStderr.java
@@ -18,19 +18,21 @@
package org.apache.hadoop.streaming;
-import junit.framework.TestCase;
import java.io.*;
import java.util.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
/**
* Test that streaming consumes stderr from the streaming process
* (before, during, and after the main processing of mapred input),
* and that stderr messages count as task progress.
*/
-public class TestStreamingStderr extends TestCase
+public class TestStreamingStderr
{
public TestStreamingStderr() throws IOException {
UtilTest utilTest = new UtilTest(getClass().getName());
@@ -71,43 +73,38 @@
}
public void runStreamJob(String baseName, boolean hasInput,
- int preLines, int duringLines, int postLines) {
- try {
- File input = setupInput(baseName, hasInput);
- File output = setupOutput(baseName);
- boolean mayExit = false;
- int returnStatus = 0;
+ int preLines, int duringLines, int postLines)
+ throws Exception {
+ File input = setupInput(baseName, hasInput);
+ File output = setupOutput(baseName);
+ boolean mayExit = false;
+ int returnStatus = 0;
- StreamJob job = new StreamJob(genArgs(input, output, preLines, duringLines, postLines), mayExit);
- returnStatus = job.go();
- assertEquals("StreamJob success", 0, returnStatus);
- } catch (Exception e) {
- failTrace(e);
- }
+ StreamJob job = new StreamJob(genArgs(input, output, preLines, duringLines, postLines), mayExit);
+ returnStatus = job.go();
+ assertEquals("StreamJob success", 0, returnStatus);
}
// This test will fail by blocking forever if the stderr isn't
// consumed by Hadoop for tasks that don't have any input.
- public void testStderrNoInput() throws IOException {
+ @Test
+ public void testStderrNoInput() throws Exception {
runStreamJob("stderr-pre", false, 10000, 0, 0);
}
// Streaming should continue to read stderr even after all input has
// been consumed.
- public void testStderrAfterOutput() throws IOException {
+ @Test
+ public void testStderrAfterOutput() throws Exception {
runStreamJob("stderr-post", false, 0, 0, 10000);
}
// This test should produce a task timeout if stderr lines aren't
// counted as progress. This won't actually work until
// LocalJobRunner supports timeouts.
- public void testStderrCountsAsProgress() throws IOException {
+ @Test
+ public void testStderrCountsAsProgress() throws Exception {
runStreamJob("stderr-progress", true, 10, 1000, 0);
}
- protected void failTrace(Exception e) {
- StringWriter sw = new StringWriter();
- e.printStackTrace(new PrintWriter(sw));
- fail(sw.toString());
- }
}
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingTaskLog.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingTaskLog.java
index c76fd52..cf22cf3 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingTaskLog.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingTaskLog.java
@@ -28,13 +28,15 @@
import org.apache.hadoop.mapred.TestMiniMRWithDFS;
import org.apache.hadoop.util.Shell;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
/**
* This tests the environment set by TT for the child of task jvm.
* This will launch a streaming job with a shell script as mapper.
*/
-public class TestStreamingTaskLog extends TestCase {
+public class TestStreamingTaskLog {
String input = "the dummy input";
Path inputPath = new Path("inDir");
Path outputPath = new Path("outDir");
@@ -65,6 +67,7 @@
* (b) hadoop.tasklog.totalLogFileSize
* for the children of java tasks in streaming jobs.
*/
+ @Test
public void testStreamingTaskLogWithHadoopCmd() {
try {
final int numSlaves = 1;
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java
index 578817f..9fc2634 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java
@@ -25,7 +25,8 @@
import java.io.PrintWriter;
import java.io.StringWriter;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -40,7 +41,7 @@
* This test case tests the symlink creation
* utility provided by distributed caching
*/
-public class TestSymLink extends TestCase
+public class TestSymLink
{
String INPUT_FILE = "/testing-streaming/input.txt";
String OUTPUT_DIR = "/testing-streaming/out";
@@ -52,92 +53,78 @@
String cacheString = "This is just the cache string";
StreamJob job;
- public TestSymLink() throws IOException
+ @Test
+ public void testSymLink() throws Exception
{
- }
-
- public void testSymLink()
- {
+ boolean mayExit = false;
+ MiniMRCluster mr = null;
+ MiniDFSCluster dfs = null;
try {
- boolean mayExit = false;
- MiniMRCluster mr = null;
- MiniDFSCluster dfs = null;
- try{
- Configuration conf = new Configuration();
- dfs = new MiniDFSCluster(conf, 1, true, null);
- FileSystem fileSys = dfs.getFileSystem();
- String namenode = fileSys.getUri().toString();
- mr = new MiniMRCluster(1, namenode, 3);
- // During tests, the default Configuration will use a local mapred
- // So don't specify -config or -cluster
- String strJobtracker = JTConfig.JT_IPC_ADDRESS + "=localhost:" + mr.getJobTrackerPort();
- String strNamenode = "fs.default.name=" + namenode;
- String argv[] = new String[] {
- "-input", INPUT_FILE,
- "-output", OUTPUT_DIR,
- "-mapper", map,
- "-reducer", reduce,
- //"-verbose",
- //"-jobconf", "stream.debug=set"
- "-jobconf", strNamenode,
- "-jobconf", strJobtracker,
- "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp"),
- "-jobconf",
- JobConf.MAPRED_MAP_TASK_JAVA_OPTS+ "=" +
- "-Dcontrib.name=" + System.getProperty("contrib.name") + " " +
- "-Dbuild.test=" + System.getProperty("build.test") + " " +
- conf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS,
- conf.get(JobConf.MAPRED_TASK_JAVA_OPTS, "")),
- "-jobconf",
- JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS+ "=" +
- "-Dcontrib.name=" + System.getProperty("contrib.name") + " " +
- "-Dbuild.test=" + System.getProperty("build.test") + " " +
- conf.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS,
- conf.get(JobConf.MAPRED_TASK_JAVA_OPTS, "")),
- "-cacheFile", fileSys.getUri() + CACHE_FILE + "#testlink"
- };
+ Configuration conf = new Configuration();
+ dfs = new MiniDFSCluster(conf, 1, true, null);
+ FileSystem fileSys = dfs.getFileSystem();
+ String namenode = fileSys.getUri().toString();
+ mr = new MiniMRCluster(1, namenode, 3);
+ // During tests, the default Configuration will use a local mapred
+ // So don't specify -config or -cluster
+ String strJobtracker = JTConfig.JT_IPC_ADDRESS + "=localhost:" + mr.getJobTrackerPort();
+ String strNamenode = "fs.default.name=" + namenode;
+ String argv[] = new String[] {
+ "-input", INPUT_FILE,
+ "-output", OUTPUT_DIR,
+ "-mapper", map,
+ "-reducer", reduce,
+ //"-verbose",
+ //"-jobconf", "stream.debug=set"
+ "-jobconf", strNamenode,
+ "-jobconf", strJobtracker,
+ "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp"),
+ "-jobconf",
+ JobConf.MAPRED_MAP_TASK_JAVA_OPTS+ "=" +
+ "-Dcontrib.name=" + System.getProperty("contrib.name") + " " +
+ "-Dbuild.test=" + System.getProperty("build.test") + " " +
+ conf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS,
+ conf.get(JobConf.MAPRED_TASK_JAVA_OPTS, "")),
+ "-jobconf",
+ JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS+ "=" +
+ "-Dcontrib.name=" + System.getProperty("contrib.name") + " " +
+ "-Dbuild.test=" + System.getProperty("build.test") + " " +
+ conf.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS,
+ conf.get(JobConf.MAPRED_TASK_JAVA_OPTS, "")),
+ "-cacheFile", fileSys.getUri() + CACHE_FILE + "#testlink"
+ };
- fileSys.delete(new Path(OUTPUT_DIR), true);
-
- DataOutputStream file = fileSys.create(new Path(INPUT_FILE));
- file.writeBytes(mapString);
- file.close();
- file = fileSys.create(new Path(CACHE_FILE));
- file.writeBytes(cacheString);
- file.close();
-
- job = new StreamJob(argv, mayExit);
- job.go();
-
- fileSys = dfs.getFileSystem();
- String line = null;
- Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
- new Path(OUTPUT_DIR),
- new Utils.OutputFileUtils
- .OutputFilesFilter()));
- for (int i = 0; i < fileList.length; i++){
- System.out.println(fileList[i].toString());
- BufferedReader bread =
- new BufferedReader(new InputStreamReader(fileSys.open(fileList[i])));
- line = bread.readLine();
- System.out.println(line);
- }
- assertEquals(cacheString + "\t", line);
- } finally{
- if (dfs != null) { dfs.shutdown(); }
- if (mr != null) { mr.shutdown();}
- }
+ fileSys.delete(new Path(OUTPUT_DIR), true);
- } catch(Exception e) {
- failTrace(e);
- }
- }
+ DataOutputStream file = fileSys.create(new Path(INPUT_FILE));
+ file.writeBytes(mapString);
+ file.close();
+ file = fileSys.create(new Path(CACHE_FILE));
+ file.writeBytes(cacheString);
+ file.close();
+
+ job = new StreamJob(argv, mayExit);
+ job.go();
- void failTrace(Exception e)
- {
- StringWriter sw = new StringWriter();
- e.printStackTrace(new PrintWriter(sw));
- fail(sw.toString());
+ fileSys = dfs.getFileSystem();
+ String line = null;
+ Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
+ new Path(OUTPUT_DIR),
+ new Utils.OutputFileUtils
+ .OutputFilesFilter()));
+ for (int i = 0; i < fileList.length; i++){
+ System.out.println(fileList[i].toString());
+ BufferedReader bread =
+ new BufferedReader(new InputStreamReader(fileSys.open(fileList[i])));
+ line = bread.readLine();
+ System.out.println(line);
+ }
+ assertEquals(cacheString + "\t", line);
+ } finally{
+ if (dfs != null) { dfs.shutdown(); }
+ if (mr != null) { mr.shutdown();}
+ }
+
}
public static void main(String[]args) throws Exception
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestTypedBytesStreaming.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestTypedBytesStreaming.java
index a0c03c4..6e91687 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestTypedBytesStreaming.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestTypedBytesStreaming.java
@@ -26,9 +26,12 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
-import junit.framework.TestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
-public class TestTypedBytesStreaming extends TestCase {
+public class TestTypedBytesStreaming {
protected File INPUT_FILE = new File("input.txt");
protected File OUTPUT_DIR = new File("out");
@@ -60,33 +63,29 @@
"-io", "typedbytes"
};
}
-
+
+ @Before
+ @After
+ public void cleanupOutput() throws Exception {
+ FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
+ INPUT_FILE.delete();
+ createInput();
+ }
+
+ @Test
public void testCommandLine() throws Exception {
- try {
- try {
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- } catch (Exception e) {
- }
-
- createInput();
- OUTPUT_DIR.delete();
-
- // During tests, the default Configuration will use a local mapred
- // So don't specify -config or -cluster
- StreamJob job = new StreamJob();
- job.setConf(new Configuration());
- job.run(genArgs());
- File outFile = new File(OUTPUT_DIR, "part-00000").getAbsoluteFile();
- String output = StreamUtil.slurp(outFile);
- outFile.delete();
- System.out.println(" map=" + map);
- System.out.println("reduce=" + reduce);
- System.err.println("outEx1=" + outputExpect);
- System.err.println(" out1=" + output);
- assertEquals(outputExpect, output);
- } finally {
- INPUT_FILE.delete();
- FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
- }
+ // During tests, the default Configuration will use a local mapred
+ // So don't specify -config or -cluster
+ StreamJob job = new StreamJob();
+ job.setConf(new Configuration());
+ job.run(genArgs());
+ File outFile = new File(OUTPUT_DIR, "part-00000").getAbsoluteFile();
+ String output = StreamUtil.slurp(outFile);
+ outFile.delete();
+ System.out.println(" map=" + map);
+ System.out.println("reduce=" + reduce);
+ System.err.println("outEx1=" + outputExpect);
+ System.err.println(" out1=" + output);
+ assertEquals(outputExpect, output);
}
}
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestUlimit.java b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestUlimit.java
index dd07190..df2bbed 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestUlimit.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestUlimit.java
@@ -31,7 +31,8 @@
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.util.StringUtils;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
/**
* This tests the setting of memory limit for streaming processes.
@@ -40,7 +41,7 @@
* it to succeed. Then program is launched with insufficient memory and
* is expected to be a failure.
*/
-public class TestUlimit extends TestCase {
+public class TestUlimit {
String input = "the dummy input";
Path inputPath = new Path("/testing/in");
Path outputPath = new Path("/testing/out");
@@ -75,6 +76,7 @@
* it to succeed. Then program is launched with insufficient memory and
* is expected to be a failure.
*/
+ @Test
public void testCommandLine() {
if (StreamUtil.isCygwin()) {
return;
diff --git a/src/contrib/streaming/src/test/org/apache/hadoop/typedbytes/TestIO.java b/src/contrib/streaming/src/test/org/apache/hadoop/typedbytes/TestIO.java
index c376d77..a7ca587 100644
--- a/src/contrib/streaming/src/test/org/apache/hadoop/typedbytes/TestIO.java
+++ b/src/contrib/streaming/src/test/org/apache/hadoop/typedbytes/TestIO.java
@@ -49,21 +49,34 @@
import org.apache.hadoop.record.RecRecord0;
import org.apache.hadoop.record.RecRecord1;
-import junit.framework.TestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
-public class TestIO extends TestCase {
+public class TestIO {
private File tmpfile;
+ private File tmpdir;
- protected void setUp() throws Exception {
- this.tmpfile = new File(System.getProperty("test.build.data", "/tmp"),
- "typedbytes.bin");
+ @Before
+ public void setUp() throws Exception {
+ this.tmpdir = new File(System.getProperty("test.build.data", "/tmp"));
+ if(this.tmpdir.exists() || this.tmpdir.mkdirs()) {
+ this.tmpfile = new File(this.tmpdir,
+ "typedbytes.bin");
+ } else {
+ throw new IOException("Failed to create directory " + tmpdir.getAbsolutePath());
+ }
}
- protected void tearDown() throws Exception {
+ @After
+ public void tearDown() throws Exception {
tmpfile.delete();
}
+ @Test
public void testIO() throws IOException {
ArrayList<Object> vector = new ArrayList<Object>();
vector.add("test");
@@ -121,6 +134,7 @@
istream.close();
}
+ @Test
public void testCustomTypesIO() throws IOException {
byte[] rawBytes = new byte[] { 100, 0, 0, 0, 3, 1, 2, 3 };
@@ -139,6 +153,7 @@
istream.close();
}
+ @Test
public void testRecordIO() throws IOException {
RecRecord1 r1 = new RecRecord1();
r1.setBoolVal(true);
@@ -172,6 +187,7 @@
assertEquals(r1, r2);
}
+ @Test
public void testWritableIO() throws IOException {
Writable[] vectorValues = new Writable[] {
new Text("test1"), new Text("test2"), new Text("test3")
diff --git a/src/contrib/vaidya/build.xml b/src/contrib/vaidya/build.xml
index f150e05..d5ab229 100644
--- a/src/contrib/vaidya/build.xml
+++ b/src/contrib/vaidya/build.xml
@@ -20,7 +20,6 @@
<project name="vaidya" default="jar">
<import file="../build-contrib.xml" />
- <import file="../../../build.xml" />
<target name="init">
<mkdir dir="${build.dir}" />
diff --git a/src/contrib/vaidya/ivy.xml b/src/contrib/vaidya/ivy.xml
index 5bc73fe..303fa8c 100644
--- a/src/contrib/vaidya/ivy.xml
+++ b/src/contrib/vaidya/ivy.xml
@@ -1,4 +1,21 @@
<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<ivy-module version="1.0">
<info organisation="org.apache.hadoop" module="${ant.project.name}">
<license name="Apache 2.0"/>
diff --git a/src/contrib/vaidya/ivy/libraries.properties b/src/contrib/vaidya/ivy/libraries.properties
index a470b37..8a80dd8 100644
--- a/src/contrib/vaidya/ivy/libraries.properties
+++ b/src/contrib/vaidya/ivy/libraries.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
#This properties file lists the versions of the various artifacts used by streaming.
#It drives ivy and the generation of a maven POM
diff --git a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaConfiguration.java b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaConfiguration.java
index baa4002..df9c1dd 100644
--- a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaConfiguration.java
+++ b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaConfiguration.java
@@ -55,6 +55,9 @@
* @see VerticaOutputFormat#setOutput(Job, String, boolean, String...)
*/
public class VerticaConfiguration {
+ /** Vertica Version Constants */
+ public static final Integer VERSION_3_5 = 305;
+
/** Class name for Vertica JDBC Driver */
public static final String VERTICA_DRIVER_CLASS = "com.vertica.Driver";
diff --git a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaOutputFormat.java b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaOutputFormat.java
index cefe1ed..feda95e 100644
--- a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaOutputFormat.java
+++ b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaOutputFormat.java
@@ -63,7 +63,7 @@
* @param dropTable
*/
public static void setOutput(Job job, String tableName, boolean dropTable) {
- setOutput(job, tableName, dropTable);
+ setOutput(job, tableName, dropTable, (String[])null);
}
/**
@@ -193,39 +193,46 @@
stmt.execute("select create_projection_design('" + designName + "', '', '"
+ designTables.toString() + "')");
- rs = stmt.executeQuery("select get_design_script('" + designName + "', '"
- + designName + "')");
- rs.next();
- String[] projSet = rs.getString(1).split(";");
- for (String proj : projSet) {
- stmt.execute(proj);
- }
- stmt.execute("select start_refresh()");
-
- // pool for refresh complete
- boolean refreshing = true;
- Long timeout = vtconfig.getOptimizePollTimeout();
- while (refreshing) {
- refreshing = false;
- rs = stmt
- .executeQuery("select table_name, projection_name, status from vt_projection_refresh");
- while (rs.next()) {
- String table = rs.getString(1);
- String stat = rs.getString(3);
- if (stat.equals("refreshing") && tablesWithTemp.contains(table))
- refreshing = true;
+ if(VerticaUtil.verticaVersion(conf, true) >= VerticaConfiguration.VERSION_3_5) {
+ stmt.execute("select deploy_design('" + designName + "', '" + designName + "')");
+ } else {
+ rs = stmt.executeQuery("select get_design_script('" + designName + "', '"
+ + designName + "')");
+ rs.next();
+ String[] projSet = rs.getString(1).split(";");
+ for (String proj : projSet) {
+ stmt.execute(proj);
+ }
+ stmt.execute("select start_refresh()");
+
+ // poll for refresh complete
+ boolean refreshing = true;
+ Long timeout = vtconfig.getOptimizePollTimeout();
+ while (refreshing) {
+ refreshing = false;
+ rs = stmt
+ .executeQuery("select table_name, status from vt_projection_refresh");
+ while (rs.next()) {
+ String table = rs.getString(1);
+ String stat = rs.getString(2);
+ if (stat.equals("refreshing") && tablesWithTemp.contains(table))
+ refreshing = true;
+ }
+ rs.close();
+
+ Thread.sleep(timeout);
+ }
+
+ // refresh done, move the ancient history mark (ahm) and drop the temp projections
+ stmt.execute("select make_ahm_now()");
+
+ for (String table : tablesWithTemp) {
+ for (String proj : tableProj.get(table)) {
+ stmt.execute("DROP PROJECTION " + proj);
+ }
}
- Thread.sleep(timeout);
- }
-
- // refresh done, move the ahm and drop the temp projections
- stmt.execute("select make_ahm_now()");
-
- for (String table : tablesWithTemp) {
- for (String proj : tableProj.get(table)) {
- stmt.execute("DROP PROJECTION " + proj);
- }
+ stmt.close();
}
}
@@ -235,4 +242,4 @@
return new FileOutputCommitter(FileOutputFormat.getOutputPath(context),
context);
}
-}
+}
\ No newline at end of file
diff --git a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaRecord.java b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaRecord.java
index 3a705f9..8f04c38 100644
--- a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaRecord.java
+++ b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaRecord.java
@@ -21,6 +21,7 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.sql.Date;
import java.sql.ResultSet;
@@ -178,7 +179,7 @@
if (i >= values.size())
throw new IndexOutOfBoundsException("Index " + i
+ " greater than input size " + values.size());
- if (validate) {
+ if (validate && value != null) {
Integer type = types.get(i);
switch (type) {
case Types.BIGINT:
@@ -204,6 +205,9 @@
case Types.REAL:
case Types.DECIMAL:
case Types.NUMERIC:
+ if (!(value instanceof BigDecimal))
+ throw new ClassCastException("Cannot cast "
+ + value.getClass().getName() + " to BigDecimal");
case Types.DOUBLE:
if (!(value instanceof Double) && !(value instanceof Float)
&& !(value instanceof DoubleWritable)
@@ -278,7 +282,7 @@
private void objectTypes() {
for (Object obj : values) {
if (obj == null) {
- this.types.add(null);
+ this.types.add(Types.NULL);
} else if (obj instanceof Long) {
this.types.add(Types.BIGINT);
} else if (obj instanceof LongWritable) {
@@ -291,6 +295,8 @@
this.types.add(Types.INTEGER);
} else if (obj instanceof Short) {
this.types.add(Types.SMALLINT);
+ } else if (obj instanceof BigDecimal) {
+ this.types.add(Types.NUMERIC);
} else if (obj instanceof DoubleWritable) {
this.types.add(Types.DOUBLE);
} else if (obj instanceof Double) {
@@ -343,20 +349,23 @@
// switch statement uses fall through to handle type variations
// e.g. type specified as BIGINT but passed in as Integer
switch (type) {
+ case Types.NULL:
+ sb.append("");
+ break;
case Types.BIGINT:
if (obj instanceof Long) {
- sb.append(((Long) obj).toString());
+ sb.append(obj.toString());
break;
}
case Types.INTEGER:
if (obj instanceof Integer) {
- sb.append(((Integer) obj).toString());
+ sb.append(obj.toString());
break;
}
case Types.TINYINT:
case Types.SMALLINT:
if (obj instanceof Short) {
- sb.append(((Short) obj).toString());
+ sb.append(obj.toString());
break;
}
if (obj instanceof LongWritable) {
@@ -374,18 +383,22 @@
case Types.REAL:
case Types.DECIMAL:
case Types.NUMERIC:
+ if (obj instanceof BigDecimal) {
+ sb.append(obj.toString());
+ break;
+ }
case Types.DOUBLE:
if (obj instanceof Double) {
- sb.append(((Double) obj).toString());
+ sb.append(obj.toString());
break;
}
if (obj instanceof DoubleWritable) {
- sb.append(((DoubleWritable) obj).toString());
+ sb.append(((DoubleWritable) obj).get());
break;
}
case Types.FLOAT:
if (obj instanceof Float) {
- sb.append(((Float) obj).toString());
+ sb.append(obj.toString());
break;
}
if (obj instanceof FloatWritable) {
@@ -395,7 +408,8 @@
case Types.BINARY:
case Types.LONGVARBINARY:
case Types.VARBINARY:
- sb.append(ByteBuffer.wrap((byte[]) obj).asCharBuffer());
+ if(obj == null) sb.append("");
+ else sb.append(ByteBuffer.wrap((byte[]) obj).asCharBuffer());
break;
case Types.BIT:
case Types.BOOLEAN:
@@ -452,7 +466,8 @@
sb.append(sqlfmt.format((Timestamp) obj));
break;
default:
- throw new RuntimeException("Unknown type value " + types.get(i));
+ if(obj == null) sb.append("");
+ else throw new RuntimeException("Unknown type value " + types.get(i));
}
if (i < columns - 1)
sb.append(delimiterArg);
@@ -473,6 +488,9 @@
for (int i = 0; i < columns; i++) {
int type = types.get(i);
switch (type) {
+ case Types.NULL:
+ values.add(null);
+ break;
case Types.BIGINT:
values.add(in.readLong());
break;
@@ -486,6 +504,8 @@
case Types.REAL:
case Types.DECIMAL:
case Types.NUMERIC:
+ values.add(new BigDecimal(Text.readString(in)));
+ break;
case Types.DOUBLE:
values.add(in.readDouble());
break;
@@ -551,13 +571,20 @@
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(columns);
- for (Integer type : types)
- out.writeInt(type);
+
+ for (int i = 0; i < columns; i++) {
+ Object obj = values.get(i);
+ Integer type = types.get(i);
+ if(obj == null) out.writeInt(Types.NULL);
+ else out.writeInt(type);
+ }
for (int i = 0; i < columns; i++) {
Object obj = values.get(i);
Integer type = types.get(i);
+ if(obj == null) continue;
+
switch (type) {
case Types.BIGINT:
out.writeLong((Long) obj);
@@ -572,6 +599,8 @@
case Types.REAL:
case Types.DECIMAL:
case Types.NUMERIC:
+ Text.writeString(out, obj.toString());
+ break;
case Types.DOUBLE:
out.writeDouble((Double) obj);
break;
@@ -628,4 +657,4 @@
}
}
-}
+}
\ No newline at end of file
diff --git a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaRecordWriter.java b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaRecordWriter.java
index 9cd0194..1cc3d7f 100644
--- a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaRecordWriter.java
+++ b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaRecordWriter.java
@@ -31,6 +31,7 @@
import java.util.List;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
@@ -99,8 +100,9 @@
@Override
public void close(TaskAttemptContext context) throws IOException {
try {
- if (statement != null)
+ if (statement != null) {
finishCopyIn.invoke(statement); // statement.finishCopyIn();
+ }
} catch (Exception e) {
throw new IOException(e);
}
diff --git a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaStreamingRecordWriter.java b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaStreamingRecordWriter.java
index 9f7fd2d..3826963 100644
--- a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaStreamingRecordWriter.java
+++ b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaStreamingRecordWriter.java
@@ -27,6 +27,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
diff --git a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaUtil.java b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaUtil.java
index b9aaa27..37e0b52 100644
--- a/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaUtil.java
+++ b/src/contrib/vertica/src/java/org/apache/hadoop/vertica/VerticaUtil.java
@@ -31,13 +31,29 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.conf.Configuration;
public class VerticaUtil {
private static final Log LOG = LogFactory.getLog(VerticaUtil.class);
+ public static int verticaVersion(Configuration conf, boolean output) throws IOException {
+ int ver = -1;
+ try {
+ VerticaConfiguration vtconfig = new VerticaConfiguration(conf);
+ Connection conn = vtconfig.getConnection(output);
+ DatabaseMetaData dbmd = conn.getMetaData();
+ ver = dbmd.getDatabaseMajorVersion() * 100;
+ ver += dbmd.getDatabaseMinorVersion();
+ } catch(ClassNotFoundException e) {
+ throw new IOException("Vertica Driver required to use Vertica Input or Output Formatters");
+ } catch (SQLException e) { throw new IOException(e); }
+ return ver;
+ }
+
public static void checkOutputSpecs(Configuration conf) throws IOException {
VerticaConfiguration vtconfig = new VerticaConfiguration(conf);
@@ -67,20 +83,24 @@
stmt = conn.createStatement();
if (tableExists && dropTable) {
- // TODO: need truncate support
- // for now drop the table if it exists
- // if def is empty, grab the columns first
- if (def == null) {
- rs = dbmd.getColumns(null, schema, table, null);
- ArrayList<String> defs = new ArrayList<String>();
- while (rs.next())
- defs.add(rs.getString(4) + " " + rs.getString(5));
- def = defs.toArray(new String[0]);
+ if(verticaVersion(conf, true) >= 305) {
+ stmt = conn.createStatement();
+ stmt.execute("TRUNCATE TABLE " + writerTable);
+ } else {
+ // for version < 3.0 drop the table if it exists
+ // if def is empty, grab the columns first to redfine the table
+ if (def == null) {
+ rs = dbmd.getColumns(null, schema, table, null);
+ ArrayList<String> defs = new ArrayList<String>();
+ while (rs.next())
+ defs.add(rs.getString(4) + " " + rs.getString(5));
+ def = defs.toArray(new String[0]);
+ }
+
+ stmt = conn.createStatement();
+ stmt.execute("DROP TABLE " + writerTable + " CASCADE");
+ tableExists = false; // force create
}
-
- stmt = conn.createStatement();
- stmt.execute("DROP TABLE " + writerTable + " CASCADE");
- tableExists = false; // force create
}
// create table if it doesn't exist
@@ -120,7 +140,7 @@
public static List<InputSplit> getSplits(JobContext context)
throws IOException {
Configuration conf = context.getConfiguration();
- int numSplits = conf.getInt("mapred.map.tasks", 1);
+ int numSplits = conf.getInt("mapreduce.job.maps", 1);
LOG.debug("creating splits up to " + numSplits);
List<InputSplit> splits = new ArrayList<InputSplit>();
int i = 0;
diff --git a/src/contrib/vertica/src/test/org/apache/hadoop/vertica/AllTests.java b/src/contrib/vertica/src/test/org/apache/hadoop/vertica/AllTests.java
index b4077bd..9fe0b8c 100644
--- a/src/contrib/vertica/src/test/org/apache/hadoop/vertica/AllTests.java
+++ b/src/contrib/vertica/src/test/org/apache/hadoop/vertica/AllTests.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.vertica;
-import java.io.FileNotFoundException;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
diff --git a/src/contrib/vertica/src/test/org/apache/hadoop/vertica/TestExample.java b/src/contrib/vertica/src/test/org/apache/hadoop/vertica/TestExample.java
index 6a40e2a..13d0d70 100644
--- a/src/contrib/vertica/src/test/org/apache/hadoop/vertica/TestExample.java
+++ b/src/contrib/vertica/src/test/org/apache/hadoop/vertica/TestExample.java
@@ -29,6 +29,7 @@
import org.apache.hadoop.io.DoubleWritable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Cluster;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
@@ -87,13 +88,16 @@
}
public Job getJob() throws IOException {
- Job job = new Job();
+ Configuration conf = new Configuration(true);
+ Cluster cluster = new Cluster(conf);
+ Job job = Job.getInstance(cluster);
+
+ conf = job.getConfiguration();
+ conf.set("mapreduce.job.tracker", "local");
+
job.setJarByClass(TestExample.class);
job.setJobName("vertica test");
- Configuration conf = job.getConfiguration();
- conf.set("mapred.job.tracker", "local");
-
job.setOutputKeyClass(Text.class);
job.setOutputValueClass(VerticaRecord.class);
job.setInputFormatClass(VerticaInputFormat.class);
diff --git a/src/contrib/vertica/src/test/org/apache/hadoop/vertica/TestVertica.java b/src/contrib/vertica/src/test/org/apache/hadoop/vertica/TestVertica.java
index 9459958..190eace 100644
--- a/src/contrib/vertica/src/test/org/apache/hadoop/vertica/TestVertica.java
+++ b/src/contrib/vertica/src/test/org/apache/hadoop/vertica/TestVertica.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.lang.reflect.Array;
+import java.math.BigDecimal;
import java.sql.Date;
import java.sql.Time;
import java.sql.Timestamp;
@@ -37,6 +38,7 @@
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Cluster;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
@@ -71,7 +73,8 @@
public Job getVerticaJob() throws IOException {
Configuration conf = new Configuration(true);
- Job job = new Job(conf, "TestVertica");
+ Cluster cluster = new Cluster(conf);
+ Job job = Job.getInstance(cluster);
job.setJarByClass(VerticaTestMR.class);
VerticaConfiguration.configureVertica(job.getConfiguration(),
@@ -123,11 +126,11 @@
types.add(Types.SMALLINT);
values.add((short) 4); // SMALLINT
types.add(Types.REAL);
- values.add(15234342345.532637); // REAL
+ values.add(new BigDecimal(15234342345.532637)); // REAL
types.add(Types.DECIMAL);
- values.add(346223093.4256); // DECIMAL
+ values.add(new BigDecimal(346223093.4256)); // DECIMAL
types.add(Types.NUMERIC);
- values.add(209232301132.4203); // NUMERIC
+ values.add(new BigDecimal(209232301132.4203)); // NUMERIC
types.add(Types.DOUBLE);
values.add(934029342.234); // DOUBLE
types.add(Types.FLOAT);
@@ -158,6 +161,51 @@
values
.add(new Timestamp(tmstmpfmt.parse("2007-08-09 6:07:05.06").getTime())); // TIMESTAMP
+ types.add(Types.BIGINT);
+ values.add(null); // BIGINT
+ types.add(Types.INTEGER);
+ values.add(null); // INTGER
+ types.add(Types.TINYINT);
+ values.add(null); // TINYINT
+ types.add(Types.SMALLINT);
+ values.add(null); // SMALLINT
+ types.add(Types.REAL);
+ values.add(null); // REAL
+ types.add(Types.DECIMAL);
+ values.add(null); // DECIMAL
+ types.add(Types.NUMERIC);
+ values.add(null); // NUMERIC
+ types.add(Types.DOUBLE);
+ values.add(null); // DOUBLE
+ types.add(Types.FLOAT);
+ values.add(null); // FLOAT
+ types.add(Types.BINARY);
+ values.add(null); // BINARY
+ types.add(Types.LONGVARBINARY);
+ values.add(null); // LONGVARBINARY
+ types.add(Types.VARBINARY);
+ values.add(null); // VARBINARY
+ types.add(Types.BOOLEAN);
+ values.add(null); // BOOLEAN
+ types.add(Types.CHAR);
+ values.add(null); // CHAR
+ types.add(Types.LONGNVARCHAR);
+ values.add(null); // LONGNVARCHAR
+ types.add(Types.LONGVARCHAR);
+ values.add(null); // LONGVARCHAR
+ types.add(Types.NCHAR);
+ values.add(null); // NCHAR
+ types.add(Types.VARCHAR);
+ values.add(null); // VARCHAR
+ types.add(Types.DATE);
+ values.add(null); // DATE
+ types.add(Types.TIME);
+ values.add(null); // TIME
+ types.add(Types.TIMESTAMP);
+ values
+ .add(null); // TIMESTAMP
+
+
String sql1 = null;
sql1 = recordTest(types, values, out, in, true);
@@ -191,7 +239,8 @@
// compare values
for(int i = 0; i < values.size(); i++)
- if(values.get(i).getClass().isArray()) {
+ if(values.get(i) == null) assertSame("Vertica Record serialized value " + i + " is null", values.get(i), new_values.get(i));
+ else if(values.get(i).getClass().isArray()) {
Object a = values.get(i);
Object b = new_values.get(i);
for(int j = 0; j < Array.getLength(a); j++)
@@ -255,17 +304,17 @@
List<InputSplit> splits = null;
Configuration conf = job.getConfiguration();
- conf.setInt("mapred.map.tasks", 1);
+ conf.setInt("mapreduce.job.maps", 1);
JobContext context = new JobContextImpl(conf, new JobID());
splits = input.getSplits(context);
assert splits.size() == 1;
- conf.setInt("mapred.map.tasks", 3);
+ conf.setInt("mapreduce.job.maps", 3);
splits = input.getSplits(context);
assert splits.size() == 3;
- conf.setInt("mapred.map.tasks", 10);
+ conf.setInt("mapreduce.job.maps", 10);
splits = input.getSplits(context);
assert splits.size() == 10;
}
diff --git a/src/docs/src/documentation/content/xdocs/capacity_scheduler.xml b/src/docs/src/documentation/content/xdocs/capacity_scheduler.xml
index 905dac4..27dfd96 100644
--- a/src/docs/src/documentation/content/xdocs/capacity_scheduler.xml
+++ b/src/docs/src/documentation/content/xdocs/capacity_scheduler.xml
@@ -40,7 +40,8 @@
<p>The Capacity Scheduler supports the following features:</p>
<ul>
<li>
- Multiple queues, where a job is submitted to a queue.
+ Multiple queues, possibly hierarchical/recursive, where a job is
+ submitted to a queue.
</li>
<li>
Queues are allocated a fraction of the capacity of the grid in the
@@ -72,11 +73,23 @@
competition for them.
</li>
<li>
+ Queues can use idle resources of other queues. In order to prevent
+ monopolizing of resources by particular queues, each queue can be
+ set a cap on the maximum number of resources it can expand to in
+ the presence of idle resources in other queues of the cluster.
+ </li>
+ <li>
Support for memory-intensive jobs, wherein a job can optionally
specify higher memory-requirements than the default, and the tasks
of the job will only be run on TaskTrackers that have enough memory
to spare.
</li>
+ <li>
+ Support for refreshing/reloading some of the queue-properties
+ without restarting the JobTracker, taking advantage of the
+ <a href="cluster_setup.html#Refreshing+queue+configuration">
+ queue-refresh</a> feature in the framework.
+ </li>
</ul>
</section>
@@ -144,54 +157,113 @@
<section>
<title>Setting Up Queues</title>
<p>
- You can define multiple queues to which users can submit jobs with
- the Capacity Scheduler. To define multiple queues, you should edit
- the site configuration for Hadoop and modify the
- <em>mapreduce.jobtracker.taskscheduler.queue.names</em> property.
- </p>
- <p>
- You can also configure ACLs for controlling which users or groups
- have access to the queues.
- </p>
- <p>
- For more details, see
- <a href="http://hadoop.apache.org/common/docs/current/cluster_setup.html#Configuring+the+Hadoop+Daemons">Configuring the Hadoop Daemons</a>.
- </p>
+ You can define multiple, possibly hierarchical queues to which users
+ can submit jobs with the Capacity Scheduler. To define queues,
+ various properties should be set in two configuration files -
+ <a href="cluster_setup.html#mapred-queues.xml">mapred-queues.xml</a>
+ and
+ <a href="ext:capacity-scheduler-conf">conf/capacity-scheduler.xml</a>
+ .</p>
+ <p><em>conf/capacity-scheduler.xml</em> can be used to configure (1)
+ job-initialization-poller related properties and (2) the
+ default values for various properties in the queues</p>
+ <p><em>conf/mapred-queues.xml</em> contains the actual queue
+ configuration including (1) framework specific properties like ACLs
+ for controlling which users or groups have access to the queues and
+ state of the queues and (2) the scheduler specific properties for
+ each queue. If any of these scheduler specific properties are
+ missing and not configured for a queue, then the properties in
+ <em>conf/capacity-scheduler.xml</em> are used to set default values.
+ More details about the properties that can be configured, and their
+ semantics is mentioned below. Also, a default template for
+ mapred-queues.xml tailored for using with
+ Capacity-scheduler can be found
+ <a href="ext:mapred-queues-capacity-scheduler">here</a>.</p>
</section>
<section>
<title>Configuring Properties for Queues</title>
<p>The Capacity Scheduler can be configured with several properties
- for each queue that control the behavior of the Scheduler. This
- configuration is in the <em>conf/capacity-scheduler.xml</em>. By
+ for each queue that control the behavior of the Scheduler. As
+ described above, this scheduler specific configuration has to be in
+ the <em>conf/mapred-queues.xml</em> along with the rest of the
+ framework specific configuration. By
default, the configuration is set up for one queue, named
<em>default</em>.</p>
- <p>To specify a property for a queue that is defined in the site
- configuration, you should use the property name as
- <em>mapred.capacity-scheduler.queue.<queue-name>.<property-name></em>.
- </p>
- <p>For example, to define the property <em>capacity</em>
- for queue named <em>research</em>, you should specify the property
- name as
- <em>mapred.capacity-scheduler.queue.research.capacity</em>.
+ <p>To specify a property for a specific queue that is defined in the
+ mapred-queues.xml, you should set the corresponding property in a
+ <property> tag explained
+ <a href="cluster_setup.html#property_tag">here</a>.
</p>
<p>The properties defined for queues and their descriptions are
listed in the table below:</p>
<table>
- <tr><th>Name</th><th>Description</th></tr>
- <tr><td>mapred.capacity-scheduler.queue.<queue-<br/>name>.capacity</td>
- <td>Percentage of the number of slots in the cluster that are made
- to be available for jobs in this queue. The sum of capacities
- for all queues should be less than or equal 100.</td>
+ <tr>
+ <th>Name</th>
+ <th>
+ <a href="commands_manual.html#RefreshQueues">
+ Refresh-able?</a>
+ </th>
+ <th>Applicable to?</th>
+ <th>Description</th>
</tr>
- <tr><td>mapred.capacity-scheduler.queue.<queue-<br/>name>.supports-priority</td>
+ <tr>
+ <td>capacity</td>
+ <td>Yes</td>
+ <td>Container queues as well as leaf queues</td>
+ <td>For a root-level container queue, this is the percentage of the
+ number of slots in the cluster that will be available for all its
+ immediate children together. For a root-level leaf-queue, this is
+ the percentage of the number of slots in the cluster that will be
+ available for all its jobs. For a non-root level container queue,
+ this is the percentage of the number of slots in its parent queue
+ that will be available for all its children together. For a
+ non-root-level leaf queue, this is the percentage of the number of
+ slots in its parent queue that will be available for jobs in this
+ queue. The sum of capacities for all children of a container queue
+ should be less than or equal 100. The sum of capacities of all the
+ root-level queues should be less than or equal to 100.
+ </td>
+ </tr>
+ <tr>
+ <td>maximum-capacity</td>
+ <td>Yes</td>
+ <td>Container queues as well as leaf queues</td>
+ <td>
+ A limit in percentage beyond which a non-root-level queue cannot use
+ the capacity of its parent queue; for a root-level queue, this is
+ the limit in percentage beyond which it cannot use the
+ cluster-capacity. This property provides a means to limit how much
+ excess capacity a queue can use. It can be used to prevent queues
+ with long running jobs from occupying more than a certain percentage
+ of the parent-queue or the cluster, which, in the absence of
+ pre-emption, can lead to capacity guarantees of other queues getting
+ affected.
+
+ The maximum-capacity of a queue can only be greater than or equal to
+ its capacity. By default, there is no limit for a queue. For a
+ non-root-level queue this means it can occupy till the
+ maximum-capacity of its parent, for a root-level queue, it means that
+ it can occupy the whole cluster. A value of 100 implies that a queue
+ can use the complete capacity of its parent, or the complete
+ cluster-capacity in case of root-level-queues.
+ </td>
+ </tr>
+ <tr>
+ <td>supports-priority</td>
+ <td>No</td>
+ <td>Leaf queues only</td>
<td>If true, priorities of jobs will be taken into account in scheduling
- decisions.</td>
+ decisions.
+ </td>
</tr>
- <tr><td>mapred.capacity-scheduler.queue.<queue-<br/>name>.minimum-user-limit-percent</td>
+ <tr>
+ <td>minimum-user-limit-percent</td>
+ <td>Yes</td>
+ <td>Leaf queues only</td>
<td>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
@@ -202,9 +274,25 @@
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.</td>
+ value of 100 implies no user limits are imposed.
+ </td>
</tr>
+ <tr>
+ <td>maximum-initialized-jobs-per-user</td>
+ <td>Yes</td>
+ <td>Leaf queues only</td>
+ <td>
+ Maximum number of jobs which are allowed to be pre-initialized for
+ a particular user in the queue. Once a job is scheduled, i.e.
+ it starts running, then that job is not considered
+ while scheduler computes the maximum job a user is allowed to
+ initialize.
+ </td>
+ </tr>
</table>
+ <p>See <a href="ext:mapred-queues-capacity-scheduler">
+ this configuration file</a> for a default configuration of queues in
+ capacity-scheduler.</p>
</section>
<section>
@@ -296,18 +384,6 @@
<tr><th>Name</th><th>Description</th></tr>
<tr>
<td>
- mapred.capacity-scheduler.queue.<queue-<br/>name>.maximum-initialized-jobs-per-user
- </td>
- <td>
- Maximum number of jobs which are allowed to be pre-initialized for
- a particular user in the queue. Once a job is scheduled, i.e.
- it starts running, then that job is not considered
- while scheduler computes the maximum job a user is allowed to
- initialize.
- </td>
- </tr>
- <tr>
- <td>
mapred.capacity-scheduler.init-poll-interval
</td>
<td>
diff --git a/src/docs/src/documentation/content/xdocs/cluster_setup.xml b/src/docs/src/documentation/content/xdocs/cluster_setup.xml
index 3d46ece..a4ee3a9 100644
--- a/src/docs/src/documentation/content/xdocs/cluster_setup.xml
+++ b/src/docs/src/documentation/content/xdocs/cluster_setup.xml
@@ -33,7 +33,7 @@
Hadoop clusters ranging from a few nodes to extremely large clusters with
thousands of nodes.</p>
<p>
- To play with Hadoop, you may first want to install Hadoop on a single machine (see <a href="quickstart.html"> Hadoop Quick Start</a>).
+ To play with Hadoop, you may first want to install Hadoop on a single machine (see <a href="ext:single-node-setup"> Hadoop Quick Start</a>).
</p>
</section>
@@ -42,11 +42,11 @@
<ol>
<li>
- Make sure all <a href="quickstart.html#PreReqs">requisite</a> software
+ Make sure all <a href="ext:single-node-setup/PreReqs">requisite</a> software
is installed on all nodes in your cluster.
</li>
<li>
- <a href="quickstart.html#Download">Get</a> the Hadoop software.
+ <a href="ext:single-node-setup/Download">Get</a> the Hadoop software.
</li>
</ol>
</section>
@@ -81,15 +81,17 @@
<ol>
<li>
Read-only default configuration -
- <a href="ext:core-default">src/core/core-default.xml</a>,
- <a href="ext:hdfs-default">src/hdfs/hdfs-default.xml</a> and
- <a href="ext:mapred-default">src/mapred/mapred-default.xml</a>.
+ <a href="ext:common-default">src/core/core-default.xml</a>,
+ <a href="ext:hdfs-default">src/hdfs/hdfs-default.xml</a>,
+ <a href="ext:mapred-default">src/mapred/mapred-default.xml</a> and
+ <a href="ext:mapred-queues">conf/mapred-queues.xml.template</a>.
</li>
<li>
Site-specific configuration -
- <em>conf/core-site.xml</em>,
- <em>conf/hdfs-site.xml</em> and
- <em>conf/mapred-site.xml</em>.
+ <a href="#core-site.xml">conf/core-site.xml</a>,
+ <a href="#hdfs-site.xml">conf/hdfs-site.xml</a>,
+ <a href="#mapred-site.xml">conf/mapred-site.xml</a> and
+ <a href="#mapred-queues.xml">conf/mapred-queues.xml</a>.
</li>
</ol>
@@ -163,9 +165,8 @@
<title>Configuring the Hadoop Daemons</title>
<p>This section deals with important parameters to be specified in the
- following:
- <br/>
- <code>conf/core-site.xml</code>:</p>
+ following:</p>
+ <anchor id="core-site.xml"/><p><code>conf/core-site.xml</code>:</p>
<table>
<tr>
@@ -180,7 +181,7 @@
</tr>
</table>
- <p><br/><code>conf/hdfs-site.xml</code>:</p>
+ <anchor id="hdfs-site.xml"/><p><code>conf/hdfs-site.xml</code>:</p>
<table>
<tr>
@@ -212,7 +213,7 @@
</tr>
</table>
- <p><br/><code>conf/mapred-site.xml</code>:</p>
+ <anchor id="mapred-site.xml"/><p><code>conf/mapred-site.xml</code>:</p>
<table>
<tr>
@@ -271,83 +272,321 @@
TaskTrackers.
</td>
</tr>
- <tr>
- <td>mapred.queue.names</td>
- <td>Comma separated list of queues to which jobs can be submitted.</td>
- <td>
- The Map/Reduce system always supports atleast one queue
- with the name as <em>default</em>. Hence, this parameter's
- value should always contain the string <em>default</em>.
- Some job schedulers supported in Hadoop, like the
- <a href="capacity_scheduler.html">Capacity
- Scheduler</a>, support multiple queues. If such a scheduler is
- being used, the list of configured queue names must be
- specified here. Once queues are defined, users can submit
- jobs to a queue using the property name
- <em>mapreduce.job.queuename</em> in the job configuration.
- There could be a separate
- configuration file for configuring properties of these
- queues that is managed by the scheduler.
- Refer to the documentation of the scheduler for information on
- the same.
- </td>
- </tr>
- <tr>
- <td>mapred.acls.enabled</td>
- <td>Specifies whether ACLs are supported for controlling job
- submission and administration</td>
- <td>
- If <em>true</em>, ACLs would be checked while submitting
- and administering jobs. ACLs can be specified using the
- configuration parameters of the form
- <em>mapred.queue.queue-name.acl-name</em>, defined below.
- </td>
- </tr>
- </table>
-
- <p><br/><code> conf/mapred-queue-acls.xml</code></p>
-
- <table>
- <tr>
- <th>Parameter</th>
- <th>Value</th>
- <th>Notes</th>
- </tr>
- <tr>
- <td>mapred.queue.<em>queue-name</em>.acl-submit-job</td>
- <td>List of users and groups that can submit jobs to the
- specified <em>queue-name</em>.</td>
- <td>
- The list of users and groups are both comma separated
- list of names. The two lists are separated by a blank.
- Example: <em>user1,user2 group1,group2</em>.
- If you wish to define only a list of groups, provide
- a blank at the beginning of the value.
- </td>
- </tr>
- <tr>
- <td>mapred.queue.<em>queue-name</em>.acl-administer-job</td>
- <td>List of users and groups that can change the priority
- or kill jobs that have been submitted to the
- specified <em>queue-name</em>.</td>
- <td>
- The list of users and groups are both comma separated
- list of names. The two lists are separated by a blank.
- Example: <em>user1,user2 group1,group2</em>.
- If you wish to define only a list of groups, provide
- a blank at the beginning of the value. Note that an
- owner of a job can always change the priority or kill
- his/her own job, irrespective of the ACLs.
- </td>
- </tr>
- </table>
-
+ </table>
<p>Typically all the above parameters are marked as
<a href="ext:api/org/apache/hadoop/conf/configuration/final_parameters">
final</a> to ensure that they cannot be overriden by user-applications.
</p>
+ <anchor id="mapred-queues.xml"/><p><code>conf/mapred-queues.xml
+ </code>:</p>
+ <p>This file is used to configure the queues in the Map/Reduce
+ system. Queues are abstract entities in the JobTracker that can be
+ used to manage collections of jobs. They provide a way for
+ administrators to organize jobs in specific ways and to enforce
+ certain policies on such collections, thus providing varying
+ levels of administrative control and management functions on jobs.
+ </p>
+ <p>One can imagine the following sample scenarios:</p>
+ <ul>
+ <li> Jobs submitted by a particular group of users can all be
+ submitted to one queue. </li>
+ <li> Long running jobs in an organization can be submitted to a
+ queue. </li>
+ <li> Short running jobs can be submitted to a queue and the number
+ of jobs that can run concurrently can be restricted. </li>
+ </ul>
+ <p>The usage of queues is closely tied to the scheduler configured
+ at the JobTracker via <em>mapreduce.jobtracker.taskscheduler</em>.
+ The degree of support of queues depends on the scheduler used. Some
+ schedulers support a single queue, while others support more complex
+ configurations. Schedulers also implement the policies that apply
+ to jobs in a queue. Some schedulers, such as the Fairshare scheduler,
+ implement their own mechanisms for collections of jobs and do not rely
+ on queues provided by the framework. The administrators are
+ encouraged to refer to the documentation of the scheduler they are
+ interested in for determining the level of support for queues.</p>
+ <p>The Map/Reduce framework supports some basic operations on queues
+ such as job submission to a specific queue, access control for queues,
+ queue states, viewing configured queues and their properties
+ and refresh of queue properties. In order to fully implement some of
+ these operations, the framework takes the help of the configured
+ scheduler.</p>
+ <p>The following types of queue configurations are possible:</p>
+ <ul>
+ <li> Single queue: The default configuration in Map/Reduce comprises
+ of a single queue, as supported by the default scheduler. All jobs
+ are submitted to this default queue which maintains jobs in a priority
+ based FIFO order.</li>
+ <li> Multiple single level queues: Multiple queues are defined, and
+ jobs can be submitted to any of these queues. Different policies
+ can be applied to these queues by schedulers that support this
+ configuration to provide a better level of support. For example,
+ the <a href="capacity_scheduler.html">capacity scheduler</a>
+ provides ways of configuring different
+ capacity and fairness guarantees on these queues.</li>
+ <li> Hierarchical queues: Hierarchical queues are a configuration in
+ which queues can contain other queues within them recursively. The
+ queues that contain other queues are referred to as
+ container queues. Queues that do not contain other queues are
+ referred as leaf or job queues. Jobs can only be submitted to leaf
+ queues. Hierarchical queues can potentially offer a higher level
+ of control to administrators, as schedulers can now build a
+ hierarchy of policies where policies applicable to a container
+ queue can provide context for policies applicable to queues it
+ contains. It also opens up possibilities for delegating queue
+ administration where administration of queues in a container queue
+ can be turned over to a different set of administrators, within
+ the context provided by the container queue. For example, the
+ <a href="capacity_scheduler.html">capacity scheduler</a>
+ uses hierarchical queues to partition capacity of a cluster
+ among container queues, and allowing queues they contain to divide
+ that capacity in more ways.</li>
+ </ul>
+
+ <p>Most of the configuration of the queues can be refreshed/reloaded
+ without restarting the Map/Reduce sub-system by editing this
+ configuration file as described in the section on
+ <a href="commands_manual.html#RefreshQueues">reloading queue
+ configuration</a>.
+ Not all configuration properties can be reloaded of course,
+ as will description of each property below explain.</p>
+
+ <p>The format of conf/mapred-queues.xml is different from the other
+ configuration files, supporting nested configuration
+ elements to support hierarchical queues. The format is as follows:
+ </p>
+
+ <source>
+ <queues aclsEnabled="$aclsEnabled">
+ <queue>
+ <name>$queue-name</name>
+ <state>$state</state>
+ <queue>
+ <name>$child-queue1</name>
+ <properties>
+ <property key="$key" value="$value"/>
+ ...
+ </properties>
+ <queue>
+ <name>$grand-child-queue1</name>
+ ...
+ </queue>
+ </queue>
+ <queue>
+ <name>$child-queue2</name>
+ ...
+ </queue>
+ ...
+ ...
+ ...
+ <queue>
+ <name>$leaf-queue</name>
+ <acl-submit-job>$acls</acl-submit-job>
+ <acl-administer-jobs>$acls</acl-administer-jobs>
+ <properties>
+ <property key="$key" value="$value"/>
+ ...
+ </properties>
+ </queue>
+ </queue>
+ </queues>
+ </source>
+ <table>
+ <tr>
+ <th>Tag/Attribute</th>
+ <th>Value</th>
+ <th>
+ <a href="commands_manual.html#RefreshQueues">Refresh-able?</a>
+ </th>
+ <th>Notes</th>
+ </tr>
+
+ <tr>
+ <td><anchor id="queues_tag"/>queues</td>
+ <td>Root element of the configuration file.</td>
+ <td>Not-applicable</td>
+ <td>All the queues are nested inside this root element of the
+ file. There can be only one root queues element in the file.</td>
+ </tr>
+
+ <tr>
+ <td>aclsEnabled</td>
+ <td>Boolean attribute to the
+ <a href="#queues_tag"><em><queues></em></a> tag
+ specifying whether ACLs are supported for controlling job
+ submission and administration for <em>all</em> the queues
+ configured.
+ </td>
+ <td>Yes</td>
+ <td>If <em>false</em>, ACLs are ignored for <em>all</em> the
+ configured queues. <br/><br/>
+ If <em>true</em>, the user and group details of the user
+ are checked against the configured ACLs of the corresponding
+ job-queue while submitting and administering jobs. ACLs can be
+ specified for each queue using the queue-specific tags
+ "acl-$acl_name", defined below. ACLs are checked only against
+ the job-queues, i.e. the leaf-level queues; ACLs configured
+ for the rest of the queues in the hierarchy are ignored.
+ </td>
+ </tr>
+
+ <tr>
+ <td><anchor id="queue_tag"/>queue</td>
+ <td>A child element of the
+ <a href="#queues_tag"><em><queues></em></a> tag or another
+ <a href="#queue_tag"><em><queue></em></a>. Denotes a queue
+ in the system.
+ </td>
+ <td>Not applicable</td>
+ <td>Queues can be hierarchical and so this element can contain
+ children of this same type.</td>
+ </tr>
+
+ <tr>
+ <td>name</td>
+ <td>Child element of a
+ <a href="#queue_tag"><em><queue></em></a> specifying the
+ name of the queue.</td>
+ <td>No</td>
+ <td>Name of the queue cannot contain the character <em>":"</em>
+ which is reserved as the queue-name delimiter when addressing a
+ queue in a hierarchy.</td>
+ </tr>
+
+ <tr>
+ <td>state</td>
+ <td>Child element of a
+ <a href="#queue_tag"><em><queue></em></a> specifying the
+ state of the queue.
+ </td>
+ <td>Yes</td>
+ <td>Each queue has a corresponding state. A queue in
+ <em>'running'</em> state can accept new jobs, while a queue in
+ <em>'stopped'</em> state will stop accepting any new jobs. State
+ is defined and respected by the framework only for the
+ leaf-level queues and is ignored for all other queues.
+ <br/><br/>
+ The state of the queue can be viewed from the command line using
+ <code>'bin/mapred queue'</code> command and also on the the Web
+ UI.<br/><br/>
+ Administrators can stop and start queues at runtime using the
+ feature of <a href="commands_manual.html#RefreshQueues">reloading
+ queue configuration</a>. If a queue is stopped at runtime, it
+ will complete all the existing running jobs and will stop
+ accepting any new jobs.
+ </td>
+ </tr>
+
+ <tr>
+ <td>acl-submit-job</td>
+ <td>Child element of a
+ <a href="#queue_tag"><em><queue></em></a> specifying the
+ list of users and groups that can submit jobs to the specified
+ queue.</td>
+ <td>Yes</td>
+ <td>
+ Applicable only to leaf-queues.<br/><br/>
+ The list of users and groups are both comma separated
+ list of names. The two lists are separated by a blank.
+ Example: <em>user1,user2 group1,group2</em>.
+ If you wish to define only a list of groups, provide
+ a blank at the beginning of the value.
+ <br/><br/>
+ </td>
+ </tr>
+
+ <tr>
+ <td>acl-administer-job</td>
+ <td>Child element of a
+ <a href="#queue_tag"><em><queue></em></a> specifying the
+ list of users and groups that can change the priority of a job
+ or kill a job that has been submitted to the specified queue.
+ </td>
+ <td>Yes</td>
+ <td>
+ Applicable only to leaf-queues.<br/><br/>
+ The list of users and groups are both comma separated
+ list of names. The two lists are separated by a blank.
+ Example: <em>user1,user2 group1,group2</em>.
+ If you wish to define only a list of groups, provide
+ a blank at the beginning of the value. Note that an
+ owner of a job can always change the priority or kill
+ his/her own job, irrespective of the ACLs.
+ </td>
+ </tr>
+
+ <tr>
+ <td><anchor id="properties_tag"/>properties</td>
+ <td>Child element of a
+ <a href="#queue_tag"><em><queue></em></a> specifying the
+ scheduler specific properties.</td>
+ <td>Not applicable</td>
+ <td>The scheduler specific properties are the children of this
+ element specified as a group of <property> tags described
+ below. The JobTracker completely ignores these properties. These
+ can be used as per-queue properties needed by the scheduler
+ being configured. Please look at the scheduler specific
+ documentation as to how these properties are used by that
+ particular scheduler.
+ </td>
+ </tr>
+
+ <tr>
+ <td><anchor id="property_tag"/>property</td>
+ <td>Child element of
+ <a href="#properties_tag"><em><properties></em></a> for a
+ specific queue.</td>
+ <td>Not applicable</td>
+ <td>A single scheduler specific queue-property. Ignored by
+ the JobTracker and used by the scheduler that is configured.</td>
+ </tr>
+
+ <tr>
+ <td>key</td>
+ <td>Attribute of a
+ <a href="#property_tag"><em><property></em></a> for a
+ specific queue.</td>
+ <td>Scheduler-specific</td>
+ <td>The name of a single scheduler specific queue-property.</td>
+ </tr>
+
+ <tr>
+ <td>value</td>
+ <td>Attribute of a
+ <a href="#property_tag"><em><property></em></a> for a
+ specific queue.</td>
+ <td>Scheduler-specific</td>
+ <td>The value of a single scheduler specific queue-property.
+ The value can be anything that is left for the proper
+ interpretation by the scheduler that is configured.</td>
+ </tr>
+
+ </table>
+
+ <p>Once the queues are configured properly and the Map/Reduce
+ system is up and running, from the command line one can
+ <a href="commands_manual.html#QueuesList">get the list
+ of queues</a> and
+ <a href="commands_manual.html#QueuesInfo">obtain
+ information specific to each queue</a>. This information is also
+ available from the web UI. On the web UI, queue information can be
+ seen by going to queueinfo.jsp, linked to from the queues table-cell
+ in the cluster-summary table. The queueinfo.jsp prints the hierarchy
+ of queues as well as the specific information for each queue.
+ </p>
+
+ <p> Users can submit jobs only to a
+ leaf-level queue by specifying the fully-qualified queue-name for
+ the property name <em>mapreduce.job.queuename</em> in the job
+ configuration. The character ':' is the queue-name delimiter and so,
+ for e.g., if one wants to submit to a configured job-queue 'Queue-C'
+ which is one of the sub-queues of 'Queue-B' which in-turn is a
+ sub-queue of 'Queue-A', then the job configuration should contain
+ property <em>mapreduce.job.queuename</em> set to the <em>
+ <value>Queue-A:Queue-B:Queue-C</value></em></p>
+ </section>
<section>
<title>Real-World Cluster Configurations</title>
@@ -881,7 +1120,6 @@
<code>$ bin/hadoop job -history all output-dir</code><br/></p>
</section>
</section>
- </section>
<p>Once all the necessary configuration is complete, distribute the files
to the <code>HADOOP_CONF_DIR</code> directory on all the machines,
@@ -952,7 +1190,7 @@
and starts the <code>TaskTracker</code> daemon on all the listed slaves.
</p>
</section>
-
+
<section>
<title>Hadoop Shutdown</title>
diff --git a/src/docs/src/documentation/content/xdocs/commands_manual.xml b/src/docs/src/documentation/content/xdocs/commands_manual.xml
index 1161264..1bf2a69 100644
--- a/src/docs/src/documentation/content/xdocs/commands_manual.xml
+++ b/src/docs/src/documentation/content/xdocs/commands_manual.xml
@@ -369,13 +369,13 @@
<th> COMMAND_OPTION </th><th> Description </th>
</tr>
<tr>
- <td><code>-list</code> </td>
+ <td><anchor id="QueuesList"/><code>-list</code> </td>
<td>Gets list of Job Queues configured in the system. Along with scheduling information
associated with the job queues.
</td>
</tr>
<tr>
- <td><code>-info <job-queue-name> [-showJobs]</code></td>
+ <td><anchor id="QueuesInfo"/><code>-info <job-queue-name> [-showJobs]</code></td>
<td>
Displays the job queue information and associated scheduling information of particular
job queue. If -showJobs options is present a list of jobs submitted to the particular job
@@ -581,16 +581,61 @@
<p>Runs MR admin client</p>
<p><code>Usage: hadoop mradmin [</code>
<a href="commands_manual.html#Generic+Options">GENERIC_OPTIONS</a>
- <code>] [-refreshQueueAcls] </code></p>
+ <code>] [-refreshServiceAcl] [-refreshQueues] [-refreshNodes] [-help [cmd]] </code></p>
<table>
<tr>
<th> COMMAND_OPTION </th><th> Description </th>
</tr>
<tr>
- <td><code>-refreshQueueAcls</code></td>
- <td> Refresh the queue acls used by Hadoop, to check access during submissions
- and administration of the job by the user. The properties present in
- <code>mapred-queue-acls.xml</code> is reloaded by the queue manager.</td>
+ <td><code>-refreshServiceAcl</code></td>
+ <td> Reload the service-level authorization policies. Jobtracker
+ will reload the authorization policy file.</td>
+ </tr>
+ <tr>
+ <td><anchor id="RefreshQueues"/><code>-refreshQueues</code></td>
+ <td><p> Reload the queues' configuration at the JobTracker.
+ Most of the configuration of the queues can be refreshed/reloaded
+ without restarting the Map/Reduce sub-system. Administrators
+ typically own the
+ <a href="cluster_setup.html#mapred-queues.xml">
+ <em>conf/mapred-queues.xml</em></a>
+ file, can edit it while the JobTracker is still running, and can do
+ a reload by running this command.</p>
+ <p>It should be noted that while trying to refresh queues'
+ configuration, one cannot change the hierarchy of queues itself.
+ This means no operation that involves a change in either the
+ hierarchy structure itself or the queues' names will be allowed.
+ Only selected properties of queues can be changed during refresh.
+ For example, new queues cannot be added dynamically, neither can an
+ existing queue be deleted.</p>
+ <p>If during a reload of queue configuration,
+ a syntactic or semantic error in made during the editing of the
+ configuration file, the refresh command fails with an exception that
+ is printed on the standard output of this command, thus informing the
+ requester with any helpful messages of what has gone wrong during
+ the edit/reload. Importantly, the existing queue configuration is
+ untouched and the system is left in a consistent state.
+ </p>
+ <p>As described in the
+ <a href="cluster_setup.html#mapred-queues.xml"><em>
+ conf/mapred-queues.xml</em></a> section, the
+ <a href="cluster_setup.html#properties_tag"><em>
+ <properties></em></a> tag in the queue configuration file can
+ also be used to specify per-queue properties needed by the scheduler.
+ When the framework's queue configuration is reloaded using this
+ command, this scheduler specific configuration will also be reloaded
+ , provided the scheduler being configured supports this reload.
+ Please see the documentation of the particular scheduler in use.</p>
+ </td>
+ </tr>
+ <tr>
+ <td><code>-refreshNodes</code></td>
+ <td> Refresh the hosts information at the jobtracker.</td>
+ </tr>
+ <tr>
+ <td><code>-help [cmd]</code></td>
+ <td>Displays help for the given command or all commands if none
+ is specified.</td>
</tr>
</table>
</section>
diff --git a/src/docs/src/documentation/content/xdocs/fair_scheduler.xml b/src/docs/src/documentation/content/xdocs/fair_scheduler.xml
index b5abc4d..900dcc3 100644
--- a/src/docs/src/documentation/content/xdocs/fair_scheduler.xml
+++ b/src/docs/src/documentation/content/xdocs/fair_scheduler.xml
@@ -79,7 +79,7 @@
jobs tolerate losing tasks; it only makes them take longer to finish.
</p>
<p>
- Finally, the Fair Scheduler can limit the number of concurrent
+ The Fair Scheduler can limit the number of concurrent
running jobs per user and per pool. This can be useful when a
user must submit hundreds of jobs at once, or for ensuring that
intermediate data does not fill up disk space on a cluster when too many
@@ -89,6 +89,13 @@
Jobs to run from each user/pool are chosen in order of priority and then
submit time.
</p>
+ <p>
+ Finally, the Fair Scheduler can limit the number of concurrent
+ running tasks per pool. This can be useful when jobs have a
+ dependency on an external service like a database or web
+ service that could be overloaded if too many map or reduce
+ tasks are run at once.
+ </p>
</section>
<section>
@@ -176,7 +183,7 @@
</td>
<td>
Specify which jobconf property is used to determine the pool that a
- job belongs in. String, default: <em>mapreduce.job.mapreduce.job.user.name</em>
+ job belongs in. String, default: <em>mapreduce.job.user.name</em>
(i.e. one pool for each user).
Another useful value is <em>group.name</em> to create a
pool per Unix group.
@@ -351,6 +358,8 @@
<ul>
<li><em>minMaps</em> and <em>minReduces</em>,
to set the pool's minimum share of task slots.</li>
+ <li><em>maxMaps</em> and <em>maxReduces</em>, to set the
+ pool's maximum concurrent task slots.</li>
<li><em>schedulingMode</em>, the pool's internal scheduling mode,
which can be <em>fair</em> for fair sharing or <em>fifo</em> for
first-in-first-out.</li>
@@ -398,9 +407,11 @@
<pool name="sample_pool">
<minMaps>5</minMaps>
<minReduces>5</minReduces>
+ <maxMaps>25</maxMaps>
+ <maxReduces>25</maxReduces>
<minSharePreemptionTimeout>300</minSharePreemptionTimeout>
</pool>
- <mapreduce.job.mapreduce.job.user.name="sample_user">
+ <mapreduce.job.user.name="sample_user">
<maxRunningJobs>6</maxRunningJobs>
</user>
<userMaxJobsDefault>3</userMaxJobsDefault>
@@ -412,7 +423,9 @@
slots and 5 reduce slots. The pool also has a minimum share preemption
timeout of 300 seconds (5 minutes), meaning that if it does not get its
guaranteed share within this time, it is allowed to kill tasks from
- other pools to achieve its share.
+ other pools to achieve its share. The pool has a cap of 25 map and 25
+ reduce slots, which means that once 25 tasks are running, no more will
+ be scheduled even if the pool's fair share is higher.
The example also limits the number of running jobs
per user to 3, except for sample_user, who can run 6 jobs concurrently.
Finally, the example sets a fair share preemption timeout of 600 seconds
diff --git a/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml b/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
index 3e6b9de..b842502 100644
--- a/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
+++ b/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
@@ -41,10 +41,10 @@
</p>
<ul>
<li>
- <a href="http://hadoop.apache.org/common/docs/current/single_node_setup.html">Single Node Setup</a> for first-time users.
+ <a href="ext:single-node-setup">Single Node Setup</a> for first-time users.
</li>
<li>
- <a href="http://hadoop.apache.org/common/docs/current/cluster_setup.html">Cluster Setup</a> for large, distributed clusters.
+ <a href="cluster_setup.html">Cluster Setup</a> for large, distributed clusters.
</li>
</ul>
</section>
@@ -152,8 +152,8 @@
occurences of each word in a given input set.</p>
<p>This example works with a
- pseudo-distributed (<a href="http://hadoop.apache.org/common/docs/current/single_node_setup.html#SingleNodeSetup">Single Node Setup</a>)
- or fully-distributed (<a href="http://hadoop.apache.org/common/docs/current/cluster_setup.html">Cluster Setup</a>)
+ pseudo-distributed (<a href="ext:single-node-setup">Single Node Setup</a>)
+ or fully-distributed (<a href="cluster_setup.html">Cluster Setup</a>)
Hadoop installation.</p>
<section>
@@ -947,6 +947,177 @@
map-outputs before writing them out to the <code>FileSystem</code>.
</p>
</section>
+
+ <section>
+ <title>Mark-Reset</title>
+
+ <p>While applications iterate through the values for a given key, it is
+ possible to mark the current position and later reset the iterator to
+ this position and continue the iteration process. The corresponding
+ methods are <code>mark()</code> and <code>reset()</code>.
+ </p>
+
+ <p><code>mark()</code> and <code>reset()</code> can be called any
+ number of times during the iteration cycle. The <code>reset()</code>
+ method will reset the iterator to the last record before a call to
+ the previous <code>mark()</code>.
+ </p>
+
+ <p>This functionality is available only with the new context based
+ reduce iterator.
+ </p>
+
+ <p> The following code snippet demonstrates the use of this
+ functionality.
+ </p>
+
+ <section>
+ <title>Source Code</title>
+
+ <table>
+ <tr><td>
+ <code>
+ public void reduce(IntWritable key,
+ Iterable<IntWritable> values,
+ Context context) throws IOException, InterruptedException {
+ </code>
+ </td></tr>
+
+ <tr><td></td></tr>
+
+ <tr><td>
+ <code>
+
+ MarkableIterator<IntWritable> mitr =
+ new MarkableIterator<IntWritable>(values.iterator());
+ </code>
+ </td></tr>
+
+ <tr><td></td></tr>
+
+ <tr><td>
+ <code>
+
+ // Mark the position
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ values.mark();
+ </code>
+ </td></tr>
+
+ <tr><td></td></tr>
+
+ <tr><td>
+ <code>
+
+ while (values.hasNext()) {
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ i = values.next();
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ // Do the necessary processing
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ }
+ </code>
+ </td></tr>
+
+ <tr><td></td></tr>
+
+ <tr><td>
+ <code>
+
+ // Reset
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ values.reset();
+ </code>
+ </td></tr>
+
+ <tr><td></td></tr>
+
+ <tr><td>
+ <code>
+
+ // Iterate all over again. Since mark was called before the first
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ // call to values.next() in this example, we will iterate over all
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ // the values now
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ while (values.hasNext()) {
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ i = values.next();
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ // Do the necessary processing
+ </code>
+ </td></tr>
+
+ <tr><td>
+ <code>
+
+ }
+ </code>
+ </td></tr>
+
+ <tr><td></td></tr>
+
+ <tr><td>
+ <code>
+ }
+ </code>
+ </td></tr>
+
+ </table>
+ </section>
+
+ </section>
</section>
<section>
@@ -1130,7 +1301,7 @@
<p>Note: <code>mapred.{map|reduce}.child.java.opts</code> are used only
for configuring the launched child tasks from task tracker. Configuring
the memory options for daemons is documented under
- <a href="http://hadoop.apache.org/common/docs/current/cluster_setup.html#Configuring+the+Environment+of+the+Hadoop+Daemons">
+ <a href="cluster_setup.html#Configuring+the+Environment+of+the+Hadoop+Daemons">
Configuring the Environment of the Hadoop Daemons</a> (Cluster Setup).</p>
<p>The memory available to some parts of the framework is also
@@ -1336,9 +1507,11 @@
<li><code>${mapreduce.cluster.local.dir}/taskTracker/jobcache/$jobid/jars/</code>
: The jars directory, which has the job jar file and expanded jar.
The <code>job.jar</code> is the application's jar file that is
- automatically distributed to each machine. It is expanded in jars
- directory before the tasks for the job start. The job.jar location
- is accessible to the application through the api
+ automatically distributed to each machine. Any library jars that are dependencies
+ of the application code may be packaged inside this jar in a <code>lib/</code> directory.
+ This directory is extracted from <code>job.jar</code> and its contents are
+ automatically added to the classpath for each task.
+ The job.jar location is accessible to the application through the api
<a href="ext:api/org/apache/hadoop/mapred/jobconf/getjar">
JobConf.getJar() </a>. To access the unjarred directory,
JobConf.getJar().getParent() can be called.</li>
@@ -2239,8 +2412,8 @@
<p>This example needs the HDFS to be up and running, especially for the
<code>DistributedCache</code>-related features. Hence it only works with a
- pseudo-distributed (<a href="http://hadoop.apache.org/common/docs/current/single_node_setup.html#SingleNodeSetup">Single Node Setup</a>)
- or fully-distributed (<a href="http://hadoop.apache.org/common/docs/current/cluster_setup.html#Fully-Distributed+Operation">Cluster Setup</a>)
+ pseudo-distributed (<a href="ext:single-node-setup">Single Node Setup</a>)
+ or fully-distributed (<a href="cluster_setup.html#Fully-Distributed+Operation">Cluster Setup</a>)
Hadoop installation.</p>
<section>
diff --git a/src/docs/src/documentation/content/xdocs/site.xml b/src/docs/src/documentation/content/xdocs/site.xml
index d5f35b4..bc37b67 100644
--- a/src/docs/src/documentation/content/xdocs/site.xml
+++ b/src/docs/src/documentation/content/xdocs/site.xml
@@ -34,6 +34,7 @@
<docs label="Getting Started">
<overview label="Overview" href="index.html" />
+ <setup label="Cluster Setup" href="cluster_setup.html"/>
<mapred label="MapReduce Tutorial" href="mapred_tutorial.html" />
<streaming label="Hadoop Streaming" href="streaming.html" />
</docs>
@@ -71,11 +72,18 @@
<jira href="http://hadoop.apache.org/mapreduce/issue_tracking.html"/>
<wiki href="http://wiki.apache.org/hadoop/MapReduce" />
<faq href="http://wiki.apache.org/hadoop/MapReduce/FAQ" />
-
<common-default href="http://hadoop.apache.org/common/docs/current/common-default.html" />
<hdfs-default href="http://hadoop.apache.org/hdfs/docs/current/hdfs-default.html" />
<mapred-default href="http://hadoop.apache.org/mapreduce/docs/current/mapred-default.html" />
-
+ <mapred-queues href="http://hadoop.apache.org/mapreduce/docs/current/mapred-queues.xml" />
+ <mapred-queues-capacity-scheduler href="http://hadoop.apache.org/mapreduce/docs/current/mapred-queues-capacity-scheduler.xml" />
+ <capacity-scheduler-conf href="http://hadoop.apache.org/mapreduce/docs/current/capacity-scheduler-conf.html" />
+
+ <single-node-setup href="http://hadoop.apache.org/common/docs/current/single_node_setup.html">
+ <PreReqs href="#PreReqs" />
+ <Download href="#Download" />
+ </single-node-setup>
+
<zlib href="http://www.zlib.net/" />
<gzip href="http://www.gzip.org/" />
<bzip href="http://www.bzip.org/" />
diff --git a/src/docs/src/documentation/content/xdocs/streaming.xml b/src/docs/src/documentation/content/xdocs/streaming.xml
index 1acadee..e34b3a2 100644
--- a/src/docs/src/documentation/content/xdocs/streaming.xml
+++ b/src/docs/src/documentation/content/xdocs/streaming.xml
@@ -580,8 +580,8 @@
-D map.output.key.field.separa=. \
-D mapreduce.partition.keypartitioner.options=-k1,2 \
-D mapreduce.fieldsel.data.field.separator=. \
- -D mapreduce.fieldsel.mapreduce.fieldsel.map.output.key.value.fields.spec=6,5,1-3:0- \
- -D mapreduce.fieldsel.mapreduce.fieldsel.reduce.output.key.value.fields.spec=0-2:5- \
+ -D mapreduce.fieldsel.map.output.key.value.fields.spec=6,5,1-3:0- \
+ -D mapreduce.fieldsel.reduce.output.key.value.fields.spec=0-2:5- \
-D mapreduce.job.reduces=12 \
-input myInputDirs \
-output myOutputDir \
@@ -591,13 +591,13 @@
</source>
<p>
-The option "-D mapreduce.fieldsel.mapreduce.fieldsel.map.output.key.value.fields.spec=6,5,1-3:0-" specifies key/value selection for the map outputs.
+The option "-D mapreduce.fieldsel.map.output.key.value.fields.spec=6,5,1-3:0-" specifies key/value selection for the map outputs.
Key selection spec and value selection spec are separated by ":".
In this case, the map output key will consist of fields 6, 5, 1, 2, and 3.
The map output value will consist of all fields (0- means field 0 and all the subsequent fields).
</p>
<p>
-The option "-D mapreduce.fieldsel.mapreduce.fieldsel.reduce.output.key.value.fields.spec=0-2:5-" specifies
+The option "-D mapreduce.fieldsel.reduce.output.key.value.fields.spec=0-2:5-" specifies
key/value selection for the reduce outputs. In this case, the reduce
output key will consist of fields 0, 1, 2 (corresponding to the original
fields 6, 5, 1). The reduce output value will consist of all fields starting
diff --git a/src/examples/org/apache/hadoop/examples/pi/math/package.html b/src/examples/org/apache/hadoop/examples/pi/math/package.html
new file mode 100644
index 0000000..d090ed5
--- /dev/null
+++ b/src/examples/org/apache/hadoop/examples/pi/math/package.html
@@ -0,0 +1,22 @@
+<html>
+<!--
+ 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.
+-->
+<body>
+ This package provides useful mathematical library classes
+ for the distbbp program.
+</body>
+</html>
diff --git a/src/examples/org/apache/hadoop/examples/pi/package.html b/src/examples/org/apache/hadoop/examples/pi/package.html
new file mode 100644
index 0000000..55913a9
--- /dev/null
+++ b/src/examples/org/apache/hadoop/examples/pi/package.html
@@ -0,0 +1,181 @@
+<html>
+<!--
+ 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.
+-->
+<body>
+ This package consists of a map/reduce application,
+ <i>distbbp</i>,
+ which computes exact binary digits of the mathematical constant π.
+ distbbp is designed for computing the n<sup>th</sup> bit of π,
+ for large n, say n > 100,000,000.
+ For computing the lower bits of π, consider using <i>bbp</i>.
+
+ <h3>The distbbp Program</h3>
+ The main class is DistBbp
+ and the actually computation is done by DistSum jobs.
+ The steps for launching the jobs are:
+
+ <ol>
+ <li>Initialize parameters.</li>
+ <li>Create a list of sums.</li>
+ <li>Read computed values from the given local directory.</li>
+ <li>Remove the computed values from the sums.</li>
+ <li>Partition the remaining sums into computation jobs.</li>
+ <li>Submit the computation jobs to a cluster and then wait for the results.</li>
+ <li>Write job outputs to the given local directory.</li>
+ <li>Combine the job outputs and print the π bits.</li>
+ </ol>
+
+<table><tr valign=top><td width=420>
+ <h3>The Bits of π</h3>
+ <p>
+The table on the right are the results computed by distbbp.
+</p>
+<ul>
+<p><li>Row 0 to Row 7
+<ul><li>They were computed by a single machine.</li>
+
+ <li>A single run of Row 7 took several seconds.</li>
+</ul></li></p>
+<p><li>Row 8 to Row 14
+<ul><li>They were computed by a 7600-task-capacity cluster.</li>
+ <li>A single run of Row 14 took 27 hours.</li>
+ <li>The computations in Row 13 and Row 14 were completed on May 20, 2009.
+ It seems that the corresponding bits were never computed before.</li>
+</ul></li></p>
+<p><li>The first part of Row 15 (<tt>6216B06</tt>)
+
+<ul><li>The first 30% of the computation was done in idle cycles of some
+ clusters spread over 20 days.</li>
+ <li>The remaining 70% was finished over a weekend on <i>Hammer</i>,
+ a 30,000-task-capacity cluster, which was also used for the
+ <a href="http://developer.yahoo.net/blogs/hadoop/2009/05/hadoop_sorts_a_petabyte_in_162.html">petabyte sort benchmark</a>.</li>
+ <li>The log files are available
+ <a href="https://issues.apache.org/jira/secure/attachment/12408543/1e15log.zip">here</a>.</li>
+ <li>The result was posted in
+ <a href="http://developer.yahoo.net/blogs/hadoop/2009/05/hadoop_computes_the_10151st_bi.html">this YDN blog</a>.</li>
+
+</ul></li></p>
+<p><li>The second part of Row 15 (<tt>D3611</tt>)
+<ul><li>The starting position is 1,000,000,000,000,053, totally 20 bits.</li>
+ <li>Two computations, at positions <i>n</i> and <i>n</i>+4, were performed.
+ <li>A single computation was divided into 14,000 jobs
+ and totally 7,000,000 tasks.
+ It took 208 years of CPU-time
+ or 12 days of cluster (with 7600-task-capacity) time. </li>
+ <li>The log files are available
+ <a href="https://issues.apache.org/jira/secure/attachment/12412297/D3611.tar.gz">here</a>.</li>
+
+ <li>The computations were completed on June 30, 2009.
+ The last bit, the 1,000,000,000,000,072<sup>nd</sup> bit,
+ probably is the highest bit (or the least significant bit) of π
+ computed ever in the history.</li>
+</ul></li></p>
+
+</td><td width=20></td><td>
+<table border=1 width=400 cellpadding=5>
+<tr><th width=30></th><th>Position <i>n</i></th><th>π bits (in hex) starting at <i>n</i></th></tr>
+
+<tr><td align=right>0</td><td align=right>1</td><td><tt>243F6A8885A3</tt><sup>*</sup></td></tr>
+<tr><td align=right>1</td><td align=right>11</td><td><tt>FDAA22168C23</tt></td></tr>
+<tr><td align=right>2</td><td align=right>101</td><td><tt>3707344A409</tt></td></tr>
+<tr><td align=right>3</td><td align=right>1,001</td><td><tt>574E69A458F</tt></td></tr>
+
+<tr><td align=right>4</td><td align=right>10,001</td><td><tt>44EC5716F2B</tt></td></tr>
+<tr><td align=right>5</td><td align=right>100,001</td><td><tt>944F7A204</tt></td></tr>
+<tr><td align=right>6</td><td align=right>1,000,001</td><td><tt>6FFFA4103</tt></td></tr>
+<tr><td align=right>7</td><td align=right>10,000,001</td><td><tt>6CFDD54E3</tt></td></tr>
+<tr><td align=right>8</td><td align=right>100,000,001</td><td><tt>A306CFA7</tt></td></tr>
+
+<tr><td align=right>9</td><td align=right>1,000,000,001</td><td><tt>3E08FF2B</tt></td></tr>
+<tr><td align=right>10</td><td align=right>10,000,000,001</td><td><tt>0A8BD8C0</tt></td></tr>
+<tr><td align=right>11</td><td align=right>100,000,000,001</td><td><tt>B2238C1</tt></td></tr>
+<tr><td align=right>12</td><td align=right>1,000,000,000,001</td><td><tt>0FEE563</tt></td></tr>
+<tr><td align=right>13</td><td align=right>10,000,000,000,001</td><td><tt>896DC3</tt></td></tr>
+
+<tr><td align=right>14</td><td align=right>100,000,000,000,001</td><td><tt>C216EC</tt></td></tr>
+<tr><td align=right>15</td><td align=right>1,000,000,000,000,001</td><td><tt>6216B06</tt> ... <tt>D3611</tt></td></tr>
+</table>
+
+<p><sup>*</sup>
+By representing π in decimal, hexadecimal and binary, we have
+
+<ul><table><tr>
+ <td>π</td><td>=</td><td><tt>3.1415926535 8979323846 2643383279</tt> ...</td>
+</tr><tr>
+ <td></td><td>=</td><td><tt>3.243F6A8885 A308D31319 8A2E037073</tt> ...</td>
+</tr><tr>
+ <td></td><td>=</td><td><tt>11.0010010000 1111110110 1010100010</tt> ...</td>
+
+</td></tr></table></ul>
+The first ten bits of π are <tt>0010010000</tt>.
+</p>
+</td></tr></table>
+
+
+ <h3>Command Line Usages</h3>
+ The command line format is:
+ <ul><pre>
+$ hadoop org.apache.hadoop.examples.pi.DistBbp \
+ <b> <nThreads> <nJobs> <type> <nPart> <remoteDir> <localDir></pre></ul>
+ And the parameters are:
+ <ul><table>
+ <tr>
+ <td><b></td>
+ <td>The number of bits to skip, i.e. compute the (b+1)th position.</td>
+ </tr>
+ <tr>
+ <td><nThreads></td>
+ <td>The number of working threads.</td>
+ </tr>
+ <tr>
+ <td><nJobs></td><td>The number of jobs per sum.
+ </tr>
+ <tr>
+ <td><type></td><td>'m' for map side job, 'r' for reduce side job, 'x' for mix type.</td>
+ </tr>
+ <tr>
+ <td><nPart></td>
+ <td>The number of parts per job.</td>
+ </tr>
+ <tr>
+ <td><remoteDir></td>
+ <td>Remote directory for submitting jobs.</td>
+ </tr>
+ <tr>
+ <td><localDir></td>
+ <td>Local directory for storing output files.</td>
+ </tr>
+ </table></ul>
+ Note that it may take a long time to finish all the jobs when <b> is large.
+ If the program is killed in the middle of the execution, the same command with
+ a different <remoteDir> can be used to resume the execution. For example, suppose
+ we use the following command to compute the (10^15+57)th bit of π.
+ <ul><pre>
+$ hadoop org.apache.hadoop.examples.pi.DistBbp \
+ 1,000,000,000,000,056 20 1000 x 500 remote/a local/output</pre></ul>
+ It uses 20 threads to summit jobs so that there are at most 20 concurrent jobs.
+ Each sum (there are totally 14 sums) is partitioned into 1000 jobs.
+ The jobs will be executed in map-side or reduce-side. Each job has 500 parts.
+ The remote directory for the jobs is remote/a and the local directory
+ for storing output is local/output. Depends on the cluster configuration,
+ it may take many days to finish the entire execution. If the execution is killed,
+ we may resume it by
+ <ul><pre>
+$ hadoop org.apache.hadoop.examples.pi.DistBbp \
+ 1,000,000,000,000,056 20 1000 x 500 remote/b local/output</pre></ul>
+</body>
+</html>
diff --git a/src/examples/org/apache/hadoop/examples/terasort/TeraChecksum.java b/src/examples/org/apache/hadoop/examples/terasort/TeraChecksum.java
index ce67126..1664742 100644
--- a/src/examples/org/apache/hadoop/examples/terasort/TeraChecksum.java
+++ b/src/examples/org/apache/hadoop/examples/terasort/TeraChecksum.java
@@ -18,40 +18,31 @@
package org.apache.hadoop.examples.terasort;
import java.io.IOException;
-import java.util.Iterator;
import java.util.zip.Checksum;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MapReduceBase;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.PureJavaCrc32;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
public class TeraChecksum extends Configured implements Tool {
- static class ChecksumMapper extends MapReduceBase
- implements Mapper<Text,Text,NullWritable,Unsigned16> {
- private OutputCollector<NullWritable,Unsigned16> output;
+ static class ChecksumMapper
+ extends Mapper<Text, Text, NullWritable, Unsigned16> {
private Unsigned16 checksum = new Unsigned16();
private Unsigned16 sum = new Unsigned16();
private Checksum crc32 = new PureJavaCrc32();
public void map(Text key, Text value,
- OutputCollector<NullWritable,Unsigned16> output,
- Reporter reporter) throws IOException {
- if (this.output == null) {
- this.output = output;
- }
+ Context context) throws IOException {
crc32.reset();
crc32.update(key.getBytes(), 0, key.getLength());
crc32.update(value.getBytes(), 0, value.getLength());
@@ -59,23 +50,22 @@
sum.add(checksum);
}
- public void close() throws IOException {
- if (output != null) {
- output.collect(NullWritable.get(), sum);
- }
+ public void cleanup(Context context)
+ throws IOException, InterruptedException {
+ context.write(NullWritable.get(), sum);
}
}
- static class ChecksumReducer extends MapReduceBase
- implements Reducer<NullWritable,Unsigned16,NullWritable,Unsigned16> {
- public void reduce(NullWritable key, Iterator<Unsigned16> values,
- OutputCollector<NullWritable, Unsigned16> output,
- Reporter reporter) throws IOException {
+ static class ChecksumReducer
+ extends Reducer<NullWritable, Unsigned16, NullWritable, Unsigned16> {
+
+ public void reduce(NullWritable key, Iterable<Unsigned16> values,
+ Context context) throws IOException, InterruptedException {
Unsigned16 sum = new Unsigned16();
- while (values.hasNext()) {
- sum.add(values.next());
+ for (Unsigned16 val : values) {
+ sum.add(val);
}
- output.collect(key, sum);
+ context.write(key, sum);
}
}
@@ -84,10 +74,10 @@
}
public int run(String[] args) throws Exception {
- JobConf job = (JobConf) getConf();
+ Job job = Job.getInstance(new Cluster(getConf()), getConf());
if (args.length != 2) {
usage();
- return 1;
+ return 2;
}
TeraInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
@@ -99,16 +89,15 @@
job.setOutputValueClass(Unsigned16.class);
// force a single reducer
job.setNumReduceTasks(1);
- job.setInputFormat(TeraInputFormat.class);
- JobClient.runJob(job);
- return 0;
+ job.setInputFormatClass(TeraInputFormat.class);
+ return job.waitForCompletion(true) ? 0 : 1;
}
/**
* @param args
*/
public static void main(String[] args) throws Exception {
- int res = ToolRunner.run(new JobConf(), new TeraChecksum(), args);
+ int res = ToolRunner.run(new Configuration(), new TeraChecksum(), args);
System.exit(res);
}
diff --git a/src/examples/org/apache/hadoop/examples/terasort/TeraGen.java b/src/examples/org/apache/hadoop/examples/terasort/TeraGen.java
index 0533d0e..dba0138 100644
--- a/src/examples/org/apache/hadoop/examples/terasort/TeraGen.java
+++ b/src/examples/org/apache/hadoop/examples/terasort/TeraGen.java
@@ -21,27 +21,30 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
import java.util.zip.Checksum;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MapReduceBase;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.PureJavaCrc32;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@@ -71,12 +74,12 @@
* An input format that assigns ranges of longs to each mapper.
*/
static class RangeInputFormat
- implements InputFormat<LongWritable, NullWritable> {
+ extends InputFormat<LongWritable, NullWritable> {
/**
* An input split consisting of a range on numbers.
*/
- static class RangeInputSplit implements InputSplit {
+ static class RangeInputSplit extends InputSplit implements Writable {
long firstRow;
long rowCount;
@@ -110,39 +113,42 @@
* A record reader that will generate a range of numbers.
*/
static class RangeRecordReader
- implements RecordReader<LongWritable, NullWritable> {
+ extends RecordReader<LongWritable, NullWritable> {
long startRow;
long finishedRows;
long totalRows;
+ LongWritable key = null;
- public RangeRecordReader(RangeInputSplit split) {
- startRow = split.firstRow;
+ public RangeRecordReader() {
+ }
+
+ public void initialize(InputSplit split, TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ startRow = ((RangeInputSplit)split).firstRow;
finishedRows = 0;
- totalRows = split.rowCount;
+ totalRows = ((RangeInputSplit)split).rowCount;
}
public void close() throws IOException {
// NOTHING
}
- public LongWritable createKey() {
- return new LongWritable();
+ public LongWritable getCurrentKey() {
+ return key;
}
- public NullWritable createValue() {
+ public NullWritable getCurrentValue() {
return NullWritable.get();
}
- public long getPos() throws IOException {
- return finishedRows;
- }
-
public float getProgress() throws IOException {
return finishedRows / (float) totalRows;
}
- public boolean next(LongWritable key,
- NullWritable value) {
+ public boolean nextKeyValue() {
+ if (key == null) {
+ key = new LongWritable();
+ }
if (finishedRows < totalRows) {
key.set(startRow + finishedRows);
finishedRows += 1;
@@ -155,24 +161,25 @@
}
public RecordReader<LongWritable, NullWritable>
- getRecordReader(InputSplit split, JobConf job,
- Reporter reporter) throws IOException {
- return new RangeRecordReader((RangeInputSplit) split);
+ createRecordReader(InputSplit split, TaskAttemptContext context)
+ throws IOException {
+ return new RangeRecordReader();
}
/**
* Create the desired number of splits, dividing the number of rows
* between the mappers.
*/
- public InputSplit[] getSplits(JobConf job,
- int numSplits) {
+ public List<InputSplit> getSplits(JobContext job) {
long totalRows = getNumberOfRows(job);
+ int numSplits = job.getConfiguration().getInt(JobContext.NUM_MAPS, 1);
LOG.info("Generating " + totalRows + " using " + numSplits);
- InputSplit[] splits = new InputSplit[numSplits];
+ List<InputSplit> splits = new ArrayList<InputSplit>();
long currentRow = 0;
- for(int split=0; split < numSplits; ++split) {
- long goal = (long) Math.ceil(totalRows * (double)(split+1) / numSplits);
- splits[split] = new RangeInputSplit(currentRow, goal - currentRow);
+ for(int split = 0; split < numSplits; ++split) {
+ long goal =
+ (long) Math.ceil(totalRows * (double)(split + 1) / numSplits);
+ splits.add(new RangeInputSplit(currentRow, goal - currentRow));
currentRow = goal;
}
return splits;
@@ -180,20 +187,20 @@
}
- static long getNumberOfRows(JobConf job) {
- return job.getLong(NUM_ROWS, 0);
+ static long getNumberOfRows(JobContext job) {
+ return job.getConfiguration().getLong(NUM_ROWS, 0);
}
- static void setNumberOfRows(JobConf job, long numRows) {
- job.setLong(NUM_ROWS, numRows);
+ static void setNumberOfRows(Job job, long numRows) {
+ job.getConfiguration().setLong(NUM_ROWS, numRows);
}
/**
* The Mapper class that given a row number, will generate the appropriate
* output line.
*/
- public static class SortGenMapper extends MapReduceBase
- implements Mapper<LongWritable, NullWritable, Text, Text> {
+ public static class SortGenMapper
+ extends Mapper<LongWritable, NullWritable, Text, Text> {
private Text key = new Text();
private Text value = new Text();
@@ -208,19 +215,18 @@
private Counter checksumCounter;
public void map(LongWritable row, NullWritable ignored,
- OutputCollector<Text, Text> output,
- Reporter reporter) throws IOException {
+ Context context) throws IOException, InterruptedException {
if (rand == null) {
rowId = new Unsigned16(row.get());
rand = Random16.skipAhead(rowId);
- checksumCounter = reporter.getCounter(Counters.CHECKSUM);
+ checksumCounter = context.getCounter(Counters.CHECKSUM);
}
Random16.nextRand(rand);
GenSort.generateRecord(buffer, rand, rowId);
key.set(buffer, 0, TeraInputFormat.KEY_LENGTH);
value.set(buffer, TeraInputFormat.KEY_LENGTH,
TeraInputFormat.VALUE_LENGTH);
- output.collect(key, value);
+ context.write(key, value);
crc32.reset();
crc32.update(buffer, 0,
TeraInputFormat.KEY_LENGTH + TeraInputFormat.VALUE_LENGTH);
@@ -230,7 +236,7 @@
}
@Override
- public void close() {
+ public void cleanup(Context context) {
checksumCounter.increment(total.getLow8());
}
}
@@ -271,15 +277,16 @@
/**
* @param args the cli arguments
*/
- public int run(String[] args) throws IOException {
- JobConf job = (JobConf) getConf();
+ public int run(String[] args)
+ throws IOException, InterruptedException, ClassNotFoundException {
+ Job job = Job.getInstance(new Cluster(getConf()), getConf());
if (args.length != 2) {
usage();
- return 1;
+ return 2;
}
setNumberOfRows(job, parseHumanLong(args[0]));
Path outputDir = new Path(args[1]);
- if (outputDir.getFileSystem(job).exists(outputDir)) {
+ if (outputDir.getFileSystem(getConf()).exists(outputDir)) {
throw new IOException("Output directory " + outputDir +
" already exists.");
}
@@ -290,14 +297,13 @@
job.setNumReduceTasks(0);
job.setOutputKeyClass(Text.class);
job.setOutputValueClass(Text.class);
- job.setInputFormat(RangeInputFormat.class);
- job.setOutputFormat(TeraOutputFormat.class);
- JobClient.runJob(job);
- return 0;
+ job.setInputFormatClass(RangeInputFormat.class);
+ job.setOutputFormatClass(TeraOutputFormat.class);
+ return job.waitForCompletion(true) ? 0 : 1;
}
public static void main(String[] args) throws Exception {
- int res = ToolRunner.run(new JobConf(), new TeraGen(), args);
+ int res = ToolRunner.run(new Configuration(), new TeraGen(), args);
System.exit(res);
}
diff --git a/src/examples/org/apache/hadoop/examples/terasort/TeraInputFormat.java b/src/examples/org/apache/hadoop/examples/terasort/TeraInputFormat.java
index 73f0c35..00e8cdb 100644
--- a/src/examples/org/apache/hadoop/examples/terasort/TeraInputFormat.java
+++ b/src/examples/org/apache/hadoop/examples/terasort/TeraInputFormat.java
@@ -22,19 +22,21 @@
import java.io.EOFException;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.LineRecordReader;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
import org.apache.hadoop.util.IndexedSortable;
import org.apache.hadoop.util.QuickSort;
import org.apache.hadoop.util.StringUtils;
@@ -47,13 +49,15 @@
public class TeraInputFormat extends FileInputFormat<Text,Text> {
static final String PARTITION_FILENAME = "_partition.lst";
- private static final String NUM_PARTITIONS = "terasort.num.partitions";
- private static final String SAMPLE_SIZE = "terasort.partitions.sample";
+ private static final String NUM_PARTITIONS =
+ "mapreduce.terasort.num.partitions";
+ private static final String SAMPLE_SIZE =
+ "mapreduce.terasort.partitions.sample";
static final int KEY_LENGTH = 10;
static final int VALUE_LENGTH = 90;
static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH;
- private static JobConf lastConf = null;
- private static InputSplit[] lastResult = null;
+ private static JobContext lastContext = null;
+ private static List<InputSplit> lastResult = null;
static class TeraFileSplit extends FileSplit {
private String[] locations;
@@ -136,24 +140,25 @@
* Use the input splits to take samples of the input and generate sample
* keys. By default reads 100,000 keys from 10 locations in the input, sorts
* them and picks N-1 keys to generate N equally sized partitions.
- * @param conf the job to sample
+ * @param job the job to sample
* @param partFile where to write the output file to
* @throws IOException if something goes wrong
*/
- public static void writePartitionFile(final JobConf conf,
- Path partFile) throws IOException {
+ public static void writePartitionFile(final JobContext job,
+ Path partFile) throws IOException, InterruptedException {
long t1 = System.currentTimeMillis();
+ Configuration conf = job.getConfiguration();
final TeraInputFormat inFormat = new TeraInputFormat();
final TextSampler sampler = new TextSampler();
- int partitions = conf.getNumReduceTasks();
+ int partitions = job.getNumReduceTasks();
long sampleSize = conf.getLong(SAMPLE_SIZE, 100000);
- final InputSplit[] splits = inFormat.getSplits(conf, conf.getNumMapTasks());
+ final List<InputSplit> splits = inFormat.getSplits(job);
long t2 = System.currentTimeMillis();
System.out.println("Computing input splits took " + (t2 - t1) + "ms");
- int samples = Math.min(conf.getInt(NUM_PARTITIONS, 10), splits.length);
- System.out.println("Sampling " + samples + " splits of " + splits.length);
+ int samples = Math.min(conf.getInt(NUM_PARTITIONS, 10), splits.size());
+ System.out.println("Sampling " + samples + " splits of " + splits.size());
final long recordsPerSample = sampleSize / samples;
- final int sampleStep = splits.length / samples;
+ final int sampleStep = splits.size() / samples;
Thread[] samplerReader = new Thread[samples];
// take N samples from different parts of the input
for(int i=0; i < samples; ++i) {
@@ -164,14 +169,16 @@
setDaemon(true);
}
public void run() {
- Text key = new Text();
- Text value = new Text();
long records = 0;
try {
- RecordReader<Text,Text> reader =
- inFormat.getRecordReader(splits[sampleStep * idx], conf, null);
- while (reader.next(key, value)) {
- sampler.addKey(key);
+ TaskAttemptContext context = new TaskAttemptContextImpl(
+ job.getConfiguration(), new TaskAttemptID());
+ RecordReader<Text, Text> reader =
+ inFormat.createRecordReader(splits.get(sampleStep * idx),
+ context);
+ reader.initialize(splits.get(sampleStep * idx), context);
+ while (reader.nextKeyValue()) {
+ sampler.addKey(new Text(reader.getCurrentKey()));
records += 1;
if (recordsPerSample <= records) {
break;
@@ -181,6 +188,8 @@
System.err.println("Got an exception while reading splits " +
StringUtils.stringifyException(ie));
System.exit(-1);
+ } catch (InterruptedException e) {
+
}
}
};
@@ -203,46 +212,47 @@
System.out.println("Computing parititions took " + (t3 - t2) + "ms");
}
- static class TeraRecordReader implements RecordReader<Text,Text> {
+ static class TeraRecordReader extends RecordReader<Text,Text> {
private FSDataInputStream in;
private long offset;
private long length;
private static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH;
private byte[] buffer = new byte[RECORD_LENGTH];
+ private Text key;
+ private Text value;
- public TeraRecordReader(Configuration job,
- FileSplit split) throws IOException {
- Path p = split.getPath();
- FileSystem fs = p.getFileSystem(job);
+ public TeraRecordReader() throws IOException {
+ }
+
+ public void initialize(InputSplit split, TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ Path p = ((FileSplit)split).getPath();
+ FileSystem fs = p.getFileSystem(context.getConfiguration());
in = fs.open(p);
- long start = split.getStart();
+ long start = ((FileSplit)split).getStart();
// find the offset to start at a record boundary
offset = (RECORD_LENGTH - (start % RECORD_LENGTH)) % RECORD_LENGTH;
in.seek(start + offset);
- length = split.getLength();
+ length = ((FileSplit)split).getLength();
}
public void close() throws IOException {
in.close();
}
- public Text createKey() {
- return new Text();
+ public Text getCurrentKey() {
+ return key;
}
- public Text createValue() {
- return new Text();
- }
-
- public long getPos() throws IOException {
- return in.getPos();
+ public Text getCurrentValue() {
+ return value;
}
public float getProgress() throws IOException {
return (float) offset / length;
}
- public boolean next(Text key, Text value) throws IOException {
+ public boolean nextKeyValue() throws IOException {
if (offset >= length) {
return false;
}
@@ -258,6 +268,12 @@
}
read += newRead;
}
+ if (key == null) {
+ key = new Text();
+ }
+ if (value == null) {
+ value = new Text();
+ }
key.set(buffer, 0, KEY_LENGTH);
value.set(buffer, KEY_LENGTH, VALUE_LENGTH);
offset += RECORD_LENGTH;
@@ -267,31 +283,30 @@
@Override
public RecordReader<Text, Text>
- getRecordReader(InputSplit split,
- JobConf job,
- Reporter reporter) throws IOException {
- return new TeraRecordReader(job, (FileSplit) split);
+ createRecordReader(InputSplit split, TaskAttemptContext context)
+ throws IOException {
+ return new TeraRecordReader();
}
- @Override
protected FileSplit makeSplit(Path file, long start, long length,
String[] hosts) {
return new TeraFileSplit(file, start, length, hosts);
}
@Override
- public InputSplit[] getSplits(JobConf conf, int splits) throws IOException {
- if (conf == lastConf) {
+ public List<InputSplit> getSplits(JobContext job) throws IOException {
+ if (job == lastContext) {
return lastResult;
}
long t1, t2, t3;
t1 = System.currentTimeMillis();
- lastConf = conf;
- lastResult = super.getSplits(conf, splits);
+ lastContext = job;
+ lastResult = super.getSplits(job);
t2 = System.currentTimeMillis();
System.out.println("Spent " + (t2 - t1) + "ms computing base-splits.");
- if (conf.getBoolean("terasort.use.terascheduler", true)) {
- TeraScheduler scheduler = new TeraScheduler((FileSplit[]) lastResult, conf);
+ if (job.getConfiguration().getBoolean(TeraScheduler.USE, true)) {
+ TeraScheduler scheduler = new TeraScheduler(
+ lastResult.toArray(new TeraFileSplit[0]), job.getConfiguration());
lastResult = scheduler.getNewFileSplits();
t3 = System.currentTimeMillis();
System.out.println("Spent " + (t3 - t2) + "ms computing TeraScheduler splits.");
diff --git a/src/examples/org/apache/hadoop/examples/terasort/TeraOutputFormat.java b/src/examples/org/apache/hadoop/examples/terasort/TeraOutputFormat.java
index 9e4936c..2796ba6 100644
--- a/src/examples/org/apache/hadoop/examples/terasort/TeraOutputFormat.java
+++ b/src/examples/org/apache/hadoop/examples/terasort/TeraOutputFormat.java
@@ -24,46 +24,43 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileOutputCommitter;
-import org.apache.hadoop.mapred.FileOutputFormat;
import org.apache.hadoop.mapred.InvalidJobConfException;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.OutputCommitter;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.TaskAttemptContext;
-import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
/**
* An output format that writes the key and value appended together.
*/
public class TeraOutputFormat extends FileOutputFormat<Text,Text> {
- static final String FINAL_SYNC_ATTRIBUTE = "terasort.final.sync";
+ static final String FINAL_SYNC_ATTRIBUTE = "mapreduce.terasort.final.sync";
+ private OutputCommitter committer = null;
/**
* Set the requirement for a final sync before the stream is closed.
*/
- public static void setFinalSync(JobConf conf, boolean newValue) {
- conf.setBoolean(FINAL_SYNC_ATTRIBUTE, newValue);
+ static void setFinalSync(JobContext job, boolean newValue) {
+ job.getConfiguration().setBoolean(FINAL_SYNC_ATTRIBUTE, newValue);
}
/**
* Does the user want a final sync at close?
*/
- public static boolean getFinalSync(JobConf conf) {
- return conf.getBoolean(FINAL_SYNC_ATTRIBUTE, false);
+ public static boolean getFinalSync(JobContext job) {
+ return job.getConfiguration().getBoolean(FINAL_SYNC_ATTRIBUTE, false);
}
- static class TeraRecordWriter implements RecordWriter<Text,Text> {
+ static class TeraRecordWriter extends RecordWriter<Text,Text> {
private boolean finalSync = false;
private FSDataOutputStream out;
public TeraRecordWriter(FSDataOutputStream out,
- JobConf conf) {
- finalSync = getFinalSync(conf);
+ JobContext job) {
+ finalSync = getFinalSync(job);
this.out = out;
}
@@ -73,7 +70,7 @@
out.write(value.getBytes(), 0, value.getLength());
}
- public void close(Reporter reporter) throws IOException {
+ public void close(TaskAttemptContext context) throws IOException {
if (finalSync) {
out.sync();
}
@@ -82,37 +79,41 @@
}
@Override
- public void checkOutputSpecs(FileSystem ignored,
- JobConf job
+ public void checkOutputSpecs(JobContext job
) throws InvalidJobConfException, IOException {
- // Ensure that the output directory is set and not already there
+ // Ensure that the output directory is set
Path outDir = getOutputPath(job);
if (outDir == null) {
throw new InvalidJobConfException("Output directory not set in JobConf.");
}
}
- public RecordWriter<Text,Text> getRecordWriter(FileSystem ignored,
- JobConf job,
- String name,
- Progressable progress
+ public RecordWriter<Text,Text> getRecordWriter(TaskAttemptContext job
) throws IOException {
- Path dir = getWorkOutputPath(job);
- FileSystem fs = dir.getFileSystem(job);
- FSDataOutputStream fileOut = fs.create(new Path(dir, name), progress);
+ Path file = getDefaultWorkFile(job, "");
+ FileSystem fs = file.getFileSystem(job.getConfiguration());
+ FSDataOutputStream fileOut = fs.create(file);
return new TeraRecordWriter(fileOut, job);
}
+ public OutputCommitter getOutputCommitter(TaskAttemptContext context)
+ throws IOException {
+ if (committer == null) {
+ Path output = getOutputPath(context);
+ committer = new TeraOutputCommitter(output, context);
+ }
+ return committer;
+ }
+
public static class TeraOutputCommitter extends FileOutputCommitter {
- @Override
- public void commitJob(JobContext jobContext) {
+ public TeraOutputCommitter(Path outputPath, TaskAttemptContext context)
+ throws IOException {
+ super(outputPath, context);
}
@Override
- public boolean needsTaskCommit(TaskAttemptContext taskContext) {
- return taskContext.getTaskAttemptID().getTaskID().getTaskType() ==
- TaskType.REDUCE;
+ public void commitJob(JobContext jobContext) {
}
@Override
diff --git a/src/examples/org/apache/hadoop/examples/terasort/TeraScheduler.java b/src/examples/org/apache/hadoop/examples/terasort/TeraScheduler.java
index bc30d0c..180c924 100644
--- a/src/examples/org/apache/hadoop/examples/terasort/TeraScheduler.java
+++ b/src/examples/org/apache/hadoop/examples/terasort/TeraScheduler.java
@@ -25,21 +25,24 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.examples.terasort.TeraInputFormat.TeraFileSplit;
-import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
class TeraScheduler {
+ static String USE = "mapreduce.terasort.use.terascheduler";
private static final Log LOG = LogFactory.getLog(TeraScheduler.class);
- private InputSplit[] splits;
+ private Split[] splits;
private List<Host> hosts = new ArrayList<Host>();
private int slotsPerHost;
private int remainingSplits = 0;
private FileSplit[] realSplits = null;
- static class InputSplit {
+ static class Split {
String filename;
boolean isAssigned = false;
List<Host> locations = new ArrayList<Host>();
- InputSplit(String filename) {
+ Split(String filename) {
this.filename = filename;
}
public String toString() {
@@ -55,7 +58,7 @@
}
static class Host {
String hostname;
- List<InputSplit> splits = new ArrayList<InputSplit>();
+ List<Split> splits = new ArrayList<Split>();
Host(String hostname) {
this.hostname = hostname;
}
@@ -92,11 +95,11 @@
}
// read the blocks
List<String> splitLines = readFile(splitFilename);
- splits = new InputSplit[splitLines.size()];
+ splits = new Split[splitLines.size()];
remainingSplits = 0;
for(String line: splitLines) {
StringTokenizer itr = new StringTokenizer(line);
- InputSplit newSplit = new InputSplit(itr.nextToken());
+ Split newSplit = new Split(itr.nextToken());
splits[remainingSplits++] = newSplit;
while (itr.hasMoreTokens()) {
Host host = hostIds.get(itr.nextToken());
@@ -109,11 +112,11 @@
public TeraScheduler(FileSplit[] realSplits,
Configuration conf) throws IOException {
this.realSplits = realSplits;
- this.slotsPerHost = conf.getInt("mapred.tasktracker.map.tasks.maximum", 4);
+ this.slotsPerHost = conf.getInt(TTConfig.TT_MAP_SLOTS, 4);
Map<String, Host> hostTable = new HashMap<String, Host>();
- splits = new InputSplit[realSplits.length];
+ splits = new Split[realSplits.length];
for(FileSplit realSplit: realSplits) {
- InputSplit split = new InputSplit(realSplit.getPath().toString());
+ Split split = new Split(realSplit.getPath().toString());
splits[remainingSplits++] = split;
for(String hostname: realSplit.getLocations()) {
Host host = hostTable.get(hostname);
@@ -148,8 +151,8 @@
int tasksToPick = Math.min(slotsPerHost,
(int) Math.ceil((double) remainingSplits /
hosts.size()));
- InputSplit[] best = new InputSplit[tasksToPick];
- for(InputSplit cur: host.splits) {
+ Split[] best = new Split[tasksToPick];
+ for(Split cur: host.splits) {
LOG.debug(" examine: " + cur.filename + " " + cur.locations.size());
int i = 0;
while (i < tasksToPick && best[i] != null &&
@@ -177,7 +180,7 @@
}
}
// for the non-chosen blocks, remove this host
- for(InputSplit cur: host.splits) {
+ for(Split cur: host.splits) {
if (!cur.isAssigned) {
cur.locations.remove(host);
}
@@ -200,7 +203,7 @@
* best host as the only host.
* @throws IOException
*/
- public FileSplit[] getNewFileSplits() throws IOException {
+ public List<InputSplit> getNewFileSplits() throws IOException {
solve();
FileSplit[] result = new FileSplit[realSplits.length];
int left = 0;
@@ -215,7 +218,11 @@
result[right--] = realSplits[i];
}
}
- return result;
+ List<InputSplit> ret = new ArrayList<InputSplit>();
+ for (FileSplit fs : result) {
+ ret.add(fs);
+ }
+ return ret;
}
public static void main(String[] args) throws IOException {
@@ -225,7 +232,7 @@
}
LOG.info("starting solve");
problem.solve();
- List<InputSplit> leftOvers = new ArrayList<InputSplit>();
+ List<Split> leftOvers = new ArrayList<Split>();
for(int i=0; i < problem.splits.length; ++i) {
if (problem.splits[i].isAssigned) {
System.out.println("sched: " + problem.splits[i]);
@@ -233,7 +240,7 @@
leftOvers.add(problem.splits[i]);
}
}
- for(InputSplit cur: leftOvers) {
+ for(Split cur: leftOvers) {
System.out.println("left: " + cur);
}
System.out.println("left over: " + leftOvers.size());
diff --git a/src/examples/org/apache/hadoop/examples/terasort/TeraSort.java b/src/examples/org/apache/hadoop/examples/terasort/TeraSort.java
index c820451..e4fc57a 100644
--- a/src/examples/org/apache/hadoop/examples/terasort/TeraSort.java
+++ b/src/examples/org/apache/hadoop/examples/terasort/TeraSort.java
@@ -25,17 +25,17 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.examples.terasort.TeraOutputFormat.TeraOutputCommitter;
-import org.apache.hadoop.filecache.DistributedCache;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Partitioner;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@@ -48,14 +48,18 @@
*/
public class TeraSort extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(TeraSort.class);
+ static String SIMPLE_PARTITIONER = "mapreduce.terasort.simplepartitioner";
+ static String OUTPUT_REPLICATION = "mapreduce.terasort.output.replication";
/**
* A partitioner that splits text keys into roughly equal partitions
* in a global sorted order.
*/
- static class TotalOrderPartitioner implements Partitioner<Text,Text>{
+ static class TotalOrderPartitioner extends Partitioner<Text,Text>
+ implements Configurable {
private TrieNode trie;
private Text[] splitPoints;
+ private Configuration conf;
/**
* A generic trie node
@@ -147,9 +151,9 @@
* @return the strings to split the partitions on
* @throws IOException
*/
- private static Text[] readPartitions(FileSystem fs, Path p,
- JobConf job) throws IOException {
- int reduces = job.getNumReduceTasks();
+ private static Text[] readPartitions(FileSystem fs, Path p,
+ Configuration conf) throws IOException {
+ int reduces = conf.getInt(JobContext.NUM_REDUCES, 1);
Text[] result = new Text[reduces - 1];
DataInputStream reader = fs.open(p);
for(int i=0; i < reduces - 1; ++i) {
@@ -201,17 +205,22 @@
return result;
}
- public void configure(JobConf job) {
+ public void setConf(Configuration conf) {
try {
- FileSystem fs = FileSystem.getLocal(job);
+ FileSystem fs = FileSystem.getLocal(conf);
+ this.conf = conf;
Path partFile = new Path(TeraInputFormat.PARTITION_FILENAME);
- splitPoints = readPartitions(fs, partFile, job);
+ splitPoints = readPartitions(fs, partFile, conf);
trie = buildTrie(splitPoints, 0, splitPoints.length, new Text(), 2);
} catch (IOException ie) {
throw new IllegalArgumentException("can't read paritions file", ie);
}
}
+ public Configuration getConf() {
+ return conf;
+ }
+
public TotalOrderPartitioner() {
}
@@ -225,13 +234,21 @@
* A total order partitioner that assigns keys based on their first
* PREFIX_LENGTH bytes, assuming a flat distribution.
*/
- public static class SimplePartitioner implements Partitioner<Text, Text>{
+ public static class SimplePartitioner extends Partitioner<Text, Text>
+ implements Configurable {
int prefixesPerReduce;
private static final int PREFIX_LENGTH = 3;
- public void configure(JobConf job) {
+ private Configuration conf = null;
+ public void setConf(Configuration conf) {
+ this.conf = conf;
prefixesPerReduce = (int) Math.ceil((1 << (8 * PREFIX_LENGTH)) /
- (float) job.getNumReduceTasks());
+ (float) conf.getInt(JobContext.NUM_REDUCES, 1));
}
+
+ public Configuration getConf() {
+ return conf;
+ }
+
@Override
public int getPartition(Text key, Text value, int numPartitions) {
byte[] bytes = key.getBytes();
@@ -244,35 +261,36 @@
}
}
- public static boolean getUseSimplePartitioner(Configuration conf) {
- return conf.getBoolean("terasort.partitioner.simple", false);
+ public static boolean getUseSimplePartitioner(JobContext job) {
+ return job.getConfiguration().getBoolean(SIMPLE_PARTITIONER, false);
}
- public static void setUseSimplePartitioner(Configuration conf,
- boolean value) {
- conf.setBoolean("terasort.partitioner.simple", value);
+ public static void setUseSimplePartitioner(Job job, boolean value) {
+ job.getConfiguration().setBoolean(SIMPLE_PARTITIONER, value);
+ }
+
+ public static int getOutputReplication(JobContext job) {
+ return job.getConfiguration().getInt(OUTPUT_REPLICATION, 1);
+ }
+
+ public static void setOutputReplication(Job job, int value) {
+ job.getConfiguration().setInt(OUTPUT_REPLICATION, value);
}
public int run(String[] args) throws Exception {
LOG.info("starting");
- JobConf job = (JobConf) getConf();
+ Job job = Job.getInstance(new Cluster(getConf()), getConf());
Path inputDir = new Path(args[0]);
Path outputDir = new Path(args[1]);
boolean useSimplePartitioner = getUseSimplePartitioner(job);
- FileSystem outputFileSystem = outputDir.getFileSystem(job);
- outputDir = outputDir.makeQualified(outputFileSystem);
- if (outputFileSystem.exists(outputDir)) {
- throw new IOException("Output directory " + outputDir +
- " already exists.");
- }
TeraInputFormat.setInputPaths(job, inputDir);
FileOutputFormat.setOutputPath(job, outputDir);
job.setJobName("TeraSort");
job.setJarByClass(TeraSort.class);
job.setOutputKeyClass(Text.class);
job.setOutputValueClass(Text.class);
- job.setInputFormat(TeraInputFormat.class);
- job.setOutputFormat(TeraOutputFormat.class);
+ job.setInputFormatClass(TeraInputFormat.class);
+ job.setOutputFormatClass(TeraOutputFormat.class);
if (useSimplePartitioner) {
job.setPartitionerClass(SimplePartitioner.class);
} else {
@@ -282,27 +300,25 @@
URI partitionUri = new URI(partitionFile.toString() +
"#" + TeraInputFormat.PARTITION_FILENAME);
TeraInputFormat.writePartitionFile(job, partitionFile);
- DistributedCache.addCacheFile(partitionUri, job);
- DistributedCache.createSymlink(job);
+ job.addCacheFile(partitionUri);
+ job.createSymlink();
long end = System.currentTimeMillis();
System.out.println("Spent " + (end - start) + "ms computing partitions.");
job.setPartitionerClass(TotalOrderPartitioner.class);
}
- job.setOutputCommitter(TeraOutputCommitter.class);
- job.setInt("dfs.replication",
- job.getInt("terasort.output.replication", 1));
+ job.getConfiguration().setInt("dfs.replication", getOutputReplication(job));
TeraOutputFormat.setFinalSync(job, true);
- JobClient.runJob(job);
+ int ret = job.waitForCompletion(true) ? 0 : 1;
LOG.info("done");
- return 0;
+ return ret;
}
/**
* @param args
*/
public static void main(String[] args) throws Exception {
- int res = ToolRunner.run(new JobConf(), new TeraSort(), args);
+ int res = ToolRunner.run(new Configuration(), new TeraSort(), args);
System.exit(res);
}
diff --git a/src/examples/org/apache/hadoop/examples/terasort/TeraValidate.java b/src/examples/org/apache/hadoop/examples/terasort/TeraValidate.java
index 472128c..c053952 100644
--- a/src/examples/org/apache/hadoop/examples/terasort/TeraValidate.java
+++ b/src/examples/org/apache/hadoop/examples/terasort/TeraValidate.java
@@ -19,23 +19,20 @@
package org.apache.hadoop.examples.terasort;
import java.io.IOException;
-import java.util.Iterator;
import java.util.zip.Checksum;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MapReduceBase;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Partitioner;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.PureJavaCrc32;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@@ -63,10 +60,8 @@
return b.toString();
}
- static class ValidateMapper extends MapReduceBase
- implements Mapper<Text,Text,Text,Text> {
+ static class ValidateMapper extends Mapper<Text,Text,Text,Text> {
private Text lastKey;
- private OutputCollector<Text,Text> output;
private String filename;
private Unsigned16 checksum = new Unsigned16();
private Unsigned16 tmp = new Unsigned16();
@@ -75,27 +70,22 @@
/**
* Get the final part of the input name
* @param split the input split
- * @return the "part-00000" for the input
+ * @return the "part-r-00000" for the input
*/
private String getFilename(FileSplit split) {
return split.getPath().getName();
}
- private int getPartition(FileSplit split) {
- return Integer.parseInt(split.getPath().getName().substring(5));
- }
-
- public void map(Text key, Text value, OutputCollector<Text,Text> output,
- Reporter reporter) throws IOException {
+ public void map(Text key, Text value, Context context)
+ throws IOException, InterruptedException {
if (lastKey == null) {
- FileSplit fs = (FileSplit) reporter.getInputSplit();
+ FileSplit fs = (FileSplit) context.getInputSplit();
filename = getFilename(fs);
- output.collect(new Text(filename + ":begin"), key);
+ context.write(new Text(filename + ":begin"), key);
lastKey = new Text();
- this.output = output;
} else {
if (key.compareTo(lastKey) < 0) {
- output.collect(ERROR, new Text("misorder in " + filename +
+ context.write(ERROR, new Text("misorder in " + filename +
" between " + textifyBytes(lastKey) +
" and " + textifyBytes(key)));
}
@@ -109,10 +99,11 @@
lastKey.set(key);
}
- public void close() throws IOException {
+ public void cleanup(Context context)
+ throws IOException, InterruptedException {
if (lastKey != null) {
- output.collect(new Text(filename + ":end"), lastKey);
- output.collect(CHECKSUM, new Text(checksum.toString()));
+ context.write(new Text(filename + ":end"), lastKey);
+ context.write(CHECKSUM, new Text(checksum.toString()));
}
}
}
@@ -122,34 +113,31 @@
* boundary keys are always increasing.
* Also passes any error reports along intact.
*/
- static class ValidateReducer extends MapReduceBase
- implements Reducer<Text,Text,Text,Text> {
+ static class ValidateReducer extends Reducer<Text,Text,Text,Text> {
private boolean firstKey = true;
private Text lastKey = new Text();
private Text lastValue = new Text();
- public void reduce(Text key, Iterator<Text> values,
- OutputCollector<Text, Text> output,
- Reporter reporter) throws IOException {
+ public void reduce(Text key, Iterable<Text> values,
+ Context context) throws IOException, InterruptedException {
if (ERROR.equals(key)) {
- while(values.hasNext()) {
- output.collect(key, values.next());
+ for (Text val : values) {
+ context.write(key, val);
}
} else if (CHECKSUM.equals(key)) {
Unsigned16 tmp = new Unsigned16();
Unsigned16 sum = new Unsigned16();
- while (values.hasNext()) {
- String val = values.next().toString();
- tmp.set(val);
+ for (Text val : values) {
+ tmp.set(val.toString());
sum.add(tmp);
}
- output.collect(CHECKSUM, new Text(sum.toString()));
+ context.write(CHECKSUM, new Text(sum.toString()));
} else {
- Text value = values.next();
+ Text value = values.iterator().next();
if (firstKey) {
firstKey = false;
} else {
if (value.compareTo(lastValue) < 0) {
- output.collect(ERROR,
+ context.write(ERROR,
new Text("bad key partitioning:\n file " +
lastKey + " key " +
textifyBytes(lastValue) +
@@ -169,7 +157,7 @@
}
public int run(String[] args) throws Exception {
- JobConf job = (JobConf) getConf();
+ Job job = Job.getInstance(new Cluster(getConf()), getConf());
if (args.length != 2) {
usage();
return 1;
@@ -185,18 +173,16 @@
// force a single reducer
job.setNumReduceTasks(1);
// force a single split
- job.setLong(org.apache.hadoop.mapreduce.lib.input.
- FileInputFormat.SPLIT_MINSIZE, Long.MAX_VALUE);
- job.setInputFormat(TeraInputFormat.class);
- JobClient.runJob(job);
- return 0;
+ FileInputFormat.setMinInputSplitSize(job, Long.MAX_VALUE);
+ job.setInputFormatClass(TeraInputFormat.class);
+ return job.waitForCompletion(true) ? 0 : 1;
}
/**
* @param args
*/
public static void main(String[] args) throws Exception {
- int res = ToolRunner.run(new JobConf(), new TeraValidate(), args);
+ int res = ToolRunner.run(new Configuration(), new TeraValidate(), args);
System.exit(res);
}
diff --git a/src/examples/python/compile b/src/examples/python/compile
index a29b75f..e5503e0 100644
--- a/src/examples/python/compile
+++ b/src/examples/python/compile
@@ -1,4 +1,16 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
export HADOOP_HOME=../../..
diff --git a/src/examples/python/pyAbacus/compile b/src/examples/python/pyAbacus/compile
index b37fd41..2e19ce9 100644
--- a/src/examples/python/pyAbacus/compile
+++ b/src/examples/python/pyAbacus/compile
@@ -1,4 +1,16 @@
#!/usr/bin/env bash
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
export HADOOP_HOME=../../../../..
diff --git a/src/examples/python/pyAbacus/wordcountaggregator.spec b/src/examples/python/pyAbacus/wordcountaggregator.spec
index 5270a65..47902e6 100644
--- a/src/examples/python/pyAbacus/wordcountaggregator.spec
+++ b/src/examples/python/pyAbacus/wordcountaggregator.spec
@@ -1,4 +1,21 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<configuration>
diff --git a/src/java/mapred-default.xml b/src/java/mapred-default.xml
index 655ea5e..55ed310 100644
--- a/src/java/mapred-default.xml
+++ b/src/java/mapred-default.xml
@@ -1,4 +1,20 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!-- Do not modify this file directly. Instead, copy entries that you -->
@@ -175,14 +191,14 @@
</property>
<property>
- <name>mapreduce.tasktracker.memorycalculatorplugin</name>
+ <name>mapreduce.tasktracker.resourcecalculatorplugin</name>
<value></value>
<description>
- Name of the class whose instance will be used to query memory information
+ Name of the class whose instance will be used to query resource information
on the tasktracker.
The class must be an instance of
- org.apache.hadoop.util.MemoryCalculatorPlugin. If the value is null, the
+ org.apache.hadoop.util.ResourceCalculatorPlugin. If the value is null, the
tasktracker attempts to use a class appropriate to the platform.
Currently, the only platform supported is Linux.
</description>
@@ -248,6 +264,17 @@
<description>The class responsible for scheduling the tasks.</description>
</property>
+
+<property>
+ <name>mapreduce.job.split.metainfo.maxsize</name>
+ <value>10000000</value>
+ <description>The maximum permissible size of the split metainfo file.
+ The JobTracker won't attempt to read split metainfo files bigger than
+ the configured value.
+ No limits if set to -1.
+ </description>
+</property>
+
<property>
<name>mapreduce.jobtracker.taskscheduler.maxrunningtasks.perjob</name>
<value></value>
@@ -285,18 +312,17 @@
<property>
<name>mapreduce.reduce.shuffle.connect.timeout</name>
<value>180000</value>
- <description>Expert: Cluster-wide configuration. The maximum amount of
- time (in milli seconds) reduce task spends in trying to connect to a
- tasktracker for getting map output.
+ <description>Expert: The maximum amount of time (in milli seconds) reduce
+ task spends in trying to connect to a tasktracker for getting map output.
</description>
</property>
<property>
<name>mapreduce.reduce.shuffle.read.timeout</name>
- <value>30000</value>
- <description>Expert: Cluster-wide configuration. The maximum amount of time
- (in milli seconds) reduce task waits for map output data to be available
- for reading after obtaining connection.
+ <value>180000</value>
+ <description>Expert: The maximum amount of time (in milli seconds) reduce
+ task waits for map output data to be available for reading after obtaining
+ connection.
</description>
</property>
@@ -718,7 +744,7 @@
</property>
<property>
- <name>mapreduce.client.progerssmonitor.pollinterval</name>
+ <name>mapreduce.client.progressmonitor.pollinterval</name>
<value>1000</value>
<description>The interval (in milliseconds) between which the JobClient
reports status to the console and checks for job completion. You may want to set this
diff --git a/src/java/mapred-queues-default.xml b/src/java/mapred-queues-default.xml
new file mode 100644
index 0000000..4a33f36
--- /dev/null
+++ b/src/java/mapred-queues-default.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<!-- This is the default mapred-queues.xml file that is loaded in the case
+ that the user does not have such a file on their classpath. -->
+<queues aclsEnabled="false">
+ <queue>
+ <name>default</name>
+ <properties>
+ </properties>
+ <state>running</state>
+ <acl-submit-job>*</acl-submit-job>
+ <acl-administer-jobs>*</acl-administer-jobs>
+ </queue>
+</queues>
\ No newline at end of file
diff --git a/src/java/org/apache/hadoop/mapred/Child.java b/src/java/org/apache/hadoop/mapred/Child.java
index a2af6ef..ec7b108 100644
--- a/src/java/org/apache/hadoop/mapred/Child.java
+++ b/src/java/org/apache/hadoop/mapred/Child.java
@@ -26,21 +26,22 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSError;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.mapred.JvmTask;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.security.JobTokens;
+import org.apache.hadoop.mapreduce.security.TokenCache;
+import org.apache.hadoop.mapreduce.security.TokenStorage;
+import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.apache.hadoop.metrics.MetricsContext;
import org.apache.hadoop.metrics.MetricsUtil;
import org.apache.hadoop.metrics.jvm.JvmMetrics;
-import org.apache.log4j.LogManager;
+import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
/**
* The main() for child processes.
@@ -70,11 +71,12 @@
JVMId jvmId = new JVMId(firstTaskid.getJobID(),
firstTaskid.getTaskType() == TaskType.MAP,jvmIdInt);
- // file name is passed thru env
+ //load token cache storage
String jobTokenFile = System.getenv().get("JOB_TOKEN_FILE");
- FileSystem localFs = FileSystem.getLocal(defaultConf);
- JobTokens jt = loadJobTokens(jobTokenFile, localFs);
- LOG.debug("Child: got jobTokenfile=" + jobTokenFile);
+ defaultConf.set(JobContext.JOB_TOKEN_FILE, jobTokenFile);
+ TokenStorage ts = TokenCache.loadTaskTokenStorage(defaultConf);
+ LOG.debug("loading token. # keys =" +ts.numberOfSecretKeys() +
+ "; from file=" + jobTokenFile);
TaskUmbilicalProtocol umbilical =
(TaskUmbilicalProtocol)RPC.getProxy(TaskUmbilicalProtocol.class,
@@ -153,8 +155,10 @@
TaskLog.syncLogs(firstTaskid, taskid, isCleanup);
JobConf job = new JobConf(task.getJobFile());
+ // set job shuffle token
+ Token<JobTokenIdentifier> jt = (Token<JobTokenIdentifier>)ts.getJobToken();
// set the jobTokenFile into task
- task.setJobTokens(jt);
+ task.setJobTokenSecret(JobTokenSecretManager.createSecretKey(jt.getPassword()));
// setup the child's Configs.LOCAL_DIR. The child is now sandboxed and
// can only see files down and under attemtdir only.
@@ -224,22 +228,4 @@
LogManager.shutdown();
}
}
-
- /**
- * load secret keys from a file
- * @param jobTokenFile
- * @param conf
- * @throws IOException
- */
- private static JobTokens loadJobTokens(String jobTokenFile, FileSystem localFS)
- throws IOException {
- Path localJobTokenFile = new Path (jobTokenFile);
- FSDataInputStream in = localFS.open(localJobTokenFile);
- JobTokens jt = new JobTokens();
- jt.readFields(in);
-
- LOG.debug("Loaded jobTokenFile from: "+localJobTokenFile.toUri().getPath());
- in.close();
- return jt;
- }
}
diff --git a/src/java/org/apache/hadoop/mapred/CleanupQueue.java b/src/java/org/apache/hadoop/mapred/CleanupQueue.java
index 9f6d167..456ed7c 100644
--- a/src/java/org/apache/hadoop/mapred/CleanupQueue.java
+++ b/src/java/org/apache/hadoop/mapred/CleanupQueue.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.mapred;
+import java.io.IOException;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.commons.logging.Log;
@@ -38,7 +39,7 @@
* paths(directories/files) in a separate thread. This constructor creates a
* clean-up thread and also starts it as a daemon. Callers can instantiate one
* CleanupQueue per JVM and can use it for deleting paths. Use
- * {@link CleanupQueue#addToQueue(FileSystem, Path...)} to add paths for
+ * {@link CleanupQueue#addToQueue(PathDeletionContext...)} to add paths for
* deletion.
*/
public CleanupQueue() {
@@ -49,22 +50,61 @@
}
}
- public void addToQueue(FileSystem fs, Path...paths) {
- cleanupThread.addToQueue(fs, paths);
+ /**
+ * Contains info related to the path of the file/dir to be deleted
+ */
+ static class PathDeletionContext {
+ String fullPath;// full path of file or dir
+ FileSystem fs;
+
+ public PathDeletionContext(FileSystem fs, String fullPath) {
+ this.fs = fs;
+ this.fullPath = fullPath;
+ }
+
+ protected String getPathForCleanup() {
+ return fullPath;
+ }
+
+ /**
+ * Makes the path(and its subdirectories recursively) fully deletable
+ */
+ protected void enablePathForCleanup() throws IOException {
+ // Do nothing by default.
+ // Subclasses can override to provide enabling for deletion.
+ }
+ }
+
+ /**
+ * Adds the paths to the queue of paths to be deleted by cleanupThread.
+ */
+ void addToQueue(PathDeletionContext... contexts) {
+ cleanupThread.addToQueue(contexts);
+ }
+
+ protected static boolean deletePath(PathDeletionContext context)
+ throws IOException {
+ context.enablePathForCleanup();
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Trying to delete " + context.fullPath);
+ }
+ if (context.fs.exists(new Path(context.fullPath))) {
+ return context.fs.delete(new Path(context.fullPath), true);
+ }
+ return true;
+ }
+
+ // currently used by tests only
+ protected boolean isQueueEmpty() {
+ return (cleanupThread.queue.size() == 0);
}
private static class PathCleanupThread extends Thread {
- static class PathAndFS {
- FileSystem fs;
- Path path;
- PathAndFS(FileSystem fs, Path path) {
- this.fs = fs;
- this.path = path;
- }
- }
// cleanup queue which deletes files/directories of the paths queued up.
- private LinkedBlockingQueue<PathAndFS> queue = new LinkedBlockingQueue<PathAndFS>();
+ private LinkedBlockingQueue<PathDeletionContext> queue =
+ new LinkedBlockingQueue<PathDeletionContext>();
public PathCleanupThread() {
setName("Directory/File cleanup thread");
@@ -72,27 +112,34 @@
start();
}
- public void addToQueue(FileSystem fs, Path... paths) {
- for (Path p : paths) {
+ void addToQueue(PathDeletionContext[] contexts) {
+ for (PathDeletionContext context : contexts) {
try {
- queue.put(new PathAndFS(fs, p));
- } catch (InterruptedException ie) {}
+ queue.put(context);
+ } catch(InterruptedException ie) {}
}
}
public void run() {
- LOG.debug(getName() + " started.");
- PathAndFS pathAndFS = null;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(getName() + " started.");
+ }
+ PathDeletionContext context = null;
while (true) {
try {
- pathAndFS = queue.take();
+ context = queue.take();
// delete the path.
- pathAndFS.fs.delete(pathAndFS.path, true);
- LOG.debug("DELETED " + pathAndFS.path);
+ if (!deletePath(context)) {
+ LOG.warn("CleanupThread:Unable to delete path " + context.fullPath);
+ }
+ else if (LOG.isDebugEnabled()) {
+ LOG.debug("DELETED " + context.fullPath);
+ }
} catch (InterruptedException t) {
+ LOG.warn("Interrupted deletion of " + context.fullPath);
return;
} catch (Exception e) {
- LOG.warn("Error deleting path" + pathAndFS.path);
+ LOG.warn("Error deleting path " + context.fullPath + ": " + e);
}
}
}
diff --git a/src/java/org/apache/hadoop/mapred/DefaultTaskController.java b/src/java/org/apache/hadoop/mapred/DefaultTaskController.java
index bcef160..6945269 100644
--- a/src/java/org/apache/hadoop/mapred/DefaultTaskController.java
+++ b/src/java/org/apache/hadoop/mapred/DefaultTaskController.java
@@ -21,6 +21,8 @@
import java.io.IOException;
import java.util.List;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.mapred.CleanupQueue.PathDeletionContext;
import org.apache.hadoop.mapred.JvmManager.JvmEnv;
import org.apache.hadoop.mapreduce.util.ProcessTree;
import org.apache.hadoop.util.Shell;
@@ -29,6 +31,8 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
/**
* The default implementation for controlling tasks.
@@ -119,14 +123,14 @@
if (shexec != null) {
if (Shell.WINDOWS) {
//We don't do send kill process signal in case of windows as
- //already we have done a process.destroy() in termintateTaskJVM()
+ //already we have done a process.destroy() in terminateTaskJVM()
return;
}
String pid = context.pid;
if (pid != null) {
if(ProcessTree.isSetsidAvailable) {
ProcessTree.killProcessGroup(pid);
- }else {
+ } else {
ProcessTree.killProcess(pid);
}
}
@@ -134,8 +138,37 @@
}
@Override
- public void initializeDistributedCache(InitializationContext context) {
- // Do nothing.
+ void dumpTaskStack(TaskControllerContext context) {
+ ShellCommandExecutor shexec = context.shExec;
+ if (shexec != null) {
+ if (Shell.WINDOWS) {
+ // We don't use signals in Windows.
+ return;
+ }
+ String pid = context.pid;
+ if (pid != null) {
+ // Send SIGQUIT to get a stack dump
+ if (ProcessTree.isSetsidAvailable) {
+ ProcessTree.sigQuitProcessGroup(pid);
+ } else {
+ ProcessTree.sigQuitProcess(pid);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void initializeDistributedCacheFile(DistributedCacheFileContext context)
+ throws IOException {
+ Path localizedUniqueDir = context.getLocalizedUniqueDir();
+ try {
+ // Setting recursive execute permission on localized dir
+ LOG.info("Doing chmod on localdir :" + localizedUniqueDir);
+ FileUtil.chmod(localizedUniqueDir.toString(), "+x", true);
+ } catch (InterruptedException ie) {
+ LOG.warn("Exception in doing chmod on" + localizedUniqueDir, ie);
+ throw new IOException(ie);
+ }
}
@Override
@@ -157,4 +190,21 @@
+ exitCode + ".");
}
}
+
+ /**
+ * Enables the task for cleanup by changing permissions of the specified path
+ * in the local filesystem
+ */
+ @Override
+ void enableTaskForCleanup(PathDeletionContext context)
+ throws IOException {
+ try {
+ FileUtil.chmod(context.fullPath, "ug+rwx", true);
+ } catch(InterruptedException e) {
+ LOG.warn("Interrupted while setting permissions for " + context.fullPath +
+ " for deletion.");
+ } catch(IOException ioe) {
+ LOG.warn("Unable to change permissions of " + context.fullPath);
+ }
+ }
}
diff --git a/src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java b/src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java
index b00846e..1ba84e3 100644
--- a/src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java
+++ b/src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java
@@ -65,8 +65,10 @@
* Version 27: Added numRequiredSlots to TaskStatus for MAPREDUCE-516
* Version 28: Adding node health status to TaskStatus for MAPREDUCE-211
* Version 29: Adding user name to the serialized Task for use by TT.
- */
- public static final long versionID = 29L;
+ * Version 30: Adding available memory and CPU usage information on TT to
+ * TaskTrackerStatus for MAPREDUCE-1218
+ */
+ public static final long versionID = 30L;
public final static int TRACKERS_OK = 0;
public final static int UNKNOWN_TASKTRACKER = 1;
diff --git a/src/java/org/apache/hadoop/mapred/IsolationRunner.java b/src/java/org/apache/hadoop/mapred/IsolationRunner.java
index 54d584b..8f557a0 100644
--- a/src/java/org/apache/hadoop/mapred/IsolationRunner.java
+++ b/src/java/org/apache/hadoop/mapred/IsolationRunner.java
@@ -30,10 +30,9 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JvmTask;
import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
/**
* IsolationRunner is intended to facilitate debugging by re-running a specific
@@ -180,19 +179,21 @@
Thread.currentThread().setContextClassLoader(classLoader);
conf.setClassLoader(classLoader);
- // split.dta file is used only by IsolationRunner. The file can now be in
- // any of the configured local disks, so use LocalDirAllocator to find out
- // where it is.
- Path localSplit =
+ // split.dta/split.meta files are used only by IsolationRunner.
+ // The file can now be in any of the configured local disks,
+ // so use LocalDirAllocator to find out where it is.
+ Path localMetaSplit =
new LocalDirAllocator(MRConfig.LOCAL_DIR).getLocalPathToRead(
- TaskTracker.getLocalSplitFile(conf.getUser(), taskId.getJobID()
- .toString(), taskId.toString()), conf);
- DataInputStream splitFile = FileSystem.getLocal(conf).open(localSplit);
- String splitClass = Text.readString(splitFile);
- BytesWritable split = new BytesWritable();
- split.readFields(splitFile);
+ TaskTracker.getLocalSplitMetaFile(conf.getUser(),
+ taskId.getJobID().toString(), taskId
+ .toString()), conf);
+ DataInputStream splitFile = FileSystem.getLocal(conf).open(localMetaSplit);
+ TaskSplitIndex splitIndex = new TaskSplitIndex();
+ splitIndex.readFields(splitFile);
splitFile.close();
- Task task = new MapTask(jobFilename.toString(), taskId, partition, splitClass, split, 1);
+
+ Task task =
+ new MapTask(jobFilename.toString(), taskId, partition, splitIndex, 1);
task.setConf(conf);
task.run(conf, new FakeUmbilical());
return true;
diff --git a/src/java/org/apache/hadoop/mapred/JobClient.java b/src/java/org/apache/hadoop/mapred/JobClient.java
index 523585a..d026077 100644
--- a/src/java/org/apache/hadoop/mapred/JobClient.java
+++ b/src/java/org/apache/hadoop/mapred/JobClient.java
@@ -476,6 +476,13 @@
}
/**
+ * Get a handle to the Cluster
+ */
+ public Cluster getClusterHandle() {
+ return cluster;
+ }
+
+ /**
* Submit a job to the MR system.
*
* This returns a handle to the {@link RunningJob} which can be used to track
@@ -523,37 +530,6 @@
}
}
- /**
- * Checks if the job directory is clean and has all the required components
- * for (re) starting the job
- */
- public static boolean isJobDirValid(Path jobDirPath, FileSystem fs)
- throws IOException {
- FileStatus[] contents = null;
-
- try {
- contents = fs.listStatus(jobDirPath);
- } catch(FileNotFoundException fnfe) {
- return false;
- }
-
- int matchCount = 0;
- if (contents.length >=2) {
- for (FileStatus status : contents) {
- if ("job.xml".equals(status.getPath().getName())) {
- ++matchCount;
- }
- if ("job.split".equals(status.getPath().getName())) {
- ++matchCount;
- }
- }
- if (matchCount == 2) {
- return true;
- }
- }
- return false;
- }
-
/**
* Get an {@link RunningJob} object to track an ongoing job. Returns
* null if the id does not correspond to any known job.
@@ -1001,8 +977,12 @@
public JobStatus[] getJobsFromQueue(String queueName) throws IOException {
try {
+ QueueInfo queue = cluster.getQueue(queueName);
+ if (queue == null) {
+ return null;
+ }
org.apache.hadoop.mapreduce.JobStatus[] stats =
- cluster.getQueue(queueName).getJobStatuses();
+ queue.getJobStatuses();
JobStatus[] ret = new JobStatus[stats.length];
for (int i = 0 ; i < stats.length; i++ ) {
ret[i] = JobStatus.downgrade(stats[i]);
@@ -1022,7 +1002,11 @@
*/
public JobQueueInfo getQueueInfo(String queueName) throws IOException {
try {
- return new JobQueueInfo(cluster.getQueue(queueName));
+ QueueInfo queueInfo = cluster.getQueue(queueName);
+ if (queueInfo != null) {
+ return new JobQueueInfo(queueInfo);
+ }
+ return null;
} catch (InterruptedException ie) {
throw new IOException(ie);
}
diff --git a/src/java/org/apache/hadoop/mapred/JobConf.java b/src/java/org/apache/hadoop/mapred/JobConf.java
index 34bac9a..f954cfe 100644
--- a/src/java/org/apache/hadoop/mapred/JobConf.java
+++ b/src/java/org/apache/hadoop/mapred/JobConf.java
@@ -24,6 +24,7 @@
import java.net.URL;
import java.net.URLDecoder;
import java.util.Enumeration;
+import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -165,6 +166,10 @@
static final String MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY =
JobContext.REDUCE_MEMORY_MB;
+ /** Pattern for the default unpacking behavior for job jars */
+ public static final Pattern UNPACK_JAR_PATTERN_DEFAULT =
+ Pattern.compile("(?:classes/|lib/).*");
+
/**
* Configuration key to set the java command line options for the child
* map and reduce tasks.
@@ -418,6 +423,14 @@
* @param jar the user jar for the map-reduce job.
*/
public void setJar(String jar) { set(JobContext.JAR, jar); }
+
+ /**
+ * Get the pattern for jar contents to unpack on the tasktracker
+ */
+ public Pattern getJarUnpackPattern() {
+ return getPattern(JobContext.JAR_UNPACK_PATTERN, UNPACK_JAR_PATTERN_DEFAULT);
+ }
+
/**
* Set the job's jar file by finding an example class location.
@@ -435,6 +448,11 @@
return getStrings(MRConfig.LOCAL_DIR);
}
+ /**
+ * Use MRAsyncDiskService.moveAndDeleteAllVolumes instead.
+ * @see org.apache.hadoop.mapreduce.util.MRAsyncDiskService#cleanupAllVolumes()
+ */
+ @Deprecated
public void deleteLocalFiles() throws IOException {
String[] localDirs = getLocalDirs();
for (int i = 0; i < localDirs.length; i++) {
diff --git a/src/java/org/apache/hadoop/mapred/JobInProgress.java b/src/java/org/apache/hadoop/mapred/JobInProgress.java
index 04b36a5..71fe0b5 100644
--- a/src/java/org/apache/hadoop/mapred/JobInProgress.java
+++ b/src/java/org/apache/hadoop/mapred/JobInProgress.java
@@ -17,7 +17,6 @@
*/
package org.apache.hadoop.mapred;
-import java.io.DataInputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
@@ -40,14 +39,16 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.CleanupQueue.PathDeletionContext;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.JobCounter;
+import org.apache.hadoop.mapreduce.JobSubmissionFiles;
import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.JobStatus.State;
import org.apache.hadoop.mapreduce.jobhistory.JobFinishedEvent;
import org.apache.hadoop.mapreduce.jobhistory.JobHistory;
import org.apache.hadoop.mapreduce.jobhistory.JobInfoChangeEvent;
@@ -63,17 +64,22 @@
import org.apache.hadoop.mapreduce.jobhistory.TaskFailedEvent;
import org.apache.hadoop.mapreduce.jobhistory.TaskFinishedEvent;
import org.apache.hadoop.mapreduce.jobhistory.TaskStartedEvent;
-import org.apache.hadoop.mapreduce.security.JobTokens;
import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
+import org.apache.hadoop.mapreduce.security.TokenStorage;
+import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
+import org.apache.hadoop.mapreduce.split.JobSplit;
+import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
import org.apache.hadoop.metrics.MetricsContext;
import org.apache.hadoop.metrics.MetricsRecord;
import org.apache.hadoop.metrics.MetricsUtil;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.fs.FSDataOutputStream;
/*************************************************************
* JobInProgress maintains all the info for keeping
@@ -101,7 +107,6 @@
JobStatus status;
Path jobFile = null;
Path localJobFile = null;
- Path localJarFile = null;
TaskInProgress maps[] = new TaskInProgress[0];
TaskInProgress reduces[] = new TaskInProgress[0];
@@ -141,6 +146,8 @@
JobPriority priority = JobPriority.NORMAL;
protected JobTracker jobtracker;
+ protected TokenStorage tokenStorage;
+
JobHistory jobHistory;
// NetworkTopology Node to the set of TIPs
@@ -298,6 +305,7 @@
new HashMap<TaskTracker, FallowSlotInfo>();
private Map<TaskTracker, FallowSlotInfo> trackersReservedForReduces =
new HashMap<TaskTracker, FallowSlotInfo>();
+ private Path jobSubmitDir = null;
/**
* Create an almost empty JobInProgress, which can be used only for tests
@@ -341,6 +349,7 @@
if (tracker != null) { // Some mock tests have null tracker
this.jobHistory = tracker.getJobHistory();
}
+ this.tokenStorage = null;
}
JobInProgress() {
@@ -352,45 +361,52 @@
* Create a JobInProgress with the given job file, plus a handle
* to the tracker.
*/
- public JobInProgress(JobID jobid, JobTracker jobtracker,
- JobConf default_conf, int rCount) throws IOException {
+ public JobInProgress(JobTracker jobtracker,
+ JobConf default_conf, int rCount,
+ JobInfo jobInfo, TokenStorage ts) throws IOException {
this.restartCount = rCount;
- this.jobId = jobid;
+ this.jobId = JobID.downgrade(jobInfo.getJobID());
String url = "http://" + jobtracker.getJobTrackerMachine() + ":"
- + jobtracker.getInfoPort() + "/jobdetails.jsp?jobid=" + jobid;
+ + jobtracker.getInfoPort() + "/jobdetails.jsp?jobid=" + this.jobId;
this.jobtracker = jobtracker;
this.jobHistory = jobtracker.getJobHistory();
this.startTime = System.currentTimeMillis();
this.localFs = jobtracker.getLocalFileSystem();
- JobConf default_job_conf = new JobConf(default_conf);
- this.localJobFile = default_job_conf.getLocalPath(JobTracker.SUBDIR
- +"/"+jobid + ".xml");
- this.localJarFile = default_job_conf.getLocalPath(JobTracker.SUBDIR
- +"/"+ jobid + ".jar");
- Path jobDir = jobtracker.getSystemDirectoryForJob(jobId);
- fs = jobtracker.getFileSystem(jobDir);
- jobFile = new Path(jobDir, "job.xml");
+ // use the user supplied token to add user credentials to the conf
+ jobSubmitDir = jobInfo.getJobSubmitDir();
+ String user = jobInfo.getUser().toString();
+ conf = new JobConf();
+ conf.set(UnixUserGroupInformation.UGI_PROPERTY_NAME,
+ new UnixUserGroupInformation(user,
+ new String[]{UnixUserGroupInformation.DEFAULT_GROUP}).toString());
+ fs = jobSubmitDir.getFileSystem(conf);
+
+ this.localJobFile =
+ default_conf.getLocalPath(JobTracker.SUBDIR + "/" + this.jobId + ".xml");
+
+ jobFile = JobSubmissionFiles.getJobConfPath(jobSubmitDir);
fs.copyToLocalFile(jobFile, localJobFile);
conf = new JobConf(localJobFile);
+ if (conf.getUser() == null) {
+ this.conf.setUser(user);
+ }
+ if (!conf.getUser().equals(user)) {
+ throw new IOException("The username obtained from the conf doesn't " +
+ "match the username the user authenticated as");
+ }
this.priority = conf.getJobPriority();
- this.profile = new JobProfile(conf.getUser(), jobid,
- jobFile.toString(), url, conf.getJobName(),
- conf.getQueueName());
- this.status = new JobStatus(jobid, 0.0f, 0.0f, JobStatus.PREP,
+ this.profile = new JobProfile(conf.getUser(), this.jobId,
+ jobFile.toString(), url,
+ conf.getJobName(), conf.getQueueName());
+ this.status = new JobStatus(this.jobId, 0.0f, 0.0f, JobStatus.PREP,
profile.getUser(), profile.getJobName(), profile.getJobFile(),
profile.getURL().toString());
- this.jobtracker.getInstrumentation().addPrepJob(conf, jobid);
+ this.jobtracker.getInstrumentation().addPrepJob(conf, this.jobId);
status.setStartTime(startTime);
this.status.setJobPriority(this.priority);
- String jarFile = conf.getJar();
- if (jarFile != null) {
- fs.copyToLocalFile(new Path(jarFile), localJarFile);
- conf.setJar(localJarFile.toString());
- }
-
this.numMapTasks = conf.getNumMapTasks();
this.numReduceTasks = conf.getNumReduceTasks();
this.taskCompletionEvents = new ArrayList<TaskCompletionEvent>
@@ -406,7 +422,7 @@
this.jobMetrics.setTag("user", conf.getUser());
this.jobMetrics.setTag("sessionId", conf.getSessionId());
this.jobMetrics.setTag("jobName", conf.getJobName());
- this.jobMetrics.setTag("jobId", jobid.toString());
+ this.jobMetrics.setTag("jobId", this.jobId.toString());
hasSpeculativeMaps = conf.getMapSpeculativeExecution();
hasSpeculativeReduces = conf.getReduceSpeculativeExecution();
this.maxLevel = jobtracker.getNumTaskCacheLevels();
@@ -429,6 +445,7 @@
JobContext.SPECULATIVECAP,0.1f);
this.slowNodeThreshold = conf.getFloat(
JobContext.SPECULATIVE_SLOWNODE_THRESHOLD,1.0f);
+ this.tokenStorage = ts;
}
@@ -473,7 +490,7 @@
}
Map<Node, List<TaskInProgress>> createCache(
- Job.RawSplit[] splits, int maxLevel) {
+ TaskSplitMetaInfo[] splits, int maxLevel) {
Map<Node, List<TaskInProgress>> cache =
new IdentityHashMap<Node, List<TaskInProgress>>(maxLevel);
@@ -581,31 +598,30 @@
//
// generate security keys needed by Tasks
//
- generateJobTokens(jobtracker.getSystemDirectoryForJob(jobId));
+ generateAndStoreTokens();
//
// read input splits and create a map per a split
//
- String jobFile = profile.getJobFile();
-
- Job.RawSplit[] splits = createSplits();
- numMapTasks = splits.length;
+ TaskSplitMetaInfo[] taskSplitMetaInfo = createSplits(jobId);
+ numMapTasks = taskSplitMetaInfo.length;
checkTaskLimits();
jobtracker.getInstrumentation().addWaitingMaps(getJobID(), numMapTasks);
jobtracker.getInstrumentation().addWaitingReduces(getJobID(), numReduceTasks);
- createMapTasks(jobFile, splits);
+ createMapTasks(jobFile.toString(), taskSplitMetaInfo);
if (numMapTasks > 0) {
- nonRunningMapCache = createCache(splits, maxLevel);
+ nonRunningMapCache = createCache(taskSplitMetaInfo,
+ maxLevel);
}
// set the launch time
this.launchTime = JobTracker.getClock().getTime();
- createReduceTasks(jobFile);
+ createReduceTasks(jobFile.toString());
// Calculate the minimum number of maps to be complete before
// we should start scheduling reduces
@@ -615,7 +631,7 @@
DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART) *
numMapTasks));
- initSetupCleanupTasks(jobFile);
+ initSetupCleanupTasks(jobFile.toString());
synchronized(jobInitKillStatus){
jobInitKillStatus.initDone = true;
@@ -633,6 +649,9 @@
jobHistory.logEvent(jie, jobId);
+ // Log the number of map and reduce tasks
+ LOG.info("Job " + jobId + " initialized successfully with " + numMapTasks
+ + " map tasks and " + numReduceTasks + " reduce tasks.");
}
// Returns true if the job is empty (0 maps, 0 reduces and no setup-cleanup)
@@ -669,16 +688,11 @@
}
- Job.RawSplit[] createSplits() throws IOException {
- DataInputStream splitFile =
- fs.open(new Path(conf.get(JobContext.SPLIT_FILE)));
- Job.RawSplit[] splits;
- try {
- splits = Job.readSplitFile(splitFile);
- } finally {
- splitFile.close();
- }
- return splits;
+ TaskSplitMetaInfo[] createSplits(org.apache.hadoop.mapreduce.JobID jobId)
+ throws IOException {
+ TaskSplitMetaInfo[] allTaskSplitMetaInfo =
+ SplitMetaInfoReader.readSplitMetaInfo(jobId, fs, conf, jobSubmitDir);
+ return allTaskSplitMetaInfo;
}
/**
@@ -695,10 +709,11 @@
}
}
- synchronized void createMapTasks(String jobFile, Job.RawSplit[] splits) {
+ synchronized void createMapTasks(String jobFile,
+ TaskSplitMetaInfo[] splits) {
maps = new TaskInProgress[numMapTasks];
for(int i=0; i < numMapTasks; ++i) {
- inputLength += splits[i].getDataLength();
+ inputLength += splits[i].getInputDataLength();
maps[i] = new TaskInProgress(jobId, jobFile,
splits[i],
jobtracker, conf, this,
@@ -720,8 +735,10 @@
}
}
+
synchronized void initSetupCleanupTasks(String jobFile) {
if (!jobSetupCleanupNeeded) {
+ LOG.info("Setup/Cleanup not needed for job " + jobId);
// nothing to initialize
return;
}
@@ -730,7 +747,7 @@
// cleanup map tip. This map doesn't use any splits. Just assign an empty
// split.
- Job.RawSplit emptySplit = new Job.RawSplit();
+ TaskSplitMetaInfo emptySplit = JobSplit.EMPTY_TASK_SPLIT;
cleanup[0] = new TaskInProgress(jobId, jobFile, emptySplit,
jobtracker, conf, this, numMapTasks, 1);
cleanup[0].setJobCleanupTask();
@@ -870,36 +887,42 @@
return launchedSetup;
}
- /**
- * Get the list of map tasks
- * @return the raw array of maps for this job
+ /**
+ * Get all the tasks of the desired type in this job.
+ * @param type {@link TaskType} of the tasks required
+ * @return An array of {@link TaskInProgress} matching the given type.
+ * Returns an empty array if no tasks are found for the given type.
*/
- TaskInProgress[] getMapTasks() {
- return maps;
- }
-
- /**
- * Get the list of cleanup tasks
- * @return the array of cleanup tasks for the job
- */
- TaskInProgress[] getCleanupTasks() {
- return cleanup;
- }
-
- /**
- * Get the list of setup tasks
- * @return the array of setup tasks for the job
- */
- TaskInProgress[] getSetupTasks() {
- return setup;
- }
-
- /**
- * Get the list of reduce tasks
- * @return the raw array of reduce tasks for this job
- */
- TaskInProgress[] getReduceTasks() {
- return reduces;
+ TaskInProgress[] getTasks(TaskType type) {
+ TaskInProgress[] tasks = null;
+ switch (type) {
+ case MAP:
+ {
+ tasks = maps;
+ }
+ break;
+ case REDUCE:
+ {
+ tasks = reduces;
+ }
+ break;
+ case JOB_SETUP:
+ {
+ tasks = setup;
+ }
+ break;
+ case JOB_CLEANUP:
+ {
+ tasks = cleanup;
+ }
+ break;
+ default:
+ {
+ tasks = new TaskInProgress[0];
+ }
+ break;
+ }
+ return tasks;
}
/**
@@ -1003,6 +1026,8 @@
boolean wasComplete = tip.isComplete();
boolean wasPending = tip.isOnlyCommitPending();
TaskAttemptID taskid = status.getTaskID();
+ boolean wasAttemptRunning = tip.isAttemptRunning(taskid);
+
// If the TIP is already completed and the task reports as SUCCEEDED then
// mark the task as KILLED.
@@ -1099,7 +1124,7 @@
// Tell the job to fail the relevant task
failedTask(tip, taskid, status, taskTracker,
- wasRunning, wasComplete);
+ wasRunning, wasComplete, wasAttemptRunning);
// Did the task failure lead to tip failure?
TaskCompletionEvent.Status taskCompletionStatus =
@@ -2950,11 +2975,10 @@
* we need to schedule reexecution so that downstream reduce tasks can
* obtain the map task's output.
*/
- private void failedTask(TaskInProgress tip, TaskAttemptID taskid,
- TaskStatus status,
- TaskTracker taskTracker,
- boolean wasRunning, boolean wasComplete) {
- final JobTrackerInstrumentation metrics = jobtracker.getInstrumentation();
+ private void failedTask(TaskInProgress tip, TaskAttemptID taskid,
+ TaskStatus status,
+ TaskTracker taskTracker, boolean wasRunning,
+ boolean wasComplete, boolean wasAttemptRunning) {
// check if the TIP is already failed
boolean wasFailed = tip.isFailed();
boolean wasSpeculating = tip.isSpeculating();
@@ -2965,6 +2989,25 @@
boolean isRunning = tip.isRunning();
boolean isComplete = tip.isComplete();
+
+ if(wasAttemptRunning) {
+ // We are decrementing counters without looking for isRunning ,
+ // because we increment the counters when we obtain
+ // new map task attempt or reduce task attempt.We do not really check
+ // for tip being running.
+ // Whenever we obtain new task attempt runningMapTasks incremented.
+ // hence we are decrementing the same.
+ if(!tip.isJobCleanupTask() && !tip.isJobSetupTask()) {
+ if(tip.isMapTask()) {
+ runningMapTasks -= 1;
+ } else {
+ runningReduceTasks -= 1;
+ }
+ }
+
+ // Metering
+ meterTaskAttempt(tip, status);
+ }
//update running count on task failure.
if (wasRunning && !isRunning) {
@@ -2973,8 +3016,6 @@
} else if (tip.isJobSetupTask()) {
launchedSetup = false;
} else if (tip.isMapTask()) {
- runningMapTasks -= 1;
- metrics.failedMap(taskid);
// remove from the running queue and put it in the non-running cache
// if the tip is not complete i.e if the tip still needs to be run
if (!isComplete) {
@@ -2982,8 +3023,6 @@
failMap(tip);
}
} else {
- runningReduceTasks -= 1;
- metrics.failedReduce(taskid);
// remove from the running queue and put in the failed queue if the tip
// is not complete
if (!isComplete) {
@@ -2991,9 +3030,6 @@
failReduce(tip);
}
}
-
- // Metering
- meterTaskAttempt(tip, status);
}
// The case when the map was complete but the task tracker went down.
@@ -3203,22 +3239,10 @@
localFs.delete(localJobFile, true);
localJobFile = null;
}
- if (localJarFile != null) {
- localFs.delete(localJarFile, true);
- localJarFile = null;
- }
- // clean up splits
- for (int i = 0; i < maps.length; i++) {
- maps[i].clearSplit();
- }
-
- // JobClient always creates a new directory with job files
- // so we remove that directory to cleanup
- // Delete temp dfs dirs created if any, like in case of
- // speculative exn of reduces.
Path tempDir = jobtracker.getSystemDirectoryForJob(getJobID());
- new CleanupQueue().addToQueue(jobtracker.getFileSystem(tempDir), tempDir);
+ new CleanupQueue().addToQueue(new PathDeletionContext(
+ jobtracker.getFileSystem(tempDir), tempDir.toUri().getPath()));
} catch (IOException e) {
LOG.warn("Error cleaning up "+profile.getJobID()+": "+e);
}
@@ -3448,11 +3472,11 @@
"submitTime" + EQUALS + job.getStartTime() + StringUtils.COMMA +
"launchTime" + EQUALS + job.getLaunchTime() + StringUtils.COMMA +
"finishTime" + EQUALS + job.getFinishTime() + StringUtils.COMMA +
- "numMaps" + EQUALS + job.getMapTasks().length +
+ "numMaps" + EQUALS + job.getTasks(TaskType.MAP).length +
StringUtils.COMMA +
"numSlotsPerMap" + EQUALS + job.getNumSlotsPerMap() +
StringUtils.COMMA +
- "numReduces" + EQUALS + job.getReduceTasks().length +
+ "numReduces" + EQUALS + job.getTasks(TaskType.REDUCE).length +
StringUtils.COMMA +
"numSlotsPerReduce" + EQUALS + job.getNumSlotsPerReduce() +
StringUtils.COMMA +
@@ -3515,28 +3539,33 @@
}
/**
- * generate keys and save it into the file
- * @param jobDir
+ * generate job token and save it into the file
* @throws IOException
*/
- private void generateJobTokens(Path jobDir) throws IOException{
- Path keysFile = new Path(jobDir, JobTokens.JOB_TOKEN_FILENAME);
- FSDataOutputStream os = fs.create(keysFile);
- //create JobTokens file and add key to it
- JobTokens jt = new JobTokens();
- byte [] key;
- try {
- // new key
- key = SecureShuffleUtils.getNewEncodedKey();
- } catch (java.security.GeneralSecurityException e) {
- throw new IOException(e);
- }
- // remember the key
- jt.setShuffleJobToken(key);
- // other keys..
- jt.write(os);
+ private void generateAndStoreTokens() throws IOException{
+ Path jobDir = jobtracker.getSystemDirectoryForJob(jobId);
+ Path keysFile = new Path(jobDir, SecureShuffleUtils.JOB_TOKEN_FILENAME);
+ // we need to create this file using the jobtracker's filesystem
+ FSDataOutputStream os = jobtracker.getFileSystem().create(keysFile);
+
+ //create JobToken file and write token to it
+ JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(jobId
+ .toString()));
+ Token<JobTokenIdentifier> token = new Token<JobTokenIdentifier>(identifier,
+ jobtracker.getJobTokenSecretManager());
+ token.setService(identifier.getJobId());
+
+ // add this token to the tokenStorage
+ if(tokenStorage == null)
+ tokenStorage = new TokenStorage();
+
+ tokenStorage.setJobToken(token);
+
+ // write TokenStorage out
+ tokenStorage.write(os);
os.close();
- LOG.debug("jobTokens generated and stored in "+ keysFile.toUri().getPath());
+ LOG.info("jobToken generated and stored with users keys in "
+ + keysFile.toUri().getPath());
}
}
diff --git a/src/java/org/apache/hadoop/mapred/JobInfo.java b/src/java/org/apache/hadoop/mapred/JobInfo.java
new file mode 100644
index 0000000..bed9abe
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapred/JobInfo.java
@@ -0,0 +1,83 @@
+/**
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * Represents the basic information that is saved per a job when the
+ * JobTracker receives a submitJob request. The information is saved
+ * so that the JobTracker can recover incomplete jobs upon restart.
+ */
+class JobInfo implements Writable {
+ private org.apache.hadoop.mapreduce.JobID id;
+ private Text user;
+ private Path jobSubmitDir;
+ public JobInfo() {}
+
+ public JobInfo(org.apache.hadoop.mapreduce.JobID id,
+ Text user,
+ Path jobSubmitDir) {
+ this.id = id;
+ this.user = user;
+ this.jobSubmitDir = jobSubmitDir;
+ }
+
+ /**
+ * Get the job id.
+ */
+ public org.apache.hadoop.mapreduce.JobID getJobID() {
+ return id;
+ }
+
+ /**
+ * Get the configured job's user-name.
+ */
+ public Text getUser() {
+ return user;
+ }
+
+ /**
+ * Get the job submission directory
+ */
+ public Path getJobSubmitDir() {
+ return this.jobSubmitDir;
+ }
+
+ public void readFields(DataInput in) throws IOException {
+ id = new org.apache.hadoop.mapreduce.JobID();
+ id.readFields(in);
+ user = new Text();
+ user.readFields(in);
+ jobSubmitDir = new Path(WritableUtils.readString(in));
+ }
+
+ public void write(DataOutput out) throws IOException {
+ id.write(out);
+ user.write(out);
+ WritableUtils.writeString(out, jobSubmitDir.toString());
+ }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/hadoop/mapred/JobQueueClient.java b/src/java/org/apache/hadoop/mapred/JobQueueClient.java
index 47fa7d7..129b175 100644
--- a/src/java/org/apache/hadoop/mapred/JobQueueClient.java
+++ b/src/java/org/apache/hadoop/mapred/JobQueueClient.java
@@ -175,6 +175,11 @@
private void displayQueueInfo(String queue, boolean showJobs)
throws IOException {
JobQueueInfo jobQueueInfo = jc.getQueueInfo(queue);
+
+ if (jobQueueInfo == null) {
+ System.out.println("Queue \"" + queue + "\" does not exist.");
+ return;
+ }
printJobQueueInfo(jobQueueInfo, new PrintWriter(System.out));
if (showJobs && (jobQueueInfo.getChildren() == null ||
jobQueueInfo.getChildren().size() == 0)) {
diff --git a/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java b/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java
deleted file mode 100644
index e69de29..0000000
--- a/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java
+++ /dev/null
diff --git a/src/java/org/apache/hadoop/mapred/JobTracker.java b/src/java/org/apache/hadoop/mapred/JobTracker.java
index bed32e1..451b251 100644
--- a/src/java/org/apache/hadoop/mapred/JobTracker.java
+++ b/src/java/org/apache/hadoop/mapred/JobTracker.java
@@ -22,11 +22,9 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.PrintWriter;
-import java.io.UnsupportedEncodingException;
import java.io.Writer;
import java.net.BindException;
import java.net.InetSocketAddress;
-import java.net.URLEncoder;
import java.net.UnknownHostException;
import java.text.ParseException;
import java.text.SimpleDateFormat;
@@ -64,6 +62,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.ipc.RPC.VersionMismatch;
@@ -76,9 +75,14 @@
import org.apache.hadoop.mapreduce.QueueInfo;
import org.apache.hadoop.mapreduce.TaskTrackerInfo;
import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
import org.apache.hadoop.mapreduce.jobhistory.JobHistory;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.mapreduce.security.TokenStorage;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
+import org.apache.hadoop.mapreduce.util.ConfigUtil;
+import org.apache.hadoop.mapreduce.util.MRAsyncDiskService;
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NetworkTopology;
@@ -87,6 +91,7 @@
import org.apache.hadoop.net.ScriptBasedMapping;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.PermissionChecker;
+import org.apache.hadoop.security.RefreshUserToGroupMappingsProtocol;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UnixUserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation;
@@ -100,8 +105,6 @@
import org.apache.hadoop.util.Service;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.VersionInfo;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.mapreduce.util.ConfigUtil;
/*******************************************************
* JobTracker is the central location for submitting and
@@ -110,7 +113,7 @@
*******************************************************/
public class JobTracker extends Service
implements MRConstants, InterTrackerProtocol,
- ClientProtocol, TaskTrackerManager,
+ ClientProtocol, TaskTrackerManager, RefreshUserToGroupMappingsProtocol,
RefreshAuthorizationPolicyProtocol, AdminOperationsProtocol, JTConfig {
static{
@@ -145,6 +148,8 @@
public static enum State { INITIALIZING, RUNNING }
private static final int FS_ACCESS_RETRY_PERIOD = 10000;
+
+ static final String JOB_INFO_FILE = "job-info";
private DNSToSwitchMapping dnsToSwitchMapping;
NetworkTopology clusterMap = new NetworkTopology();
@@ -154,9 +159,9 @@
private final List<JobInProgressListener> jobInProgressListeners =
new CopyOnWriteArrayList<JobInProgressListener>();
- // system directories are world-wide readable and owner readable
+ // system directory is completely owned by the JobTracker
final static FsPermission SYSTEM_DIR_PERMISSION =
- FsPermission.createImmutable((short) 0733); // rwx-wx-wx
+ FsPermission.createImmutable((short) 0700); // rwx------
// system files should have 700 permission
final static FsPermission SYSTEM_FILE_PERMISSION =
@@ -167,7 +172,18 @@
static final Clock DEFAULT_CLOCK = new Clock();
private JobHistory jobHistory;
+
+ private final JobTokenSecretManager jobTokenSecretManager
+ = new JobTokenSecretManager();
+
+ JobTokenSecretManager getJobTokenSecretManager() {
+ return jobTokenSecretManager;
+ }
+ private MRAsyncDiskService asyncDiskService;
+
+ private String defaultStagingBaseDir;
+
/**
* A client tried to submit a job before the Job Tracker was ready.
*/
@@ -269,6 +285,8 @@
return RefreshAuthorizationPolicyProtocol.versionID;
} else if (protocol.equals(AdminOperationsProtocol.class.getName())){
return AdminOperationsProtocol.versionID;
+ } else if (protocol.equals(RefreshUserToGroupMappingsProtocol.class.getName())){
+ return RefreshUserToGroupMappingsProtocol.versionID;
} else {
throw new IOException("Unknown protocol to job tracker: " + protocol);
}
@@ -441,6 +459,7 @@
}
}
+ // Assumes JobTracker, taskTrackers and trackerExpiryQueue are locked on entry
private void removeTracker(TaskTracker tracker) {
lostTaskTracker(tracker);
String trackerName = tracker.getStatus().getTrackerName();
@@ -460,19 +479,10 @@
if (job != null) {
JobStatus status = job.getStatus();
- //set the historyfile and update the tracking url
- String trackingUrl = "";
+ //set the historyfile
if (historyFile != null) {
status.setHistoryFile(historyFile);
- try {
- trackingUrl = "http://" + getJobTrackerMachine() + ":" +
- getInfoPort() + "/jobdetailshistory.jsp?jobid=" +
- jobid + "&logFile=" + URLEncoder.encode(historyFile, "UTF-8");
- } catch (UnsupportedEncodingException e) {
- LOG.warn("Could not create trackingUrl", e);
- }
}
- status.setTrackingUrl(trackingUrl);
// clean up job files from the local disk
job.cleanupLocalizedJobConf(job.getProfile().getJobID());
@@ -639,6 +649,7 @@
* Increments faults(blacklist by job) for the tracker by one.
*
* Adds the tracker to the potentially faulty list.
+ * Assumes JobTracker is locked on the entry.
*
* @param hostName
*/
@@ -729,13 +740,17 @@
}
}
+ // Assumes JobTracker is locked on entry.
private FaultInfo getFaultInfo(String hostName,
boolean createIfNeccessary) {
- FaultInfo fi = potentiallyFaultyTrackers.get(hostName);
- long now = clock.getTime();
- if (fi == null && createIfNeccessary) {
- fi = new FaultInfo(now);
- potentiallyFaultyTrackers.put(hostName, fi);
+ FaultInfo fi = null;
+ synchronized (potentiallyFaultyTrackers) {
+ fi = potentiallyFaultyTrackers.get(hostName);
+ long now = clock.getTime();
+ if (fi == null && createIfNeccessary) {
+ fi = new FaultInfo(now);
+ potentiallyFaultyTrackers.put(hostName, fi);
+ }
}
return fi;
}
@@ -773,6 +788,8 @@
* Removes the tracker from blacklist and
* from potentially faulty list, when it is restarted.
*
+ * Assumes JobTracker is locked on the entry.
+ *
* @param hostName
*/
void markTrackerHealthy(String hostName) {
@@ -791,6 +808,7 @@
* One fault of the tracker is discarded if there
* are no faults during one day. So, the tracker will get a
* chance again to run tasks of a job.
+ * Assumes JobTracker is locked on the entry.
*
* @param hostName The tracker name
* @param now The current time
@@ -819,17 +837,21 @@
private void removeHostCapacity(String hostName) {
synchronized (taskTrackers) {
// remove the capacity of trackers on this host
+ int numTrackersOnHost = 0;
for (TaskTrackerStatus status : getStatusesOnHost(hostName)) {
int mapSlots = status.getMaxMapSlots();
totalMapTaskCapacity -= mapSlots;
int reduceSlots = status.getMaxReduceSlots();
totalReduceTaskCapacity -= reduceSlots;
+ ++numTrackersOnHost;
getInstrumentation().addBlackListedMapSlots(
mapSlots);
getInstrumentation().addBlackListedReduceSlots(
reduceSlots);
}
- incrBlackListedTrackers(uniqueHostsMap.remove(hostName));
+ // remove the host
+ uniqueHostsMap.remove(hostName);
+ incrBlackListedTrackers(numTrackersOnHost);
}
}
@@ -856,6 +878,7 @@
/**
* Whether a host is blacklisted across all the jobs.
*
+ * Assumes JobTracker is locked on the entry.
* @param hostName
* @return
*/
@@ -869,6 +892,7 @@
return false;
}
+ // Assumes JobTracker is locked on the entry.
int getFaultCount(String hostName) {
synchronized (potentiallyFaultyTrackers) {
FaultInfo fi = null;
@@ -879,6 +903,7 @@
return 0;
}
+ // Assumes JobTracker is locked on the entry.
Set<ReasonForBlackListing> getReasonForBlackListing(String hostName) {
synchronized (potentiallyFaultyTrackers) {
FaultInfo fi = null;
@@ -890,6 +915,7 @@
}
+ // Assumes JobTracker is locked on the entry.
void setNodeHealthStatus(String hostName, boolean isHealthy, String reason) {
FaultInfo fi = null;
// If tracker is not healthy, create a fault info object
@@ -947,6 +973,7 @@
/**
* Get all task tracker statuses on given host
*
+ * Assumes JobTracker is locked on the entry
* @param hostName
* @return {@link java.util.List} of {@link TaskTrackerStatus}
*/
@@ -996,36 +1023,10 @@
return jobsToRecover;
}
- /** Check if the given string represents a job-id or not
- */
- private boolean isJobNameValid(String str) {
- if(str == null) {
- return false;
- }
- String[] parts = str.split("_");
- if(parts.length == 3) {
- if(parts[0].equals("job")) {
- // other 2 parts should be parseable
- return JobTracker.validateIdentifier(parts[1])
- && JobTracker.validateJobNumber(parts[2]);
- }
- }
- return false;
- }
-
- // checks if the job dir has the required files
- public void checkAndAddJob(FileStatus status) throws IOException {
- String fileName = status.getPath().getName();
- if (isJobNameValid(fileName)) {
- if (JobClient.isJobDirValid(status.getPath(), fs)) {
- recoveryManager.addJobForRecovery(JobID.forName(fileName));
- shouldRecover = true; // enable actual recovery if num-files > 1
- } else {
- LOG.info("Found an incomplete job directory " + fileName + "."
- + " Deleting it!!");
- fs.delete(status.getPath(), true);
- }
- }
+ // add the job
+ void addJobForRecovery(FileStatus status) throws IOException {
+ recoveryManager.addJobForRecovery(JobID.forName(status.getPath().getName()));
+ shouldRecover = true; // enable actual recovery if num-files > 1
}
@@ -1132,7 +1133,16 @@
for (JobID jobId : jobsToRecover) {
LOG.info("Submitting job "+ jobId);
try {
- submitJob(jobId, restartCount);
+ Path jobInfoFile = getSystemFileForJob(jobId);
+ FSDataInputStream in = fs.open(jobInfoFile);
+ JobInfo token = new JobInfo();
+ token.readFields(in);
+ in.close();
+ UnixUserGroupInformation ugi = new UnixUserGroupInformation(
+ token.getUser().toString(),
+ new String[]{UnixUserGroupInformation.DEFAULT_GROUP});
+ submitJob(token.getJobID(), restartCount,
+ ugi, token.getJobSubmitDir().toString(), true, null);
recovered++;
} catch (Exception e) {
LOG.warn("Could not recover job " + jobId, e);
@@ -1300,6 +1310,26 @@
private final QueueManager queueManager;
+ //TO BE USED BY TEST CLASSES ONLY
+ //ONLY BUILD THE STATE WHICH IS REQUIRED BY TESTS
+ JobTracker() {
+ hostsReader = null;
+ retiredJobsCacheSize = 0;
+ infoServer = null;
+ queueManager = null;
+ supergroup = null;
+ taskScheduler = null;
+ trackerIdentifier = null;
+ recoveryManager = null;
+ jobHistory = null;
+ completedJobStatusStore = null;
+ tasktrackerExpiryInterval = 0;
+ myInstrumentation = new JobTrackerMetricsInst(this, new JobConf());
+ mrOwner = null;
+ defaultStagingBaseDir = "/Users";
+ }
+
+
JobTracker(JobConf conf)
throws IOException,InterruptedException, LoginException {
this(conf, new Clock());
@@ -1482,6 +1512,18 @@
if(systemDir == null) {
systemDir = new Path(getSystemDir());
}
+ try {
+ FileStatus systemDirStatus = fs.getFileStatus(systemDir);
+ if (!systemDirStatus.getOwner().equals(mrOwner.getUserName())) {
+ throw new AccessControlException("The systemdir " + systemDir +
+ " is not owned by " + mrOwner.getUserName());
+ }
+ if (!systemDirStatus.getPermission().equals(SYSTEM_DIR_PERMISSION)) {
+ LOG.warn("Incorrect permissions on " + systemDir +
+ ". Setting it to " + SYSTEM_DIR_PERMISSION);
+ fs.setPermission(systemDir, SYSTEM_DIR_PERMISSION);
+ }
+ } catch (FileNotFoundException fnf) {} //ignore
// Make sure that the backup data is preserved
FileStatus[] systemDirData;
try {
@@ -1496,7 +1538,7 @@
&& systemDirData != null) {
for (FileStatus status : systemDirData) {
try {
- recoveryManager.checkAndAddJob(status);
+ recoveryManager.addJobForRecovery(status);
} catch (Throwable t) {
LOG.warn("Failed to add the job " + status.getPath().getName(),
t);
@@ -1539,7 +1581,8 @@
}
// Same with 'localDir' except it's always on the local disk.
- jobConf.deleteLocalFiles(SUBDIR);
+ asyncDiskService = new MRAsyncDiskService(FileSystem.getLocal(conf), conf.getLocalDirs());
+ asyncDiskService.moveAndDeleteFromEachVolume(SUBDIR);
// Initialize history DONE folder
jobHistory.initDone(conf, fs);
@@ -1561,6 +1604,8 @@
synchronized (this) {
completedJobStatusStore = new CompletedJobStatusStore(conf);
}
+ Path homeDir = fs.getHomeDirectory();
+ defaultStagingBaseDir = homeDir.getParent().toString();
}
private static SimpleDateFormat getDateFormat() {
@@ -1860,8 +1905,7 @@
// and TaskInProgress
///////////////////////////////////////////////////////
void createTaskEntry(TaskAttemptID taskid, String taskTracker, TaskInProgress tip) {
- LOG.info("Adding task " +
- (tip.isCleanupAttempt(taskid) ? "(cleanup)" : "") +
+ LOG.info("Adding task (" + tip.getAttemptType(taskid) + ") " +
"'" + taskid + "' to tip " +
tip.getTIPId() + ", for tracker '" + taskTracker + "'");
@@ -1894,9 +1938,10 @@
}
// taskid --> TIP
- taskidToTIPMap.remove(taskid);
-
- LOG.debug("Removing task '" + taskid + "'");
+ if (taskidToTIPMap.remove(taskid) != null) {
+ // log the task removal in case of success
+ LOG.info("Removing task '" + taskid + "'");
+ }
}
/**
@@ -1925,7 +1970,7 @@
* @param job the completed job
*/
void markCompletedJob(JobInProgress job) {
- for (TaskInProgress tip : job.getSetupTasks()) {
+ for (TaskInProgress tip : job.getTasks(TaskType.JOB_SETUP)) {
for (TaskStatus taskStatus : tip.getTaskStatuses()) {
if (taskStatus.getRunState() != TaskStatus.State.RUNNING &&
taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING &&
@@ -1935,7 +1980,7 @@
}
}
}
- for (TaskInProgress tip : job.getMapTasks()) {
+ for (TaskInProgress tip : job.getTasks(TaskType.MAP)) {
for (TaskStatus taskStatus : tip.getTaskStatuses()) {
if (taskStatus.getRunState() != TaskStatus.State.RUNNING &&
taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING &&
@@ -1947,7 +1992,7 @@
}
}
}
- for (TaskInProgress tip : job.getReduceTasks()) {
+ for (TaskInProgress tip : job.getTasks(TaskType.REDUCE)) {
for (TaskStatus taskStatus : tip.getTaskStatuses()) {
if (taskStatus.getRunState() != TaskStatus.State.RUNNING &&
taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING &&
@@ -1975,8 +2020,10 @@
if (markedTaskSet != null) {
for (TaskAttemptID taskid : markedTaskSet) {
removeTaskEntry(taskid);
- LOG.info("Removed completed task '" + taskid + "' from '" +
- taskTracker + "'");
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Removed marked completed task '" + taskid + "' from '" +
+ taskTracker + "'");
+ }
}
// Clear
trackerToMarkedTasksMap.remove(taskTracker);
@@ -1990,15 +2037,16 @@
*
* @param job the job about to be 'retired'
*/
- synchronized private void removeJobTasks(JobInProgress job) {
- for (TaskInProgress tip : job.getMapTasks()) {
- for (TaskStatus taskStatus : tip.getTaskStatuses()) {
- removeTaskEntry(taskStatus.getTaskID());
- }
- }
- for (TaskInProgress tip : job.getReduceTasks()) {
- for (TaskStatus taskStatus : tip.getTaskStatuses()) {
- removeTaskEntry(taskStatus.getTaskID());
+ synchronized void removeJobTasks(JobInProgress job) {
+ // iterate over all the task types
+ for (TaskType type : TaskType.values()) {
+ // iterate over all the tips of the type under consideration
+ for (TaskInProgress tip : job.getTasks(type)) {
+ // iterate over all the task-ids in the tip under consideration
+ for (TaskAttemptID id : tip.getAllTaskAttemptIDs()) {
+ // remove the task-id entry from the jobtracker
+ removeTaskEntry(id);
+ }
}
}
}
@@ -2134,7 +2182,8 @@
*
* @return {@link Collection} of {@link TaskTrackerStatus}
*/
- public Collection<TaskTrackerStatus> taskTrackers() {
+ // lock to taskTrackers should hold JT lock first.
+ public synchronized Collection<TaskTrackerStatus> taskTrackers() {
Collection<TaskTrackerStatus> ttStatuses;
synchronized (taskTrackers) {
ttStatuses =
@@ -2151,7 +2200,10 @@
*
* @return {@link Collection} of active {@link TaskTrackerStatus}
*/
- public Collection<TaskTrackerStatus> activeTaskTrackers() {
+ // This method is synchronized to make sure that the locking order
+ // "taskTrackers lock followed by faultyTrackers.potentiallyFaultyTrackers
+ // lock" is under JobTracker lock to avoid deadlocks.
+ synchronized public Collection<TaskTrackerStatus> activeTaskTrackers() {
Collection<TaskTrackerStatus> activeTrackers =
new ArrayList<TaskTrackerStatus>();
synchronized (taskTrackers) {
@@ -2171,7 +2223,10 @@
* The second element in the returned list contains the list of blacklisted
* tracker names.
*/
- public List<List<String>> taskTrackerNames() {
+ // This method is synchronized to make sure that the locking order
+ // "taskTrackers lock followed by faultyTrackers.potentiallyFaultyTrackers
+ // lock" is under JobTracker lock to avoid deadlocks.
+ synchronized public List<List<String>> taskTrackerNames() {
List<String> activeTrackers =
new ArrayList<String>();
List<String> blacklistedTrackers =
@@ -2197,7 +2252,10 @@
*
* @return {@link Collection} of blacklisted {@link TaskTrackerStatus}
*/
- public Collection<TaskTrackerStatus> blacklistedTaskTrackers() {
+ // This method is synchronized to make sure that the locking order
+ // "taskTrackers lock followed by faultyTrackers.potentiallyFaultyTrackers
+ // lock" is under JobTracker lock to avoid deadlocks.
+ synchronized public Collection<TaskTrackerStatus> blacklistedTaskTrackers() {
Collection<TaskTrackerStatus> blacklistedTrackers =
new ArrayList<TaskTrackerStatus>();
synchronized (taskTrackers) {
@@ -2211,7 +2269,7 @@
return blacklistedTrackers;
}
- int getFaultCount(String hostName) {
+ synchronized int getFaultCount(String hostName) {
return faultyTrackers.getFaultCount(hostName);
}
@@ -2231,7 +2289,7 @@
*
* @return true if blacklisted, false otherwise
*/
- public boolean isBlacklisted(String trackerID) {
+ synchronized public boolean isBlacklisted(String trackerID) {
TaskTrackerStatus status = getTaskTrackerStatus(trackerID);
if (status != null) {
return faultyTrackers.isBlacklisted(status.getHost());
@@ -2239,7 +2297,8 @@
return false;
}
- public TaskTrackerStatus getTaskTrackerStatus(String trackerID) {
+ // lock to taskTrackers should hold JT lock first.
+ synchronized public TaskTrackerStatus getTaskTrackerStatus(String trackerID) {
TaskTracker taskTracker;
synchronized (taskTrackers) {
taskTracker = taskTrackers.get(trackerID);
@@ -2247,7 +2306,8 @@
return (taskTracker == null) ? null : taskTracker.getStatus();
}
- public TaskTracker getTaskTracker(String trackerID) {
+ // lock to taskTrackers should hold JT lock first.
+ synchronized public TaskTracker getTaskTracker(String trackerID) {
synchronized (taskTrackers) {
return taskTrackers.get(trackerID);
}
@@ -2260,7 +2320,7 @@
* Adds a new node to the jobtracker. It involves adding it to the expiry
* thread and adding it for resolution
*
- * Assuming trackerExpiryQueue is locked on entry
+ * Assumes JobTracker, taskTrackers and trackerExpiryQueue are locked on entry
*
* @param status Task Tracker's status
*/
@@ -2596,12 +2656,14 @@
taskTrackers.remove(trackerName);
Integer numTaskTrackersInHost =
uniqueHostsMap.get(oldStatus.getHost());
- numTaskTrackersInHost --;
- if (numTaskTrackersInHost > 0) {
- uniqueHostsMap.put(oldStatus.getHost(), numTaskTrackersInHost);
- }
- else {
- uniqueHostsMap.remove(oldStatus.getHost());
+ if (numTaskTrackersInHost != null) {
+ numTaskTrackersInHost --;
+ if (numTaskTrackersInHost > 0) {
+ uniqueHostsMap.put(oldStatus.getHost(), numTaskTrackersInHost);
+ }
+ else {
+ uniqueHostsMap.remove(oldStatus.getHost());
+ }
}
}
}
@@ -2987,8 +3049,9 @@
* the JobTracker alone.
*/
public synchronized org.apache.hadoop.mapreduce.JobStatus submitJob(
- org.apache.hadoop.mapreduce.JobID jobId) throws IOException {
- return submitJob(JobID.downgrade(jobId));
+ org.apache.hadoop.mapreduce.JobID jobId,String jobSubmitDir, TokenStorage ts)
+ throws IOException {
+ return submitJob(JobID.downgrade(jobId), jobSubmitDir, ts);
}
/**
@@ -2999,46 +3062,53 @@
* of the JobTracker. But JobInProgress adds info that's useful for
* the JobTracker alone.
* @deprecated Use
- * {@link #submitJob(org.apache.hadoop.mapreduce.JobID)} instead
+ * {@link #submitJob(org.apache.hadoop.mapreduce.JobID, String, TokenStorage)}
+ * instead
*/
@Deprecated
- public synchronized JobStatus submitJob(JobID jobId) throws IOException {
- verifyServiceState(ServiceState.LIVE);
- return submitJob(jobId, 0);
+ public synchronized JobStatus submitJob(
+ JobID jobId, String jobSubmitDir, TokenStorage ts)
+ throws IOException {
+ return submitJob(jobId, 0,
+ UserGroupInformation.getCurrentUGI(),
+ jobSubmitDir, false, ts);
}
/**
* Submits either a new job or a job from an earlier run.
*/
- private synchronized JobStatus submitJob(JobID jobId,
- int restartCount) throws IOException {
+ private synchronized JobStatus submitJob(org.apache.hadoop.mapreduce.JobID jobID,
+ int restartCount, UserGroupInformation ugi, String jobSubmitDir,
+ boolean recovered, TokenStorage ts) throws IOException {
+ verifyServiceState(ServiceState.LIVE);
+ JobID jobId = JobID.downgrade(jobID);
if(jobs.containsKey(jobId)) {
//job already running, don't start twice
return jobs.get(jobId).getStatus();
}
-
- JobInProgress job = new JobInProgress(jobId, this, this.conf, restartCount);
+
+ //the conversion from String to Text for the UGI's username will
+ //not be required when we have the UGI to return us the username as
+ //Text.
+ JobInfo jobInfo = new JobInfo(jobId, new Text(ugi.getUserName()),
+ new Path(jobSubmitDir));
+ JobInProgress job =
+ new JobInProgress(this, this.conf, restartCount, jobInfo, ts);
String queue = job.getProfile().getQueueName();
if(!(queueManager.getLeafQueueNames().contains(queue))) {
- new CleanupQueue().addToQueue(fs, getSystemDirectoryForJob(jobId));
throw new IOException("Queue \"" + queue + "\" does not exist");
}
//check if queue is RUNNING
if(!queueManager.isRunning(queue)) {
- new CleanupQueue().addToQueue(fs, getSystemDirectoryForJob(jobId));
throw new IOException("Queue \"" + queue + "\" is not running");
}
try {
- // check for access
- UserGroupInformation ugi =
- UserGroupInformation.readFrom(job.getJobConf());
checkAccess(job, Queue.QueueOperation.SUBMIT_JOB, ugi);
} catch (IOException ioe) {
- LOG.warn("Access denied for user " + job.getJobConf().getUser()
- + ". Ignoring job " + jobId, ioe);
- new CleanupQueue().addToQueue(fs, getSystemDirectoryForJob(jobId));
+ LOG.warn("Access denied for user " + job.getJobConf().getUser()
+ + ". Ignoring job " + jobId, ioe);
throw ioe;
}
@@ -3047,11 +3117,19 @@
try {
checkMemoryRequirements(job);
} catch (IOException ioe) {
- new CleanupQueue().addToQueue(fs, getSystemDirectoryForJob(jobId));
throw ioe;
}
- return addJob(jobId, job);
+ if (!recovered) {
+ //Store the information in a file so that the job can be recovered
+ //later (if at all)
+ Path jobDir = getSystemDirectoryForJob(jobId);
+ FileSystem.mkdirs(fs, jobDir, new FsPermission(SYSTEM_DIR_PERMISSION));
+ FSDataOutputStream out = fs.create(getSystemFileForJob(jobId));
+ jobInfo.write(out);
+ out.close();
+ }
+ return addJob(jobId, job);
}
/**
@@ -3076,6 +3154,9 @@
}
}
myInstrumentation.submitJob(job.getJobConf(), jobId);
+ LOG.info("Job " + jobId + " added successfully for user '"
+ + job.getJobConf().getUser() + "' to queue '"
+ + job.getJobConf().getQueueName() + "'");
return job.getStatus();
}
@@ -3726,6 +3807,17 @@
}
/**
+ * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getStagingAreaDir()
+ */
+ public String getStagingAreaDir() {
+ Path stagingRootDir = new Path(conf.get(JTConfig.JT_STAGING_AREA_ROOT,
+ defaultStagingBaseDir));
+ String user = UserGroupInformation.getCurrentUGI().getUserName();
+ return fs.makeQualified(new Path(stagingRootDir,
+ user+"/.staging")).toString();
+ }
+
+ /**
* @see
* org.apache.hadoop.mapreduce.protocol.ClientProtocol#getJobHistoryDir()
*/
@@ -3740,10 +3832,15 @@
return jobs.get(jobid);
}
- // Get the job directory in system directory
+ //Get the job directory in system directory
Path getSystemDirectoryForJob(JobID id) {
return new Path(getSystemDir(), id.toString());
}
+
+ //Get the job token file in system directory
+ Path getSystemFileForJob(JobID id) {
+ return new Path(getSystemDirectoryForJob(id)+"/" + JOB_INFO_FILE);
+ }
/**
* Change the run-time priority of the given job.
@@ -3980,8 +4077,8 @@
Set<TaskTracker> trackers = hostnameToTaskTracker.remove(host);
if (trackers != null) {
for (TaskTracker tracker : trackers) {
- LOG.info("Decommission: Losing tracker " + tracker +
- " on host " + host);
+ LOG.info("Decommission: Losing tracker "
+ + tracker.getTrackerName() + " on host " + host);
removeTracker(tracker);
}
trackersDecommissioned += trackers.size();
@@ -4105,7 +4202,9 @@
@Override
public QueueInfo getQueue(String queue) throws IOException {
JobQueueInfo jqueue = queueManager.getJobQueueInfo(queue);
- jqueue.setJobStatuses(getJobsFromQueue(jqueue.getQueueName()));
+ if (jqueue != null) {
+ jqueue.setJobStatuses(getJobsFromQueue(jqueue.getQueueName()));
+ }
return jqueue;
}
@@ -4221,6 +4320,15 @@
limitMaxMemForReduceTasks).append(")"));
}
+
+ @Override
+ public void refreshUserToGroupsMappings(Configuration conf) throws IOException {
+ LOG.info("Refreshing all user-to-groups mappings. Requested by user: " +
+ UserGroupInformation.getCurrentUGI().getUserName());
+
+ SecurityUtil.getUserToGroupsMappingService(conf).refresh();
+ }
+
private boolean perTaskMemoryConfigurationSetOnJT() {
if (limitMaxMemForMapTasks == JobConf.DISABLED_MEMORY_LIMIT
|| limitMaxMemForReduceTasks == JobConf.DISABLED_MEMORY_LIMIT
@@ -4273,7 +4381,7 @@
}
}
- String getFaultReport(String host) {
+ synchronized String getFaultReport(String host) {
FaultInfo fi = faultyTrackers.getFaultInfo(host, false);
if (fi == null) {
return "";
@@ -4281,7 +4389,7 @@
return fi.getTrackerFaultReport();
}
- Set<ReasonForBlackListing> getReasonForBlackList(String host) {
+ synchronized Set<ReasonForBlackListing> getReasonForBlackList(String host) {
FaultInfo fi = faultyTrackers.getFaultInfo(host, false);
if (fi == null) {
return new HashSet<ReasonForBlackListing>();
@@ -4289,7 +4397,7 @@
return fi.getReasonforblacklisting();
}
- Collection<BlackListInfo> getBlackListedTrackers() {
+ synchronized Collection<BlackListInfo> getBlackListedTrackers() {
Collection<BlackListInfo> blackListedTrackers =
new ArrayList<BlackListInfo>();
for(TaskTrackerStatus tracker : blacklistedTaskTrackers()) {
@@ -4314,9 +4422,12 @@
return blackListedTrackers;
}
- /** Test method to increment the fault*/
-
- void incrementFaults(String hostName) {
+ /** Test method to increment the fault
+ * This method is synchronized to make sure that the locking order
+ * "faultyTrackers.potentiallyFaultyTrackers lock followed by taskTrackers
+ * lock" is under JobTracker lock to avoid deadlocks.
+ */
+ synchronized void incrementFaults(String hostName) {
faultyTrackers.incrementFaults(hostName);
}
@@ -4419,6 +4530,8 @@
//initializes the job status store
completedJobStatusStore = new CompletedJobStatusStore(conf);
+ Path homeDir = fs.getHomeDirectory();
+ defaultStagingBaseDir = homeDir.getParent().toString();
}
/**
diff --git a/src/java/org/apache/hadoop/mapred/JvmManager.java b/src/java/org/apache/hadoop/mapred/JvmManager.java
index 058e22d..142d64b 100644
--- a/src/java/org/apache/hadoop/mapred/JvmManager.java
+++ b/src/java/org/apache/hadoop/mapred/JvmManager.java
@@ -30,11 +30,11 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.mapred.TaskController.TaskControllerContext;
import org.apache.hadoop.mapred.TaskTracker.TaskInProgress;
import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.mapreduce.util.ProcessTree;
class JvmManager {
@@ -136,6 +136,14 @@
}
}
+ void dumpStack(TaskRunner tr) {
+ if (tr.getTask().isMapTask()) {
+ mapJvmManager.dumpStack(tr);
+ } else {
+ reduceJvmManager.dumpStack(tr);
+ }
+ }
+
public void killJvm(JVMId jvmId) {
if (jvmId.isMap) {
mapJvmManager.killJvm(jvmId);
@@ -144,6 +152,22 @@
}
}
+ /**
+ * Adds the task's work dir to the cleanup queue of taskTracker for
+ * asynchronous deletion of work dir.
+ * @param tracker taskTracker
+ * @param task the task whose work dir needs to be deleted
+ * @throws IOException
+ */
+ static void deleteWorkDir(TaskTracker tracker, Task task) throws IOException {
+ tracker.getCleanupThread().addToQueue(
+ TaskTracker.buildTaskControllerPathDeletionContexts(
+ tracker.getLocalFileSystem(),
+ tracker.getLocalFiles(tracker.getJobConf(), ""),
+ task, true /* workDir */,
+ tracker.getTaskController()));
+ }
+
private static class JvmManagerForType {
//Mapping from the JVM IDs to running Tasks
Map <JVMId,TaskRunner> jvmToRunningTask =
@@ -243,6 +267,16 @@
}
}
+ synchronized void dumpStack(TaskRunner tr) {
+ JVMId jvmId = runningTaskToJvm.get(tr);
+ if (null != jvmId) {
+ JvmRunner jvmRunner = jvmIdToRunner.get(jvmId);
+ if (null != jvmRunner) {
+ jvmRunner.dumpChildStacks();
+ }
+ }
+ }
+
synchronized public void stop() {
//since the kill() method invoked later on would remove
//an entry from the jvmIdToRunner map, we create a
@@ -428,7 +462,7 @@
//task at the beginning of each task in the task JVM.
//For the last task, we do it here.
if (env.conf.getNumTasksToExecutePerJvm() != 1) {
- FileUtil.fullyDelete(env.workDir);
+ deleteWorkDir(tracker, initalContext.task);
}
} catch (IOException ie){}
}
@@ -459,7 +493,38 @@
removeJvm(jvmId);
}
}
-
+
+ /** Send a signal to the JVM requesting that it dump a stack trace,
+ * and wait for a timeout interval to give this signal time to be
+ * processed.
+ */
+ void dumpChildStacks() {
+ if (!killed) {
+ TaskController controller = tracker.getTaskController();
+ // Check inital context before issuing a signal to prevent situations
+ // where signal is issued before task is launched.
+ if (initalContext != null && initalContext.env != null) {
+ initalContext.pid = jvmIdToPid.get(jvmId);
+ initalContext.sleeptimeBeforeSigkill = tracker.getJobConf()
+ .getLong(TTConfig.TT_SLEEP_TIME_BEFORE_SIG_KILL,
+ ProcessTree.DEFAULT_SLEEPTIME_BEFORE_SIGKILL);
+
+ // signal the task jvm
+ controller.dumpTaskStack(initalContext);
+
+ // We're going to kill the jvm with SIGKILL after this,
+ // so we should wait for a few seconds first to ensure that
+ // the SIGQUIT has time to be processed.
+ try {
+ Thread.sleep(initalContext.sleeptimeBeforeSigkill);
+ } catch (InterruptedException e) {
+ LOG.warn("Sleep interrupted : " +
+ StringUtils.stringifyException(e));
+ }
+ }
+ }
+ }
+
public void taskRan() {
busy = false;
numTasksRan++;
diff --git a/src/java/org/apache/hadoop/mapred/LinuxTaskController.java b/src/java/org/apache/hadoop/mapred/LinuxTaskController.java
index de434b0..1505233 100644
--- a/src/java/org/apache/hadoop/mapred/LinuxTaskController.java
+++ b/src/java/org/apache/hadoop/mapred/LinuxTaskController.java
@@ -29,7 +29,9 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.mapred.CleanupQueue.PathDeletionContext;
import org.apache.hadoop.mapred.JvmManager.JvmEnv;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
@@ -83,12 +85,14 @@
enum TaskCommands {
INITIALIZE_USER,
INITIALIZE_JOB,
- INITIALIZE_DISTRIBUTEDCACHE,
+ INITIALIZE_DISTRIBUTEDCACHE_FILE,
LAUNCH_TASK_JVM,
INITIALIZE_TASK,
TERMINATE_TASK_JVM,
KILL_TASK_JVM,
RUN_DEBUG_SCRIPT,
+ SIGQUIT_TASK_JVM,
+ ENABLE_TASK_FOR_CLEANUP
}
/**
@@ -228,12 +232,75 @@
@Override
void initializeTask(TaskControllerContext context)
throws IOException {
- LOG.debug("Going to do " + TaskCommands.INITIALIZE_TASK.toString()
- + " for " + context.task.getTaskID().toString());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Going to do " + TaskCommands.INITIALIZE_TASK.toString()
+ + " for " + context.task.getTaskID().toString());
+ }
runCommand(TaskCommands.INITIALIZE_TASK, context.env.conf.getUser(),
buildInitializeTaskArgs(context), context.env.workDir, context.env.env);
}
+ /**
+ * Builds the args to be passed to task-controller for enabling of task for
+ * cleanup. Last arg in this List is either $attemptId or $attemptId/work
+ */
+ private List<String> buildTaskCleanupArgs(
+ TaskControllerPathDeletionContext context) {
+ List<String> commandArgs = new ArrayList<String>(3);
+ commandArgs.add(context.mapredLocalDir.toUri().getPath());
+ commandArgs.add(context.task.getJobID().toString());
+
+ String workDir = "";
+ if (context.isWorkDir) {
+ workDir = "/work";
+ }
+ if (context.task.isTaskCleanupTask()) {
+ commandArgs.add(context.task.getTaskID() + TaskTracker.TASK_CLEANUP_SUFFIX
+ + workDir);
+ } else {
+ commandArgs.add(context.task.getTaskID() + workDir);
+ }
+
+ return commandArgs;
+ }
+
+ /**
+ * Enables the task for cleanup by changing permissions of the specified path
+ * in the local filesystem
+ */
+ @Override
+ void enableTaskForCleanup(PathDeletionContext context)
+ throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Going to do " + TaskCommands.ENABLE_TASK_FOR_CLEANUP.toString()
+ + " for " + context.fullPath);
+ }
+
+ if (context instanceof TaskControllerPathDeletionContext) {
+ TaskControllerPathDeletionContext tContext =
+ (TaskControllerPathDeletionContext) context;
+
+ if (tContext.task.getUser() != null &&
+ tContext.fs instanceof LocalFileSystem) {
+ try {
+ runCommand(TaskCommands.ENABLE_TASK_FOR_CLEANUP,
+ tContext.task.getUser(),
+ buildTaskCleanupArgs(tContext), null, null);
+ } catch(IOException e) {
+ LOG.warn("Uanble to change permissions for " + tContext.fullPath);
+ }
+ }
+ else {
+ throw new IllegalArgumentException("Either user is null or the " +
+ "file system is not local file system.");
+ }
+ }
+ else {
+ throw new IllegalArgumentException("PathDeletionContext provided is not "
+ + "TaskControllerPathDeletionContext.");
+ }
+ }
+
private void logOutput(String output) {
String shExecOutput = output;
if (shExecOutput != null) {
@@ -408,12 +475,21 @@
}
@Override
- public void initializeDistributedCache(InitializationContext context)
+ public void initializeDistributedCacheFile(DistributedCacheFileContext context)
throws IOException {
- LOG.debug("Going to initialize distributed cache for " + context.user
- + " on the TT");
- runCommand(TaskCommands.INITIALIZE_DISTRIBUTEDCACHE, context.user,
- new ArrayList<String>(), context.workDir, null);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Going to initialize distributed cache for " + context.user
+ + " with localizedBaseDir " + context.localizedBaseDir +
+ " and uniqueString " + context.uniqueString);
+ }
+ List<String> args = new ArrayList<String>();
+ // Here, uniqueString might start with '-'. Adding -- in front of the
+ // arguments indicates that they are non-option parameters.
+ args.add("--");
+ args.add(context.localizedBaseDir.toString());
+ args.add(context.uniqueString);
+ runCommand(TaskCommands.INITIALIZE_DISTRIBUTEDCACHE_FILE, context.user,
+ args, context.workDir, null);
}
@Override
@@ -443,21 +519,22 @@
}
/**
- * Convenience method used to sending appropriate Kill signal to the task
+ * Convenience method used to sending appropriate signal to the task
* VM
* @param context
* @param command
* @throws IOException
*/
- private void finishTask(TaskControllerContext context,
+ protected void signalTask(TaskControllerContext context,
TaskCommands command) throws IOException{
if(context.task == null) {
- LOG.info("Context task null not killing the JVM");
+ LOG.info("Context task is null; not signaling the JVM");
return;
}
ShellCommandExecutor shExec = buildTaskControllerExecutor(
command, context.env.conf.getUser(),
- buildKillTaskCommandArgs(context), context.env.workDir, context.env.env);
+ buildKillTaskCommandArgs(context), context.env.workDir,
+ context.env.env);
try {
shExec.execute();
} catch (Exception e) {
@@ -469,7 +546,7 @@
@Override
void terminateTask(TaskControllerContext context) {
try {
- finishTask(context, TaskCommands.TERMINATE_TASK_JVM);
+ signalTask(context, TaskCommands.TERMINATE_TASK_JVM);
} catch (Exception e) {
LOG.warn("Exception thrown while sending kill to the Task VM " +
StringUtils.stringifyException(e));
@@ -479,13 +556,23 @@
@Override
void killTask(TaskControllerContext context) {
try {
- finishTask(context, TaskCommands.KILL_TASK_JVM);
+ signalTask(context, TaskCommands.KILL_TASK_JVM);
} catch (Exception e) {
LOG.warn("Exception thrown while sending destroy to the Task VM " +
StringUtils.stringifyException(e));
}
}
+ @Override
+ void dumpTaskStack(TaskControllerContext context) {
+ try {
+ signalTask(context, TaskCommands.SIGQUIT_TASK_JVM);
+ } catch (Exception e) {
+ LOG.warn("Exception thrown while sending SIGQUIT to the Task VM " +
+ StringUtils.stringifyException(e));
+ }
+ }
+
protected String getTaskControllerExecutablePath() {
return taskControllerExe;
}
diff --git a/src/java/org/apache/hadoop/mapred/LocalJobRunner.java b/src/java/org/apache/hadoop/mapred/LocalJobRunner.java
index 9d77763..3361bad 100644
--- a/src/java/org/apache/hadoop/mapred/LocalJobRunner.java
+++ b/src/java/org/apache/hadoop/mapred/LocalJobRunner.java
@@ -23,32 +23,32 @@
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.HashMap;
-import java.util.List;
import java.util.Map;
+import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Job.RawSplit;
-import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
import org.apache.hadoop.mapreduce.ClusterMetrics;
+import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.QueueInfo;
import org.apache.hadoop.mapreduce.TaskCompletionEvent;
import org.apache.hadoop.mapreduce.TaskTrackerInfo;
-import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.mapreduce.filecache.TaskDistributedCacheManager;
import org.apache.hadoop.mapreduce.filecache.TrackerDistributedCacheManager;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.mapreduce.security.TokenCache;
+import org.apache.hadoop.mapreduce.security.TokenStorage;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.mapreduce.server.jobtracker.State;
-import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
+import org.apache.hadoop.security.UserGroupInformation;
/** Implements MapReduce locally, in-process, for debugging. */
public class LocalJobRunner implements ClientProtocol {
@@ -60,7 +60,8 @@
private JobConf conf;
private int map_tasks = 0;
private int reduce_tasks = 0;
-
+ final Random rand = new Random();
+
private JobTrackerInstrumentation myMetrics = null;
private static final String jobDir = "localRunner/";
@@ -68,33 +69,6 @@
public long getProtocolVersion(String protocol, long clientVersion) {
return ClientProtocol.versionID;
}
-
- @SuppressWarnings("unchecked")
- static RawSplit[] getRawSplits(JobContext jContext, JobConf job)
- throws Exception {
- JobConf jobConf = jContext.getJobConf();
- org.apache.hadoop.mapreduce.InputFormat<?,?> input =
- ReflectionUtils.newInstance(jContext.getInputFormatClass(), jobConf);
-
- List<org.apache.hadoop.mapreduce.InputSplit> splits = input.getSplits(jContext);
- RawSplit[] rawSplits = new RawSplit[splits.size()];
- DataOutputBuffer buffer = new DataOutputBuffer();
- SerializationFactory factory = new SerializationFactory(jobConf);
- Serializer serializer =
- factory.getSerializer(splits.get(0).getClass());
- serializer.open(buffer);
- for (int i = 0; i < splits.size(); i++) {
- buffer.reset();
- serializer.serialize(splits.get(i));
- RawSplit rawSplit = new RawSplit();
- rawSplit.setClassName(splits.get(i).getClass().getName());
- rawSplit.setDataLength(splits.get(i).getLength());
- rawSplit.setBytes(buffer.getData(), 0, buffer.getLength());
- rawSplit.setLocations(splits.get(i).getLocations());
- rawSplits[i] = rawSplit;
- }
- return rawSplits;
- }
private class Job extends Thread implements TaskUmbilicalProtocol {
// The job directory on the system: JobClient places job configurations here.
@@ -130,8 +104,8 @@
return TaskUmbilicalProtocol.versionID;
}
- public Job(JobID jobid) throws IOException {
- this.systemJobDir = new Path(getSystemDir(), jobid.toString());
+ public Job(JobID jobid, String jobSubmitDir) throws IOException {
+ this.systemJobDir = new Path(jobSubmitDir);
this.systemJobFile = new Path(systemJobDir, "job.xml");
this.id = jobid;
JobConf conf = new JobConf(systemJobFile);
@@ -142,13 +116,13 @@
// Manage the distributed cache. If there are files to be copied,
// this will trigger localFile to be re-written again.
this.trackerDistributerdCacheManager =
- new TrackerDistributedCacheManager(conf);
+ new TrackerDistributedCacheManager(conf, new DefaultTaskController());
this.taskDistributedCacheManager =
trackerDistributerdCacheManager.newTaskDistributedCacheManager(conf);
taskDistributedCacheManager.setup(
new LocalDirAllocator(MRConfig.LOCAL_DIR),
new File(systemJobDir.toString()),
- "archive");
+ "archive", "archive");
if (DistributedCache.getSymlink(conf)) {
// This is not supported largely because,
@@ -202,26 +176,10 @@
JobContext jContext = new JobContextImpl(job, jobId);
OutputCommitter outputCommitter = job.getOutputCommitter();
try {
- // split input into minimum number of splits
- RawSplit[] rawSplits;
- if (job.getUseNewMapper()) {
- rawSplits = getRawSplits(jContext, job);
- } else {
- InputSplit[] splits = job.getInputFormat().getSplits(job, 1);
- rawSplits = new RawSplit[splits.length];
- DataOutputBuffer buffer = new DataOutputBuffer();
- for (int i = 0; i < splits.length; i++) {
- buffer.reset();
- splits[i].write(buffer);
- RawSplit rawSplit = new RawSplit();
- rawSplit.setClassName(splits[i].getClass().getName());
- rawSplit.setDataLength(splits[i].getLength());
- rawSplit.setBytes(buffer.getData(), 0, buffer.getLength());
- rawSplit.setLocations(splits[i].getLocations());
- rawSplits[i] = rawSplit;
- }
- }
+ TaskSplitMetaInfo[] taskSplitMetaInfos =
+ SplitMetaInfoReader.readSplitMetaInfo(jobId, localFs, conf, systemJobDir);
+
int numReduceTasks = job.getNumReduceTasks();
if (numReduceTasks > 1 || numReduceTasks < 0) {
// we only allow 0 or 1 reducer in local mode
@@ -233,15 +191,14 @@
Map<TaskAttemptID, MapOutputFile> mapOutputFiles =
new HashMap<TaskAttemptID, MapOutputFile>();
- for (int i = 0; i < rawSplits.length; i++) {
+ for (int i = 0; i < taskSplitMetaInfos.length; i++) {
if (!this.isInterrupted()) {
TaskAttemptID mapId = new TaskAttemptID(
new TaskID(jobId, TaskType.MAP, i),0);
mapIds.add(mapId);
MapTask map = new MapTask(systemJobFile.toString(),
mapId, i,
- rawSplits[i].getClassName(),
- rawSplits[i].getBytes(), 1);
+ taskSplitMetaInfos[i].getSplitIndex(), 1);
JobConf localConf = new JobConf(job);
TaskRunner.setupChildMapredLocalDirs(map, localConf);
@@ -459,9 +416,10 @@
}
public org.apache.hadoop.mapreduce.JobStatus submitJob(
- org.apache.hadoop.mapreduce.JobID jobid)
+ org.apache.hadoop.mapreduce.JobID jobid, String jobSubmitDir, TokenStorage ts)
throws IOException {
- return new Job(JobID.downgrade(jobid)).status;
+ TokenCache.setTokenStorage(ts);
+ return new Job(JobID.downgrade(jobid), jobSubmitDir).status;
}
public void killJob(org.apache.hadoop.mapreduce.JobID id) {
@@ -564,6 +522,22 @@
return fs.makeQualified(sysDir).toString();
}
+ /**
+ * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getStagingAreaDir()
+ */
+ public String getStagingAreaDir() {
+ Path stagingRootDir = new Path(conf.get(JTConfig.JT_STAGING_AREA_ROOT,
+ "/tmp/hadoop/mapred/staging"));
+ UserGroupInformation ugi = UserGroupInformation.getCurrentUGI();
+ String user;
+ if (ugi != null) {
+ user = ugi.getUserName() + rand.nextInt();
+ } else {
+ user = "dummy" + rand.nextInt();
+ }
+ return fs.makeQualified(new Path(stagingRootDir, user+"/.staging")).toString();
+ }
+
public String getJobHistoryDir() {
return null;
}
diff --git a/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java b/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java
index 41b958b..23a0118 100644
--- a/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java
+++ b/src/java/org/apache/hadoop/mapred/MapReducePolicyProvider.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.mapred;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.security.RefreshUserToGroupMappingsProtocol;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
import org.apache.hadoop.security.authorize.Service;
@@ -36,6 +37,8 @@
TaskUmbilicalProtocol.class),
new Service("security.refresh.policy.protocol.acl",
RefreshAuthorizationPolicyProtocol.class),
+ new Service("security.refresh.usertogroups.mappings.protocol.acl",
+ RefreshUserToGroupMappingsProtocol.class),
};
@Override
diff --git a/src/java/org/apache/hadoop/mapred/MapTask.java b/src/java/org/apache/hadoop/mapred/MapTask.java
index 61256a0..e86b2c5 100644
--- a/src/java/org/apache/hadoop/mapred/MapTask.java
+++ b/src/java/org/apache/hadoop/mapred/MapTask.java
@@ -33,6 +33,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
@@ -43,6 +44,7 @@
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.DefaultCodec;
@@ -54,6 +56,10 @@
import org.apache.hadoop.mapred.SortedRanges.SkipRangeIterator;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
+import org.apache.hadoop.mapreduce.split.JobSplit;
+import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
import org.apache.hadoop.mapreduce.task.MapContextImpl;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
@@ -72,7 +78,7 @@
*/
public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
- private BytesWritable split = new BytesWritable();
+ private TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
private String splitClass;
private final static int APPROX_HEADER_LENGTH = 150;
@@ -81,7 +87,6 @@
private Progress mapPhase;
private Progress sortPhase;
-
{ // set phase for this task
setPhase(TaskStatus.Phase.MAP);
getProgress().setStatus("map");
@@ -92,11 +97,10 @@
}
public MapTask(String jobFile, TaskAttemptID taskId,
- int partition, String splitClass, BytesWritable split,
+ int partition, TaskSplitIndex splitIndex,
int numSlotsRequired) {
super(jobFile, taskId, partition, numSlotsRequired);
- this.splitClass = splitClass;
- this.split = split;
+ this.splitMetaInfo = splitIndex;
}
@Override
@@ -108,26 +112,26 @@
public void localizeConfiguration(JobConf conf)
throws IOException {
super.localizeConfiguration(conf);
- // split.dta file is used only by IsolationRunner.
+ // split.dta/split.info files are used only by IsolationRunner.
// Write the split file to the local disk if it is a normal map task (not a
// job-setup or a job-cleanup task) and if the user wishes to run
// IsolationRunner either by setting keep.failed.tasks.files to true or by
// using keep.tasks.files.pattern
- if (isMapOrReduce()
- && (conf.getKeepTaskFilesPattern() != null || conf
- .getKeepFailedTaskFiles())) {
- Path localSplit =
+ if (supportIsolationRunner(conf) && isMapOrReduce()) {
+ // localize the split meta-information
+ Path localSplitMeta =
new LocalDirAllocator(MRConfig.LOCAL_DIR).getLocalPathForWrite(
- TaskTracker.getLocalSplitFile(conf.getUser(), getJobID()
- .toString(), getTaskID().toString()), conf);
- LOG.debug("Writing local split to " + localSplit);
- DataOutputStream out = FileSystem.getLocal(conf).create(localSplit);
- Text.writeString(out, splitClass);
- split.write(out);
+ TaskTracker.getLocalSplitMetaFile(conf.getUser(),
+ getJobID().toString(), getTaskID()
+ .toString()), conf);
+ LOG.debug("Writing local split to " + localSplitMeta);
+ DataOutputStream out = FileSystem.getLocal(conf).create(localSplitMeta);
+ splitMetaInfo.write(out);
out.close();
}
}
+
@Override
public TaskRunner createRunner(TaskTracker tracker,
TaskTracker.TaskInProgress tip) {
@@ -138,9 +142,8 @@
public void write(DataOutput out) throws IOException {
super.write(out);
if (isMapOrReduce()) {
- Text.writeString(out, splitClass);
- split.write(out);
- split = null;
+ splitMetaInfo.write(out);
+ splitMetaInfo = null;
}
}
@@ -148,8 +151,7 @@
public void readFields(DataInput in) throws IOException {
super.readFields(in);
if (isMapOrReduce()) {
- splitClass = Text.readString(in);
- split.readFields(in);
+ splitMetaInfo.readFields(in);
}
}
@@ -320,36 +322,52 @@
}
if (useNewApi) {
- runNewMapper(job, split, umbilical, reporter);
+ runNewMapper(job, splitMetaInfo, umbilical, reporter);
} else {
- runOldMapper(job, split, umbilical, reporter);
+ runOldMapper(job, splitMetaInfo, umbilical, reporter);
}
done(umbilical, reporter);
}
+ @SuppressWarnings("unchecked")
+ private <T> T getSplitDetails(Path file, long offset)
+ throws IOException {
+ FileSystem fs = file.getFileSystem(conf);
+ FSDataInputStream inFile = fs.open(file);
+ inFile.seek(offset);
+ String className = Text.readString(inFile);
+ Class<T> cls;
+ try {
+ cls = (Class<T>) conf.getClassByName(className);
+ } catch (ClassNotFoundException ce) {
+ IOException wrap = new IOException("Split class " + className +
+ " not found");
+ wrap.initCause(ce);
+ throw wrap;
+ }
+ SerializationFactory factory = new SerializationFactory(conf);
+ Deserializer<T> deserializer =
+ (Deserializer<T>) factory.getDeserializer(cls);
+ deserializer.open(inFile);
+ T split = deserializer.deserialize(null);
+ long pos = inFile.getPos();
+ getCounters().findCounter(
+ TaskCounter.SPLIT_RAW_BYTES).increment(pos - offset);
+ inFile.close();
+ return split;
+ }
+
@SuppressWarnings("unchecked")
private <INKEY,INVALUE,OUTKEY,OUTVALUE>
void runOldMapper(final JobConf job,
- final BytesWritable rawSplit,
+ final TaskSplitIndex splitIndex,
final TaskUmbilicalProtocol umbilical,
TaskReporter reporter
) throws IOException, InterruptedException,
ClassNotFoundException {
- InputSplit inputSplit = null;
- // reinstantiate the split
- try {
- inputSplit = (InputSplit)
- ReflectionUtils.newInstance(job.getClassByName(splitClass), job);
- } catch (ClassNotFoundException exp) {
- IOException wrap = new IOException("Split class " + splitClass +
- " not found");
- wrap.initCause(exp);
- throw wrap;
- }
- DataInputBuffer splitBuffer = new DataInputBuffer();
- splitBuffer.reset(split.getBytes(), 0, split.getLength());
- inputSplit.readFields(splitBuffer);
-
+ InputSplit inputSplit = getSplitDetails(new Path(splitIndex.getSplitLocation()),
+ splitIndex.getStartOffset());
+
updateJobWithSplit(job, inputSplit);
reporter.setInputSplit(inputSplit);
@@ -465,7 +483,7 @@
@SuppressWarnings("unchecked")
OldOutputCollector(MapOutputCollector<K,V> collector, JobConf conf) {
numPartitions = conf.getNumReduceTasks();
- if (numPartitions > 0) {
+ if (numPartitions > 1) {
partitioner = (Partitioner<K,V>)
ReflectionUtils.newInstance(conf.getPartitionerClass(), conf);
} else {
@@ -474,7 +492,7 @@
public void configure(JobConf job) { }
@Override
public int getPartition(K key, V value, int numPartitions) {
- return -1;
+ return numPartitions - 1;
}
};
}
@@ -544,14 +562,14 @@
) throws IOException, ClassNotFoundException {
collector = new MapOutputBuffer<K,V>(umbilical, job, reporter);
partitions = jobContext.getNumReduceTasks();
- if (partitions > 0) {
+ if (partitions > 1) {
partitioner = (org.apache.hadoop.mapreduce.Partitioner<K,V>)
ReflectionUtils.newInstance(jobContext.getPartitionerClass(), job);
} else {
partitioner = new org.apache.hadoop.mapreduce.Partitioner<K,V>() {
@Override
public int getPartition(K key, V value, int numPartitions) {
- return -1;
+ return partitions - 1;
}
};
}
@@ -578,7 +596,7 @@
@SuppressWarnings("unchecked")
private <INKEY,INVALUE,OUTKEY,OUTVALUE>
void runNewMapper(final JobConf job,
- final BytesWritable rawSplit,
+ final TaskSplitIndex splitIndex,
final TaskUmbilicalProtocol umbilical,
TaskReporter reporter
) throws IOException, ClassNotFoundException,
@@ -597,15 +615,8 @@
ReflectionUtils.newInstance(taskContext.getInputFormatClass(), job);
// rebuild the input split
org.apache.hadoop.mapreduce.InputSplit split = null;
- DataInputBuffer splitBuffer = new DataInputBuffer();
- splitBuffer.reset(rawSplit.getBytes(), 0, rawSplit.getLength());
- SerializationFactory factory = new SerializationFactory(job);
- Deserializer<? extends org.apache.hadoop.mapreduce.InputSplit>
- deserializer =
- (Deserializer<? extends org.apache.hadoop.mapreduce.InputSplit>)
- factory.getDeserializer(job.getClassByName(splitClass));
- deserializer.open(splitBuffer);
- split = deserializer.deserialize(null);
+ split = getSplitDetails(new Path(splitIndex.getSplitLocation()),
+ splitIndex.getStartOffset());
org.apache.hadoop.mapreduce.RecordReader<INKEY,INVALUE> input =
new NewTrackingRecordReader<INKEY,INVALUE>
@@ -782,11 +793,11 @@
throw new IOException("Invalid \"mapreduce.map.sort.record.percent\": " + recper);
}
if ((sortmb & 0x7FF) != sortmb) {
- throw new IOException("Invalid \"mapreduce.task.mapreduce.task.io.sort.mb\": " + sortmb);
+ throw new IOException("Invalid " + JobContext.IO_SORT_MB + ": " + sortmb);
}
sorter = ReflectionUtils.newInstance(job.getClass("map.sort.class",
QuickSort.class, IndexedSorter.class), job);
- LOG.info("mapreduce.task.mapreduce.task.io.sort.mb = " + sortmb);
+ LOG.info(JobContext.IO_SORT_MB + " = " + sortmb);
// buffers and accounting
int maxMemUsage = sortmb << 20;
int recordCapacity = (int)(maxMemUsage * recper);
@@ -832,7 +843,7 @@
} else {
combineCollector = null;
}
- minSpillsForCombine = job.getInt(JobContext.MAP_COMBINE_MIN_SPISS, 3);
+ minSpillsForCombine = job.getInt(JobContext.MAP_COMBINE_MIN_SPILLS, 3);
spillThread.setDaemon(true);
spillThread.setName("SpillThread");
spillLock.lock();
diff --git a/src/java/org/apache/hadoop/mapred/Merger.java b/src/java/org/apache/hadoop/mapred/Merger.java
index 84c9014..bec2051 100644
--- a/src/java/org/apache/hadoop/mapred/Merger.java
+++ b/src/java/org/apache/hadoop/mapred/Merger.java
@@ -747,7 +747,7 @@
* calculating mergeProgress. This simulates the above merge() method and
* tries to obtain the number of bytes that are going to be merged in all
* merges(assuming that there is no combiner called while merging).
- * @param factor mapreduce.task.mapreduce.task.io.sort.factor
+ * @param factor mapreduce.task.io.sort.factor
* @param inMem number of segments in memory to be merged
*/
long computeBytesInMerges(int factor, int inMem) {
diff --git a/src/java/org/apache/hadoop/mapred/QueueConfigurationParser.java b/src/java/org/apache/hadoop/mapred/QueueConfigurationParser.java
index 447704b..ef7289e 100644
--- a/src/java/org/apache/hadoop/mapred/QueueConfigurationParser.java
+++ b/src/java/org/apache/hadoop/mapred/QueueConfigurationParser.java
@@ -19,6 +19,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.mapred.Queue.QueueOperation;
import org.apache.hadoop.mapreduce.QueueState;
import org.apache.hadoop.security.SecurityUtil.AccessControlList;
@@ -87,9 +88,30 @@
}
- QueueConfigurationParser(String file) {
+ QueueConfigurationParser(String confFile) {
+ File file = new File(confFile).getAbsoluteFile();
+ if (!file.exists()) {
+ throw new RuntimeException("Configuration file not found at " +
+ confFile);
+ }
+ InputStream in = null;
try {
- this.root = loadResource(file);
+ in = new BufferedInputStream(new FileInputStream(file));
+ loadFrom(in);
+ } catch (IOException ioe) {
+ throw new RuntimeException(ioe);
+ } finally {
+ IOUtils.closeStream(in);
+ }
+ }
+
+ QueueConfigurationParser(InputStream xmlInput) {
+ loadFrom(xmlInput);
+ }
+
+ private void loadFrom(InputStream xmlInput) {
+ try {
+ this.root = loadResource(xmlInput);
} catch (ParserConfigurationException e) {
throw new RuntimeException(e);
} catch (SAXException e) {
@@ -120,13 +142,13 @@
* Method to load the resource file.
* generates the root.
*
- * @param confFile
+ * @param resourceInput InputStream that provides the XML to parse
* @return
* @throws ParserConfigurationException
* @throws SAXException
* @throws IOException
*/
- protected Queue loadResource(String confFile)
+ protected Queue loadResource(InputStream resourceInput)
throws ParserConfigurationException, SAXException, IOException {
DocumentBuilderFactory docBuilderFactory
= DocumentBuilderFactory.newInstance();
@@ -146,19 +168,8 @@
DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
Document doc = null;
Element queuesNode = null;
- File file = new File(confFile).getAbsoluteFile();
- if (file.exists()) {
- InputStream in = new BufferedInputStream(new FileInputStream(file));
- try {
- doc = builder.parse(in);
- } finally {
- in.close();
- }
- }
- if (doc == null) {
- throw new RuntimeException(file.getAbsolutePath() + " not found");
- }
+ doc = builder.parse(resourceInput);
queuesNode = doc.getDocumentElement();
return this.parseResource(queuesNode);
}
diff --git a/src/java/org/apache/hadoop/mapred/QueueManager.java b/src/java/org/apache/hadoop/mapred/QueueManager.java
index 2c02314..3055355 100644
--- a/src/java/org/apache/hadoop/mapred/QueueManager.java
+++ b/src/java/org/apache/hadoop/mapred/QueueManager.java
@@ -22,6 +22,7 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.mapred.TaskScheduler.QueueRefresher;
import org.apache.hadoop.mapreduce.QueueState;
import org.apache.hadoop.security.SecurityUtil.AccessControlList;
@@ -31,6 +32,8 @@
import org.codehaus.jackson.JsonGenerationException;
import org.codehaus.jackson.JsonGenerator;
+import java.io.BufferedInputStream;
+import java.io.InputStream;
import java.io.IOException;
import java.io.Writer;
import java.util.ArrayList;
@@ -87,6 +90,7 @@
private Map<String, Queue> leafQueues = new HashMap<String,Queue>();
private Map<String, Queue> allQueues = new HashMap<String, Queue>();
static final String QUEUE_CONF_FILE_NAME = "mapred-queues.xml";
+ static final String QUEUE_CONF_DEFAULT_FILE_NAME = "mapred-queues-default.xml";
// Prefix in configuration for queue related keys
static final String QUEUE_CONF_PROPERTY_NAME_PREFIX
@@ -120,10 +124,24 @@
}
return new DeprecatedQueueConfigurationParser(conf);
} else {
- URL filePath =
+ URL xmlInUrl =
Thread.currentThread().getContextClassLoader()
.getResource(QUEUE_CONF_FILE_NAME);
- return new QueueConfigurationParser(filePath.getPath());
+ if (xmlInUrl == null) {
+ xmlInUrl = Thread.currentThread().getContextClassLoader()
+ .getResource(QUEUE_CONF_DEFAULT_FILE_NAME);
+ assert xmlInUrl != null; // this should be in our jar
+ }
+ InputStream stream = null;
+ try {
+ stream = xmlInUrl.openStream();
+ return new QueueConfigurationParser(new BufferedInputStream(stream));
+ } catch (IOException ioe) {
+ throw new RuntimeException("Couldn't open queue configuration at " +
+ xmlInUrl, ioe);
+ } finally {
+ IOUtils.closeStream(stream);
+ }
}
}
diff --git a/src/java/org/apache/hadoop/mapred/ReduceTask.java b/src/java/org/apache/hadoop/mapred/ReduceTask.java
index 2748a49..d7471f3 100644
--- a/src/java/org/apache/hadoop/mapred/ReduceTask.java
+++ b/src/java/org/apache/hadoop/mapred/ReduceTask.java
@@ -45,6 +45,7 @@
import org.apache.hadoop.mapred.SortedRanges.SkipRangeIterator;
import org.apache.hadoop.mapred.TaskTracker.TaskInProgress;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.mapreduce.task.reduce.Shuffle;
import org.apache.hadoop.util.Progress;
@@ -77,19 +78,19 @@
private Progress sortPhase;
private Progress reducePhase;
private Counters.Counter shuffledMapsCounter =
- getCounters().findCounter(Counter.SHUFFLED_MAPS);
+ getCounters().findCounter(TaskCounter.SHUFFLED_MAPS);
private Counters.Counter reduceShuffleBytes =
- getCounters().findCounter(Counter.REDUCE_SHUFFLE_BYTES);
+ getCounters().findCounter(TaskCounter.REDUCE_SHUFFLE_BYTES);
private Counters.Counter reduceInputKeyCounter =
- getCounters().findCounter(Counter.REDUCE_INPUT_GROUPS);
+ getCounters().findCounter(TaskCounter.REDUCE_INPUT_GROUPS);
private Counters.Counter reduceInputValueCounter =
- getCounters().findCounter(Counter.REDUCE_INPUT_RECORDS);
+ getCounters().findCounter(TaskCounter.REDUCE_INPUT_RECORDS);
private Counters.Counter reduceOutputCounter =
- getCounters().findCounter(Counter.REDUCE_OUTPUT_RECORDS);
+ getCounters().findCounter(TaskCounter.REDUCE_OUTPUT_RECORDS);
private Counters.Counter reduceCombineInputCounter =
- getCounters().findCounter(Counter.COMBINE_INPUT_RECORDS);
+ getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
private Counters.Counter reduceCombineOutputCounter =
- getCounters().findCounter(Counter.COMBINE_OUTPUT_RECORDS);
+ getCounters().findCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
// A custom comparator for map output files. Here the ordering is determined
// by the file's size and path. In case of files with same size and different
@@ -237,9 +238,9 @@
super(in, comparator, keyClass, valClass, conf, reporter);
this.umbilical = umbilical;
this.skipGroupCounter =
- reporter.getCounter(Counter.REDUCE_SKIPPED_GROUPS);
+ reporter.getCounter(TaskCounter.REDUCE_SKIPPED_GROUPS);
this.skipRecCounter =
- reporter.getCounter(Counter.REDUCE_SKIPPED_RECORDS);
+ reporter.getCounter(TaskCounter.REDUCE_SKIPPED_RECORDS);
this.toWriteSkipRecs = toWriteSkipRecs() &&
SkipBadRecords.getSkipOutputPath(conf)!=null;
this.keyClass = keyClass;
diff --git a/src/java/org/apache/hadoop/mapred/Task.java b/src/java/org/apache/hadoop/mapred/Task.java
index 879d5d7..94b7b75 100644
--- a/src/java/org/apache/hadoop/mapred/Task.java
+++ b/src/java/org/apache/hadoop/mapred/Task.java
@@ -28,6 +28,8 @@
import java.util.NoSuchElementException;
import java.util.concurrent.atomic.AtomicBoolean;
+import javax.crypto.SecretKey;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable;
@@ -44,13 +46,12 @@
import org.apache.hadoop.io.serializer.Deserializer;
import org.apache.hadoop.io.serializer.SerializationFactory;
import org.apache.hadoop.mapred.IFile.Writer;
+import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.JobStatus;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
-import org.apache.hadoop.mapreduce.security.JobTokens;
-import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.Progress;
@@ -68,27 +69,6 @@
private static final Log LOG =
LogFactory.getLog(Task.class);
- // Counters used by Task subclasses
- protected static enum Counter {
- MAP_INPUT_RECORDS,
- MAP_OUTPUT_RECORDS,
- MAP_SKIPPED_RECORDS,
- MAP_INPUT_BYTES,
- MAP_OUTPUT_BYTES,
- COMBINE_INPUT_RECORDS,
- COMBINE_OUTPUT_RECORDS,
- REDUCE_INPUT_GROUPS,
- REDUCE_SHUFFLE_BYTES,
- REDUCE_INPUT_RECORDS,
- REDUCE_OUTPUT_RECORDS,
- REDUCE_SKIPPED_GROUPS,
- REDUCE_SKIPPED_RECORDS,
- SPILLED_RECORDS,
- FAILED_SHUFFLE,
- SHUFFLED_MAPS,
- MERGED_MAP_OUTPUTS,
- }
-
public static String MERGED_OUTPUT_PREFIX = ".merged";
@@ -160,7 +140,7 @@
protected final Counters.Counter mergedMapOutputsCounter;
private int numSlotsRequired;
protected TaskUmbilicalProtocol umbilical;
- protected JobTokens jobTokens=null; // storage of the secret keys
+ protected SecretKey tokenSecret;
////////////////////////////////////////////
// Constructors
@@ -169,9 +149,12 @@
public Task() {
taskStatus = TaskStatus.createTaskStatus(isMapTask());
taskId = new TaskAttemptID();
- spilledRecordsCounter = counters.findCounter(TaskCounter.SPILLED_RECORDS);
- failedShuffleCounter = counters.findCounter(Counter.FAILED_SHUFFLE);
- mergedMapOutputsCounter = counters.findCounter(Counter.MERGED_MAP_OUTPUTS);
+ spilledRecordsCounter =
+ counters.findCounter(TaskCounter.SPILLED_RECORDS);
+ failedShuffleCounter =
+ counters.findCounter(TaskCounter.FAILED_SHUFFLE);
+ mergedMapOutputsCounter =
+ counters.findCounter(TaskCounter.MERGED_MAP_OUTPUTS);
}
public Task(String jobFile, TaskAttemptID taskId, int partition,
@@ -190,8 +173,9 @@
TaskStatus.Phase.SHUFFLE,
counters);
spilledRecordsCounter = counters.findCounter(TaskCounter.SPILLED_RECORDS);
- failedShuffleCounter = counters.findCounter(Counter.FAILED_SHUFFLE);
- mergedMapOutputsCounter = counters.findCounter(Counter.MERGED_MAP_OUTPUTS);
+ failedShuffleCounter = counters.findCounter(TaskCounter.FAILED_SHUFFLE);
+ mergedMapOutputsCounter =
+ counters.findCounter(TaskCounter.MERGED_MAP_OUTPUTS);
}
////////////////////////////////////////////
@@ -215,19 +199,19 @@
}
/**
- * set JobToken storage
- * @param jt
+ * Set the job token secret
+ * @param tokenSecret the secret
*/
- public void setJobTokens(JobTokens jt) {
- this.jobTokens = jt;
+ public void setJobTokenSecret(SecretKey tokenSecret) {
+ this.tokenSecret = tokenSecret;
}
/**
- * get JobToken storage
- * @return storage object
+ * Get the job token secret
+ * @return the token secret
*/
- public JobTokens getJobTokens() {
- return this.jobTokens;
+ public SecretKey getJobTokenSecret() {
+ return this.tokenSecret;
}
@@ -938,8 +922,22 @@
+ JobStatus.State.FAILED + " or "
+ JobStatus.State.KILLED);
}
+
+ // delete the staging area for the job
+ JobConf conf = new JobConf(jobContext.getConfiguration());
+ if (!supportIsolationRunner(conf)) {
+ String jobTempDir = conf.get("mapreduce.job.dir");
+ Path jobTempDirPath = new Path(jobTempDir);
+ FileSystem fs = jobTempDirPath.getFileSystem(conf);
+ fs.delete(jobTempDirPath, true);
+ }
done(umbilical, reporter);
}
+
+ protected boolean supportIsolationRunner(JobConf conf) {
+ return (conf.getKeepTaskFilesPattern() != null || conf
+ .getKeepFailedTaskFiles());
+ }
protected void runJobSetupTask(TaskUmbilicalProtocol umbilical,
TaskReporter reporter
diff --git a/src/java/org/apache/hadoop/mapred/TaskController.java b/src/java/org/apache/hadoop/mapred/TaskController.java
index 9ee4f90..15e9521 100644
--- a/src/java/org/apache/hadoop/mapred/TaskController.java
+++ b/src/java/org/apache/hadoop/mapred/TaskController.java
@@ -25,6 +25,9 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.CleanupQueue.PathDeletionContext;
import org.apache.hadoop.mapred.JvmManager.JvmEnv;
import org.apache.hadoop.mapreduce.server.tasktracker.Localizer;
import org.apache.hadoop.mapreduce.MRConfig;
@@ -69,12 +72,10 @@
* disks:
* <ul>
* <li>mapreduce.cluster.local.directories</li>
- * <li>Job cache directories</li>
- * <li>Archive directories</li>
* <li>Hadoop log directories</li>
* </ul>
*/
- void setup() {
+ public void setup() {
for (String localDir : this.mapredLocalDirs) {
// Set up the mapreduce.cluster.local.directories.
File mapredlocalDir = new File(localDir);
@@ -108,13 +109,13 @@
/**
* Take task-controller specific actions to initialize the distributed cache
- * files. This involves setting appropriate permissions for these files so as
+ * file. This involves setting appropriate permissions for these files so as
* to secure them to be accessible only their owners.
*
* @param context
* @throws IOException
*/
- public abstract void initializeDistributedCache(InitializationContext context)
+ public abstract void initializeDistributedCacheFile(DistributedCacheFileContext context)
throws IOException;
/**
@@ -134,26 +135,26 @@
/**
* Top level cleanup a task JVM method.
- *
- * The current implementation does the following.
* <ol>
- * <li>Sends a graceful terminate signal to task JVM allowing its sub-process
+ * <li>Sends a graceful termiante signal to task JVM to allow subprocesses
* to cleanup.</li>
- * <li>Waits for stipulated period</li>
* <li>Sends a forceful kill signal to task JVM, terminating all its
- * sub-process forcefully.</li>
+ * sub-processes forcefully.</li>
* </ol>
- *
+ *
* @param context the task for which kill signal has to be sent.
*/
final void destroyTaskJVM(TaskControllerContext context) {
+ // Send SIGTERM to try to ask for a polite exit.
terminateTask(context);
+
try {
Thread.sleep(context.sleeptimeBeforeSigkill);
} catch (InterruptedException e) {
- LOG.warn("Sleep interrupted : " +
+ LOG.warn("Sleep interrupted : " +
StringUtils.stringifyException(e));
}
+
killTask(context);
}
@@ -191,12 +192,104 @@
}
/**
+ * Contains info related to the path of the file/dir to be deleted. This info
+ * is needed by task-controller to build the full path of the file/dir
+ */
+ static class TaskControllerPathDeletionContext extends PathDeletionContext {
+ Task task;
+ boolean isWorkDir;
+ TaskController taskController;
+
+ /**
+ * mapredLocalDir is the base dir under which to-be-deleted taskWorkDir or
+ * taskAttemptDir exists. fullPath of taskAttemptDir or taskWorkDir
+ * is built using mapredLocalDir, jobId, taskId, etc.
+ */
+ Path mapredLocalDir;
+
+ public TaskControllerPathDeletionContext(FileSystem fs, Path mapredLocalDir,
+ Task task, boolean isWorkDir, TaskController taskController) {
+ super(fs, null);
+ this.task = task;
+ this.isWorkDir = isWorkDir;
+ this.taskController = taskController;
+ this.mapredLocalDir = mapredLocalDir;
+ }
+
+ @Override
+ protected String getPathForCleanup() {
+ if (fullPath == null) {
+ fullPath = buildPathForDeletion();
+ }
+ return fullPath;
+ }
+
+ /**
+ * Builds the path of taskAttemptDir OR taskWorkDir based on
+ * mapredLocalDir, jobId, taskId, etc
+ */
+ String buildPathForDeletion() {
+ String subDir = (isWorkDir) ? TaskTracker.getTaskWorkDir(task.getUser(),
+ task.getJobID().toString(), task.getTaskID().toString(),
+ task.isTaskCleanupTask())
+ : TaskTracker.getLocalTaskDir(task.getUser(),
+ task.getJobID().toString(), task.getTaskID().toString(),
+ task.isTaskCleanupTask());
+
+ return mapredLocalDir.toUri().getPath() + Path.SEPARATOR + subDir;
+ }
+
+ /**
+ * Makes the path(and its subdirectories recursively) fully deletable by
+ * setting proper permissions(770) by task-controller
+ */
+ @Override
+ protected void enablePathForCleanup() throws IOException {
+ getPathForCleanup();// allow init of fullPath, if not inited already
+ if (fs.exists(new Path(fullPath))) {
+ taskController.enableTaskForCleanup(this);
+ }
+ }
+ }
+
+ /**
* NOTE: This class is internal only class and not intended for users!!
*
*/
public static class InitializationContext {
public File workDir;
public String user;
+
+ public InitializationContext() {
+ }
+
+ public InitializationContext(String user, File workDir) {
+ this.user = user;
+ this.workDir = workDir;
+ }
+ }
+
+ /**
+ * This is used for initializing the private localized files in distributed
+ * cache. Initialization would involve changing permission, ownership and etc.
+ */
+ public static class DistributedCacheFileContext extends InitializationContext {
+ // base directory under which file has been localized
+ Path localizedBaseDir;
+ // the unique string used to construct the localized path
+ String uniqueString;
+
+ public DistributedCacheFileContext(String user, File workDir,
+ Path localizedBaseDir, String uniqueString) {
+ super(user, workDir);
+ this.localizedBaseDir = localizedBaseDir;
+ this.uniqueString = uniqueString;
+ }
+
+ public Path getLocalizedUniqueDir() {
+ return new Path(localizedBaseDir, new Path(TaskTracker
+ .getPrivateDistributedCacheDir(user), uniqueString));
+ }
}
static class JobInitializationContext extends InitializationContext {
@@ -224,6 +317,15 @@
*/
abstract void killTask(TaskControllerContext context);
+
+ /**
+ * Sends a QUIT signal to direct the task JVM (and sub-processes) to
+ * dump their stack to stdout.
+ *
+ * @param context task context.
+ */
+ abstract void dumpTaskStack(TaskControllerContext context);
+
/**
* Initialize user on this TaskTracer in a TaskController specific manner.
*
@@ -242,4 +344,11 @@
abstract void runDebugScript(DebugScriptContext context)
throws IOException;
+ /**
+ * Enable the task for cleanup by changing permissions of the path
+ * @param context path deletion context
+ * @throws IOException
+ */
+ abstract void enableTaskForCleanup(PathDeletionContext context)
+ throws IOException;
}
diff --git a/src/java/org/apache/hadoop/mapred/TaskInProgress.java b/src/java/org/apache/hadoop/mapred/TaskInProgress.java
index cb352d4..1225824 100644
--- a/src/java/org/apache/hadoop/mapred/TaskInProgress.java
+++ b/src/java/org/apache/hadoop/mapred/TaskInProgress.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
@@ -31,13 +32,12 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.mapred.JobInProgress.DataStatistics;
import org.apache.hadoop.mapred.SortedRanges.Range;
-import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.jobhistory.JobHistory;
import org.apache.hadoop.mapreduce.jobhistory.TaskUpdatedEvent;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
import org.apache.hadoop.net.Node;
@@ -65,7 +65,7 @@
// Defines the TIP
private String jobFile = null;
- private Job.RawSplit rawSplit;
+ private TaskSplitMetaInfo splitInfo;
private int numMaps;
private int partition;
private JobTracker jobtracker;
@@ -140,12 +140,12 @@
* Constructor for MapTask
*/
public TaskInProgress(JobID jobid, String jobFile,
- Job.RawSplit rawSplit,
+ TaskSplitMetaInfo split,
JobTracker jobtracker, JobConf conf,
JobInProgress job, int partition,
int numSlotsRequired) {
this.jobFile = jobFile;
- this.rawSplit = rawSplit;
+ this.splitInfo = split;
this.jobtracker = jobtracker;
this.job = job;
this.conf = conf;
@@ -316,10 +316,36 @@
* Whether this is a map task
*/
public boolean isMapTask() {
- return rawSplit != null;
+ return splitInfo != null;
}
/**
+ * Returns the {@link TaskType} of the {@link TaskAttemptID} passed.
+ * The type of an attempt is determined by the nature of the task and not its
+ * id.
+ * For example,
+ * - Attempt 'attempt_123_01_m_01_0' might be a job-setup task even though it
+ * has a _m_ in its id. Hence the task type of this attempt is JOB_SETUP
+ * instead of MAP.
+ * - Similarly reduce attempt 'attempt_123_01_r_01_0' might have failed and is
+ * now supposed to do the task-level cleanup. In such a case this attempt
+ * will be of type TASK_CLEANUP instead of REDUCE.
+ */
+ TaskType getAttemptType (TaskAttemptID id) {
+ if (isCleanupAttempt(id)) {
+ return TaskType.TASK_CLEANUP;
+ } else if (isJobSetupTask()) {
+ return TaskType.JOB_SETUP;
+ } else if (isJobCleanupTask()) {
+ return TaskType.JOB_CLEANUP;
+ } else if (isMapTask()) {
+ return TaskType.MAP;
+ } else {
+ return TaskType.REDUCE;
+ }
+ }
+
+ /**
* Is the Task associated with taskid is the first attempt of the tip?
* @param taskId
* @return Returns true if the Task is the first attempt of the tip
@@ -335,6 +361,15 @@
public boolean isRunning() {
return !activeTasks.isEmpty();
}
+
+ /**
+ * Is this TaskAttemptid running
+ * @param taskId
+ * @return true if taskId attempt is running.
+ */
+ boolean isAttemptRunning(TaskAttemptID taskId) {
+ return activeTasks.containsKey(taskId);
+ }
TaskAttemptID getSuccessfulTaskid() {
return successfulTaskId;
@@ -534,15 +569,16 @@
* A status message from a client has arrived.
* It updates the status of a single component-thread-task,
* which might result in an overall TaskInProgress status update.
- * @return has the task changed its state noticably?
+ * @return has the task changed its state noticeably?
*/
synchronized boolean updateStatus(TaskStatus status) {
TaskAttemptID taskid = status.getTaskID();
+ String tracker = status.getTaskTracker();
String diagInfo = status.getDiagnosticInfo();
TaskStatus oldStatus = taskStatuses.get(taskid);
boolean changed = true;
if (diagInfo != null && diagInfo.length() > 0) {
- LOG.info("Error from "+taskid+": "+diagInfo);
+ LOG.info("Error from " + taskid + " on " + tracker + ": "+ diagInfo);
addDiagnosticInfo(taskid, diagInfo);
}
@@ -697,6 +733,12 @@
if (tasks.contains(taskid)) {
if (taskState == TaskStatus.State.FAILED) {
numTaskFailures++;
+ if (isMapTask()) {
+ jobtracker.getInstrumentation().failedMap(taskid);
+ } else {
+ jobtracker.getInstrumentation().failedReduce(taskid);
+ }
+
machinesWhereFailed.add(trackerHostName);
if(maxSkipRecords>0) {
//skipping feature enabled
@@ -707,6 +749,11 @@
} else if (taskState == TaskStatus.State.KILLED) {
numKilledTasks++;
+ if (isMapTask()) {
+ jobtracker.getInstrumentation().killedMap(taskid);
+ } else {
+ jobtracker.getInstrumentation().killedReduce(taskid);
+ }
}
}
@@ -787,7 +834,7 @@
*/
public String[] getSplitLocations() {
if (isMapTask() && !jobSetup && !jobCleanup) {
- return rawSplit.getLocations();
+ return splitInfo.getLocations();
}
return new String[0];
}
@@ -800,6 +847,13 @@
}
/**
+ * Get all the {@link TaskAttemptID}s in this {@link TaskInProgress}
+ */
+ TaskAttemptID[] getAllTaskAttemptIDs() {
+ return tasks.toArray(new TaskAttemptID[tasks.size()]);
+ }
+
+ /**
* Get the status of the specified task
* @param taskid
* @return
@@ -992,16 +1046,8 @@
if (isMapTask()) {
LOG.debug("attempt " + numTaskFailures + " sending skippedRecords "
+ failedRanges.getIndicesCount());
- String splitClass = null;
- BytesWritable split;
- if (!jobSetup && !jobCleanup) {
- splitClass = rawSplit.getClassName();
- split = rawSplit.getBytes();
- } else {
- split = new BytesWritable();
- }
- t = new MapTask(jobFile, taskid, partition, splitClass, split,
- numSlotsNeeded);
+ t = new MapTask(jobFile, taskid, partition, splitInfo.getSplitIndex(),
+ numSlotsNeeded);
} else {
t = new ReduceTask(jobFile, taskid, partition, numMaps, numSlotsNeeded);
}
@@ -1114,7 +1160,7 @@
if (!isMapTask() || jobSetup || jobCleanup) {
return "";
}
- String[] splits = rawSplit.getLocations();
+ String[] splits = splitInfo.getLocations();
Node[] nodes = new Node[splits.length];
for (int i = 0; i < splits.length; i++) {
nodes[i] = jobtracker.getNode(splits[i]);
@@ -1144,16 +1190,12 @@
public long getMapInputSize() {
if(isMapTask() && !jobSetup && !jobCleanup) {
- return rawSplit.getDataLength();
+ return splitInfo.getInputDataLength();
} else {
return 0;
}
}
- public void clearSplit() {
- rawSplit.clearBytes();
- }
-
/**
* Compare most recent task attempts dispatch time to current system time so
* that task progress rate will slow down as time proceeds even if no progress
diff --git a/src/java/org/apache/hadoop/mapred/TaskRunner.java b/src/java/org/apache/hadoop/mapred/TaskRunner.java
index a52f748..9301707 100644
--- a/src/java/org/apache/hadoop/mapred/TaskRunner.java
+++ b/src/java/org/apache/hadoop/mapred/TaskRunner.java
@@ -43,7 +43,6 @@
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.TaskController.InitializationContext;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils;
import org.apache.log4j.Level;
@@ -93,6 +92,8 @@
public TaskTracker.TaskInProgress getTaskInProgress() { return tip; }
public TaskTracker getTracker() { return tracker; }
+ public JvmManager getJvmManager() { return jvmManager; }
+
/** Called to assemble this task's input. This method is run in the parent
* process before the child is spawned. It should not execute user code,
* only system code. */
@@ -172,18 +173,14 @@
taskDistributedCacheManager = tracker.getTrackerDistributedCacheManager()
.newTaskDistributedCacheManager(conf);
taskDistributedCacheManager.setup(lDirAlloc, workDir, TaskTracker
- .getDistributedCacheDir(conf.getUser()));
+ .getPrivateDistributedCacheDir(conf.getUser()),
+ TaskTracker.getPublicDistributedCacheDir());
// Set up the child task's configuration. After this call, no localization
// of files should happen in the TaskTracker's process space. Any changes to
// the conf object after this will NOT be reflected to the child.
setupChildTaskConfiguration(lDirAlloc);
- InitializationContext context = new InitializationContext();
- context.user = conf.getUser();
- context.workDir = new File(conf.get(TaskTracker.JOB_LOCAL_DIR));
- tracker.getTaskController().initializeDistributedCache(context);
-
if (!prepare()) {
return;
}
@@ -521,7 +518,7 @@
}
hadoopClientOpts = hadoopClientOpts + "-Dhadoop.tasklog.taskid=" + taskid
+ " -Dhadoop.tasklog.totalLogFileSize=" + logSize;
- env.put("HADOOP_CLIENT_OPTS", "\"" + hadoopClientOpts + "\"");
+ env.put("HADOOP_CLIENT_OPTS", hadoopClientOpts);
// add the env variables passed by the user
String mapredChildEnv = getChildEnv(conf);
@@ -647,7 +644,40 @@
}
}
classPaths.add(new File(jobCacheDir, "classes").toString());
- classPaths.add(jobCacheDir.toString());
+ classPaths.add(new File(jobCacheDir, "job.jar").toString());
+ }
+
+ /**
+ * Sets permissions recursively and then deletes the contents of dir.
+ * Makes dir empty directory(does not delete dir itself).
+ */
+ static void deleteDirContents(JobConf conf, File dir) throws IOException {
+ FileSystem fs = FileSystem.getLocal(conf);
+ if (fs.exists(new Path(dir.getAbsolutePath()))) {
+ File contents[] = dir.listFiles();
+ if (contents != null) {
+ for (int i = 0; i < contents.length; i++) {
+ try {
+ int ret = 0;
+ if ((ret = FileUtil.chmod(contents[i].getAbsolutePath(),
+ "ug+rwx", true)) != 0) {
+ LOG.warn("Unable to chmod for " + contents[i] +
+ "; chmod exit status = " + ret);
+ }
+ } catch(InterruptedException e) {
+ LOG.warn("Interrupted while setting permissions for contents of " +
+ "workDir. Not deleting the remaining contents of workDir.");
+ return;
+ }
+ if (!fs.delete(new Path(contents[i].getAbsolutePath()), true)) {
+ LOG.warn("Unable to delete "+ contents[i]);
+ }
+ }
+ }
+ }
+ else {
+ LOG.warn(dir + " does not exist.");
+ }
}
/**
@@ -660,11 +690,14 @@
* @param workDir Working directory, which is completely deleted.
*/
public static void setupWorkDir(JobConf conf, File workDir) throws IOException {
- LOG.debug("Fully deleting and re-creating" + workDir);
- FileUtil.fullyDelete(workDir);
- if (!workDir.mkdir()) {
- LOG.debug("Did not recreate " + workDir);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Fully deleting contents of " + workDir);
}
+
+ /** delete only the contents of workDir leaving the directory empty. We
+ * can't delete the workDir as it is the current working directory.
+ */
+ deleteDirContents(conf, workDir);
if (DistributedCache.getSymlink(conf)) {
URI[] archives = DistributedCache.getCacheArchives(conf);
diff --git a/src/java/org/apache/hadoop/mapred/TaskTracker.java b/src/java/org/apache/hadoop/mapred/TaskTracker.java
index cd1355f..d840054 100644
--- a/src/java/org/apache/hadoop/mapred/TaskTracker.java
+++ b/src/java/org/apache/hadoop/mapred/TaskTracker.java
@@ -38,11 +38,11 @@
import java.util.Set;
import java.util.StringTokenizer;
import java.util.TreeMap;
-import java.util.Vector;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.regex.Pattern;
+import javax.crypto.SecretKey;
import javax.servlet.ServletContext;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
@@ -67,14 +67,19 @@
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.mapred.TaskController.DebugScriptContext;
import org.apache.hadoop.mapred.TaskController.JobInitializationContext;
+import org.apache.hadoop.mapred.CleanupQueue.PathDeletionContext;
+import org.apache.hadoop.mapred.TaskController.TaskControllerPathDeletionContext;
import org.apache.hadoop.mapred.TaskTrackerStatus.TaskTrackerHealthStatus;
import org.apache.hadoop.mapred.pipes.Submitter;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.filecache.TrackerDistributedCacheManager;
-import org.apache.hadoop.mapreduce.security.JobTokens;
import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
+import org.apache.hadoop.mapreduce.security.TokenCache;
+import org.apache.hadoop.mapreduce.security.TokenStorage;
+import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
import org.apache.hadoop.mapreduce.server.tasktracker.Localizer;
import org.apache.hadoop.mapreduce.task.reduce.ShuffleHeader;
@@ -86,13 +91,16 @@
import org.apache.hadoop.net.DNS;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UnixUserGroupInformation;
import org.apache.hadoop.security.authorize.ConfiguredPolicy;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.mapreduce.util.ConfigUtil;
import org.apache.hadoop.mapreduce.util.MemoryCalculatorPlugin;
+import org.apache.hadoop.mapreduce.util.ResourceCalculatorPlugin;
import org.apache.hadoop.mapreduce.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.RunJar;
import org.apache.hadoop.util.Service;
@@ -100,7 +108,7 @@
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.VersionInfo;
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
-import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.mapreduce.util.MRAsyncDiskService;
/*******************************************************
* TaskTracker is a process that starts and tracks MR Tasks
@@ -123,6 +131,7 @@
static final String MAPRED_TASKTRACKER_PMEM_RESERVED_PROPERTY =
"mapred.tasktracker.pmem.reserved";
+
static final long WAIT_FOR_DONE = 3 * 1000;
int httpPort;
@@ -194,6 +203,8 @@
*/
Map<TaskAttemptID, TaskInProgress> runningTasks = null;
Map<JobID, RunningJob> runningJobs = new TreeMap<JobID, RunningJob>();
+ private final JobTokenSecretManager jobTokenSecretManager
+ = new JobTokenSecretManager();
volatile int mapTotal = 0;
volatile int reduceTotal = 0;
@@ -221,13 +232,14 @@
static final String OUTPUT = "output";
private static final String JARSDIR = "jars";
static final String LOCAL_SPLIT_FILE = "split.dta";
+ static final String LOCAL_SPLIT_META_FILE = "split.info";
static final String JOBFILE = "job.xml";
static final String JOB_TOKEN_FILE="jobToken"; //localized file
static final String JOB_LOCAL_DIR = JobContext.JOB_LOCAL_DIR;
private JobConf fConf;
- FileSystem localFs;
+ private FileSystem localFs;
private Localizer localizer;
@@ -254,9 +266,7 @@
private long mapSlotMemorySizeOnTT = JobConf.DISABLED_MEMORY_LIMIT;
private long reduceSlotSizeMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
private long totalMemoryAllottedForTasks = JobConf.DISABLED_MEMORY_LIMIT;
-
- static final String MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY =
- TT_MEMORY_CALCULATOR_PLUGIN;
+ private ResourceCalculatorPlugin resourceCalculatorPlugin = null;
/**
* the minimum interval between jobtracker polls
@@ -269,6 +279,8 @@
private IndexCache indexCache;
+ private MRAsyncDiskService asyncDiskService;
+
/**
* Handle to the specific instance of the {@link TaskController} class
*/
@@ -365,6 +377,11 @@
return taskController;
}
+ // Currently this is used only by tests
+ void setTaskController(TaskController t) {
+ taskController = t;
+ }
+
private RunningJob addTaskToJob(JobID jobId,
TaskInProgress tip) {
synchronized (runningJobs) {
@@ -398,6 +415,10 @@
}
}
+ JobTokenSecretManager getJobTokenSecretManager() {
+ return jobTokenSecretManager;
+ }
+
Localizer getLocalizer() {
return localizer;
}
@@ -410,9 +431,13 @@
return TaskTracker.SUBDIR + Path.SEPARATOR + user;
}
- public static String getDistributedCacheDir(String user) {
+ public static String getPrivateDistributedCacheDir(String user) {
return getUserDir(user) + Path.SEPARATOR + TaskTracker.DISTCACHEDIR;
}
+
+ public static String getPublicDistributedCacheDir() {
+ return TaskTracker.SUBDIR + Path.SEPARATOR + TaskTracker.DISTCACHEDIR;
+ }
public static String getJobCacheSubdir(String user) {
return getUserDir(user) + Path.SEPARATOR + TaskTracker.JOBCACHE;
@@ -449,11 +474,16 @@
return getLocalJobDir(user, jobid) + Path.SEPARATOR + MRConstants.WORKDIR;
}
- static String getLocalSplitFile(String user, String jobid, String taskid) {
+ static String getLocalSplitMetaFile(String user, String jobid, String taskid){
return TaskTracker.getLocalTaskDir(user, jobid, taskid) + Path.SEPARATOR
- + TaskTracker.LOCAL_SPLIT_FILE;
+ + TaskTracker.LOCAL_SPLIT_META_FILE;
}
+ static String getLocalSplitFile(String user, String jobid, String taskid) {
+ return TaskTracker.getLocalTaskDir(user, jobid, taskid) + Path.SEPARATOR
+ + TaskTracker.LOCAL_SPLIT_FILE;
+ }
+
static String getIntermediateOutputDir(String user, String jobid,
String taskid) {
return getLocalTaskDir(user, jobid, taskid) + Path.SEPARATOR
@@ -475,10 +505,7 @@
static String getTaskWorkDir(String user, String jobid, String taskid,
boolean isCleanupAttempt) {
- String dir = getLocalJobDir(user, jobid) + Path.SEPARATOR + taskid;
- if (isCleanupAttempt) {
- dir = dir + TASK_CLEANUP_SUFFIX;
- }
+ String dir = getLocalTaskDir(user, jobid, taskid, isCleanupAttempt);
return dir + Path.SEPARATOR + MRConstants.WORKDIR;
}
@@ -529,9 +556,11 @@
fConf.get(TT_DNS_NAMESERVER,"default"));
}
- //check local disk
+ // Check local disk, start async disk service, and clean up all
+ // local directories.
checkLocalDirs(this.fConf.getLocalDirs());
- fConf.deleteLocalFiles(SUBDIR);
+ asyncDiskService = new MRAsyncDiskService(fConf);
+ asyncDiskService.cleanupAllVolumes();
// Clear out state tables
this.tasks.clear();
@@ -597,12 +626,19 @@
this.taskTrackerName = "tracker_" + localHostname + ":" + taskReportAddress;
LOG.info("Starting tracker " + taskTrackerName);
- // Initialize DistributedCache and
- // clear out temporary files that might be lying around
+ Class<? extends TaskController> taskControllerClass = fConf.getClass(
+ TT_TASK_CONTROLLER, DefaultTaskController.class, TaskController.class);
+ taskController = (TaskController) ReflectionUtils.newInstance(
+ taskControllerClass, fConf);
+
+
+ // setup and create jobcache directory with appropriate permissions
+ taskController.setup();
+
+ // Initialize DistributedCache
this.distributedCacheManager =
- new TrackerDistributedCacheManager(this.fConf);
- this.distributedCacheManager.purgeCache();
- cleanupStorage();
+ new TrackerDistributedCacheManager(this.fConf, taskController,
+ asyncDiskService);
//mark as just started; this is used in heartbeats
this.justStarted = true;
@@ -625,6 +661,12 @@
taskTrackerName);
mapEventsFetcher.start();
+ Class<? extends ResourceCalculatorPlugin> clazz =
+ fConf.getClass(TT_RESOURCE_CALCULATOR_PLUGIN,
+ null, ResourceCalculatorPlugin.class);
+ resourceCalculatorPlugin = ResourceCalculatorPlugin
+ .getResourceCalculatorPlugin(clazz, fConf);
+ LOG.info(" Using ResourceCalculatorPlugin : " + resourceCalculatorPlugin);
initializeMemoryManagement();
this.indexCache = new IndexCache(this.fConf);
@@ -633,15 +675,6 @@
reduceLauncher = new TaskLauncher(TaskType.REDUCE, maxReduceSlots);
mapLauncher.start();
reduceLauncher.start();
- Class<? extends TaskController> taskControllerClass
- = fConf.getClass(TT_TASK_CONTROLLER,
- DefaultTaskController.class,
- TaskController.class);
- taskController = (TaskController)ReflectionUtils.newInstance(
- taskControllerClass, fConf);
-
- //setup and create jobcache directory with appropriate permissions
- taskController.setup();
// create a localizer instance
setLocalizer(new Localizer(localFs, fConf.getLocalDirs(), taskController));
@@ -667,10 +700,14 @@
t, TaskTrackerInstrumentation.class);
}
- /**
- * Removes all contents of temporary storage. Called upon
+ /**
+ * Removes all contents of temporary storage. Called upon
* startup, to remove any leftovers from previous run.
+ *
+ * Use MRAsyncDiskService.moveAndDeleteAllVolumes instead.
+ * @see org.apache.hadoop.mapreduce.util.MRAsyncDiskService#cleanupAllVolumes()
*/
+ @Deprecated
public void cleanupStorage() throws IOException {
if (fConf != null) {
fConf.deleteLocalFiles();
@@ -880,18 +917,22 @@
rjob.jobConf = localJobConf;
rjob.keepJobFiles = ((localJobConf.getKeepTaskFilesPattern() != null) ||
localJobConf.getKeepFailedTaskFiles());
- FSDataInputStream in = localFs.open(new Path(
- rjob.jobConf.get(JobContext.JOB_TOKEN_FILE)));
- JobTokens jt = new JobTokens();
- jt.readFields(in);
- rjob.jobTokens = jt; // store JobToken object per job
-
+ TokenStorage ts = TokenCache.loadTokens(rjob.jobConf);
+ Token<JobTokenIdentifier> jt = (Token<JobTokenIdentifier>)ts.getJobToken();
+ getJobTokenSecretManager().addTokenForJob(jobId.toString(), jt);
rjob.localized = true;
}
}
launchTaskForJob(tip, new JobConf(rjob.jobConf));
}
+ private void setUgi(String user, Configuration conf) {
+ //The dummy-group used here will not be required once we have UGI
+ //object creation with just the user name.
+ conf.set(UnixUserGroupInformation.UGI_PROPERTY_NAME,
+ user+","+UnixUserGroupInformation.DEFAULT_GROUP);
+ }
+
/**
* Localize the job on this tasktracker. Specifically
* <ul>
@@ -934,6 +975,7 @@
}
System.setProperty(JOB_LOCAL_DIR, workDir.toUri().getPath());
localJobConf.set(JOB_LOCAL_DIR, workDir.toUri().getPath());
+ setUgi(userName, localJobConf);
// Download the job.jar for this job from the system FS
localizeJobJarFile(userName, jobId, localFs, localJobConf);
@@ -952,12 +994,17 @@
*/
private Path localizeJobConfFile(Path jobFile, String user, JobID jobId)
throws IOException {
- // Get sizes of JobFile and JarFile
+ JobConf conf = new JobConf(getJobConf());
+ setUgi(user, conf);
+
+ FileSystem userFs = jobFile.getFileSystem(conf);
+ // Get sizes of JobFile
// sizes are -1 if they are not present.
FileStatus status = null;
long jobFileSize = -1;
try {
- status = systemFS.getFileStatus(jobFile);
+
+ status = userFs.getFileStatus(jobFile);
jobFileSize = status.getLen();
} catch(FileNotFoundException fe) {
jobFileSize = -1;
@@ -968,7 +1015,7 @@
jobFileSize, fConf);
// Download job.xml
- systemFS.copyToLocalFile(jobFile, localJobFile);
+ userFs.copyToLocalFile(jobFile, localJobFile);
return localJobFile;
}
@@ -990,8 +1037,9 @@
long jarFileSize = -1;
if (jarFile != null) {
Path jarFilePath = new Path(jarFile);
+ FileSystem fs = jarFilePath.getFileSystem(localJobConf);
try {
- status = systemFS.getFileStatus(jarFilePath);
+ status = fs.getFileStatus(jarFilePath);
jarFileSize = status.getLen();
} catch (FileNotFoundException fe) {
jarFileSize = -1;
@@ -1003,14 +1051,15 @@
getJobJarFile(user, jobId.toString()), 5 * jarFileSize, fConf);
// Download job.jar
- systemFS.copyToLocalFile(jarFilePath, localJarFile);
+ fs.copyToLocalFile(jarFilePath, localJarFile);
localJobConf.setJar(localJarFile.toString());
- // Also un-jar the job.jar files. We un-jar it so that classes inside
- // sub-directories, for e.g., lib/, classes/ are available on class-path
- RunJar.unJar(new File(localJarFile.toString()), new File(localJarFile
- .getParent().toString()));
+ // Un-jar the parts of the job.jar that need to be added to the classpath
+ RunJar.unJar(
+ new File(localJarFile.toString()),
+ new File(localJarFile.getParent().toString()),
+ localJobConf.getJarUnpackPattern());
}
}
@@ -1081,9 +1130,23 @@
this.running = false;
- // Clear local storage
- cleanupStorage();
-
+ if (asyncDiskService != null) {
+ // Clear local storage
+ asyncDiskService.cleanupAllVolumes();
+
+ // Shutdown all async deletion threads with up to 10 seconds of delay
+ asyncDiskService.shutdown();
+ try {
+ if (!asyncDiskService.awaitTermination(10000)) {
+ asyncDiskService.shutdownNow();
+ asyncDiskService = null;
+ }
+ } catch (InterruptedException e) {
+ asyncDiskService.shutdownNow();
+ asyncDiskService = null;
+ }
+ }
+
// Shutdown the fetcher thread
if (mapEventsFetcher != null) {
mapEventsFetcher.interrupt();
@@ -1229,6 +1292,16 @@
directoryCleanupThread = new CleanupQueue();
}
+
+ // only used by tests
+ void setCleanupThread(CleanupQueue c) {
+ directoryCleanupThread = c;
+ }
+
+ CleanupQueue getCleanupThread() {
+ return directoryCleanupThread;
+ }
+
/**
* Tell the cleanup threads that they should end themselves
*/
@@ -1429,7 +1502,7 @@
* @return false if the tracker was unknown
* @throws IOException
*/
- private HeartbeatResponse transmitHeartBeat(long now) throws IOException {
+ HeartbeatResponse transmitHeartBeat(long now) throws IOException {
// Send Counters in the status once every COUNTER_UPDATE_INTERVAL
boolean sendCounters;
if (now > (previousUpdate + COUNTER_UPDATE_INTERVAL)) {
@@ -1478,6 +1551,12 @@
long freeDiskSpace = getFreeSpace();
long totVmem = getTotalVirtualMemoryOnTT();
long totPmem = getTotalPhysicalMemoryOnTT();
+ long availableVmem = getAvailableVirtualMemoryOnTT();
+ long availablePmem = getAvailablePhysicalMemoryOnTT();
+ long cumuCpuTime = getCumulativeCpuTimeOnTT();
+ long cpuFreq = getCpuFrequencyOnTT();
+ int numCpu = getNumProcessorsOnTT();
+ float cpuUsage = getCpuUsageOnTT();
status.getResourceStatus().setAvailableSpace(freeDiskSpace);
status.getResourceStatus().setTotalVirtualMemory(totVmem);
@@ -1486,6 +1565,12 @@
mapSlotMemorySizeOnTT);
status.getResourceStatus().setReduceSlotMemorySizeOnTT(
reduceSlotSizeMemoryOnTT);
+ status.getResourceStatus().setAvailableVirtualMemory(availableVmem);
+ status.getResourceStatus().setAvailablePhysicalMemory(availablePmem);
+ status.getResourceStatus().setCumulativeCpuTime(cumuCpuTime);
+ status.getResourceStatus().setCpuFrequency(cpuFreq);
+ status.getResourceStatus().setNumProcessors(numCpu);
+ status.getResourceStatus().setCpuUsage(cpuUsage);
}
//add node health information
@@ -1562,6 +1647,80 @@
return totalPhysicalMemoryOnTT;
}
+ /**
+ * Return the free virtual memory available on this TaskTracker.
+ * @return total size of free virtual memory.
+ */
+ long getAvailableVirtualMemoryOnTT() {
+ long availableVirtualMemoryOnTT = TaskTrackerStatus.UNAVAILABLE;
+ if (resourceCalculatorPlugin != null) {
+ availableVirtualMemoryOnTT =
+ resourceCalculatorPlugin.getAvailableVirtualMemorySize();
+ }
+ return availableVirtualMemoryOnTT;
+ }
+
+ /**
+ * Return the free physical memory available on this TaskTracker.
+ * @return total size of free physical memory in bytes
+ */
+ long getAvailablePhysicalMemoryOnTT() {
+ long availablePhysicalMemoryOnTT = TaskTrackerStatus.UNAVAILABLE;
+ if (resourceCalculatorPlugin != null) {
+ availablePhysicalMemoryOnTT =
+ resourceCalculatorPlugin.getAvailablePhysicalMemorySize();
+ }
+ return availablePhysicalMemoryOnTT;
+ }
+
+ /**
+ * Return the cumulative CPU used time on this TaskTracker since system is on
+ * @return cumulative CPU used time in millisecond
+ */
+ long getCumulativeCpuTimeOnTT() {
+ long cumulativeCpuTime = TaskTrackerStatus.UNAVAILABLE;
+ if (resourceCalculatorPlugin != null) {
+ cumulativeCpuTime = resourceCalculatorPlugin.getCumulativeCpuTime();
+ }
+ return cumulativeCpuTime;
+ }
+
+ /**
+ * Return the number of Processors on this TaskTracker
+ * @return number of processors
+ */
+ int getNumProcessorsOnTT() {
+ int numProcessors = TaskTrackerStatus.UNAVAILABLE;
+ if (resourceCalculatorPlugin != null) {
+ numProcessors = resourceCalculatorPlugin.getNumProcessors();
+ }
+ return numProcessors;
+ }
+
+ /**
+ * Return the CPU frequency of this TaskTracker
+ * @return CPU frequency in kHz
+ */
+ long getCpuFrequencyOnTT() {
+ long cpuFrequency = TaskTrackerStatus.UNAVAILABLE;
+ if (resourceCalculatorPlugin != null) {
+ cpuFrequency = resourceCalculatorPlugin.getCpuFrequency();
+ }
+ return cpuFrequency;
+ }
+
+ /**
+ * Return the CPU usage in % of this TaskTracker
+ * @return CPU usage in %
+ */
+ float getCpuUsageOnTT() {
+ float cpuUsage = TaskTrackerStatus.UNAVAILABLE;
+ if (resourceCalculatorPlugin != null) {
+ cpuUsage = resourceCalculatorPlugin.getCpuUsage();
+ }
+ return cpuUsage;
+ }
+
long getTotalMemoryAllottedForTasksOnTT() {
return totalMemoryAllottedForTasks;
}
@@ -1613,6 +1772,7 @@
ReflectionUtils.logThreadInfo(LOG, "lost task", 30);
tip.reportDiagnosticInfo(msg);
myInstrumentation.timedoutTask(tip.getTask().getTaskID());
+ dumpTaskStack(tip);
purgeTask(tip, true);
}
}
@@ -1620,6 +1780,60 @@
}
/**
+ * Builds list of PathDeletionContext objects for the given paths
+ */
+ private static PathDeletionContext[] buildPathDeletionContexts(FileSystem fs,
+ Path[] paths) {
+ int i = 0;
+ PathDeletionContext[] contexts = new PathDeletionContext[paths.length];
+
+ for (Path p : paths) {
+ contexts[i++] = new PathDeletionContext(fs, p.toUri().getPath());
+ }
+ return contexts;
+ }
+
+ /**
+ * Builds list of TaskControllerPathDeletionContext objects for a task
+ * @param fs : FileSystem in which the dirs to be deleted
+ * @param paths : mapred-local-dirs
+ * @param task : the task whose taskDir or taskWorkDir is going to be deleted
+ * @param isWorkDir : the dir to be deleted is workDir or taskDir
+ * @param taskController : the task-controller to be used for deletion of
+ * taskDir or taskWorkDir
+ */
+ static PathDeletionContext[] buildTaskControllerPathDeletionContexts(
+ FileSystem fs, Path[] paths, Task task, boolean isWorkDir,
+ TaskController taskController)
+ throws IOException {
+ int i = 0;
+ PathDeletionContext[] contexts =
+ new TaskControllerPathDeletionContext[paths.length];
+
+ for (Path p : paths) {
+ contexts[i++] = new TaskControllerPathDeletionContext(fs, p, task,
+ isWorkDir, taskController);
+ }
+ return contexts;
+ }
+
+ /**
+ * Send a signal to a stuck task commanding it to dump stack traces
+ * to stderr before we kill it with purgeTask().
+ *
+ * @param tip {@link TaskInProgress} to dump stack traces.
+ */
+ private void dumpTaskStack(TaskInProgress tip) {
+ TaskRunner runner = tip.getTaskRunner();
+ if (null == runner) {
+ return; // tip is already abandoned.
+ }
+
+ JvmManager jvmMgr = runner.getJvmManager();
+ jvmMgr.dumpStack(runner);
+ }
+
+ /**
* The task tracker is done with this job, so we need to clean up.
* @param action The action with the job
* @throws IOException
@@ -1657,6 +1871,7 @@
synchronized(runningJobs) {
runningJobs.remove(jobId);
}
+ getJobTokenSecretManager().removeTokenForJob(jobId.toString());
}
/**
@@ -1667,8 +1882,9 @@
*/
void removeJobFiles(String user, String jobId)
throws IOException {
- directoryCleanupThread.addToQueue(localFs, getLocalFiles(fConf,
- getLocalJobDir(user, jobId)));
+ PathDeletionContext[] contexts = buildPathDeletionContexts(localFs,
+ getLocalFiles(fConf, getLocalJobDir(user, jobId)));
+ directoryCleanupThread.addToQueue(contexts);
}
/**
@@ -2739,29 +2955,33 @@
runner.close();
}
- String localTaskDir =
- getLocalTaskDir(task.getUser(), task.getJobID().toString(), taskId
- .toString(), task.isTaskCleanupTask());
if (localJobConf.getNumTasksToExecutePerJvm() == 1) {
// No jvm reuse, remove everything
- directoryCleanupThread.addToQueue(localFs, getLocalFiles(
- defaultJobConf, localTaskDir));
+ PathDeletionContext[] contexts =
+ buildTaskControllerPathDeletionContexts(localFs,
+ getLocalFiles(fConf, ""), task, false/* not workDir */,
+ taskController);
+ directoryCleanupThread.addToQueue(contexts);
} else {
// Jvm reuse. We don't delete the workdir since some other task
// (running in the same JVM) might be using the dir. The JVM
// running the tasks would clean the workdir per a task in the
// task process itself.
- directoryCleanupThread.addToQueue(localFs, getLocalFiles(
- defaultJobConf, localTaskDir + Path.SEPARATOR
- + TaskTracker.JOBFILE));
+ String localTaskDir =
+ getLocalTaskDir(task.getUser(), task.getJobID().toString(), taskId
+ .toString(), task.isTaskCleanupTask());
+ PathDeletionContext[] contexts = buildPathDeletionContexts(
+ localFs, getLocalFiles(defaultJobConf, localTaskDir +
+ Path.SEPARATOR + TaskTracker.JOBFILE));
+ directoryCleanupThread.addToQueue(contexts);
}
} else {
if (localJobConf.getNumTasksToExecutePerJvm() == 1) {
- String taskWorkDir =
- getTaskWorkDir(task.getUser(), task.getJobID().toString(),
- taskId.toString(), task.isTaskCleanupTask());
- directoryCleanupThread.addToQueue(localFs, getLocalFiles(
- defaultJobConf, taskWorkDir));
+ PathDeletionContext[] contexts =
+ buildTaskControllerPathDeletionContexts(localFs,
+ getLocalFiles(fConf, ""), task, true /* workDir */,
+ taskController);
+ directoryCleanupThread.addToQueue(contexts);
}
}
}
@@ -3001,7 +3221,6 @@
boolean localized;
boolean keepJobFiles;
FetchStatus f;
- JobTokens jobTokens;
RunningJob(JobID jobid) {
this.jobid = jobid;
localized = false;
@@ -3379,14 +3598,8 @@
private void verifyRequest(HttpServletRequest request,
HttpServletResponse response, TaskTracker tracker, String jobId)
throws IOException {
- JobTokens jt = null;
- synchronized (tracker.runningJobs) {
- RunningJob rjob = tracker.runningJobs.get(JobID.forName(jobId));
- if (rjob == null) {
- throw new IOException("Unknown job " + jobId + "!!");
- }
- jt = rjob.jobTokens;
- }
+ SecretKey tokenSecret = tracker.getJobTokenSecretManager()
+ .retrieveTokenSecret(jobId);
// string to encrypt
String enc_str = SecureShuffleUtils.buildMsgFrom(request);
@@ -3400,17 +3613,16 @@
LOG.debug("verifying request. enc_str="+enc_str+"; hash=..."+
urlHashStr.substring(len-len/2, len-1)); // half of the hash for debug
- SecureShuffleUtils ssutil = new SecureShuffleUtils(jt.getShuffleJobToken());
// verify - throws exception
try {
- ssutil.verifyReply(urlHashStr, enc_str);
+ SecureShuffleUtils.verifyReply(urlHashStr, enc_str, tokenSecret);
} catch (IOException ioe) {
response.sendError(HttpServletResponse.SC_UNAUTHORIZED);
throw ioe;
}
// verification passed - encode the reply
- String reply = ssutil.generateHash(urlHashStr.getBytes());
+ String reply = SecureShuffleUtils.generateHash(urlHashStr.getBytes(), tokenSecret);
response.addHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
len = reply.length();
@@ -3419,19 +3631,29 @@
}
}
-
// get the full paths of the directory in all the local disks.
- private Path[] getLocalFiles(JobConf conf, String subdir) throws IOException{
+ Path[] getLocalFiles(JobConf conf, String subdir) throws IOException{
String[] localDirs = conf.getLocalDirs();
Path[] paths = new Path[localDirs.length];
FileSystem localFs = FileSystem.getLocal(conf);
+ boolean subdirNeeded = (subdir != null) && (subdir.length() > 0);
for (int i = 0; i < localDirs.length; i++) {
- paths[i] = new Path(localDirs[i], subdir);
+ paths[i] = (subdirNeeded) ? new Path(localDirs[i], subdir)
+ : new Path(localDirs[i]);
paths[i] = paths[i].makeQualified(localFs);
}
return paths;
}
+ FileSystem getLocalFileSystem(){
+ return localFs;
+ }
+
+ // only used by tests
+ void setLocalFileSystem(FileSystem fs){
+ localFs = fs;
+ }
+
int getMaxCurrentMapTasks() {
return maxMapSlots;
}
@@ -3515,22 +3737,24 @@
JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY));
}
- Class<? extends MemoryCalculatorPlugin> clazz =
- fConf.getClass(TT_MEMORY_CALCULATOR_PLUGIN,
- null, MemoryCalculatorPlugin.class);
- MemoryCalculatorPlugin memoryCalculatorPlugin =
- MemoryCalculatorPlugin
- .getMemoryCalculatorPlugin(clazz, fConf);
- LOG.info(" Using MemoryCalculatorPlugin : " + memoryCalculatorPlugin);
-
- if (memoryCalculatorPlugin != null) {
- totalVirtualMemoryOnTT = memoryCalculatorPlugin.getVirtualMemorySize();
+ // Use TT_MEMORY_CALCULATOR_PLUGIN if it is configured.
+ Class<? extends MemoryCalculatorPlugin> clazz =
+ fConf.getClass(TT_MEMORY_CALCULATOR_PLUGIN,
+ null, MemoryCalculatorPlugin.class);
+ MemoryCalculatorPlugin memoryCalculatorPlugin = (clazz == null ?
+ null : MemoryCalculatorPlugin.getMemoryCalculatorPlugin(clazz, fConf));
+ if (memoryCalculatorPlugin != null || resourceCalculatorPlugin != null) {
+ totalVirtualMemoryOnTT = (memoryCalculatorPlugin == null ?
+ resourceCalculatorPlugin.getVirtualMemorySize() :
+ memoryCalculatorPlugin.getVirtualMemorySize());
if (totalVirtualMemoryOnTT <= 0) {
LOG.warn("TaskTracker's totalVmem could not be calculated. "
+ "Setting it to " + JobConf.DISABLED_MEMORY_LIMIT);
totalVirtualMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
}
- totalPhysicalMemoryOnTT = memoryCalculatorPlugin.getPhysicalMemorySize();
+ totalPhysicalMemoryOnTT = (memoryCalculatorPlugin == null ?
+ resourceCalculatorPlugin.getPhysicalMemorySize() :
+ memoryCalculatorPlugin.getPhysicalMemorySize());
if (totalPhysicalMemoryOnTT <= 0) {
LOG.warn("TaskTracker's totalPmem could not be calculated. "
+ "Setting it to " + JobConf.DISABLED_MEMORY_LIMIT);
@@ -3662,7 +3886,7 @@
throws IOException {
// check if the tokenJob file is there..
Path skPath = new Path(systemDirectory,
- jobId.toString()+"/"+JobTokens.JOB_TOKEN_FILENAME);
+ jobId.toString()+"/"+SecureShuffleUtils.JOB_TOKEN_FILENAME);
FileStatus status = null;
long jobTokenSize = -1;
diff --git a/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java b/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java
index eeb555f..11608fa 100644
--- a/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java
+++ b/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java
@@ -55,6 +55,7 @@
private int maxReduceTasks;
private TaskTrackerHealthStatus healthStatus;
+ public static final int UNAVAILABLE = -1;
/**
* Class representing a collection of resources on this tasktracker.
*/
@@ -65,7 +66,14 @@
private long mapSlotMemorySizeOnTT;
private long reduceSlotMemorySizeOnTT;
private long availableSpace;
-
+
+ private long availableVirtualMemory = UNAVAILABLE; // in byte
+ private long availablePhysicalMemory = UNAVAILABLE; // in byte
+ private int numProcessors = UNAVAILABLE;
+ private long cumulativeCpuTime = UNAVAILABLE; // in millisecond
+ private long cpuFrequency = UNAVAILABLE; // in kHz
+ private float cpuUsage = UNAVAILABLE; // in %
+
ResourceStatus() {
totalVirtualMemory = JobConf.DISABLED_MEMORY_LIMIT;
totalPhysicalMemory = JobConf.DISABLED_MEMORY_LIMIT;
@@ -172,21 +180,160 @@
long getAvailableSpace() {
return availableSpace;
}
+
+ /**
+ * Set the amount of available virtual memory on the tasktracker.
+ * If the input is not a valid number, it will be set to UNAVAILABLE
+ *
+ * @param vmem amount of available virtual memory on the tasktracker
+ * in bytes.
+ */
+ void setAvailableVirtualMemory(long availableMem) {
+ availableVirtualMemory = availableMem > 0 ?
+ availableMem : UNAVAILABLE;
+ }
+
+ /**
+ * Get the amount of available virtual memory on the tasktracker.
+ * Will return UNAVAILABLE if it cannot be obtained
+ *
+ * @return the amount of available virtual memory on the tasktracker
+ * in bytes.
+ */
+ long getAvailabelVirtualMemory() {
+ return availableVirtualMemory;
+ }
+
+ /**
+ * Set the amount of available physical memory on the tasktracker.
+ * If the input is not a valid number, it will be set to UNAVAILABLE
+ *
+ * @param availableRAM amount of available physical memory on the
+ * tasktracker in bytes.
+ */
+ void setAvailablePhysicalMemory(long availableRAM) {
+ availablePhysicalMemory = availableRAM > 0 ?
+ availableRAM : UNAVAILABLE;
+ }
+
+ /**
+ * Get the amount of available physical memory on the tasktracker.
+ * Will return UNAVAILABLE if it cannot be obtained
+ *
+ * @return amount of available physical memory on the tasktracker in bytes.
+ */
+ long getAvailablePhysicalMemory() {
+ return availablePhysicalMemory;
+ }
+
+ /**
+ * Set the CPU frequency of this TaskTracker
+ * If the input is not a valid number, it will be set to UNAVAILABLE
+ *
+ * @param cpuFrequency CPU frequency in kHz
+ */
+ public void setCpuFrequency(long cpuFrequency) {
+ this.cpuFrequency = cpuFrequency > 0 ?
+ cpuFrequency : UNAVAILABLE;
+ }
+
+ /**
+ * Get the CPU frequency of this TaskTracker
+ * Will return UNAVAILABLE if it cannot be obtained
+ *
+ * @return CPU frequency in kHz
+ */
+ public long getCpuFrequency() {
+ return cpuFrequency;
+ }
+
+ /**
+ * Set the number of processors on this TaskTracker
+ * If the input is not a valid number, it will be set to UNAVAILABLE
+ *
+ * @param numProcessors number of processors
+ */
+ public void setNumProcessors(int numProcessors) {
+ this.numProcessors = numProcessors > 0 ?
+ numProcessors : UNAVAILABLE;
+ }
+
+ /**
+ * Get the number of processors on this TaskTracker
+ * Will return UNAVAILABLE if it cannot be obtained
+ *
+ * @return number of processors
+ */
+ public int getNumProcessors() {
+ return numProcessors;
+ }
+
+ /**
+ * Set the cumulative CPU time on this TaskTracker since it is up
+ * It can be set to UNAVAILABLE if it is currently unavailable.
+ *
+ * @param cumulativeCpuTime Used CPU time in millisecond
+ */
+ public void setCumulativeCpuTime(long cumulativeCpuTime) {
+ this.cumulativeCpuTime = cumulativeCpuTime > 0 ?
+ cumulativeCpuTime : UNAVAILABLE;
+ }
+
+ /**
+ * Get the cumulative CPU time on this TaskTracker since it is up
+ * Will return UNAVAILABLE if it cannot be obtained
+ *
+ * @return used CPU time in milliseconds
+ */
+ public long getCumulativeCpuTime() {
+ return cumulativeCpuTime;
+ }
+
+ /**
+ * Set the CPU usage on this TaskTracker
+ *
+ * @param cpuUsage CPU usage in %
+ */
+ public void setCpuUsage(float cpuUsage) {
+ this.cpuUsage = cpuUsage;
+ }
+
+ /**
+ * Get the CPU usage on this TaskTracker
+ * Will return UNAVAILABLE if it cannot be obtained
+ *
+ * @return CPU usage in %
+ */
+ public float getCpuUsage() {
+ return cpuUsage;
+ }
public void write(DataOutput out) throws IOException {
WritableUtils.writeVLong(out, totalVirtualMemory);
WritableUtils.writeVLong(out, totalPhysicalMemory);
+ WritableUtils.writeVLong(out, availableVirtualMemory);
+ WritableUtils.writeVLong(out, availablePhysicalMemory);
WritableUtils.writeVLong(out, mapSlotMemorySizeOnTT);
WritableUtils.writeVLong(out, reduceSlotMemorySizeOnTT);
WritableUtils.writeVLong(out, availableSpace);
+ WritableUtils.writeVLong(out, cumulativeCpuTime);
+ WritableUtils.writeVLong(out, cpuFrequency);
+ WritableUtils.writeVInt(out, numProcessors);
+ out.writeFloat(getCpuUsage());
}
public void readFields(DataInput in) throws IOException {
totalVirtualMemory = WritableUtils.readVLong(in);
totalPhysicalMemory = WritableUtils.readVLong(in);
+ availableVirtualMemory = WritableUtils.readVLong(in);
+ availablePhysicalMemory = WritableUtils.readVLong(in);
mapSlotMemorySizeOnTT = WritableUtils.readVLong(in);
reduceSlotMemorySizeOnTT = WritableUtils.readVLong(in);
availableSpace = WritableUtils.readVLong(in);
+ cumulativeCpuTime = WritableUtils.readVLong(in);
+ cpuFrequency = WritableUtils.readVLong(in);
+ numProcessors = WritableUtils.readVInt(in);
+ setCpuUsage(in.readFloat());
}
}
diff --git a/src/java/org/apache/hadoop/mapred/lib/Chain.java b/src/java/org/apache/hadoop/mapred/lib/Chain.java
index 847ec24..c14e9ad 100644
--- a/src/java/org/apache/hadoop/mapred/lib/Chain.java
+++ b/src/java/org/apache/hadoop/mapred/lib/Chain.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.mapred.lib;
import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.Stringifier;
-import org.apache.hadoop.io.DefaultStringifier;
import org.apache.hadoop.io.serializer.Deserializer;
import org.apache.hadoop.io.serializer.Serialization;
import org.apache.hadoop.io.serializer.SerializationFactory;
@@ -32,45 +30,19 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
/**
* The Chain class provides all the common functionality for the
* {@link ChainMapper} and the {@link ChainReducer} classes.
+ * @deprecated Use {@link org.apache.hadoop.mapreduce.lib.chain.Chain} instead
*/
-class Chain {
- private static final String CHAIN_MAPPER = "chain.mapper";
- private static final String CHAIN_REDUCER = "chain.reducer";
-
- private static final String CHAIN_MAPPER_SIZE = ".size";
- private static final String CHAIN_MAPPER_CLASS = ".mapper.class.";
- private static final String CHAIN_MAPPER_CONFIG = ".mapper.config.";
- private static final String CHAIN_REDUCER_CLASS = ".reducer.class";
- private static final String CHAIN_REDUCER_CONFIG = ".reducer.config";
+@Deprecated
+class Chain extends org.apache.hadoop.mapreduce.lib.chain.Chain {
private static final String MAPPER_BY_VALUE = "chain.mapper.byValue";
private static final String REDUCER_BY_VALUE = "chain.reducer.byValue";
- private static final String MAPPER_INPUT_KEY_CLASS =
- "chain.mapper.input.key.class";
- private static final String MAPPER_INPUT_VALUE_CLASS =
- "chain.mapper.input.value.class";
- private static final String MAPPER_OUTPUT_KEY_CLASS =
- "chain.mapper.output.key.class";
- private static final String MAPPER_OUTPUT_VALUE_CLASS =
- "chain.mapper.output.value.class";
- private static final String REDUCER_INPUT_KEY_CLASS =
- "chain.reducer.input.key.class";
- private static final String REDUCER_INPUT_VALUE_CLASS =
- "chain.reducer.input.value.class";
- private static final String REDUCER_OUTPUT_KEY_CLASS =
- "chain.reducer.output.key.class";
- private static final String REDUCER_OUTPUT_VALUE_CLASS =
- "chain.reducer.output.value.class";
-
- private boolean isMap;
-
private JobConf chainJobConf;
private List<Mapper> mappers = new ArrayList<Mapper>();
@@ -92,51 +64,7 @@
* Reducer.
*/
Chain(boolean isMap) {
- this.isMap = isMap;
- }
-
- /**
- * Returns the prefix to use for the configuration of the chain depending
- * if it is for a Mapper or a Reducer.
- *
- * @param isMap TRUE for Mapper, FALSE for Reducer.
- * @return the prefix to use.
- */
- private static String getPrefix(boolean isMap) {
- return (isMap) ? CHAIN_MAPPER : CHAIN_REDUCER;
- }
-
- /**
- * Creates a {@link JobConf} for one of the Maps or Reduce in the chain.
- * <p/>
- * It creates a new JobConf using the chain job's JobConf as base and adds to
- * it the configuration properties for the chain element. The keys of the
- * chain element jobConf have precedence over the given JobConf.
- *
- * @param jobConf the chain job's JobConf.
- * @param confKey the key for chain element configuration serialized in the
- * chain job's JobConf.
- * @return a new JobConf aggregating the chain job's JobConf with the chain
- * element configuration properties.
- */
- private static JobConf getChainElementConf(JobConf jobConf, String confKey) {
- JobConf conf;
- try {
- Stringifier<JobConf> stringifier =
- new DefaultStringifier<JobConf>(jobConf, JobConf.class);
- conf = stringifier.fromString(jobConf.get(confKey, null));
- } catch (IOException ioex) {
- throw new RuntimeException(ioex);
- }
- // we have to do this because the Writable desearialization clears all
- // values set in the conf making not possible do do a new JobConf(jobConf)
- // in the creation of the conf above
- jobConf = new JobConf(jobConf);
-
- for(Map.Entry<String, String> entry : conf) {
- jobConf.set(entry.getKey(), entry.getValue());
- }
- return jobConf;
+ super(isMap);
}
/**
@@ -169,82 +97,27 @@
String prefix = getPrefix(isMap);
// if a reducer chain check the Reducer has been already set
- if (!isMap) {
- if (jobConf.getClass(prefix + CHAIN_REDUCER_CLASS,
- Reducer.class) == null) {
- throw new IllegalStateException(
- "A Mapper can be added to the chain only after the Reducer has " +
- "been set");
- }
- }
- int index = jobConf.getInt(prefix + CHAIN_MAPPER_SIZE, 0);
+ checkReducerAlreadySet(isMap, jobConf, prefix, true);
+
+ // set the mapper class
+ int index = getIndex(jobConf, prefix);
jobConf.setClass(prefix + CHAIN_MAPPER_CLASS + index, klass, Mapper.class);
-
- // if it is a reducer chain and the first Mapper is being added check the
- // key and value input classes of the mapper match those of the reducer
- // output.
- if (!isMap && index == 0) {
- JobConf reducerConf =
- getChainElementConf(jobConf, prefix + CHAIN_REDUCER_CONFIG);
- if (! inputKeyClass.isAssignableFrom(
- reducerConf.getClass(REDUCER_OUTPUT_KEY_CLASS, null))) {
- throw new IllegalArgumentException("The Reducer output key class does" +
- " not match the Mapper input key class");
- }
- if (! inputValueClass.isAssignableFrom(
- reducerConf.getClass(REDUCER_OUTPUT_VALUE_CLASS, null))) {
- throw new IllegalArgumentException("The Reducer output value class" +
- " does not match the Mapper input value class");
- }
- } else if (index > 0) {
- // check the that the new Mapper in the chain key and value input classes
- // match those of the previous Mapper output.
- JobConf previousMapperConf =
- getChainElementConf(jobConf, prefix + CHAIN_MAPPER_CONFIG +
- (index - 1));
- if (! inputKeyClass.isAssignableFrom(
- previousMapperConf.getClass(MAPPER_OUTPUT_KEY_CLASS, null))) {
- throw new IllegalArgumentException("The Mapper output key class does" +
- " not match the previous Mapper input key class");
- }
- if (! inputValueClass.isAssignableFrom(
- previousMapperConf.getClass(MAPPER_OUTPUT_VALUE_CLASS, null))) {
- throw new IllegalArgumentException("The Mapper output value class" +
- " does not match the previous Mapper input value class");
- }
- }
-
+
+ validateKeyValueTypes(isMap, jobConf, inputKeyClass, inputValueClass,
+ outputKeyClass, outputValueClass, index, prefix);
+
// if the Mapper does not have a private JobConf create an empty one
if (mapperConf == null) {
- // using a JobConf without defaults to make it lightweight.
- // still the chain JobConf may have all defaults and this conf is
- // overlapped to the chain JobConf one.
+ // using a JobConf without defaults to make it lightweight.
+ // still the chain JobConf may have all defaults and this conf is
+ // overlapped to the chain JobConf one.
mapperConf = new JobConf(true);
}
-
- // store in the private mapper conf the input/output classes of the mapper
- // and if it works by value or by reference
+ // store in the private mapper conf if it works by value or by reference
mapperConf.setBoolean(MAPPER_BY_VALUE, byValue);
- mapperConf.setClass(MAPPER_INPUT_KEY_CLASS, inputKeyClass, Object.class);
- mapperConf.setClass(MAPPER_INPUT_VALUE_CLASS, inputValueClass,
- Object.class);
- mapperConf.setClass(MAPPER_OUTPUT_KEY_CLASS, outputKeyClass, Object.class);
- mapperConf.setClass(MAPPER_OUTPUT_VALUE_CLASS, outputValueClass,
- Object.class);
-
- // serialize the private mapper jobconf in the chain jobconf.
- Stringifier<JobConf> stringifier =
- new DefaultStringifier<JobConf>(jobConf, JobConf.class);
- try {
- jobConf.set(prefix + CHAIN_MAPPER_CONFIG + index,
- stringifier.toString(new JobConf(mapperConf)));
- }
- catch (IOException ioEx) {
- throw new RuntimeException(ioEx);
- }
-
- // increment the chain counter
- jobConf.setInt(prefix + CHAIN_MAPPER_SIZE, index + 1);
+
+ setMapperConf(isMap, jobConf, inputKeyClass, inputValueClass,
+ outputKeyClass, outputValueClass, mapperConf, index, prefix);
}
/**
@@ -273,13 +146,10 @@
Class<? extends V2> outputValueClass,
boolean byValue, JobConf reducerConf) {
String prefix = getPrefix(false);
-
- if (jobConf.getClass(prefix + CHAIN_REDUCER_CLASS, null) != null) {
- throw new IllegalStateException("Reducer has been already set");
- }
+ checkReducerAlreadySet(false, jobConf, prefix, false);
jobConf.setClass(prefix + CHAIN_REDUCER_CLASS, klass, Reducer.class);
-
+
// if the Reducer does not have a private JobConf create an empty one
if (reducerConf == null) {
// using a JobConf without defaults to make it lightweight.
@@ -291,24 +161,9 @@
// store in the private reducer conf the input/output classes of the reducer
// and if it works by value or by reference
reducerConf.setBoolean(MAPPER_BY_VALUE, byValue);
- reducerConf.setClass(REDUCER_INPUT_KEY_CLASS, inputKeyClass, Object.class);
- reducerConf.setClass(REDUCER_INPUT_VALUE_CLASS, inputValueClass,
- Object.class);
- reducerConf.setClass(REDUCER_OUTPUT_KEY_CLASS, outputKeyClass,
- Object.class);
- reducerConf.setClass(REDUCER_OUTPUT_VALUE_CLASS, outputValueClass,
- Object.class);
- // serialize the private mapper jobconf in the chain jobconf.
- Stringifier<JobConf> stringifier =
- new DefaultStringifier<JobConf>(jobConf, JobConf.class);
- try {
- jobConf.set(prefix + CHAIN_REDUCER_CONFIG,
- stringifier.toString(new JobConf(reducerConf)));
- }
- catch (IOException ioEx) {
- throw new RuntimeException(ioEx);
- }
+ setReducerConf(jobConf, inputKeyClass, inputValueClass, outputKeyClass,
+ outputValueClass, reducerConf, prefix);
}
/**
@@ -325,8 +180,8 @@
for (int i = 0; i < index; i++) {
Class<? extends Mapper> klass =
jobConf.getClass(prefix + CHAIN_MAPPER_CLASS + i, null, Mapper.class);
- JobConf mConf =
- getChainElementConf(jobConf, prefix + CHAIN_MAPPER_CONFIG + i);
+ JobConf mConf = new JobConf(
+ getChainElementConf(jobConf, prefix + CHAIN_MAPPER_CONFIG + i));
Mapper mapper = ReflectionUtils.newInstance(klass, mConf);
mappers.add(mapper);
@@ -343,8 +198,8 @@
Class<? extends Reducer> klass =
jobConf.getClass(prefix + CHAIN_REDUCER_CLASS, null, Reducer.class);
if (klass != null) {
- JobConf rConf =
- getChainElementConf(jobConf, prefix + CHAIN_REDUCER_CONFIG);
+ JobConf rConf = new JobConf(
+ getChainElementConf(jobConf, prefix + CHAIN_REDUCER_CONFIG));
reducer = ReflectionUtils.newInstance(klass, rConf);
if (rConf.getBoolean(REDUCER_BY_VALUE, true)) {
reducerKeySerialization = serializationFactory
diff --git a/src/java/org/apache/hadoop/mapred/lib/ChainMapper.java b/src/java/org/apache/hadoop/mapred/lib/ChainMapper.java
index b2bc246..089c9db 100644
--- a/src/java/org/apache/hadoop/mapred/lib/ChainMapper.java
+++ b/src/java/org/apache/hadoop/mapred/lib/ChainMapper.java
@@ -86,7 +86,10 @@
* RunningJob job = jc.submitJob(conf);
* ...
* </pre>
+ * @deprecated
+ * Use {@link org.apache.hadoop.mapreduce.lib.chain.ChainMapper} instead
*/
+@Deprecated
public class ChainMapper implements Mapper {
/**
diff --git a/src/java/org/apache/hadoop/mapred/lib/ChainReducer.java b/src/java/org/apache/hadoop/mapred/lib/ChainReducer.java
index f48ed46..25c2c75 100644
--- a/src/java/org/apache/hadoop/mapred/lib/ChainReducer.java
+++ b/src/java/org/apache/hadoop/mapred/lib/ChainReducer.java
@@ -86,7 +86,10 @@
* RunningJob job = jc.submitJob(conf);
* ...
* </pre>
+ * @deprecated
+ * Use {@link org.apache.hadoop.mapreduce.lib.chain.ChainReducer} instead
*/
+@Deprecated
public class ChainReducer implements Reducer {
/**
diff --git a/src/java/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.java b/src/java/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.java
index f2ebc3c..873d864 100644
--- a/src/java/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.java
+++ b/src/java/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.java
@@ -46,7 +46,8 @@
*
* The field separator is under attribute "mapreduce.fieldsel.data.field.separator"
*
- * The map output field list spec is under attribute "mapreduce.fieldsel.mapreduce.fieldsel.map.output.key.value.fields.spec".
+ * The map output field list spec is under attribute
+ * "mapreduce.fieldsel.map.output.key.value.fields.spec".
* The value is expected to be like "keyFieldsSpec:valueFieldsSpec"
* key/valueFieldsSpec are comma (,) separated field spec: fieldSpec,fieldSpec,fieldSpec ...
* Each field spec can be a simple number (e.g. 5) specifying a specific field, or a range
@@ -57,7 +58,8 @@
* Here is an example: "4,3,0,1:6,5,1-3,7-". It specifies to use fields 4,3,0 and 1 for keys,
* and use fields 6,5,1,2,3,7 and above for values.
*
- * The reduce output field list spec is under attribute "mapreduce.fieldsel.mapreduce.fieldsel.reduce.output.key.value.fields.spec".
+ * The reduce output field list spec is under attribute
+ * "mapreduce.fieldsel.reduce.output.key.value.fields.spec".
*
* The reducer extracts output key/value pairs in a similar manner, except that
* the key is never ignored.
@@ -156,13 +158,14 @@
}
public void configure(JobConf job) {
- this.fieldSeparator = job.get("mapreduce.fieldsel.data.field.separator", "\t");
- this.mapOutputKeyValueSpec = job.get("mapreduce.fieldsel.mapreduce.fieldsel.map.output.key.value.fields.spec",
- "0-:");
+ this.fieldSeparator = job.get(FieldSelectionHelper.DATA_FIELD_SEPERATOR,
+ "\t");
+ this.mapOutputKeyValueSpec = job.get(
+ FieldSelectionHelper.MAP_OUTPUT_KEY_VALUE_SPEC, "0-:");
this.ignoreInputKey = TextInputFormat.class.getCanonicalName().equals(
job.getInputFormat().getClass().getCanonicalName());
this.reduceOutputKeyValueSpec = job.get(
- "mapreduce.fieldsel.mapreduce.fieldsel.reduce.output.key.value.fields.spec", "0-:");
+ FieldSelectionHelper.REDUCE_OUTPUT_KEY_VALUE_SPEC, "0-:");
parseOutputKeyValueSpec();
LOG.info(specToString());
}
diff --git a/src/java/org/apache/hadoop/mapred/tools/MRAdmin.java b/src/java/org/apache/hadoop/mapred/tools/MRAdmin.java
index e396862..b3a2a8a 100644
--- a/src/java/org/apache/hadoop/mapred/tools/MRAdmin.java
+++ b/src/java/org/apache/hadoop/mapred/tools/MRAdmin.java
@@ -28,6 +28,7 @@
import org.apache.hadoop.mapred.JobTracker;
import org.apache.hadoop.mapred.AdminOperationsProtocol;
import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.RefreshUserToGroupMappingsProtocol;
import org.apache.hadoop.security.UnixUserGroupInformation;
import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
import org.apache.hadoop.util.Tool;
@@ -53,16 +54,20 @@
private static void printHelp(String cmd) {
String summary = "hadoop mradmin is the command to execute Map-Reduce administrative commands.\n" +
"The full syntax is: \n\n" +
- "hadoop mradmin [-refreshServiceAcl] [-refreshQueues] [-help [cmd]] "
- + "[-refreshNodes]\n";
+ "hadoop mradmin [-refreshServiceAcl] [-refreshQueues] " +
+ "[-refreshNodes] [-refreshUserToGroupsMappings] [-help [cmd]]\n";
String refreshServiceAcl = "-refreshServiceAcl: Reload the service-level authorization policy file\n" +
"\t\tJobtracker will reload the authorization policy file.\n";
String refreshQueues =
- "-refreshQueues: Reload the queue acls and state.\n"
- + "\t\tJobTracker will reload the mapred-queues.xml file.\n";
+ "-refreshQueues: Reload the queues' acls, states and "
+ + "scheduler specific properties.\n"
+ + "\t\tJobTracker will reload the mapred-queues configuration file.\n";
+ String refreshUserToGroupsMappings =
+ "-refreshUserToGroupsMappings: Refresh user-to-groups mappings\n";
+
String refreshNodes =
"-refreshNodes: Refresh the hosts information at the jobtracker.\n";
@@ -73,6 +78,8 @@
System.out.println(refreshServiceAcl);
} else if ("refreshQueues".equals(cmd)) {
System.out.println(refreshQueues);
+ } else if ("refreshUserToGroupsMappings".equals(cmd)) {
+ System.out.println(refreshUserToGroupsMappings);
} else if ("refreshNodes".equals(cmd)) {
System.out.println(refreshNodes);
} else if ("help".equals(cmd)) {
@@ -98,12 +105,15 @@
System.err.println("Usage: java MRAdmin" + " [-refreshServiceAcl]");
} else if ("-refreshQueues".equals(cmd)) {
System.err.println("Usage: java MRAdmin" + " [-refreshQueues]");
+ } else if ("-refreshUserToGroupsMappings".equals(cmd)) {
+ System.err.println("Usage: java MRAdmin" + " [-refreshUserToGroupsMappings]");
} else if ("-refreshNodes".equals(cmd)) {
System.err.println("Usage: java MRAdmin" + " [-refreshNodes]");
} else {
System.err.println("Usage: java MRAdmin");
System.err.println(" [-refreshServiceAcl]");
System.err.println(" [-refreshQueues]");
+ System.err.println(" [-refreshUserToGroupsMappings]");
System.err.println(" [-refreshNodes]");
System.err.println(" [-help [cmd]]");
System.err.println();
@@ -142,6 +152,29 @@
return 0;
}
+ /**
+ * Refresh the user-to-groups mappings on the {@link JobTracker}.
+ * @return exitcode 0 on success, non-zero on failure
+ * @throws IOException
+ */
+ private int refreshUserToGroupsMappings() throws IOException {
+ // Get the current configuration
+ Configuration conf = getConf();
+ // Create the client
+ RefreshUserToGroupMappingsProtocol refreshProtocol =
+ (RefreshUserToGroupMappingsProtocol)
+ RPC.getProxy(RefreshUserToGroupMappingsProtocol.class,
+ RefreshUserToGroupMappingsProtocol.versionID,
+ JobTracker.getAddress(conf), getUGI(conf), conf,
+ NetUtils.getSocketFactory(conf,
+ RefreshUserToGroupMappingsProtocol.class));
+
+ // Refresh the user-to-groups mappings
+ refreshProtocol.refreshUserToGroupsMappings(conf);
+
+ return 0;
+ }
+
private int refreshQueues() throws IOException {
// Get the current configuration
Configuration conf = getConf();
@@ -196,12 +229,11 @@
int exitCode = -1;
int i = 0;
String cmd = args[i++];
-
//
// verify that we have enough command line parameters
//
- if ("-refreshServiceAcl".equals(cmd) || "-refreshQueues".equals(cmd)
- || "-refreshNodes".equals(cmd)) {
+ if ("-refreshServiceAcl".equals(cmd) || "-refreshQueues".equals(cmd) ||
+ "-refreshNodes".equals(cmd) || "-refreshUserToGroupsMappings".equals(cmd)) {
if (args.length != 1) {
printUsage(cmd);
return exitCode;
@@ -214,6 +246,8 @@
exitCode = refreshAuthorizationPolicy();
} else if ("-refreshQueues".equals(cmd)) {
exitCode = refreshQueues();
+ } else if ("-refreshUserToGroupsMappings".equals(cmd)) {
+ exitCode = refreshUserToGroupsMappings();
} else if ("-refreshNodes".equals(cmd)) {
exitCode = refreshNodes();
} else if ("-help".equals(cmd)) {
diff --git a/src/java/org/apache/hadoop/mapreduce/Cluster.java b/src/java/org/apache/hadoop/mapreduce/Cluster.java
index c5807cb..6ae3db4 100644
--- a/src/java/org/apache/hadoop/mapreduce/Cluster.java
+++ b/src/java/org/apache/hadoop/mapreduce/Cluster.java
@@ -46,6 +46,7 @@
private Configuration conf;
private FileSystem fs = null;
private Path sysDir = null;
+ private Path stagingAreaDir = null;
private Path jobHistoryDir = null;
static {
@@ -76,6 +77,7 @@
ClientProtocol client;
String tracker = conf.get("mapred.job.tracker", "local");
if ("local".equals(tracker)) {
+ conf.setInt("mapreduce.job.maps", 1);
client = new LocalJobRunner(conf);
} else {
client = createRPCProxy(JobTracker.getAddress(conf), conf);
@@ -222,6 +224,19 @@
}
return sysDir;
}
+
+ /**
+ * Grab the jobtracker's view of the staging directory path where
+ * job-specific files will be placed.
+ *
+ * @return the staging directory where job-specific files are to be placed.
+ */
+ public Path getStagingAreaDir() throws IOException, InterruptedException {
+ if (stagingAreaDir == null) {
+ stagingAreaDir = new Path(client.getStagingAreaDir());
+ }
+ return stagingAreaDir;
+ }
/**
* Get the job history file path for a given job id. The job history file at
diff --git a/src/java/org/apache/hadoop/mapreduce/Job.java b/src/java/org/apache/hadoop/mapreduce/Job.java
index 39e3393..46cf937 100644
--- a/src/java/org/apache/hadoop/mapreduce/Job.java
+++ b/src/java/org/apache/hadoop/mapreduce/Job.java
@@ -20,8 +20,6 @@
import java.io.BufferedReader;
import java.io.BufferedWriter;
-import java.io.DataInput;
-import java.io.DataOutput;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
@@ -30,7 +28,6 @@
import java.io.OutputStreamWriter;
import java.net.URL;
import java.net.URLConnection;
-import java.util.Arrays;
import java.net.URI;
import javax.security.auth.login.LoginException;
@@ -40,12 +37,9 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configuration.IntegerRanges;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.mapreduce.task.JobContextImpl;
import org.apache.hadoop.mapreduce.util.ConfigUtil;
@@ -956,7 +950,7 @@
ensureState(JobState.DEFINE);
setUseNewAPI();
status = new JobSubmitter(cluster.getFileSystem(),
- cluster.getClient()).submitJobInternal(this);
+ cluster.getClient()).submitJobInternal(this, cluster);
state = JobState.RUNNING;
}
@@ -1033,12 +1027,52 @@
return isSuccessful();
}
+ /**
+ * @return true if the profile parameters indicate that this is using
+ * hprof, which generates profile files in a particular location
+ * that we can retrieve to the client.
+ */
+ private boolean shouldDownloadProfile() {
+ // Check the argument string that was used to initialize profiling.
+ // If this indicates hprof and file-based output, then we're ok to
+ // download.
+ String profileParams = getProfileParams();
+
+ if (null == profileParams) {
+ return false;
+ }
+
+ // Split this on whitespace.
+ String [] parts = profileParams.split("[ \\t]+");
+
+ // If any of these indicate hprof, and the use of output files, return true.
+ boolean hprofFound = false;
+ boolean fileFound = false;
+ for (String p : parts) {
+ if (p.startsWith("-agentlib:hprof") || p.startsWith("-Xrunhprof")) {
+ hprofFound = true;
+
+ // This contains a number of comma-delimited components, one of which
+ // may specify the file to write to. Make sure this is present and
+ // not empty.
+ String [] subparts = p.split(",");
+ for (String sub : subparts) {
+ if (sub.startsWith("file=") && sub.length() != "file=".length()) {
+ fileFound = true;
+ }
+ }
+ }
+ }
+
+ return hprofFound && fileFound;
+ }
+
private void printTaskEvents(TaskCompletionEvent[] events,
Job.TaskStatusFilter filter, boolean profiling, IntegerRanges mapRanges,
IntegerRanges reduceRanges) throws IOException, InterruptedException {
for (TaskCompletionEvent event : events) {
TaskCompletionEvent.Status status = event.getStatus();
- if (profiling &&
+ if (profiling && shouldDownloadProfile() &&
(status == TaskCompletionEvent.Status.SUCCEEDED ||
status == TaskCompletionEvent.Status.FAILED) &&
(event.isMapTask() ? mapRanges : reduceRanges).
@@ -1211,106 +1245,4 @@
}
return ugi;
}
-
- /**
- * Read a splits file into a list of raw splits.
- *
- * @param in the stream to read from
- * @return the complete list of splits
- * @throws IOException
- */
- public static RawSplit[] readSplitFile(DataInput in) throws IOException {
- byte[] header = new byte[JobSubmitter.SPLIT_FILE_HEADER.length];
- in.readFully(header);
- if (!Arrays.equals(JobSubmitter.SPLIT_FILE_HEADER, header)) {
- throw new IOException("Invalid header on split file");
- }
- int vers = WritableUtils.readVInt(in);
- if (vers != JobSubmitter.CURRENT_SPLIT_FILE_VERSION) {
- throw new IOException("Unsupported split version " + vers);
- }
- int len = WritableUtils.readVInt(in);
- RawSplit[] result = new RawSplit[len];
- for (int i=0; i < len; ++i) {
- result[i] = new RawSplit();
- result[i].readFields(in);
- }
- return result;
- }
-
- public static class RawSplit implements Writable {
- private String splitClass;
- private BytesWritable bytes = new BytesWritable();
- private String[] locations;
- long dataLength;
-
- public RawSplit() {
- }
-
- protected RawSplit(String splitClass, BytesWritable bytes,
- String[] locations, long dataLength) {
- this.splitClass = splitClass;
- this.bytes = bytes;
- this.locations = locations;
- this.dataLength = dataLength;
- }
-
- public void setBytes(byte[] data, int offset, int length) {
- bytes.set(data, offset, length);
- }
-
- public void setClassName(String className) {
- splitClass = className;
- }
-
- public String getClassName() {
- return splitClass;
- }
-
- public BytesWritable getBytes() {
- return bytes;
- }
-
- public void clearBytes() {
- bytes = null;
- }
-
- public void setLocations(String[] locations) {
- this.locations = locations;
- }
-
- public String[] getLocations() {
- return locations;
- }
-
- public long getDataLength() {
- return dataLength;
- }
-
- public void setDataLength(long l) {
- dataLength = l;
- }
-
- public void readFields(DataInput in) throws IOException {
- splitClass = Text.readString(in);
- dataLength = in.readLong();
- bytes.readFields(in);
- int len = WritableUtils.readVInt(in);
- locations = new String[len];
- for (int i=0; i < len; ++i) {
- locations[i] = Text.readString(in);
- }
- }
-
- public void write(DataOutput out) throws IOException {
- Text.writeString(out, splitClass);
- out.writeLong(dataLength);
- bytes.write(out);
- WritableUtils.writeVInt(out, locations.length);
- for (int i = 0; i < locations.length; i++) {
- Text.writeString(out, locations[i]);
- }
- }
- }
-
}
diff --git a/src/java/org/apache/hadoop/mapreduce/JobContext.java b/src/java/org/apache/hadoop/mapreduce/JobContext.java
index 839b8b5..b77f816 100644
--- a/src/java/org/apache/hadoop/mapreduce/JobContext.java
+++ b/src/java/org/apache/hadoop/mapreduce/JobContext.java
@@ -55,6 +55,7 @@
public static final String JAR = "mapreduce.job.jar";
public static final String ID = "mapreduce.job.id";
public static final String JOB_NAME = "mapreduce.job.name";
+ public static final String JAR_UNPACK_PATTERN = "mapreduce.job.jar.unpack.pattern";
public static final String USER_NAME = "mapreduce.job.user.name";
public static final String PRIORITY = "mapreduce.job.priority";
public static final String QUEUE_NAME = "mapreduce.job.queuename";
@@ -106,6 +107,10 @@
"mapreduce.job.cache.files.timestamps";
public static final String CACHE_ARCHIVES_TIMESTAMPS =
"mapreduce.job.cache.archives.timestamps";
+ public static final String CACHE_FILE_VISIBILITIES =
+ "mapreduce.job.cache.files.visibilities";
+ public static final String CACHE_ARCHIVES_VISIBILITIES =
+ "mapreduce.job.cache.archives.visibilities";
public static final String CACHE_SYMLINK =
"mapreduce.job.cache.symlink.create";
@@ -162,7 +167,7 @@
"mapreduce.map.skip.proc-count.auto-incr";
public static final String MAP_SKIP_MAX_RECORDS =
"mapreduce.map.skip.maxrecords";
- public static final String MAP_COMBINE_MIN_SPISS =
+ public static final String MAP_COMBINE_MIN_SPILLS =
"mapreduce.map.combine.minspills";
public static final String MAP_OUTPUT_COMPRESS =
"mapreduce.map.output.compress";
@@ -212,6 +217,10 @@
"mapreduce.reduce.shuffle.connect.timeout";
public static final String SHUFFLE_READ_TIMEOUT =
"mapreduce.reduce.shuffle.read.timeout";
+ public static final String SHUFFLE_FETCH_FAILURES =
+ "mapreduce.reduce.shuffle.maxfetchfailures";
+ public static final String SHUFFLE_NOTIFY_READERROR =
+ "mapreduce.reduce.shuffle.notify.readerror";
public static final String REDUCE_SKIP_INCR_PROC_COUNT =
"mapreduce.reduce.skip.proc-count.auto-incr";
public static final String REDUCE_SKIP_MAXGROUPS =
diff --git a/src/java/org/apache/hadoop/mapreduce/JobCounter.properties b/src/java/org/apache/hadoop/mapreduce/JobCounter.properties
index e6a4e22..7eaf744 100644
--- a/src/java/org/apache/hadoop/mapreduce/JobCounter.properties
+++ b/src/java/org/apache/hadoop/mapreduce/JobCounter.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
# ResourceBundle properties file for job-level counters
CounterGroupName= Job Counters
diff --git a/src/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java b/src/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
new file mode 100644
index 0000000..95a7c69
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
@@ -0,0 +1,130 @@
+/**
+ * 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.mapreduce;
+
+import java.io.IOException;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+/**
+ * A utility to manage job submission files.
+ */
+@InterfaceAudience.Private
+public class JobSubmissionFiles {
+
+ // job submission directory is private!
+ final public static FsPermission JOB_DIR_PERMISSION =
+ FsPermission.createImmutable((short) 0700); // rwx--------
+ //job files are world-wide readable and owner writable
+ final public static FsPermission JOB_FILE_PERMISSION =
+ FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+ public static Path getJobSplitFile(Path jobSubmissionDir) {
+ return new Path(jobSubmissionDir, "job.split");
+ }
+
+ public static Path getJobSplitMetaFile(Path jobSubmissionDir) {
+ return new Path(jobSubmissionDir, "job.splitmetainfo");
+ }
+
+ /**
+ * Get the job conf path.
+ */
+ public static Path getJobConfPath(Path jobSubmitDir) {
+ return new Path(jobSubmitDir, "job.xml");
+ }
+
+ /**
+ * Get the job jar path.
+ */
+ public static Path getJobJar(Path jobSubmitDir) {
+ return new Path(jobSubmitDir, "job.jar");
+ }
+
+ /**
+ * Get the job distributed cache files path.
+ * @param jobSubmitDir
+ */
+ public static Path getJobDistCacheFiles(Path jobSubmitDir) {
+ return new Path(jobSubmitDir, "files");
+ }
+ /**
+ * Get the job distributed cache archives path.
+ * @param jobSubmitDir
+ */
+ public static Path getJobDistCacheArchives(Path jobSubmitDir) {
+ return new Path(jobSubmitDir, "archives");
+ }
+ /**
+ * Get the job distributed cache libjars path.
+ * @param jobSubmitDir
+ */
+ public static Path getJobDistCacheLibjars(Path jobSubmitDir) {
+ return new Path(jobSubmitDir, "libjars");
+ }
+
+ /**
+ * Initializes the staging directory and returns the path. It also
+ * keeps track of all necessary ownership & permissions
+ * @param cluster
+ * @param conf
+ */
+ public static Path getStagingDir(Cluster cluster, Configuration conf)
+ throws IOException, InterruptedException {
+ Path stagingArea = cluster.getStagingAreaDir();
+ FileSystem fs = stagingArea.getFileSystem(conf);
+ String realUser;
+ String currentUser;
+ try {
+ UserGroupInformation ugi = UnixUserGroupInformation.login();
+ realUser = ugi.getUserName();
+ ugi = UnixUserGroupInformation.login(conf);
+ currentUser = ugi.getUserName();
+ } catch (LoginException le) {
+ throw new IOException(le);
+ }
+ if (fs.exists(stagingArea)) {
+ FileStatus fsStatus = fs.getFileStatus(stagingArea);
+ String owner = fsStatus.getOwner();
+ if (!(owner.equals(currentUser) || owner.equals(realUser)) ||
+ !fsStatus.getPermission().
+ equals(JOB_DIR_PERMISSION)) {
+ throw new IOException("The ownership/permissions on the staging " +
+ "directory " + stagingArea + " is not as expected. " +
+ "It is owned by " + owner + " and permissions are "+
+ fsStatus.getPermission() + ". The directory must " +
+ "be owned by the submitter " + currentUser + " or " +
+ "by " + realUser + " and permissions must be rwx------");
+ }
+ } else {
+ fs.mkdirs(stagingArea,
+ new FsPermission(JOB_DIR_PERMISSION));
+ }
+ return stagingArea;
+ }
+
+}
\ No newline at end of file
diff --git a/src/java/org/apache/hadoop/mapreduce/JobSubmitter.java b/src/java/org/apache/hadoop/mapreduce/JobSubmitter.java
index 79f2c83..318b312 100644
--- a/src/java/org/apache/hadoop/mapreduce/JobSubmitter.java
+++ b/src/java/org/apache/hadoop/mapreduce/JobSubmitter.java
@@ -17,7 +17,7 @@
*/
package org.apache.hadoop.mapreduce;
-import java.io.DataOutputStream;
+import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
import java.net.URI;
@@ -26,6 +26,7 @@
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
+import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -35,15 +36,15 @@
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.mapreduce.filecache.TrackerDistributedCacheManager;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.security.TokenStorage;
+import org.apache.hadoop.mapreduce.split.JobSplitWriter;
import org.apache.hadoop.util.ReflectionUtils;
+import org.codehaus.jackson.map.ObjectMapper;
class JobSubmitter {
protected static final Log LOG = LogFactory.getLog(JobSubmitter.class);
@@ -128,12 +129,7 @@
String files = conf.get("tmpfiles");
String libjars = conf.get("tmpjars");
String archives = conf.get("tmparchives");
-
- /*
- * set this user's id in job configuration, so later job files can be
- * accessed using this user's id
- */
- job.setUGIAndUserGroupNames();
+ String jobJar = job.getJar();
//
// Figure out what fs the JobTracker is using. Copy the
@@ -145,14 +141,18 @@
// Create a number of filenames in the JobTracker's fs namespace
LOG.debug("default FileSystem: " + jtFs.getUri());
- jtFs.delete(submitJobDir, true);
+ if (jtFs.exists(submitJobDir)) {
+ throw new IOException("Not submitting job. Job directory " + submitJobDir
+ +" already exists!! This is unexpected.Please check what's there in" +
+ " that directory");
+ }
submitJobDir = jtFs.makeQualified(submitJobDir);
submitJobDir = new Path(submitJobDir.toUri().getPath());
- FsPermission mapredSysPerms = new FsPermission(JOB_DIR_PERMISSION);
+ FsPermission mapredSysPerms = new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
FileSystem.mkdirs(jtFs, submitJobDir, mapredSysPerms);
- Path filesDir = new Path(submitJobDir, "files");
- Path archivesDir = new Path(submitJobDir, "archives");
- Path libjarsDir = new Path(submitJobDir, "libjars");
+ Path filesDir = JobSubmissionFiles.getJobDistCacheFiles(submitJobDir);
+ Path archivesDir = JobSubmissionFiles.getJobDistCacheArchives(submitJobDir);
+ Path libjarsDir = JobSubmissionFiles.getJobDistCacheLibjars(submitJobDir);
// add all the command line files/ jars and archive
// first copy them to jobtrackers filesystem
@@ -185,7 +185,8 @@
for (String tmpjars: libjarsArr) {
Path tmp = new Path(tmpjars);
Path newPath = copyRemoteFiles(libjarsDir, tmp, conf, replication);
- DistributedCache.addFileToClassPath(newPath, conf);
+ DistributedCache.addFileToClassPath(
+ new Path(newPath.toUri().getPath()), conf);
}
}
@@ -212,11 +213,26 @@
DistributedCache.createSymlink(conf);
}
}
-
+
+ if (jobJar != null) { // copy jar to JobTracker's fs
+ // use jar name if job is not named.
+ if ("".equals(job.getJobName())){
+ job.setJobName(new Path(jobJar).getName());
+ }
+ copyJar(new Path(jobJar), JobSubmissionFiles.getJobJar(submitJobDir),
+ replication);
+ job.setJar(JobSubmissionFiles.getJobJar(submitJobDir).toString());
+ } else {
+ LOG.warn("No job jar file set. User classes may not be found. "+
+ "See Job or Job#setJar(String).");
+ }
+
// set the timestamps of the archives and files
TrackerDistributedCacheManager.determineTimestamps(conf);
+ // set the public/private visibility of the archives and files
+ TrackerDistributedCacheManager.determineCacheVisibilities(conf);
}
-
+
private URI getPathURI(Path destPath, String fragment)
throws URISyntaxException {
URI pathURI = destPath.toUri();
@@ -234,36 +250,20 @@
short replication) throws IOException {
jtFs.copyFromLocalFile(originalJarPath, submitJarFile);
jtFs.setReplication(submitJarFile, replication);
- jtFs.setPermission(submitJarFile, new FsPermission(JOB_FILE_PERMISSION));
+ jtFs.setPermission(submitJarFile, new FsPermission(JobSubmissionFiles.JOB_FILE_PERMISSION));
}
+
/**
* configure the jobconf of the user with the command line options of
* -libjars, -files, -archives.
* @param conf
* @throws IOException
*/
- private void configureCommandLineOptions(Job job, Path submitJobDir,
- Path submitJarFile) throws IOException {
+ private void copyAndConfigureFiles(Job job, Path jobSubmitDir)
+ throws IOException {
Configuration conf = job.getConfiguration();
short replication = (short)conf.getInt(Job.SUBMIT_REPLICATION, 10);
- copyAndConfigureFiles(job, submitJobDir, replication);
-
- /* set this user's id in job configuration, so later job files can be
- * accessed using this user's id
- */
- String originalJarPath = job.getJar();
-
- if (originalJarPath != null) { // copy jar to JobTracker's fs
- // use jar name if job is not named.
- if ("".equals(job.getJobName())){
- job.setJobName(new Path(originalJarPath).getName());
- }
- job.setJar(submitJarFile.toString());
- copyJar(new Path(originalJarPath), submitJarFile, replication);
- } else {
- LOG.warn("No job jar file set. User classes may not be found. "+
- "See Job or Job#setJar(String).");
- }
+ copyAndConfigureFiles(job, jobSubmitDir, replication);
// Set the working directory
if (job.getWorkingDirectory() == null) {
@@ -271,15 +271,6 @@
}
}
-
- // job files are world-wide readable and owner writable
- final private static FsPermission JOB_FILE_PERMISSION =
- FsPermission.createImmutable((short) 0644); // rw-r--r--
-
- // job submission directory is world readable/writable/executable
- final static FsPermission JOB_DIR_PERMISSION =
- FsPermission.createImmutable((short) 0777); // rwx-rwx-rwx
-
/**
* Internal method for submitting jobs to the system.
*
@@ -305,45 +296,79 @@
* </li>
* </ol></p>
* @param job the configuration to submit
+ * @param cluster the handle to the Cluster
* @throws ClassNotFoundException
* @throws InterruptedException
* @throws IOException
*/
- JobStatus submitJobInternal(Job job) throws ClassNotFoundException,
- InterruptedException, IOException {
-
+ @SuppressWarnings("unchecked")
+ JobStatus submitJobInternal(Job job, Cluster cluster)
+ throws ClassNotFoundException, InterruptedException, IOException {
+ /*
+ * set this user's id in job configuration, so later job files can be
+ * accessed using this user's id
+ */
+ job.setUGIAndUserGroupNames();
+
+ Path jobStagingArea = JobSubmissionFiles.getStagingDir(cluster,
+ job.getConfiguration());
//configure the command line options correctly on the submitting dfs
Configuration conf = job.getConfiguration();
JobID jobId = submitClient.getNewJobID();
- Path submitJobDir = new Path(submitClient.getSystemDir(), jobId.toString());
- Path submitJarFile = new Path(submitJobDir, "job.jar");
- Path submitSplitFile = new Path(submitJobDir, "job.split");
- configureCommandLineOptions(job, submitJobDir, submitJarFile);
- Path submitJobFile = new Path(submitJobDir, "job.xml");
-
- checkSpecs(job);
+ Path submitJobDir = new Path(jobStagingArea, jobId.toString());
+ JobStatus status = null;
+ try {
+ conf.set("mapreduce.job.dir", submitJobDir.toString());
+ LOG.debug("Configuring job " + jobId + " with " + submitJobDir
+ + " as the submit dir");
+ copyAndConfigureFiles(job, submitJobDir);
+ Path submitJobFile = JobSubmissionFiles.getJobConfPath(submitJobDir);
- // Create the splits for the job
- LOG.info("Creating splits at " + jtFs.makeQualified(submitSplitFile));
- int maps = writeSplits(job, submitSplitFile);
- conf.set("mapred.job.split.file", submitSplitFile.toString());
- conf.setInt("mapred.map.tasks", maps);
- LOG.info("number of splits:" + maps);
-
- // Write job file to JobTracker's fs
- writeConf(conf, submitJobFile);
-
- //
- // Now, actually submit the job (using the submit name)
- //
- JobStatus status = submitClient.submitJob(jobId);
- if (status != null) {
- return status;
- } else {
- throw new IOException("Could not launch job");
+ checkSpecs(job);
+
+ // create TokenStorage object with user secretKeys
+ String tokensFileName = conf.get("tokenCacheFile");
+ TokenStorage tokenStorage = null;
+ if(tokensFileName != null) {
+ LOG.info("loading secret keys from " + tokensFileName);
+ String localFileName = new Path(tokensFileName).toUri().getPath();
+ tokenStorage = new TokenStorage();
+ // read JSON
+ ObjectMapper mapper = new ObjectMapper();
+ Map<String, String> nm =
+ mapper.readValue(new File(localFileName), Map.class);
+
+ for(Map.Entry<String, String> ent: nm.entrySet()) {
+ LOG.debug("adding secret key alias="+ent.getKey());
+ tokenStorage.addSecretKey(new Text(ent.getKey()), ent.getValue().getBytes());
+ }
+ }
+
+ // Create the splits for the job
+ LOG.debug("Creating splits at " + jtFs.makeQualified(submitJobDir));
+ int maps = writeSplits(job, submitJobDir);
+ conf.setInt("mapred.map.tasks", maps);
+ LOG.info("number of splits:" + maps);
+
+ // Write job file to submit dir
+ writeConf(conf, submitJobFile);
+ //
+ // Now, actually submit the job (using the submit name)
+ //
+ status = submitClient.submitJob(jobId, submitJobDir.toString(), tokenStorage);
+ if (status != null) {
+ return status;
+ } else {
+ throw new IOException("Could not launch job");
+ }
+ } finally {
+ if (status == null) {
+ LOG.info("Cleaning up the staging area " + submitJobDir);
+ jtFs.delete(submitJobDir, true);
+ }
}
}
-
+
private void checkSpecs(Job job) throws ClassNotFoundException,
InterruptedException, IOException {
JobConf jConf = (JobConf)job.getConfiguration();
@@ -364,7 +389,7 @@
// Write job file to JobTracker's fs
FSDataOutputStream out =
FileSystem.create(jtFs, jobFile,
- new FsPermission(JOB_FILE_PERMISSION));
+ new FsPermission(JobSubmissionFiles.JOB_FILE_PERMISSION));
try {
conf.writeXml(out);
} finally {
@@ -372,81 +397,42 @@
}
}
+
@SuppressWarnings("unchecked")
- private <T extends InputSplit>
- int writeNewSplits(JobContext job, Path submitSplitFile) throws IOException,
+ private <T extends InputSplit>
+ int writeNewSplits(JobContext job, Path jobSubmitDir) throws IOException,
InterruptedException, ClassNotFoundException {
Configuration conf = job.getConfiguration();
InputFormat<?, ?> input =
ReflectionUtils.newInstance(job.getInputFormatClass(), conf);
-
+
List<InputSplit> splits = input.getSplits(job);
T[] array = (T[]) splits.toArray(new InputSplit[splits.size()]);
// sort the splits into order based on size, so that the biggest
// go first
Arrays.sort(array, new SplitComparator());
- DataOutputStream out = writeSplitsFileHeader(conf, submitSplitFile,
- array.length);
- try {
- if (array.length != 0) {
- DataOutputBuffer buffer = new DataOutputBuffer();
- Job.RawSplit rawSplit = new Job.RawSplit();
- SerializationFactory factory = new SerializationFactory(conf);
- Serializer<T> serializer =
- factory.getSerializer((Class<T>) array[0].getClass());
- serializer.open(buffer);
- for (T split: array) {
- rawSplit.setClassName(split.getClass().getName());
- buffer.reset();
- serializer.serialize(split);
- rawSplit.setDataLength(split.getLength());
- rawSplit.setBytes(buffer.getData(), 0, buffer.getLength());
- rawSplit.setLocations(split.getLocations());
- rawSplit.write(out);
- }
- serializer.close();
- }
- } finally {
- out.close();
- }
+ JobSplitWriter.createSplitFiles(jobSubmitDir, conf, array);
return array.length;
}
-
- static final int CURRENT_SPLIT_FILE_VERSION = 0;
- static final byte[] SPLIT_FILE_HEADER = "SPL".getBytes();
-
- private DataOutputStream writeSplitsFileHeader(Configuration conf,
- Path filename, int length) throws IOException {
- // write the splits to a file for the job tracker
- FileSystem fs = filename.getFileSystem(conf);
- FSDataOutputStream out =
- FileSystem.create(fs, filename, new FsPermission(JOB_FILE_PERMISSION));
- out.write(SPLIT_FILE_HEADER);
- WritableUtils.writeVInt(out, CURRENT_SPLIT_FILE_VERSION);
- WritableUtils.writeVInt(out, length);
- return out;
- }
-
+
private int writeSplits(org.apache.hadoop.mapreduce.JobContext job,
- Path submitSplitFile) throws IOException,
+ Path jobSubmitDir) throws IOException,
InterruptedException, ClassNotFoundException {
JobConf jConf = (JobConf)job.getConfiguration();
- // Create the splits for the job
- LOG.debug("Creating splits at " + jtFs.makeQualified(submitSplitFile));
int maps;
if (jConf.getUseNewMapper()) {
- maps = writeNewSplits(job, submitSplitFile);
+ maps = writeNewSplits(job, jobSubmitDir);
} else {
- maps = writeOldSplits(jConf, submitSplitFile);
+ maps = writeOldSplits(jConf, jobSubmitDir);
}
return maps;
}
-
- // method to write splits for old api mapper.
- private int writeOldSplits(JobConf job,
- Path submitSplitFile) throws IOException {
- org.apache.hadoop.mapred.InputSplit[] splits =
+
+ //method to write splits for old api mapper.
+ private int writeOldSplits(JobConf job, Path jobSubmitDir)
+ throws IOException {
+ org.apache.hadoop.mapred.InputSplit[] splits =
job.getInputFormat().getSplits(job, job.getNumMapTasks());
// sort the splits into order based on size, so that the biggest
// go first
@@ -468,24 +454,7 @@
}
}
});
- DataOutputStream out = writeSplitsFileHeader(job, submitSplitFile,
- splits.length);
-
- try {
- DataOutputBuffer buffer = new DataOutputBuffer();
- Job.RawSplit rawSplit = new Job.RawSplit();
- for (org.apache.hadoop.mapred.InputSplit split: splits) {
- rawSplit.setClassName(split.getClass().getName());
- buffer.reset();
- split.write(buffer);
- rawSplit.setDataLength(split.getLength());
- rawSplit.setBytes(buffer.getData(), 0, buffer.getLength());
- rawSplit.setLocations(split.getLocations());
- rawSplit.write(out);
- }
- } finally {
- out.close();
- }
+ JobSplitWriter.createSplitFiles(jobSubmitDir, job, splits);
return splits.length;
}
@@ -505,7 +474,7 @@
} catch (IOException ie) {
throw new RuntimeException("exception in compare", ie);
} catch (InterruptedException ie) {
- throw new RuntimeException("exception in compare", ie);
+ throw new RuntimeException("exception in compare", ie);
}
}
}
diff --git a/src/java/org/apache/hadoop/mapreduce/TaskCounter.java b/src/java/org/apache/hadoop/mapreduce/TaskCounter.java
index 3086775..259d2c6 100644
--- a/src/java/org/apache/hadoop/mapreduce/TaskCounter.java
+++ b/src/java/org/apache/hadoop/mapreduce/TaskCounter.java
@@ -24,6 +24,7 @@
MAP_OUTPUT_RECORDS,
MAP_SKIPPED_RECORDS,
MAP_OUTPUT_BYTES,
+ SPLIT_RAW_BYTES,
COMBINE_INPUT_RECORDS,
COMBINE_OUTPUT_RECORDS,
REDUCE_INPUT_GROUPS,
@@ -32,5 +33,8 @@
REDUCE_OUTPUT_RECORDS,
REDUCE_SKIPPED_GROUPS,
REDUCE_SKIPPED_RECORDS,
- SPILLED_RECORDS
+ SPILLED_RECORDS,
+ SHUFFLED_MAPS,
+ FAILED_SHUFFLE,
+ MERGED_MAP_OUTPUTS
}
diff --git a/src/java/org/apache/hadoop/mapreduce/TaskCounter.properties b/src/java/org/apache/hadoop/mapreduce/TaskCounter.properties
index 3cb2206..04ecc56 100644
--- a/src/java/org/apache/hadoop/mapreduce/TaskCounter.properties
+++ b/src/java/org/apache/hadoop/mapreduce/TaskCounter.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
# ResourceBundle properties file for Map-Reduce counters
CounterGroupName= Map-Reduce Framework
@@ -15,4 +27,6 @@
REDUCE_SKIPPED_RECORDS.name= Reduce skipped records
REDUCE_SKIPPED_GROUPS.name= Reduce skipped groups
SPILLED_RECORDS.name= Spilled Records
-
+SHUFFLED_MAPS.name= Shuffled Maps
+FAILED_SHUFFLE.name= Failed Shuffles
+MERGED_MAP_OUTPUTS.name= Merged Map outputs
diff --git a/src/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java b/src/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java
index d144d5c..0fe7d98 100644
--- a/src/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java
+++ b/src/java/org/apache/hadoop/mapreduce/filecache/DistributedCache.java
@@ -24,6 +24,7 @@
import org.apache.hadoop.util.*;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapred.DefaultTaskController;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
@@ -198,9 +199,9 @@
boolean isArchive, long confFileStamp,
Path currentWorkDir, boolean honorSymLinkConf) throws IOException {
- return new TrackerDistributedCacheManager(conf).getLocalCache(cache, conf,
- baseDir.toString(), fileStatus, isArchive, confFileStamp, currentWorkDir,
- honorSymLinkConf);
+ return new TrackerDistributedCacheManager(conf, new DefaultTaskController())
+ .getLocalCache(cache, conf, baseDir.toString(), fileStatus, isArchive,
+ confFileStamp, currentWorkDir, honorSymLinkConf, false);
}
/**
@@ -277,8 +278,8 @@
if (timestamp == null) {
throw new IOException("TimeStamp of the uri couldnot be found");
}
- new TrackerDistributedCacheManager(conf).releaseCache(cache, conf,
- Long.parseLong(timestamp));
+ new TrackerDistributedCacheManager(conf, new DefaultTaskController())
+ .releaseCache(cache, conf, Long.parseLong(timestamp));
}
/**
@@ -294,7 +295,8 @@
@Deprecated
public static String makeRelative(URI cache, Configuration conf)
throws IOException {
- return new TrackerDistributedCacheManager(conf).makeRelative(cache, conf);
+ return new TrackerDistributedCacheManager(conf, new DefaultTaskController())
+ .makeRelative(cache, conf);
}
/**
@@ -657,6 +659,7 @@
*/
@Deprecated
public static void purgeCache(Configuration conf) throws IOException {
- new TrackerDistributedCacheManager(conf).purgeCache();
+ new TrackerDistributedCacheManager(conf, new DefaultTaskController())
+ .purgeCache();
}
}
diff --git a/src/java/org/apache/hadoop/mapreduce/filecache/TaskDistributedCacheManager.java b/src/java/org/apache/hadoop/mapreduce/filecache/TaskDistributedCacheManager.java
index 8464f1c..e3a6631 100644
--- a/src/java/org/apache/hadoop/mapreduce/filecache/TaskDistributedCacheManager.java
+++ b/src/java/org/apache/hadoop/mapreduce/filecache/TaskDistributedCacheManager.java
@@ -36,6 +36,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.classification.InterfaceAudience;
/**
* Helper class of {@link TrackerDistributedCacheManager} that represents
@@ -43,9 +44,8 @@
* by TaskRunner/LocalJobRunner to parse out the job configuration
* and setup the local caches.
*
- * <b>This class is internal to Hadoop, and should not be treated as a public
- * interface.</b>
*/
+@InterfaceAudience.Private
public class TaskDistributedCacheManager {
private final TrackerDistributedCacheManager distributedCacheManager;
private final Configuration taskConf;
@@ -66,6 +66,7 @@
REGULAR,
ARCHIVE
}
+ boolean isPublic = true;
/** Whether to decompress */
final FileType type;
final long timestamp;
@@ -73,10 +74,11 @@
final boolean shouldBeAddedToClassPath;
boolean localized = false;
- private CacheFile(URI uri, FileType type, long timestamp,
+ private CacheFile(URI uri, FileType type, boolean isPublic, long timestamp,
boolean classPath) {
this.uri = uri;
this.type = type;
+ this.isPublic = isPublic;
this.timestamp = timestamp;
this.shouldBeAddedToClassPath = classPath;
}
@@ -87,7 +89,7 @@
* files.
*/
private static List<CacheFile> makeCacheFiles(URI[] uris,
- String[] timestamps, Path[] paths, FileType type) {
+ String[] timestamps, String cacheVisibilities[], Path[] paths, FileType type) {
List<CacheFile> ret = new ArrayList<CacheFile>();
if (uris != null) {
if (uris.length != timestamps.length) {
@@ -103,7 +105,8 @@
URI u = uris[i];
boolean isClassPath = (null != classPaths.get(u.getPath()));
long t = Long.parseLong(timestamps[i]);
- ret.add(new CacheFile(u, type, t, isClassPath));
+ ret.add(new CacheFile(u, type, Boolean.valueOf(cacheVisibilities[i]),
+ t, isClassPath));
}
}
return ret;
@@ -127,11 +130,13 @@
this.cacheFiles.addAll(
CacheFile.makeCacheFiles(DistributedCache.getCacheFiles(taskConf),
DistributedCache.getFileTimestamps(taskConf),
+ TrackerDistributedCacheManager.getFileVisibilities(taskConf),
DistributedCache.getFileClassPaths(taskConf),
CacheFile.FileType.REGULAR));
this.cacheFiles.addAll(
CacheFile.makeCacheFiles(DistributedCache.getCacheArchives(taskConf),
DistributedCache.getArchiveTimestamps(taskConf),
+ TrackerDistributedCacheManager.getArchiveVisibilities(taskConf),
DistributedCache.getArchiveClassPaths(taskConf),
CacheFile.FileType.ARCHIVE));
}
@@ -144,9 +149,9 @@
* file, if necessary.
*/
public void setup(LocalDirAllocator lDirAlloc, File workDir,
- String cacheSubdir) throws IOException {
+ String privateCacheSubdir, String publicCacheSubDir) throws IOException {
setupCalled = true;
-
+
if (cacheFiles.isEmpty()) {
return;
}
@@ -159,11 +164,14 @@
URI uri = cacheFile.uri;
FileSystem fileSystem = FileSystem.get(uri, taskConf);
FileStatus fileStatus = fileSystem.getFileStatus(new Path(uri.getPath()));
-
+ String cacheSubdir = publicCacheSubDir;
+ if (!cacheFile.isPublic) {
+ cacheSubdir = privateCacheSubdir;
+ }
Path p = distributedCacheManager.getLocalCache(uri, taskConf,
cacheSubdir, fileStatus,
cacheFile.type == CacheFile.FileType.ARCHIVE,
- cacheFile.timestamp, workdirPath, false);
+ cacheFile.timestamp, workdirPath, false, cacheFile.isPublic);
cacheFile.setLocalized(true);
if (cacheFile.type == CacheFile.FileType.ARCHIVE) {
diff --git a/src/java/org/apache/hadoop/mapreduce/filecache/TrackerDistributedCacheManager.java b/src/java/org/apache/hadoop/mapreduce/filecache/TrackerDistributedCacheManager.java
index 1a76808..e5bda80 100644
--- a/src/java/org/apache/hadoop/mapreduce/filecache/TrackerDistributedCacheManager.java
+++ b/src/java/org/apache/hadoop/mapreduce/filecache/TrackerDistributedCacheManager.java
@@ -30,15 +30,22 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskController;
+import org.apache.hadoop.mapred.TaskController.DistributedCacheFileContext;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
+import org.apache.hadoop.mapreduce.util.MRAsyncDiskService;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.RunJar;
+import org.apache.hadoop.classification.InterfaceAudience;
/**
* Manages a single machine's instance of a cross-job
@@ -46,9 +53,8 @@
* by a TaskTracker (or something that emulates it,
* like LocalJobRunner).
*
- * <b>This class is internal to Hadoop, and should not be treated as a public
- * interface.</b>
*/
+@InterfaceAudience.Private
public class TrackerDistributedCacheManager {
// cacheID to cacheStatus mapping
private TreeMap<String, CacheStatus> cachedArchives =
@@ -66,14 +72,32 @@
private LocalDirAllocator lDirAllocator;
+ private TaskController taskController;
+
private Configuration trackerConf;
private Random random = new Random();
- public TrackerDistributedCacheManager(Configuration conf) throws IOException {
+ private MRAsyncDiskService asyncDiskService;
+
+ public TrackerDistributedCacheManager(Configuration conf,
+ TaskController taskController) throws IOException {
this.localFs = FileSystem.getLocal(conf);
this.trackerConf = conf;
this.lDirAllocator = new LocalDirAllocator(TTConfig.LOCAL_DIR);
+ this.taskController = taskController;
+ }
+
+ /**
+ * Creates a TrackerDistributedCacheManager with a MRAsyncDiskService.
+ * @param asyncDiskService Provides a set of ThreadPools for async disk
+ * operations.
+ */
+ public TrackerDistributedCacheManager(Configuration conf,
+ TaskController taskController, MRAsyncDiskService asyncDiskService)
+ throws IOException {
+ this(conf, taskController);
+ this.asyncDiskService = asyncDiskService;
}
/**
@@ -101,6 +125,7 @@
* launches
* NOTE: This is effectively always on since r696957, since there is no code
* path that does not use this.
+ * @param isPublic to know the cache file is accessible to public or private
* @return the path to directory where the archives are unjarred in case of
* archives, the path to the file where the file is copied locally
* @throws IOException
@@ -108,7 +133,7 @@
Path getLocalCache(URI cache, Configuration conf,
String subDir, FileStatus fileStatus,
boolean isArchive, long confFileStamp,
- Path currentWorkDir, boolean honorSymLinkConf)
+ Path currentWorkDir, boolean honorSymLinkConf, boolean isPublic)
throws IOException {
String key = getKey(cache, conf, confFileStamp);
CacheStatus lcacheStatus;
@@ -117,13 +142,13 @@
lcacheStatus = cachedArchives.get(key);
if (lcacheStatus == null) {
// was never localized
+ String uniqueString = String.valueOf(random.nextLong());
String cachePath = new Path (subDir,
- new Path(String.valueOf(random.nextLong()),
- makeRelative(cache, conf))).toString();
+ new Path(uniqueString, makeRelative(cache, conf))).toString();
Path localPath = lDirAllocator.getLocalPathForWrite(cachePath,
fileStatus.getLen(), trackerConf);
- lcacheStatus = new CacheStatus(
- new Path(localPath.toString().replace(cachePath, "")), localPath);
+ lcacheStatus = new CacheStatus(new Path(localPath.toString().replace(
+ cachePath, "")), localPath, new Path(subDir), uniqueString);
cachedArchives.put(key, lcacheStatus);
}
@@ -137,7 +162,7 @@
synchronized (lcacheStatus) {
if (!lcacheStatus.isInited()) {
localizedPath = localizeCache(conf, cache, confFileStamp,
- lcacheStatus, fileStatus, isArchive);
+ lcacheStatus, fileStatus, isArchive, isPublic);
lcacheStatus.initComplete();
} else {
localizedPath = checkCacheStatusValidity(conf, cache, confFileStamp,
@@ -242,23 +267,47 @@
// do the deletion, after releasing the global lock
for (CacheStatus lcacheStatus : deleteSet) {
synchronized (lcacheStatus) {
- FileSystem.getLocal(conf).delete(lcacheStatus.localLoadPath, true);
- LOG.info("Deleted path " + lcacheStatus.localLoadPath);
+ deleteLocalPath(asyncDiskService,
+ FileSystem.getLocal(conf), lcacheStatus.localizedLoadPath);
// decrement the size of the cache from baseDirSize
synchronized (baseDirSize) {
- Long dirSize = baseDirSize.get(lcacheStatus.baseDir);
+ Long dirSize = baseDirSize.get(lcacheStatus.localizedBaseDir);
if ( dirSize != null ) {
dirSize -= lcacheStatus.size;
- baseDirSize.put(lcacheStatus.baseDir, dirSize);
+ baseDirSize.put(lcacheStatus.localizedBaseDir, dirSize);
} else {
LOG.warn("Cannot find record of the baseDir: " +
- lcacheStatus.baseDir + " during delete!");
+ lcacheStatus.localizedBaseDir + " during delete!");
}
}
}
}
}
+ /**
+ * Delete a local path with asyncDiskService if available,
+ * or otherwise synchronously with local file system.
+ */
+ private static void deleteLocalPath(MRAsyncDiskService asyncDiskService,
+ LocalFileSystem fs, Path path) throws IOException {
+ boolean deleted = false;
+ if (asyncDiskService != null) {
+ // Try to delete using asyncDiskService
+ String localPathToDelete =
+ path.toUri().getPath();
+ deleted = asyncDiskService.moveAndDeleteAbsolutePath(localPathToDelete);
+ if (!deleted) {
+ LOG.warn("Cannot find DistributedCache path " + localPathToDelete
+ + " on any of the asyncDiskService volumes!");
+ }
+ }
+ if (!deleted) {
+ // If no asyncDiskService, we will delete the files synchronously
+ fs.delete(path, true);
+ }
+ LOG.info("Deleted path " + path);
+ }
+
/*
* Returns the relative path of the dir this cache will be localized in
* relative path that this cache will be localized in. For
@@ -305,6 +354,51 @@
return fileSystem.getFileStatus(filePath).getModificationTime();
}
+
+ /**
+ * Returns a boolean to denote whether a cache file is visible to all(public)
+ * or not
+ * @param conf
+ * @param uri
+ * @return true if the path in the uri is visible to all, false otherwise
+ * @throws IOException
+ */
+ static boolean isPublic(Configuration conf, URI uri) throws IOException {
+ FileSystem fs = FileSystem.get(uri, conf);
+ Path current = new Path(uri.getPath());
+ //the leaf level file should be readable by others
+ if (!checkPermissionOfOther(fs, current, FsAction.READ)) {
+ return false;
+ }
+ current = current.getParent();
+ while (current != null) {
+ //the subdirs in the path should have execute permissions for others
+ if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE)) {
+ return false;
+ }
+ current = current.getParent();
+ }
+ return true;
+ }
+ /**
+ * Checks for a given path whether the Other permissions on it
+ * imply the permission in the passed FsAction
+ * @param fs
+ * @param path
+ * @param action
+ * @return true if the path in the uri is visible to all, false otherwise
+ * @throws IOException
+ */
+ private static boolean checkPermissionOfOther(FileSystem fs, Path path,
+ FsAction action) throws IOException {
+ FileStatus status = fs.getFileStatus(path);
+ FsPermission perms = status.getPermission();
+ FsAction otherAction = perms.getOtherAction();
+ if (otherAction.implies(action)) {
+ return true;
+ }
+ return false;
+ }
private Path checkCacheStatusValidity(Configuration conf,
URI cache, long confFileStamp,
@@ -316,13 +410,13 @@
// Has to be
if (!ifExistsAndFresh(conf, fs, cache, confFileStamp,
cacheStatus, fileStatus)) {
- throw new IOException("Stale cache file: " + cacheStatus.localLoadPath +
+ throw new IOException("Stale cache file: " + cacheStatus.localizedLoadPath +
" for cache-file: " + cache);
}
LOG.info(String.format("Using existing cache of %s->%s",
- cache.toString(), cacheStatus.localLoadPath));
- return cacheStatus.localLoadPath;
+ cache.toString(), cacheStatus.localizedLoadPath));
+ return cacheStatus.localizedLoadPath;
}
private void createSymlink(Configuration conf, URI cache,
@@ -337,7 +431,7 @@
File flink = new File(link);
if (doSymlink){
if (!flink.exists()) {
- FileUtil.symLink(cacheStatus.localLoadPath.toString(), link);
+ FileUtil.symLink(cacheStatus.localizedLoadPath.toString(), link);
}
}
}
@@ -348,21 +442,21 @@
URI cache, long confFileStamp,
CacheStatus cacheStatus,
FileStatus fileStatus,
- boolean isArchive)
+ boolean isArchive, boolean isPublic)
throws IOException {
FileSystem fs = FileSystem.get(cache, conf);
FileSystem localFs = FileSystem.getLocal(conf);
Path parchive = null;
if (isArchive) {
- parchive = new Path(cacheStatus.localLoadPath,
- new Path(cacheStatus.localLoadPath.getName()));
+ parchive = new Path(cacheStatus.localizedLoadPath,
+ new Path(cacheStatus.localizedLoadPath.getName()));
} else {
- parchive = cacheStatus.localLoadPath;
+ parchive = cacheStatus.localizedLoadPath;
}
if (!localFs.mkdirs(parchive.getParent())) {
throw new IOException("Mkdirs failed to create directory " +
- cacheStatus.localLoadPath.toString());
+ cacheStatus.localizedLoadPath.toString());
}
String cacheId = cache.getPath();
@@ -392,29 +486,45 @@
FileUtil.getDU(new File(parchive.getParent().toString()));
cacheStatus.size = cacheSize;
synchronized (baseDirSize) {
- Long dirSize = baseDirSize.get(cacheStatus.baseDir);
+ Long dirSize = baseDirSize.get(cacheStatus.localizedBaseDir);
if( dirSize == null ) {
dirSize = Long.valueOf(cacheSize);
} else {
dirSize += cacheSize;
}
- baseDirSize.put(cacheStatus.baseDir, dirSize);
+ baseDirSize.put(cacheStatus.localizedBaseDir, dirSize);
}
- // do chmod here
- try {
- //Setting recursive permission to grant everyone read and execute
- FileUtil.chmod(cacheStatus.baseDir.toString(), "ugo+rx",true);
- } catch(InterruptedException e) {
- LOG.warn("Exception in chmod" + e.toString());
- }
+ // set proper permissions for the localized directory
+ setPermissions(conf, cacheStatus, isPublic);
// update cacheStatus to reflect the newly cached file
cacheStatus.mtime = getTimestamp(conf, cache);
LOG.info(String.format("Cached %s as %s",
- cache.toString(), cacheStatus.localLoadPath));
- return cacheStatus.localLoadPath;
+ cache.toString(), cacheStatus.localizedLoadPath));
+ return cacheStatus.localizedLoadPath;
+ }
+
+ private void setPermissions(Configuration conf, CacheStatus cacheStatus,
+ boolean isPublic) throws IOException {
+ if (isPublic) {
+ Path localizedUniqueDir = cacheStatus.getLocalizedUniqueDir();
+ LOG.info("Doing chmod on localdir :" + localizedUniqueDir);
+ try {
+ FileUtil.chmod(localizedUniqueDir.toString(), "ugo+rx", true);
+ } catch (InterruptedException e) {
+ LOG.warn("Exception in chmod" + e.toString());
+ throw new IOException(e);
+ }
+ } else {
+ // invoke taskcontroller to set permissions
+ DistributedCacheFileContext context = new DistributedCacheFileContext(
+ conf.get(JobContext.USER_NAME), new File(cacheStatus.localizedBaseDir
+ .toString()), cacheStatus.localizedBaseDir,
+ cacheStatus.uniqueString);
+ taskController.initializeDistributedCacheFile(context);
+ }
}
private static boolean isTarFile(String filename) {
@@ -485,10 +595,10 @@
static class CacheStatus {
// the local load path of this cache
- Path localLoadPath;
+ Path localizedLoadPath;
//the base dir where the cache lies
- Path baseDir;
+ Path localizedBaseDir;
//the size of this cache
long size;
@@ -501,18 +611,28 @@
// is it initialized ?
boolean inited = false;
+
+ // The sub directory (tasktracker/archive or tasktracker/user/archive),
+ // under which the file will be localized
+ Path subDir;
- public CacheStatus(Path baseDir, Path localLoadPath) {
+ // unique string used in the construction of local load path
+ String uniqueString;
+
+ public CacheStatus(Path baseDir, Path localLoadPath, Path subDir,
+ String uniqueString) {
super();
- this.localLoadPath = localLoadPath;
+ this.localizedLoadPath = localLoadPath;
this.refcount = 0;
this.mtime = -1;
- this.baseDir = baseDir;
+ this.localizedBaseDir = baseDir;
this.size = 0;
+ this.subDir = subDir;
+ this.uniqueString = uniqueString;
}
Path getBaseDir(){
- return this.baseDir;
+ return this.localizedBaseDir;
}
// mark it as initialized
@@ -524,6 +644,10 @@
boolean isInited() {
return inited;
}
+
+ Path getLocalizedUniqueDir() {
+ return new Path(localizedBaseDir, new Path(subDir, uniqueString));
+ }
}
/**
@@ -535,7 +659,7 @@
synchronized (cachedArchives) {
for (Map.Entry<String,CacheStatus> f: cachedArchives.entrySet()) {
try {
- localFs.delete(f.getValue().localLoadPath, true);
+ deleteLocalPath(asyncDiskService, localFs, f.getValue().localizedLoadPath);
} catch (IOException ie) {
LOG.debug("Error cleaning up cache", ie);
}
@@ -585,8 +709,60 @@
setFileTimestamps(job, fileTimestamps.toString());
}
}
+ /**
+ * Determines the visibilities of the distributed cache files and
+ * archives. The visibility of a cache path is "public" if the leaf component
+ * has READ permissions for others, and the parent subdirs have
+ * EXECUTE permissions for others
+ * @param job
+ * @throws IOException
+ */
+ public static void determineCacheVisibilities(Configuration job)
+ throws IOException {
+ URI[] tarchives = DistributedCache.getCacheArchives(job);
+ if (tarchives != null) {
+ StringBuffer archiveVisibilities =
+ new StringBuffer(String.valueOf(isPublic(job, tarchives[0])));
+ for (int i = 1; i < tarchives.length; i++) {
+ archiveVisibilities.append(",");
+ archiveVisibilities.append(String.valueOf(isPublic(job, tarchives[i])));
+ }
+ setArchiveVisibilities(job, archiveVisibilities.toString());
+ }
+ URI[] tfiles = DistributedCache.getCacheFiles(job);
+ if (tfiles != null) {
+ StringBuffer fileVisibilities =
+ new StringBuffer(String.valueOf(isPublic(job, tfiles[0])));
+ for (int i = 1; i < tfiles.length; i++) {
+ fileVisibilities.append(",");
+ fileVisibilities.append(String.valueOf(isPublic(job, tfiles[i])));
+ }
+ setFileVisibilities(job, fileVisibilities.toString());
+ }
+ }
/**
+ * Get the booleans on whether the files are public or not. Used by
+ * internal DistributedCache and MapReduce code.
+ * @param conf The configuration which stored the timestamps
+ * @return a string array of booleans
+ * @throws IOException
+ */
+ static String[] getFileVisibilities(Configuration conf) {
+ return conf.getStrings(JobContext.CACHE_FILE_VISIBILITIES);
+ }
+
+ /**
+ * Get the booleans on whether the archives are public or not. Used by
+ * internal DistributedCache and MapReduce code.
+ * @param conf The configuration which stored the timestamps
+ * @return a string array of booleans
+ */
+ static String[] getArchiveVisibilities(Configuration conf) {
+ return conf.getStrings(JobContext.CACHE_ARCHIVES_VISIBILITIES);
+ }
+
+ /**
* This method checks if there is a conflict in the fragment names
* of the uris. Also makes sure that each uri has a fragment. It
* is only to be called if you want to create symlinks for
@@ -631,6 +807,28 @@
}
return true;
}
+ /**
+ * This is to check the public/private visibility of the archives to be
+ * localized.
+ *
+ * @param conf Configuration which stores the timestamp's
+ * @param booleans comma separated list of booleans (true - public)
+ * The order should be the same as the order in which the archives are added.
+ */
+ static void setArchiveVisibilities(Configuration conf, String booleans) {
+ conf.set(JobContext.CACHE_ARCHIVES_VISIBILITIES, booleans);
+ }
+
+ /**
+ * This is to check the public/private visibility of the files to be localized
+ *
+ * @param conf Configuration which stores the timestamp's
+ * @param booleans comma separated list of booleans (true - public)
+ * The order should be the same as the order in which the files are added.
+ */
+ static void setFileVisibilities(Configuration conf, String booleans) {
+ conf.set(JobContext.CACHE_FILE_VISIBILITIES, booleans);
+ }
/**
* This is to check the timestamp of the archives to be localized.
diff --git a/src/java/org/apache/hadoop/mapreduce/jobhistory/JobHistory.java b/src/java/org/apache/hadoop/mapreduce/jobhistory/JobHistory.java
index fad6815..74d188e 100644
--- a/src/java/org/apache/hadoop/mapreduce/jobhistory/JobHistory.java
+++ b/src/java/org/apache/hadoop/mapreduce/jobhistory/JobHistory.java
@@ -24,8 +24,11 @@
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
+import java.util.Map.Entry;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
@@ -80,6 +83,18 @@
private HistoryCleaner historyCleanerThread = null;
+ private Map<JobID, MovedFileInfo> jobHistoryFileMap =
+ Collections.<JobID,MovedFileInfo>synchronizedMap(
+ new LinkedHashMap<JobID, MovedFileInfo>());
+
+ private static class MovedFileInfo {
+ private final String historyFile;
+ private final long timestamp;
+ public MovedFileInfo(String historyFile, long timestamp) {
+ this.historyFile = historyFile;
+ this.timestamp = timestamp;
+ }
+ }
/**
* Initialize Job History Module
* @param jt Job Tracker handle
@@ -196,6 +211,16 @@
}
/**
+ * Given the job id, return the history file path from the cache
+ */
+ public String getHistoryFilePath(JobID jobId) {
+ MovedFileInfo info = jobHistoryFileMap.get(jobId);
+ if (info == null) {
+ return null;
+ }
+ return info.historyFile;
+ }
+ /**
* Create an event writer for the Job represented by the jobID.
* This should be the first call to history for a job
* @param jobId
@@ -383,7 +408,8 @@
historyFileDonePath = new Path(done,
historyFile.getName()).toString();
}
-
+ jobHistoryFileMap.put(id, new MovedFileInfo(historyFileDonePath,
+ System.currentTimeMillis()));
jobTracker.retireJob(org.apache.hadoop.mapred.JobID.downgrade(id),
historyFileDonePath);
@@ -481,6 +507,21 @@
}
}
}
+ //walking over the map to purge entries from jobHistoryFileMap
+ synchronized (jobHistoryFileMap) {
+ Iterator<Entry<JobID, MovedFileInfo>> it =
+ jobHistoryFileMap.entrySet().iterator();
+ while (it.hasNext()) {
+ MovedFileInfo info = it.next().getValue();
+ if (now - info.timestamp > maxAgeOfHistoryFiles) {
+ it.remove();
+ } else {
+ //since entries are in sorted timestamp order, no more entries
+ //are required to be checked
+ break;
+ }
+ }
+ }
} catch (IOException ie) {
LOG.info("Error cleaning up history directory" +
StringUtils.stringifyException(ie));
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java b/src/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java
new file mode 100644
index 0000000..5749a76
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java
@@ -0,0 +1,903 @@
+/**
+ * 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.mapreduce.lib.chain;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DefaultStringifier;
+import org.apache.hadoop.io.Stringifier;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.ReduceContext;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskInputOutputContext;
+import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
+import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * The Chain class provides all the common functionality for the
+ * {@link ChainMapper} and the {@link ChainReducer} classes.
+ */
+public class Chain {
+ protected static final String CHAIN_MAPPER = "mapreduce.chain.mapper";
+ protected static final String CHAIN_REDUCER = "mapreduce.chain.reducer";
+
+ protected static final String CHAIN_MAPPER_SIZE = ".size";
+ protected static final String CHAIN_MAPPER_CLASS = ".mapper.class.";
+ protected static final String CHAIN_MAPPER_CONFIG = ".mapper.config.";
+ protected static final String CHAIN_REDUCER_CLASS = ".reducer.class";
+ protected static final String CHAIN_REDUCER_CONFIG = ".reducer.config";
+
+ protected static final String MAPPER_INPUT_KEY_CLASS =
+ "mapreduce.chain.mapper.input.key.class";
+ protected static final String MAPPER_INPUT_VALUE_CLASS =
+ "mapreduce.chain.mapper.input.value.class";
+ protected static final String MAPPER_OUTPUT_KEY_CLASS =
+ "mapreduce.chain.mapper.output.key.class";
+ protected static final String MAPPER_OUTPUT_VALUE_CLASS =
+ "mapreduce.chain.mapper.output.value.class";
+ protected static final String REDUCER_INPUT_KEY_CLASS =
+ "mapreduce.chain.reducer.input.key.class";
+ protected static final String REDUCER_INPUT_VALUE_CLASS =
+ "maperduce.chain.reducer.input.value.class";
+ protected static final String REDUCER_OUTPUT_KEY_CLASS =
+ "mapreduce.chain.reducer.output.key.class";
+ protected static final String REDUCER_OUTPUT_VALUE_CLASS =
+ "mapreduce.chain.reducer.output.value.class";
+
+ protected boolean isMap;
+
+ @SuppressWarnings("unchecked")
+ private List<Mapper> mappers = new ArrayList<Mapper>();
+ private Reducer<?, ?, ?, ?> reducer;
+ private List<Configuration> confList = new ArrayList<Configuration>();
+ private Configuration rConf;
+ private List<Thread> threads = new ArrayList<Thread>();
+ private List<ChainBlockingQueue<?>> blockingQueues =
+ new ArrayList<ChainBlockingQueue<?>>();
+ private Throwable throwable = null;
+
+ /**
+ * Creates a Chain instance configured for a Mapper or a Reducer.
+ *
+ * @param isMap
+ * TRUE indicates the chain is for a Mapper, FALSE that is for a
+ * Reducer.
+ */
+ protected Chain(boolean isMap) {
+ this.isMap = isMap;
+ }
+
+ static class KeyValuePair<K, V> {
+ K key;
+ V value;
+ boolean endOfInput;
+
+ KeyValuePair(K key, V value) {
+ this.key = key;
+ this.value = value;
+ this.endOfInput = false;
+ }
+
+ KeyValuePair(boolean eof) {
+ this.key = null;
+ this.value = null;
+ this.endOfInput = eof;
+ }
+ }
+
+ // ChainRecordReader either reads from blocking queue or task context.
+ private static class ChainRecordReader<KEYIN, VALUEIN> extends
+ RecordReader<KEYIN, VALUEIN> {
+ private Class<?> keyClass;
+ private Class<?> valueClass;
+ private KEYIN key;
+ private VALUEIN value;
+ private Configuration conf;
+ TaskInputOutputContext<KEYIN, VALUEIN, ?, ?> inputContext = null;
+ ChainBlockingQueue<KeyValuePair<KEYIN, VALUEIN>> inputQueue = null;
+
+ // constructor to read from a blocking queue
+ ChainRecordReader(Class<?> keyClass, Class<?> valueClass,
+ ChainBlockingQueue<KeyValuePair<KEYIN, VALUEIN>> inputQueue,
+ Configuration conf) {
+ this.keyClass = keyClass;
+ this.valueClass = valueClass;
+ this.inputQueue = inputQueue;
+ this.conf = conf;
+ }
+
+ // constructor to read from the context
+ ChainRecordReader(TaskInputOutputContext<KEYIN, VALUEIN, ?, ?> context) {
+ inputContext = context;
+ }
+
+ public void initialize(InputSplit split, TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ }
+
+ /**
+ * Advance to the next key, value pair, returning null if at end.
+ *
+ * @return the key object that was read into, or null if no more
+ */
+ public boolean nextKeyValue() throws IOException, InterruptedException {
+ if (inputQueue != null) {
+ return readFromQueue();
+ } else if (inputContext.nextKeyValue()) {
+ this.key = inputContext.getCurrentKey();
+ this.value = inputContext.getCurrentValue();
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private boolean readFromQueue() throws IOException, InterruptedException {
+ KeyValuePair<KEYIN, VALUEIN> kv = null;
+
+ // wait for input on queue
+ kv = inputQueue.dequeue();
+ if (kv.endOfInput) {
+ return false;
+ }
+ key = (KEYIN) ReflectionUtils.newInstance(keyClass, conf);
+ value = (VALUEIN) ReflectionUtils.newInstance(valueClass, conf);
+ ReflectionUtils.copy(conf, kv.key, this.key);
+ ReflectionUtils.copy(conf, kv.value, this.value);
+ return true;
+ }
+
+ /**
+ * Get the current key.
+ *
+ * @return the current key object or null if there isn't one
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public KEYIN getCurrentKey() throws IOException, InterruptedException {
+ return this.key;
+ }
+
+ /**
+ * Get the current value.
+ *
+ * @return the value object that was read into
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public VALUEIN getCurrentValue() throws IOException, InterruptedException {
+ return this.value;
+ }
+
+ @Override
+ public void close() throws IOException {
+ }
+
+ @Override
+ public float getProgress() throws IOException, InterruptedException {
+ return 0;
+ }
+ }
+
+ // ChainRecordWriter either writes to blocking queue or task context
+
+ private static class ChainRecordWriter<KEYOUT, VALUEOUT> extends
+ RecordWriter<KEYOUT, VALUEOUT> {
+ TaskInputOutputContext<?, ?, KEYOUT, VALUEOUT> outputContext = null;
+ ChainBlockingQueue<KeyValuePair<KEYOUT, VALUEOUT>> outputQueue = null;
+ KEYOUT keyout;
+ VALUEOUT valueout;
+ Configuration conf;
+ Class<?> keyClass;
+ Class<?> valueClass;
+
+ // constructor to write to context
+ ChainRecordWriter(TaskInputOutputContext<?, ?, KEYOUT, VALUEOUT> context) {
+ outputContext = context;
+ }
+
+ // constructor to write to blocking queue
+ ChainRecordWriter(Class<?> keyClass, Class<?> valueClass,
+ ChainBlockingQueue<KeyValuePair<KEYOUT, VALUEOUT>> output,
+ Configuration conf) {
+ this.keyClass = keyClass;
+ this.valueClass = valueClass;
+ this.outputQueue = output;
+ this.conf = conf;
+ }
+
+ /**
+ * Writes a key/value pair.
+ *
+ * @param key
+ * the key to write.
+ * @param value
+ * the value to write.
+ * @throws IOException
+ */
+ public void write(KEYOUT key, VALUEOUT value) throws IOException,
+ InterruptedException {
+ if (outputQueue != null) {
+ writeToQueue(key, value);
+ } else {
+ outputContext.write(key, value);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private void writeToQueue(KEYOUT key, VALUEOUT value) throws IOException,
+ InterruptedException {
+ this.keyout = (KEYOUT) ReflectionUtils.newInstance(keyClass, conf);
+ this.valueout = (VALUEOUT) ReflectionUtils.newInstance(valueClass, conf);
+ ReflectionUtils.copy(conf, key, this.keyout);
+ ReflectionUtils.copy(conf, value, this.valueout);
+
+ // wait to write output to queuue
+ outputQueue.enqueue(new KeyValuePair<KEYOUT, VALUEOUT>(keyout, valueout));
+ }
+
+ /**
+ * Close this <code>RecordWriter</code> to future operations.
+ *
+ * @param context
+ * the context of the task
+ * @throws IOException
+ */
+ public void close(TaskAttemptContext context) throws IOException,
+ InterruptedException {
+ if (outputQueue != null) {
+ // write end of input
+ outputQueue.enqueue(new KeyValuePair<KEYOUT, VALUEOUT>(true));
+ }
+ }
+
+ }
+
+ private synchronized Throwable getThrowable() {
+ return throwable;
+ }
+
+ private synchronized boolean setIfUnsetThrowable(Throwable th) {
+ if (throwable == null) {
+ throwable = th;
+ return true;
+ }
+ return false;
+ }
+
+ private class MapRunner<KEYIN, VALUEIN, KEYOUT, VALUEOUT> extends Thread {
+ private Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> mapper;
+ private Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context chainContext;
+ private RecordReader<KEYIN, VALUEIN> rr;
+ private RecordWriter<KEYOUT, VALUEOUT> rw;
+
+ public MapRunner(Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> mapper,
+ Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context mapperContext,
+ RecordReader<KEYIN, VALUEIN> rr, RecordWriter<KEYOUT, VALUEOUT> rw)
+ throws IOException, InterruptedException {
+ this.mapper = mapper;
+ this.rr = rr;
+ this.rw = rw;
+ this.chainContext = mapperContext;
+ }
+
+ @Override
+ public void run() {
+ if (getThrowable() != null) {
+ return;
+ }
+ try {
+ mapper.run(chainContext);
+ rr.close();
+ rw.close(chainContext);
+ } catch (Throwable th) {
+ if (setIfUnsetThrowable(th)) {
+ interruptAllThreads();
+ }
+ }
+ }
+ }
+
+ private class ReduceRunner<KEYIN, VALUEIN, KEYOUT, VALUEOUT> extends Thread {
+ private Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> reducer;
+ private Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context chainContext;
+ private RecordWriter<KEYOUT, VALUEOUT> rw;
+
+ ReduceRunner(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context context,
+ Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> reducer,
+ RecordWriter<KEYOUT, VALUEOUT> rw) throws IOException,
+ InterruptedException {
+ this.reducer = reducer;
+ this.chainContext = context;
+ this.rw = rw;
+ }
+
+ @Override
+ public void run() {
+ try {
+ reducer.run(chainContext);
+ rw.close(chainContext);
+ } catch (Throwable th) {
+ if (setIfUnsetThrowable(th)) {
+ interruptAllThreads();
+ }
+ }
+ }
+ }
+
+ Configuration getConf(int index) {
+ return confList.get(index);
+ }
+
+ /**
+ * Create a map context that is based on ChainMapContext and the given record
+ * reader and record writer
+ */
+ private <KEYIN, VALUEIN, KEYOUT, VALUEOUT>
+ Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context createMapContext(
+ RecordReader<KEYIN, VALUEIN> rr, RecordWriter<KEYOUT, VALUEOUT> rw,
+ TaskInputOutputContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> context,
+ Configuration conf) {
+ MapContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> mapContext =
+ new ChainMapContextImpl<KEYIN, VALUEIN, KEYOUT, VALUEOUT>(
+ context, rr, rw, conf);
+ Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context mapperContext =
+ new WrappedMapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT>()
+ .getMapContext(mapContext);
+ return mapperContext;
+ }
+
+ @SuppressWarnings("unchecked")
+ void runMapper(TaskInputOutputContext context, int index) throws IOException,
+ InterruptedException {
+ Mapper mapper = mappers.get(index);
+ RecordReader rr = new ChainRecordReader(context);
+ RecordWriter rw = new ChainRecordWriter(context);
+ Mapper.Context mapperContext = createMapContext(rr, rw, context,
+ getConf(index));
+ mapper.run(mapperContext);
+ rr.close();
+ rw.close(context);
+ }
+
+ /**
+ * Add mapper(the first mapper) that reads input from the input
+ * context and writes to queue
+ */
+ @SuppressWarnings("unchecked")
+ void addMapper(TaskInputOutputContext inputContext,
+ ChainBlockingQueue<KeyValuePair<?, ?>> output, int index)
+ throws IOException, InterruptedException {
+ Configuration conf = getConf(index);
+ Class<?> keyOutClass = conf.getClass(MAPPER_OUTPUT_KEY_CLASS, Object.class);
+ Class<?> valueOutClass = conf.getClass(MAPPER_OUTPUT_VALUE_CLASS,
+ Object.class);
+
+ RecordReader rr = new ChainRecordReader(inputContext);
+ RecordWriter rw = new ChainRecordWriter(keyOutClass, valueOutClass, output,
+ conf);
+ Mapper.Context mapperContext = createMapContext(rr, rw,
+ (MapContext) inputContext, getConf(index));
+ MapRunner runner = new MapRunner(mappers.get(index), mapperContext, rr, rw);
+ threads.add(runner);
+ }
+
+ /**
+ * Add mapper(the last mapper) that reads input from
+ * queue and writes output to the output context
+ */
+ @SuppressWarnings("unchecked")
+ void addMapper(ChainBlockingQueue<KeyValuePair<?, ?>> input,
+ TaskInputOutputContext outputContext, int index) throws IOException,
+ InterruptedException {
+ Configuration conf = getConf(index);
+ Class<?> keyClass = conf.getClass(MAPPER_INPUT_KEY_CLASS, Object.class);
+ Class<?> valueClass = conf.getClass(MAPPER_INPUT_VALUE_CLASS, Object.class);
+ RecordReader rr = new ChainRecordReader(keyClass, valueClass, input, conf);
+ RecordWriter rw = new ChainRecordWriter(outputContext);
+ MapRunner runner = new MapRunner(mappers.get(index), createMapContext(rr,
+ rw, outputContext, getConf(index)), rr, rw);
+ threads.add(runner);
+ }
+
+ /**
+ * Add mapper that reads and writes from/to the queue
+ */
+ @SuppressWarnings("unchecked")
+ void addMapper(ChainBlockingQueue<KeyValuePair<?, ?>> input,
+ ChainBlockingQueue<KeyValuePair<?, ?>> output,
+ TaskInputOutputContext context, int index) throws IOException,
+ InterruptedException {
+ Configuration conf = getConf(index);
+ Class<?> keyClass = conf.getClass(MAPPER_INPUT_KEY_CLASS, Object.class);
+ Class<?> valueClass = conf.getClass(MAPPER_INPUT_VALUE_CLASS, Object.class);
+ Class<?> keyOutClass = conf.getClass(MAPPER_OUTPUT_KEY_CLASS, Object.class);
+ Class<?> valueOutClass = conf.getClass(MAPPER_OUTPUT_VALUE_CLASS,
+ Object.class);
+ RecordReader rr = new ChainRecordReader(keyClass, valueClass, input, conf);
+ RecordWriter rw = new ChainRecordWriter(keyOutClass, valueOutClass, output,
+ conf);
+ MapRunner runner = new MapRunner(mappers.get(index), createMapContext(rr,
+ rw, context, getConf(index)), rr, rw);
+ threads.add(runner);
+ }
+
+ /**
+ * Create a reduce context that is based on ChainMapContext and the given
+ * record writer
+ */
+ private <KEYIN, VALUEIN, KEYOUT, VALUEOUT>
+ Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context createReduceContext(
+ RecordWriter<KEYOUT, VALUEOUT> rw,
+ ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> context,
+ Configuration conf) {
+ ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> reduceContext =
+ new ChainReduceContextImpl<KEYIN, VALUEIN, KEYOUT, VALUEOUT>(
+ context, rw, conf);
+ Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context reducerContext =
+ new WrappedReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>()
+ .getReducerContext(reduceContext);
+ return reducerContext;
+ }
+
+ // Run the reducer directly.
+ @SuppressWarnings("unchecked")
+ <KEYIN, VALUEIN, KEYOUT, VALUEOUT> void runReducer(
+ TaskInputOutputContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> context)
+ throws IOException, InterruptedException {
+ RecordWriter<KEYOUT, VALUEOUT> rw = new ChainRecordWriter<KEYOUT, VALUEOUT>(
+ context);
+ Reducer.Context reducerContext = createReduceContext(rw,
+ (ReduceContext) context, rConf);
+ reducer.run(reducerContext);
+ rw.close(context);
+ }
+
+ /**
+ * Add reducer that reads from context and writes to a queue
+ */
+ @SuppressWarnings("unchecked")
+ void addReducer(TaskInputOutputContext inputContext,
+ ChainBlockingQueue<KeyValuePair<?, ?>> outputQueue) throws IOException,
+ InterruptedException {
+
+ Class<?> keyOutClass = rConf.getClass(REDUCER_OUTPUT_KEY_CLASS,
+ Object.class);
+ Class<?> valueOutClass = rConf.getClass(REDUCER_OUTPUT_VALUE_CLASS,
+ Object.class);
+ RecordWriter rw = new ChainRecordWriter(keyOutClass, valueOutClass,
+ outputQueue, rConf);
+ Reducer.Context reducerContext = createReduceContext(rw,
+ (ReduceContext) inputContext, rConf);
+ ReduceRunner runner = new ReduceRunner(reducerContext, reducer, rw);
+ threads.add(runner);
+ }
+
+ // start all the threads
+ void startAllThreads() {
+ for (Thread thread : threads) {
+ thread.start();
+ }
+ }
+
+ // wait till all threads finish
+ void joinAllThreads() throws IOException, InterruptedException {
+ for (Thread thread : threads) {
+ thread.join();
+ }
+ Throwable th = getThrowable();
+ if (th != null) {
+ if (th instanceof IOException) {
+ throw (IOException) th;
+ } else if (th instanceof InterruptedException) {
+ throw (InterruptedException) th;
+ } else {
+ throw new RuntimeException(th);
+ }
+ }
+ }
+
+ // interrupt all threads
+ private synchronized void interruptAllThreads() {
+ for (Thread th : threads) {
+ th.interrupt();
+ }
+ for (ChainBlockingQueue<?> queue : blockingQueues) {
+ queue.interrupt();
+ }
+ }
+
+ /**
+ * Returns the prefix to use for the configuration of the chain depending if
+ * it is for a Mapper or a Reducer.
+ *
+ * @param isMap
+ * TRUE for Mapper, FALSE for Reducer.
+ * @return the prefix to use.
+ */
+ protected static String getPrefix(boolean isMap) {
+ return (isMap) ? CHAIN_MAPPER : CHAIN_REDUCER;
+ }
+
+ protected static int getIndex(Configuration conf, String prefix) {
+ return conf.getInt(prefix + CHAIN_MAPPER_SIZE, 0);
+ }
+
+ /**
+ * Creates a {@link Configuration} for the Map or Reduce in the chain.
+ *
+ * <p>
+ * It creates a new Configuration using the chain job's Configuration as base
+ * and adds to it the configuration properties for the chain element. The keys
+ * of the chain element Configuration have precedence over the given
+ * Configuration.
+ * </p>
+ *
+ * @param jobConf
+ * the chain job's Configuration.
+ * @param confKey
+ * the key for chain element configuration serialized in the chain
+ * job's Configuration.
+ * @return a new Configuration aggregating the chain job's Configuration with
+ * the chain element configuration properties.
+ */
+ protected static Configuration getChainElementConf(Configuration jobConf,
+ String confKey) {
+ Configuration conf = null;
+ try {
+ Stringifier<Configuration> stringifier =
+ new DefaultStringifier<Configuration>(jobConf, Configuration.class);
+ String confString = jobConf.get(confKey, null);
+ if (confString != null) {
+ conf = stringifier.fromString(jobConf.get(confKey, null));
+ }
+ } catch (IOException ioex) {
+ throw new RuntimeException(ioex);
+ }
+ // we have to do this because the Writable desearialization clears all
+ // values set in the conf making not possible do a
+ // new Configuration(jobConf) in the creation of the conf above
+ jobConf = new Configuration(jobConf);
+
+ if (conf != null) {
+ for (Map.Entry<String, String> entry : conf) {
+ jobConf.set(entry.getKey(), entry.getValue());
+ }
+ }
+ return jobConf;
+ }
+
+ /**
+ * Adds a Mapper class to the chain job.
+ *
+ * <p/>
+ * The configuration properties of the chain job have precedence over the
+ * configuration properties of the Mapper.
+ *
+ * @param isMap
+ * indicates if the Chain is for a Mapper or for a Reducer.
+ * @param job
+ * chain job.
+ * @param klass
+ * the Mapper class to add.
+ * @param inputKeyClass
+ * mapper input key class.
+ * @param inputValueClass
+ * mapper input value class.
+ * @param outputKeyClass
+ * mapper output key class.
+ * @param outputValueClass
+ * mapper output value class.
+ * @param mapperConf
+ * a configuration for the Mapper class. It is recommended to use a
+ * Configuration without default values using the
+ * <code>Configuration(boolean loadDefaults)</code> constructor with
+ * FALSE.
+ */
+ @SuppressWarnings("unchecked")
+ protected static void addMapper(boolean isMap, Job job,
+ Class<? extends Mapper> klass, Class<?> inputKeyClass,
+ Class<?> inputValueClass, Class<?> outputKeyClass,
+ Class<?> outputValueClass, Configuration mapperConf) {
+ String prefix = getPrefix(isMap);
+ Configuration jobConf = job.getConfiguration();
+
+ // if a reducer chain check the Reducer has been already set
+ checkReducerAlreadySet(isMap, jobConf, prefix, true);
+
+ // set the mapper class
+ int index = getIndex(jobConf, prefix);
+ jobConf.setClass(prefix + CHAIN_MAPPER_CLASS + index, klass, Mapper.class);
+
+ validateKeyValueTypes(isMap, jobConf, inputKeyClass, inputValueClass,
+ outputKeyClass, outputValueClass, index, prefix);
+
+ setMapperConf(isMap, jobConf, inputKeyClass, inputValueClass,
+ outputKeyClass, outputValueClass, mapperConf, index, prefix);
+ }
+
+ // if a reducer chain check the Reducer has been already set or not
+ protected static void checkReducerAlreadySet(boolean isMap,
+ Configuration jobConf, String prefix, boolean shouldSet) {
+ if (!isMap) {
+ if (shouldSet) {
+ if (jobConf.getClass(prefix + CHAIN_REDUCER_CLASS, null) == null) {
+ throw new IllegalStateException(
+ "A Mapper can be added to the chain only after the Reducer has "
+ + "been set");
+ }
+ } else {
+ if (jobConf.getClass(prefix + CHAIN_REDUCER_CLASS, null) != null) {
+ throw new IllegalStateException("Reducer has been already set");
+ }
+ }
+ }
+ }
+
+ protected static void validateKeyValueTypes(boolean isMap,
+ Configuration jobConf, Class<?> inputKeyClass, Class<?> inputValueClass,
+ Class<?> outputKeyClass, Class<?> outputValueClass, int index,
+ String prefix) {
+ // if it is a reducer chain and the first Mapper is being added check the
+ // key and value input classes of the mapper match those of the reducer
+ // output.
+ if (!isMap && index == 0) {
+ Configuration reducerConf = getChainElementConf(jobConf, prefix
+ + CHAIN_REDUCER_CONFIG);
+ if (!inputKeyClass.isAssignableFrom(reducerConf.getClass(
+ REDUCER_OUTPUT_KEY_CLASS, null))) {
+ throw new IllegalArgumentException("The Reducer output key class does"
+ + " not match the Mapper input key class");
+ }
+ if (!inputValueClass.isAssignableFrom(reducerConf.getClass(
+ REDUCER_OUTPUT_VALUE_CLASS, null))) {
+ throw new IllegalArgumentException("The Reducer output value class"
+ + " does not match the Mapper input value class");
+ }
+ } else if (index > 0) {
+ // check the that the new Mapper in the chain key and value input classes
+ // match those of the previous Mapper output.
+ Configuration previousMapperConf = getChainElementConf(jobConf, prefix
+ + CHAIN_MAPPER_CONFIG + (index - 1));
+ if (!inputKeyClass.isAssignableFrom(previousMapperConf.getClass(
+ MAPPER_OUTPUT_KEY_CLASS, null))) {
+ throw new IllegalArgumentException("The Mapper output key class does"
+ + " not match the previous Mapper input key class");
+ }
+ if (!inputValueClass.isAssignableFrom(previousMapperConf.getClass(
+ MAPPER_OUTPUT_VALUE_CLASS, null))) {
+ throw new IllegalArgumentException("The Mapper output value class"
+ + " does not match the previous Mapper input value class");
+ }
+ }
+ }
+
+ protected static void setMapperConf(boolean isMap, Configuration jobConf,
+ Class<?> inputKeyClass, Class<?> inputValueClass,
+ Class<?> outputKeyClass, Class<?> outputValueClass,
+ Configuration mapperConf, int index, String prefix) {
+ // if the Mapper does not have a configuration, create an empty one
+ if (mapperConf == null) {
+ // using a Configuration without defaults to make it lightweight.
+ // still the chain's conf may have all defaults and this conf is
+ // overlapped to the chain configuration one.
+ mapperConf = new Configuration(true);
+ }
+
+ // store the input/output classes of the mapper in the mapper conf
+ mapperConf.setClass(MAPPER_INPUT_KEY_CLASS, inputKeyClass, Object.class);
+ mapperConf
+ .setClass(MAPPER_INPUT_VALUE_CLASS, inputValueClass, Object.class);
+ mapperConf.setClass(MAPPER_OUTPUT_KEY_CLASS, outputKeyClass, Object.class);
+ mapperConf.setClass(MAPPER_OUTPUT_VALUE_CLASS, outputValueClass,
+ Object.class);
+ // serialize the mapper configuration in the chain configuration.
+ Stringifier<Configuration> stringifier =
+ new DefaultStringifier<Configuration>(jobConf, Configuration.class);
+ try {
+ jobConf.set(prefix + CHAIN_MAPPER_CONFIG + index, stringifier
+ .toString(new Configuration(mapperConf)));
+ } catch (IOException ioEx) {
+ throw new RuntimeException(ioEx);
+ }
+
+ // increment the chain counter
+ jobConf.setInt(prefix + CHAIN_MAPPER_SIZE, index + 1);
+ }
+
+ /**
+ * Sets the Reducer class to the chain job.
+ *
+ * <p/>
+ * The configuration properties of the chain job have precedence over the
+ * configuration properties of the Reducer.
+ *
+ * @param job
+ * the chain job.
+ * @param klass
+ * the Reducer class to add.
+ * @param inputKeyClass
+ * reducer input key class.
+ * @param inputValueClass
+ * reducer input value class.
+ * @param outputKeyClass
+ * reducer output key class.
+ * @param outputValueClass
+ * reducer output value class.
+ * @param reducerConf
+ * a configuration for the Reducer class. It is recommended to use a
+ * Configuration without default values using the
+ * <code>Configuration(boolean loadDefaults)</code> constructor with
+ * FALSE.
+ */
+ @SuppressWarnings("unchecked")
+ protected static void setReducer(Job job, Class<? extends Reducer> klass,
+ Class<?> inputKeyClass, Class<?> inputValueClass,
+ Class<?> outputKeyClass, Class<?> outputValueClass,
+ Configuration reducerConf) {
+ String prefix = getPrefix(false);
+ Configuration jobConf = job.getConfiguration();
+ checkReducerAlreadySet(false, jobConf, prefix, false);
+
+ jobConf.setClass(prefix + CHAIN_REDUCER_CLASS, klass, Reducer.class);
+
+ setReducerConf(jobConf, inputKeyClass, inputValueClass, outputKeyClass,
+ outputValueClass, reducerConf, prefix);
+ }
+
+ protected static void setReducerConf(Configuration jobConf,
+ Class<?> inputKeyClass, Class<?> inputValueClass,
+ Class<?> outputKeyClass, Class<?> outputValueClass,
+ Configuration reducerConf, String prefix) {
+ // if the Reducer does not have a Configuration, create an empty one
+ if (reducerConf == null) {
+ // using a Configuration without defaults to make it lightweight.
+ // still the chain's conf may have all defaults and this conf is
+ // overlapped to the chain's Configuration one.
+ reducerConf = new Configuration(false);
+ }
+
+ // store the input/output classes of the reducer in
+ // the reducer configuration
+ reducerConf.setClass(REDUCER_INPUT_KEY_CLASS, inputKeyClass, Object.class);
+ reducerConf.setClass(REDUCER_INPUT_VALUE_CLASS, inputValueClass,
+ Object.class);
+ reducerConf
+ .setClass(REDUCER_OUTPUT_KEY_CLASS, outputKeyClass, Object.class);
+ reducerConf.setClass(REDUCER_OUTPUT_VALUE_CLASS, outputValueClass,
+ Object.class);
+
+ // serialize the reducer configuration in the chain's configuration.
+ Stringifier<Configuration> stringifier =
+ new DefaultStringifier<Configuration>(jobConf, Configuration.class);
+ try {
+ jobConf.set(prefix + CHAIN_REDUCER_CONFIG, stringifier
+ .toString(new Configuration(reducerConf)));
+ } catch (IOException ioEx) {
+ throw new RuntimeException(ioEx);
+ }
+ }
+
+ /**
+ * Setup the chain.
+ *
+ * @param jobConf
+ * chain job's {@link Configuration}.
+ */
+ @SuppressWarnings("unchecked")
+ void setup(Configuration jobConf) {
+ String prefix = getPrefix(isMap);
+
+ int index = jobConf.getInt(prefix + CHAIN_MAPPER_SIZE, 0);
+ for (int i = 0; i < index; i++) {
+ Class<? extends Mapper> klass = jobConf.getClass(prefix
+ + CHAIN_MAPPER_CLASS + i, null, Mapper.class);
+ Configuration mConf = getChainElementConf(jobConf, prefix
+ + CHAIN_MAPPER_CONFIG + i);
+ confList.add(mConf);
+ Mapper mapper = ReflectionUtils.newInstance(klass, mConf);
+ mappers.add(mapper);
+
+ }
+
+ Class<? extends Reducer> klass = jobConf.getClass(prefix
+ + CHAIN_REDUCER_CLASS, null, Reducer.class);
+ if (klass != null) {
+ rConf = getChainElementConf(jobConf, prefix + CHAIN_REDUCER_CONFIG);
+ reducer = ReflectionUtils.newInstance(klass, rConf);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ List<Mapper> getAllMappers() {
+ return mappers;
+ }
+
+ /**
+ * Returns the Reducer instance in the chain.
+ *
+ * @return the Reducer instance in the chain or NULL if none.
+ */
+ Reducer<?, ?, ?, ?> getReducer() {
+ return reducer;
+ }
+
+ /**
+ * Creates a ChainBlockingQueue with KeyValuePair as element
+ *
+ * @return the ChainBlockingQueue
+ */
+ ChainBlockingQueue<KeyValuePair<?, ?>> createBlockingQueue() {
+ return new ChainBlockingQueue<KeyValuePair<?, ?>>();
+ }
+
+ /**
+ * A blocking queue with one element.
+ *
+ * @param <E>
+ */
+ class ChainBlockingQueue<E> {
+ E element = null;
+ boolean isInterrupted = false;
+
+ ChainBlockingQueue() {
+ blockingQueues.add(this);
+ }
+
+ synchronized void enqueue(E e) throws InterruptedException {
+ while (element != null) {
+ if (isInterrupted) {
+ throw new InterruptedException();
+ }
+ this.wait();
+ }
+ element = e;
+ this.notify();
+ }
+
+ synchronized E dequeue() throws InterruptedException {
+ while (element == null) {
+ if (isInterrupted) {
+ throw new InterruptedException();
+ }
+ this.wait();
+ }
+ E e = element;
+ element = null;
+ this.notify();
+ return e;
+ }
+
+ synchronized void interrupt() {
+ isInterrupted = true;
+ this.notifyAll();
+ }
+ }
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainMapContextImpl.java b/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainMapContextImpl.java
new file mode 100644
index 0000000..85c395f
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainMapContextImpl.java
@@ -0,0 +1,308 @@
+/**
+ * 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.mapreduce.lib.chain;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskInputOutputContext;
+
+/**
+ * A simple wrapper class that delegates most of its functionality to the
+ * underlying context, but overrides the methods to do with record readers ,
+ * record writers and configuration.
+ */
+class ChainMapContextImpl<KEYIN, VALUEIN, KEYOUT, VALUEOUT> implements
+ MapContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
+
+ private RecordReader<KEYIN, VALUEIN> reader;
+ private RecordWriter<KEYOUT, VALUEOUT> output;
+ private TaskInputOutputContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> base;
+ private Configuration conf;
+
+ ChainMapContextImpl(
+ TaskInputOutputContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> base,
+ RecordReader<KEYIN, VALUEIN> rr, RecordWriter<KEYOUT, VALUEOUT> rw,
+ Configuration conf) {
+ this.reader = rr;
+ this.output = rw;
+ this.base = base;
+ this.conf = conf;
+ }
+
+ @Override
+ public KEYIN getCurrentKey() throws IOException, InterruptedException {
+ return reader.getCurrentKey();
+ }
+
+ @Override
+ public VALUEIN getCurrentValue() throws IOException, InterruptedException {
+ return reader.getCurrentValue();
+ }
+
+ @Override
+ public boolean nextKeyValue() throws IOException, InterruptedException {
+ return reader.nextKeyValue();
+ }
+
+ @Override
+ public InputSplit getInputSplit() {
+ if (base instanceof MapContext) {
+ MapContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> mc =
+ (MapContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT>) base;
+ return mc.getInputSplit();
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public Counter getCounter(Enum<?> counterName) {
+ return base.getCounter(counterName);
+ }
+
+ @Override
+ public Counter getCounter(String groupName, String counterName) {
+ return base.getCounter(groupName, counterName);
+ }
+
+ @Override
+ public OutputCommitter getOutputCommitter() {
+ return base.getOutputCommitter();
+ }
+
+ @Override
+ public void write(KEYOUT key, VALUEOUT value) throws IOException,
+ InterruptedException {
+ output.write(key, value);
+ }
+
+ @Override
+ public String getStatus() {
+ return base.getStatus();
+ }
+
+ @Override
+ public TaskAttemptID getTaskAttemptID() {
+ return base.getTaskAttemptID();
+ }
+
+ @Override
+ public void setStatus(String msg) {
+ base.setStatus(msg);
+ }
+
+ @Override
+ public Path[] getArchiveClassPaths() {
+ return base.getArchiveClassPaths();
+ }
+
+ @Override
+ public String[] getArchiveTimestamps() {
+ return base.getArchiveTimestamps();
+ }
+
+ @Override
+ public URI[] getCacheArchives() throws IOException {
+ return base.getCacheArchives();
+ }
+
+ @Override
+ public URI[] getCacheFiles() throws IOException {
+ return base.getCacheFiles();
+ }
+
+ @Override
+ public Class<? extends Reducer<?, ?, ?, ?>> getCombinerClass()
+ throws ClassNotFoundException {
+ return base.getCombinerClass();
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return conf;
+ }
+
+ @Override
+ public Path[] getFileClassPaths() {
+ return base.getFileClassPaths();
+ }
+
+ @Override
+ public String[] getFileTimestamps() {
+ return base.getFileTimestamps();
+ }
+
+ @Override
+ public RawComparator<?> getGroupingComparator() {
+ return base.getGroupingComparator();
+ }
+
+ @Override
+ public Class<? extends InputFormat<?, ?>> getInputFormatClass()
+ throws ClassNotFoundException {
+ return base.getInputFormatClass();
+ }
+
+ @Override
+ public String getJar() {
+ return base.getJar();
+ }
+
+ @Override
+ public JobID getJobID() {
+ return base.getJobID();
+ }
+
+ @Override
+ public String getJobName() {
+ return base.getJobName();
+ }
+
+ @Override
+ public boolean getJobSetupCleanupNeeded() {
+ return base.getJobSetupCleanupNeeded();
+ }
+
+ @Override
+ public Path[] getLocalCacheArchives() throws IOException {
+ return base.getLocalCacheArchives();
+ }
+
+ @Override
+ public Path[] getLocalCacheFiles() throws IOException {
+ return base.getLocalCacheArchives();
+ }
+
+ @Override
+ public Class<?> getMapOutputKeyClass() {
+ return base.getMapOutputKeyClass();
+ }
+
+ @Override
+ public Class<?> getMapOutputValueClass() {
+ return base.getMapOutputValueClass();
+ }
+
+ @Override
+ public Class<? extends Mapper<?, ?, ?, ?>> getMapperClass()
+ throws ClassNotFoundException {
+ return base.getMapperClass();
+ }
+
+ @Override
+ public int getMaxMapAttempts() {
+ return base.getMaxMapAttempts();
+ }
+
+ @Override
+ public int getMaxReduceAttempts() {
+ return base.getMaxReduceAttempts();
+ }
+
+ @Override
+ public int getNumReduceTasks() {
+ return base.getNumReduceTasks();
+ }
+
+ @Override
+ public Class<? extends OutputFormat<?, ?>> getOutputFormatClass()
+ throws ClassNotFoundException {
+ return base.getOutputFormatClass();
+ }
+
+ @Override
+ public Class<?> getOutputKeyClass() {
+ return base.getMapOutputKeyClass();
+ }
+
+ @Override
+ public Class<?> getOutputValueClass() {
+ return base.getOutputValueClass();
+ }
+
+ @Override
+ public Class<? extends Partitioner<?, ?>> getPartitionerClass()
+ throws ClassNotFoundException {
+ return base.getPartitionerClass();
+ }
+
+ @Override
+ public boolean getProfileEnabled() {
+ return base.getProfileEnabled();
+ }
+
+ @Override
+ public String getProfileParams() {
+ return base.getProfileParams();
+ }
+
+ @Override
+ public IntegerRanges getProfileTaskRange(boolean isMap) {
+ return base.getProfileTaskRange(isMap);
+ }
+
+ @Override
+ public Class<? extends Reducer<?, ?, ?, ?>> getReducerClass()
+ throws ClassNotFoundException {
+ return base.getReducerClass();
+ }
+
+ @Override
+ public RawComparator<?> getSortComparator() {
+ return base.getSortComparator();
+ }
+
+ @Override
+ public boolean getSymlink() {
+ return base.getSymlink();
+ }
+
+ @Override
+ public String getUser() {
+ return base.getUser();
+ }
+
+ @Override
+ public Path getWorkingDirectory() throws IOException {
+ return base.getWorkingDirectory();
+ }
+
+ @Override
+ public void progress() {
+ base.progress();
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainMapper.java b/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainMapper.java
new file mode 100644
index 0000000..2b2b0fb
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainMapper.java
@@ -0,0 +1,167 @@
+/**
+ * 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.mapreduce.lib.chain;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.chain.Chain.ChainBlockingQueue;
+
+/**
+ * The ChainMapper class allows to use multiple Mapper classes within a single
+ * Map task.
+ *
+ * <p>
+ * The Mapper classes are invoked in a chained (or piped) fashion, the output of
+ * the first becomes the input of the second, and so on until the last Mapper,
+ * the output of the last Mapper will be written to the task's output.
+ * </p>
+ * <p>
+ * The key functionality of this feature is that the Mappers in the chain do not
+ * need to be aware that they are executed in a chain. This enables having
+ * reusable specialized Mappers that can be combined to perform composite
+ * operations within a single task.
+ * </p>
+ * <p>
+ * Special care has to be taken when creating chains that the key/values output
+ * by a Mapper are valid for the following Mapper in the chain. It is assumed
+ * all Mappers and the Reduce in the chain use matching output and input key and
+ * value classes as no conversion is done by the chaining code.
+ * </p>
+ * <p>
+ * Using the ChainMapper and the ChainReducer classes is possible to compose
+ * Map/Reduce jobs that look like <code>[MAP+ / REDUCE MAP*]</code>. And
+ * immediate benefit of this pattern is a dramatic reduction in disk IO.
+ * </p>
+ * <p>
+ * IMPORTANT: There is no need to specify the output key/value classes for the
+ * ChainMapper, this is done by the addMapper for the last mapper in the chain.
+ * </p>
+ * ChainMapper usage pattern:
+ * <p/>
+ *
+ * <pre>
+ * ...
+ * Job = new Job(conf);
+ * <p/>
+ * Configuration mapAConf = new Configuration(false);
+ * ...
+ * ChainMapper.addMapper(job, AMap.class, LongWritable.class, Text.class,
+ * Text.class, Text.class, true, mapAConf);
+ * <p/>
+ * Configuration mapBConf = new Configuration(false);
+ * ...
+ * ChainMapper.addMapper(job, BMap.class, Text.class, Text.class,
+ * LongWritable.class, Text.class, false, mapBConf);
+ * <p/>
+ * ...
+ * <p/>
+ * job.waitForComplettion(true);
+ * ...
+ * </pre>
+ */
+public class ChainMapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> extends
+ Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
+
+ /**
+ * Adds a {@link Mapper} class to the chain mapper.
+ *
+ * <p>
+ * The key and values are passed from one element of the chain to the next, by
+ * value. For the added Mapper the configuration given for it,
+ * <code>mapperConf</code>, have precedence over the job's Configuration. This
+ * precedence is in effect when the task is running.
+ * </p>
+ * <p>
+ * IMPORTANT: There is no need to specify the output key/value classes for the
+ * ChainMapper, this is done by the addMapper for the last mapper in the chain
+ * </p>
+ *
+ * @param job
+ * The job.
+ * @param klass
+ * the Mapper class to add.
+ * @param inputKeyClass
+ * mapper input key class.
+ * @param inputValueClass
+ * mapper input value class.
+ * @param outputKeyClass
+ * mapper output key class.
+ * @param outputValueClass
+ * mapper output value class.
+ * @param mapperConf
+ * a configuration for the Mapper class. It is recommended to use a
+ * Configuration without default values using the
+ * <code>Configuration(boolean loadDefaults)</code> constructor with
+ * FALSE.
+ */
+ public static void addMapper(Job job, Class<? extends Mapper> klass,
+ Class<?> inputKeyClass, Class<?> inputValueClass,
+ Class<?> outputKeyClass, Class<?> outputValueClass,
+ Configuration mapperConf) throws IOException {
+ job.setMapperClass(ChainMapper.class);
+ job.setMapOutputKeyClass(outputKeyClass);
+ job.setMapOutputValueClass(outputValueClass);
+ Chain.addMapper(true, job, klass, inputKeyClass, inputValueClass,
+ outputKeyClass, outputValueClass, mapperConf);
+ }
+
+ private Chain chain;
+
+ protected void setup(Context context) {
+ chain = new Chain(true);
+ chain.setup(context.getConfiguration());
+ }
+
+ public void run(Context context) throws IOException, InterruptedException {
+
+ setup(context);
+
+ int numMappers = chain.getAllMappers().size();
+ if (numMappers == 0) {
+ return;
+ }
+
+ ChainBlockingQueue<Chain.KeyValuePair<?, ?>> inputqueue;
+ ChainBlockingQueue<Chain.KeyValuePair<?, ?>> outputqueue;
+ if (numMappers == 1) {
+ chain.runMapper(context, 0);
+ } else {
+ // add all the mappers with proper context
+ // add first mapper
+ outputqueue = chain.createBlockingQueue();
+ chain.addMapper(context, outputqueue, 0);
+ // add other mappers
+ for (int i = 1; i < numMappers - 1; i++) {
+ inputqueue = outputqueue;
+ outputqueue = chain.createBlockingQueue();
+ chain.addMapper(inputqueue, outputqueue, context, i);
+ }
+ // add last mapper
+ chain.addMapper(outputqueue, context, numMappers - 1);
+ }
+
+ // start all threads
+ chain.startAllThreads();
+
+ // wait for all threads
+ chain.joinAllThreads();
+ }
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainReduceContextImpl.java b/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainReduceContextImpl.java
new file mode 100644
index 0000000..c74ba04
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainReduceContextImpl.java
@@ -0,0 +1,300 @@
+/**
+ * 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.mapreduce.lib.chain;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.ReduceContext;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+/**
+ * A simple wrapper class that delegates most of its functionality to the
+ * underlying context, but overrides the methods to do with record writer and
+ * configuration
+ */
+class ChainReduceContextImpl<KEYIN, VALUEIN, KEYOUT, VALUEOUT> implements
+ ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
+
+ private final ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> base;
+ private final RecordWriter<KEYOUT, VALUEOUT> rw;
+ private final Configuration conf;
+
+ public ChainReduceContextImpl(
+ ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> base,
+ RecordWriter<KEYOUT, VALUEOUT> output, Configuration conf) {
+ this.base = base;
+ this.rw = output;
+ this.conf = conf;
+ }
+
+ @Override
+ public Iterable<VALUEIN> getValues() throws IOException, InterruptedException {
+ return base.getValues();
+ }
+
+ @Override
+ public boolean nextKey() throws IOException, InterruptedException {
+ return base.nextKey();
+ }
+
+ @Override
+ public Counter getCounter(Enum<?> counterName) {
+ return base.getCounter(counterName);
+ }
+
+ @Override
+ public Counter getCounter(String groupName, String counterName) {
+ return base.getCounter(groupName, counterName);
+ }
+
+ @Override
+ public KEYIN getCurrentKey() throws IOException, InterruptedException {
+ return base.getCurrentKey();
+ }
+
+ @Override
+ public VALUEIN getCurrentValue() throws IOException, InterruptedException {
+ return base.getCurrentValue();
+ }
+
+ @Override
+ public OutputCommitter getOutputCommitter() {
+ return base.getOutputCommitter();
+ }
+
+ @Override
+ public boolean nextKeyValue() throws IOException, InterruptedException {
+ return base.nextKeyValue();
+ }
+
+ @Override
+ public void write(KEYOUT key, VALUEOUT value) throws IOException,
+ InterruptedException {
+ rw.write(key, value);
+ }
+
+ @Override
+ public String getStatus() {
+ return base.getStatus();
+ }
+
+ @Override
+ public TaskAttemptID getTaskAttemptID() {
+ return base.getTaskAttemptID();
+ }
+
+ @Override
+ public void setStatus(String msg) {
+ base.setStatus(msg);
+ }
+
+ @Override
+ public Path[] getArchiveClassPaths() {
+ return base.getArchiveClassPaths();
+ }
+
+ @Override
+ public String[] getArchiveTimestamps() {
+ return base.getArchiveTimestamps();
+ }
+
+ @Override
+ public URI[] getCacheArchives() throws IOException {
+ return base.getCacheArchives();
+ }
+
+ @Override
+ public URI[] getCacheFiles() throws IOException {
+ return base.getCacheFiles();
+ }
+
+ @Override
+ public Class<? extends Reducer<?, ?, ?, ?>> getCombinerClass()
+ throws ClassNotFoundException {
+ return base.getCombinerClass();
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return conf;
+ }
+
+ @Override
+ public Path[] getFileClassPaths() {
+ return base.getFileClassPaths();
+ }
+
+ @Override
+ public String[] getFileTimestamps() {
+ return base.getFileTimestamps();
+ }
+
+ @Override
+ public RawComparator<?> getGroupingComparator() {
+ return base.getGroupingComparator();
+ }
+
+ @Override
+ public Class<? extends InputFormat<?, ?>> getInputFormatClass()
+ throws ClassNotFoundException {
+ return base.getInputFormatClass();
+ }
+
+ @Override
+ public String getJar() {
+ return base.getJar();
+ }
+
+ @Override
+ public JobID getJobID() {
+ return base.getJobID();
+ }
+
+ @Override
+ public String getJobName() {
+ return base.getJobName();
+ }
+
+ @Override
+ public boolean getJobSetupCleanupNeeded() {
+ return base.getJobSetupCleanupNeeded();
+ }
+
+ @Override
+ public Path[] getLocalCacheArchives() throws IOException {
+ return base.getLocalCacheArchives();
+ }
+
+ @Override
+ public Path[] getLocalCacheFiles() throws IOException {
+ return base.getLocalCacheFiles();
+ }
+
+ @Override
+ public Class<?> getMapOutputKeyClass() {
+ return base.getMapOutputKeyClass();
+ }
+
+ @Override
+ public Class<?> getMapOutputValueClass() {
+ return base.getMapOutputValueClass();
+ }
+
+ @Override
+ public Class<? extends Mapper<?, ?, ?, ?>> getMapperClass()
+ throws ClassNotFoundException {
+ return base.getMapperClass();
+ }
+
+ @Override
+ public int getMaxMapAttempts() {
+ return base.getMaxMapAttempts();
+ }
+
+ @Override
+ public int getMaxReduceAttempts() {
+ return base.getMaxMapAttempts();
+ }
+
+ @Override
+ public int getNumReduceTasks() {
+ return base.getNumReduceTasks();
+ }
+
+ @Override
+ public Class<? extends OutputFormat<?, ?>> getOutputFormatClass()
+ throws ClassNotFoundException {
+ return base.getOutputFormatClass();
+ }
+
+ @Override
+ public Class<?> getOutputKeyClass() {
+ return base.getOutputKeyClass();
+ }
+
+ @Override
+ public Class<?> getOutputValueClass() {
+ return base.getOutputValueClass();
+ }
+
+ @Override
+ public Class<? extends Partitioner<?, ?>> getPartitionerClass()
+ throws ClassNotFoundException {
+ return base.getPartitionerClass();
+ }
+
+ @Override
+ public boolean getProfileEnabled() {
+ return base.getProfileEnabled();
+ }
+
+ @Override
+ public String getProfileParams() {
+ return base.getProfileParams();
+ }
+
+ @Override
+ public IntegerRanges getProfileTaskRange(boolean isMap) {
+ return base.getProfileTaskRange(isMap);
+ }
+
+ @Override
+ public Class<? extends Reducer<?, ?, ?, ?>> getReducerClass()
+ throws ClassNotFoundException {
+ return base.getReducerClass();
+ }
+
+ @Override
+ public RawComparator<?> getSortComparator() {
+ return base.getSortComparator();
+ }
+
+ @Override
+ public boolean getSymlink() {
+ return base.getSymlink();
+ }
+
+ @Override
+ public String getUser() {
+ return base.getUser();
+ }
+
+ @Override
+ public Path getWorkingDirectory() throws IOException {
+ return base.getWorkingDirectory();
+ }
+
+ @Override
+ public void progress() {
+ base.progress();
+ }
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainReducer.java b/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainReducer.java
new file mode 100644
index 0000000..95973c9
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/lib/chain/ChainReducer.java
@@ -0,0 +1,216 @@
+/**
+ * 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.mapreduce.lib.chain;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.chain.Chain.ChainBlockingQueue;
+
+import java.io.IOException;
+
+/**
+ * The ChainReducer class allows to chain multiple Mapper classes after a
+ * Reducer within the Reducer task.
+ *
+ * <p>
+ * For each record output by the Reducer, the Mapper classes are invoked in a
+ * chained (or piped) fashion. The output of the reducer becomes the input of
+ * the first mapper and output of first becomes the input of the second, and so
+ * on until the last Mapper, the output of the last Mapper will be written to
+ * the task's output.
+ * </p>
+ * <p>
+ * The key functionality of this feature is that the Mappers in the chain do not
+ * need to be aware that they are executed after the Reducer or in a chain. This
+ * enables having reusable specialized Mappers that can be combined to perform
+ * composite operations within a single task.
+ * </p>
+ * <p>
+ * Special care has to be taken when creating chains that the key/values output
+ * by a Mapper are valid for the following Mapper in the chain. It is assumed
+ * all Mappers and the Reduce in the chain use matching output and input key and
+ * value classes as no conversion is done by the chaining code.
+ * </p>
+ * </p> Using the ChainMapper and the ChainReducer classes is possible to
+ * compose Map/Reduce jobs that look like <code>[MAP+ / REDUCE MAP*]</code>. And
+ * immediate benefit of this pattern is a dramatic reduction in disk IO. </p>
+ * <p>
+ * IMPORTANT: There is no need to specify the output key/value classes for the
+ * ChainReducer, this is done by the setReducer or the addMapper for the last
+ * element in the chain.
+ * </p>
+ * ChainReducer usage pattern:
+ * <p/>
+ *
+ * <pre>
+ * ...
+ * Job = new Job(conf);
+ * ....
+ * <p/>
+ * Configuration reduceConf = new Configuration(false);
+ * ...
+ * ChainReducer.setReducer(job, XReduce.class, LongWritable.class, Text.class,
+ * Text.class, Text.class, true, reduceConf);
+ * <p/>
+ * ChainReducer.addMapper(job, CMap.class, Text.class, Text.class,
+ * LongWritable.class, Text.class, false, null);
+ * <p/>
+ * ChainReducer.addMapper(job, DMap.class, LongWritable.class, Text.class,
+ * LongWritable.class, LongWritable.class, true, null);
+ * <p/>
+ * ...
+ * <p/>
+ * job.waitForCompletion(true);
+ * ...
+ * </pre>
+ */
+public class ChainReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> extends
+ Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
+
+ /**
+ * Sets the {@link Reducer} class to the chain job.
+ *
+ * <p>
+ * The key and values are passed from one element of the chain to the next, by
+ * value. For the added Reducer the configuration given for it,
+ * <code>reducerConf</code>, have precedence over the job's Configuration.
+ * This precedence is in effect when the task is running.
+ * </p>
+ * <p>
+ * IMPORTANT: There is no need to specify the output key/value classes for the
+ * ChainReducer, this is done by the setReducer or the addMapper for the last
+ * element in the chain.
+ * </p>
+ *
+ * @param job
+ * the job
+ * @param klass
+ * the Reducer class to add.
+ * @param inputKeyClass
+ * reducer input key class.
+ * @param inputValueClass
+ * reducer input value class.
+ * @param outputKeyClass
+ * reducer output key class.
+ * @param outputValueClass
+ * reducer output value class.
+ * @param reducerConf
+ * a configuration for the Reducer class. It is recommended to use a
+ * Configuration without default values using the
+ * <code>Configuration(boolean loadDefaults)</code> constructor with
+ * FALSE.
+ */
+ public static void setReducer(Job job, Class<? extends Reducer> klass,
+ Class<?> inputKeyClass, Class<?> inputValueClass,
+ Class<?> outputKeyClass, Class<?> outputValueClass,
+ Configuration reducerConf) {
+ job.setReducerClass(ChainReducer.class);
+ job.setOutputKeyClass(outputKeyClass);
+ job.setOutputValueClass(outputValueClass);
+ Chain.setReducer(job, klass, inputKeyClass, inputValueClass,
+ outputKeyClass, outputValueClass, reducerConf);
+ }
+
+ /**
+ * Adds a {@link Mapper} class to the chain reducer.
+ *
+ * <p>
+ * The key and values are passed from one element of the chain to the next, by
+ * value For the added Mapper the configuration given for it,
+ * <code>mapperConf</code>, have precedence over the job's Configuration. This
+ * precedence is in effect when the task is running.
+ * </p>
+ * <p>
+ * IMPORTANT: There is no need to specify the output key/value classes for the
+ * ChainMapper, this is done by the addMapper for the last mapper in the
+ * chain.
+ * </p>
+ *
+ * @param job
+ * The job.
+ * @param klass
+ * the Mapper class to add.
+ * @param inputKeyClass
+ * mapper input key class.
+ * @param inputValueClass
+ * mapper input value class.
+ * @param outputKeyClass
+ * mapper output key class.
+ * @param outputValueClass
+ * mapper output value class.
+ * @param mapperConf
+ * a configuration for the Mapper class. It is recommended to use a
+ * Configuration without default values using the
+ * <code>Configuration(boolean loadDefaults)</code> constructor with
+ * FALSE.
+ */
+ public static void addMapper(Job job, Class<? extends Mapper> klass,
+ Class<?> inputKeyClass, Class<?> inputValueClass,
+ Class<?> outputKeyClass, Class<?> outputValueClass,
+ Configuration mapperConf) throws IOException {
+ job.setOutputKeyClass(outputKeyClass);
+ job.setOutputValueClass(outputValueClass);
+ Chain.addMapper(false, job, klass, inputKeyClass, inputValueClass,
+ outputKeyClass, outputValueClass, mapperConf);
+ }
+
+ private Chain chain;
+
+ protected void setup(Context context) {
+ chain = new Chain(false);
+ chain.setup(context.getConfiguration());
+ }
+
+ public void run(Context context) throws IOException, InterruptedException {
+ setup(context);
+
+ // if no reducer is set, just do nothing
+ if (chain.getReducer() == null) {
+ return;
+ }
+ int numMappers = chain.getAllMappers().size();
+ // if there are no mappers in chain, run the reducer
+ if (numMappers == 0) {
+ chain.runReducer(context);
+ return;
+ }
+
+ // add reducer and all mappers with proper context
+ ChainBlockingQueue<Chain.KeyValuePair<?, ?>> inputqueue;
+ ChainBlockingQueue<Chain.KeyValuePair<?, ?>> outputqueue;
+ // add reducer
+ outputqueue = chain.createBlockingQueue();
+ chain.addReducer(context, outputqueue);
+ // add all mappers except last one
+ for (int i = 0; i < numMappers - 1; i++) {
+ inputqueue = outputqueue;
+ outputqueue = chain.createBlockingQueue();
+ chain.addMapper(inputqueue, outputqueue, context, i);
+ }
+ // add last mapper
+ chain.addMapper(outputqueue, context, numMappers - 1);
+
+ // start all threads
+ chain.startAllThreads();
+
+ // wait for all threads
+ chain.joinAllThreads();
+ }
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/db/OracleDBRecordReader.java b/src/java/org/apache/hadoop/mapreduce/lib/db/OracleDBRecordReader.java
index 7fe45c0..db6bc9e 100644
--- a/src/java/org/apache/hadoop/mapreduce/lib/db/OracleDBRecordReader.java
+++ b/src/java/org/apache/hadoop/mapreduce/lib/db/OracleDBRecordReader.java
@@ -23,18 +23,25 @@
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
+import java.util.TimeZone;
+import java.lang.reflect.Method;
import org.apache.hadoop.conf.Configuration;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
/**
* A RecordReader that reads records from an Oracle SQL table.
*/
public class OracleDBRecordReader<T extends DBWritable> extends DBRecordReader<T> {
+ private static final Log LOG = LogFactory.getLog(OracleDBRecordReader.class);
+
public OracleDBRecordReader(DBInputFormat.DBInputSplit split,
Class<T> inputClass, Configuration conf, Connection conn, DBConfiguration dbConfig,
String cond, String [] fields, String table) throws SQLException {
super(split, inputClass, conf, conn, dbConfig, cond, fields, table);
+ setSessionTimeZone(conn);
}
/** Returns the query for selecting the records from an Oracle DB. */
@@ -86,4 +93,43 @@
return query.toString();
}
+
+ /**
+ * Set session time zone
+ * @param conn Connection object
+ * @throws SQLException instance
+ */
+ private void setSessionTimeZone(Connection conn) throws SQLException {
+ // need to use reflection to call the method setSessionTimeZone on the OracleConnection class
+ // because oracle specific java libraries are not accessible in this context
+ Method method;
+ try {
+ method = conn.getClass().getMethod(
+ "setSessionTimeZone", new Class [] {String.class});
+ } catch (Exception ex) {
+ LOG.error("Could not find method setSessionTimeZone in " + conn.getClass().getName(), ex);
+ // rethrow SQLException
+ throw new SQLException(ex);
+ }
+
+ // Need to set the time zone in order for Java
+ // to correctly access the column "TIMESTAMP WITH LOCAL TIME ZONE"
+ String clientTimeZone = TimeZone.getDefault().getID();
+ try {
+ method.setAccessible(true);
+ method.invoke(conn, clientTimeZone);
+ LOG.info("Time zone has been set");
+ } catch (Exception ex) {
+ LOG.warn("Time zone " + clientTimeZone +
+ " could not be set on oracle database.");
+ LOG.info("Setting default time zone: UTC");
+ try {
+ method.invoke(conn, "UTC");
+ } catch (Exception ex2) {
+ LOG.error("Could not set time zone for oracle connection", ex2);
+ // rethrow SQLException
+ throw new SQLException(ex);
+ }
+ }
+ }
}
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionHelper.java b/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionHelper.java
index 1428530..646246b 100644
--- a/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionHelper.java
+++ b/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionHelper.java
@@ -34,7 +34,8 @@
*
* The field separator is under attribute "mapreduce.fieldsel.data.field.separator"
*
- * The map output field list spec is under attribute "mapreduce.fieldsel.mapreduce.fieldsel.map.output.key.value.fields.spec".
+ * The map output field list spec is under attribute
+ * "mapreduce.fieldsel.map.output.key.value.fields.spec".
* The value is expected to be like "keyFieldsSpec:valueFieldsSpec"
* key/valueFieldsSpec are comma (,) separated field spec: fieldSpec,fieldSpec,fieldSpec ...
* Each field spec can be a simple number (e.g. 5) specifying a specific field, or a range
@@ -45,7 +46,8 @@
* Here is an example: "4,3,0,1:6,5,1-3,7-". It specifies to use fields 4,3,0 and 1 for keys,
* and use fields 6,5,1,2,3,7 and above for values.
*
- * The reduce output field list spec is under attribute "mapreduce.fieldsel.mapreduce.fieldsel.reduce.output.key.value.fields.spec".
+ * The reduce output field list spec is under attribute
+ * "mapreduce.fieldsel.reduce.output.key.value.fields.spec".
*
* The reducer extracts output key/value pairs in a similar manner, except that
* the key is never ignored.
@@ -57,9 +59,9 @@
public static final String DATA_FIELD_SEPERATOR =
"mapreduce.fieldsel.data.field.separator";
public static final String MAP_OUTPUT_KEY_VALUE_SPEC =
- "mapreduce.fieldsel.mapreduce.fieldsel.mapreduce.fieldsel.map.output.key.value.fields.spec";
+ "mapreduce.fieldsel.map.output.key.value.fields.spec";
public static final String REDUCE_OUTPUT_KEY_VALUE_SPEC =
- "mapreduce.fieldsel.mapreduce.fieldsel.mapreduce.fieldsel.reduce.output.key.value.fields.spec";
+ "mapreduce.fieldsel.reduce.output.key.value.fields.spec";
/**
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionMapper.java b/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionMapper.java
index f9ecefb..dcd6cae 100644
--- a/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionMapper.java
+++ b/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionMapper.java
@@ -42,7 +42,8 @@
* The field separator is under attribute "mapreduce.fieldsel.data.field.separator"
*
* The map output field list spec is under attribute
- * "mapreduce.fieldsel.mapreduce.fieldsel.map.output.key.value.fields.spec". The value is expected to be like
+ * "mapreduce.fieldsel.map.output.key.value.fields.spec".
+ * The value is expected to be like
* "keyFieldsSpec:valueFieldsSpec" key/valueFieldsSpec are comma (,) separated
* field spec: fieldSpec,fieldSpec,fieldSpec ... Each field spec can be a
* simple number (e.g. 5) specifying a specific field, or a range (like 2-5)
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionReducer.java b/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionReducer.java
index 4f6de6c..9e700fb 100644
--- a/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionReducer.java
+++ b/src/java/org/apache/hadoop/mapreduce/lib/fieldsel/FieldSelectionReducer.java
@@ -41,7 +41,8 @@
* The field separator is under attribute "mapreduce.fieldsel.data.field.separator"
*
* The reduce output field list spec is under attribute
- * "mapreduce.fieldsel.mapreduce.fieldsel.reduce.output.key.value.fields.spec". The value is expected to be like
+ * "mapreduce.fieldsel.reduce.output.key.value.fields.spec".
+ * The value is expected to be like
* "keyFieldsSpec:valueFieldsSpec" key/valueFieldsSpec are comma (,)
* separated field spec: fieldSpec,fieldSpec,fieldSpec ... Each field spec
* can be a simple number (e.g. 5) specifying a specific field, or a range
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java b/src/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
index 657a454..07ac4c6 100644
--- a/src/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
+++ b/src/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
@@ -238,6 +238,14 @@
return result;
}
+ /**
+ * A factory that makes the split for this class. It can be overridden
+ * by sub-classes to make sub-types
+ */
+ protected FileSplit makeSplit(Path file, long start, long length,
+ String[] hosts) {
+ return new FileSplit(file, start, length, hosts);
+ }
/**
* Generate the list of files and make them into FileSplits.
@@ -261,20 +269,20 @@
long bytesRemaining = length;
while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) {
int blkIndex = getBlockIndex(blkLocations, length-bytesRemaining);
- splits.add(new FileSplit(path, length-bytesRemaining, splitSize,
+ splits.add(makeSplit(path, length-bytesRemaining, splitSize,
blkLocations[blkIndex].getHosts()));
bytesRemaining -= splitSize;
}
if (bytesRemaining != 0) {
- splits.add(new FileSplit(path, length-bytesRemaining, bytesRemaining,
+ splits.add(makeSplit(path, length-bytesRemaining, bytesRemaining,
blkLocations[blkLocations.length-1].getHosts()));
}
} else if (length != 0) {
- splits.add(new FileSplit(path, 0, length, blkLocations[0].getHosts()));
+ splits.add(makeSplit(path, 0, length, blkLocations[0].getHosts()));
} else {
//Create empty hosts array for zero length files
- splits.add(new FileSplit(path, 0, length, new String[0]));
+ splits.add(makeSplit(path, 0, length, new String[0]));
}
}
LOG.debug("Total # of splits: " + splits.size());
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/input/SequenceFileInputFilter.java b/src/java/org/apache/hadoop/mapreduce/lib/input/SequenceFileInputFilter.java
index c1b8db2..bbbfa3d 100644
--- a/src/java/org/apache/hadoop/mapreduce/lib/input/SequenceFileInputFilter.java
+++ b/src/java/org/apache/hadoop/mapreduce/lib/input/SequenceFileInputFilter.java
@@ -47,11 +47,11 @@
public static final Log LOG = LogFactory.getLog(FileInputFormat.class);
final public static String FILTER_CLASS =
- "mapreduce.input.mapreduce.input.mapreduce.input.sequencefileinputfilter.class";
+ "mapreduce.input.sequencefileinputfilter.class";
final public static String FILTER_FREQUENCY =
- "mapreduce.input.mapreduce.input.mapreduce.input.sequencefileinputfilter.frequency";
+ "mapreduce.input.sequencefileinputfilter.frequency";
final public static String FILTER_REGEX =
- "mapreduce.input.mapreduce.input.mapreduce.input.sequencefileinputfilter.regex";
+ "mapreduce.input.sequencefileinputfilter.regex";
public SequenceFileInputFilter() {
}
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/input/TaggedInputSplit.java b/src/java/org/apache/hadoop/mapreduce/lib/input/TaggedInputSplit.java
index 57d800e..68bb789 100644
--- a/src/java/org/apache/hadoop/mapreduce/lib/input/TaggedInputSplit.java
+++ b/src/java/org/apache/hadoop/mapreduce/lib/input/TaggedInputSplit.java
@@ -125,7 +125,6 @@
Deserializer deserializer = factory.getDeserializer(inputSplitClass);
deserializer.open((DataInputStream)in);
inputSplit = (InputSplit)deserializer.deserialize(inputSplit);
- deserializer.close();
}
private Class<?> readClass(DataInput in) throws IOException {
@@ -147,7 +146,6 @@
factory.getSerializer(inputSplitClass);
serializer.open((DataOutputStream)out);
serializer.serialize(inputSplit);
- serializer.close();
}
public Configuration getConf() {
diff --git a/src/java/org/apache/hadoop/mapreduce/lib/join/CompositeInputSplit.java b/src/java/org/apache/hadoop/mapreduce/lib/join/CompositeInputSplit.java
index 7f03868..31305eb 100644
--- a/src/java/org/apache/hadoop/mapreduce/lib/join/CompositeInputSplit.java
+++ b/src/java/org/apache/hadoop/mapreduce/lib/join/CompositeInputSplit.java
@@ -128,7 +128,6 @@
factory.getSerializer(s.getClass());
serializer.open((DataOutputStream)out);
serializer.serialize(s);
- serializer.close();
}
}
@@ -155,7 +154,6 @@
Deserializer deserializer = factory.getDeserializer(cls[i]);
deserializer.open((DataInputStream)in);
splits[i] = (InputSplit)deserializer.deserialize(splits[i]);
- deserializer.close();
}
} catch (ClassNotFoundException e) {
throw new IOException("Failed split init", e);
diff --git a/src/java/org/apache/hadoop/mapreduce/protocol/ClientProtocol.java b/src/java/org/apache/hadoop/mapreduce/protocol/ClientProtocol.java
index 4e6838e..7dd013b 100644
--- a/src/java/org/apache/hadoop/mapreduce/protocol/ClientProtocol.java
+++ b/src/java/org/apache/hadoop/mapreduce/protocol/ClientProtocol.java
@@ -20,7 +20,6 @@
import java.io.IOException;
-import org.apache.hadoop.fs.Path;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.mapreduce.ClusterMetrics;
import org.apache.hadoop.mapreduce.Counters;
@@ -33,6 +32,7 @@
import org.apache.hadoop.mapreduce.TaskReport;
import org.apache.hadoop.mapreduce.TaskTrackerInfo;
import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.security.TokenStorage;
import org.apache.hadoop.mapreduce.server.jobtracker.State;
/**
@@ -87,8 +87,12 @@
* Version 28: Added getJobHistoryDir() as part of MAPREDUCE-975.
* Version 29: Added reservedSlots, runningTasks and totalJobSubmissions
* to ClusterMetrics as part of MAPREDUCE-1048.
+ * Version 30: Job submission files are uploaded to a staging area under
+ * user home dir. JobTracker reads the required files from the
+ * staging area using user credentials passed via the rpc.
+ * Version 31: Added TokenStorage to submitJob
*/
- public static final long versionID = 29L;
+ public static final long versionID = 31L;
/**
* Allocate a name for the job.
@@ -100,9 +104,8 @@
/**
* Submit a Job for execution. Returns the latest profile for
* that job.
- * The job files should be submitted in <b>system-dir</b>/<b>jobName</b>.
*/
- public JobStatus submitJob(JobID jobName)
+ public JobStatus submitJob(JobID jobId, String jobSubmitDir, TokenStorage ts)
throws IOException, InterruptedException;
/**
@@ -219,7 +222,15 @@
*
* @return the system directory where job-specific files are to be placed.
*/
- public String getSystemDir() throws IOException, InterruptedException;
+ public String getSystemDir() throws IOException, InterruptedException;
+
+ /**
+ * Get a hint from the JobTracker
+ * where job-specific files are to be placed.
+ *
+ * @return the directory where job-specific files are to be placed.
+ */
+ public String getStagingAreaDir() throws IOException, InterruptedException;
/**
* Gets the directory location of the completed job history files.
diff --git a/src/java/org/apache/hadoop/mapreduce/security/JobTokens.java b/src/java/org/apache/hadoop/mapreduce/security/JobTokens.java
deleted file mode 100644
index bfbcd72..0000000
--- a/src/java/org/apache/hadoop/mapreduce/security/JobTokens.java
+++ /dev/null
@@ -1,81 +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.
- */
-
-package org.apache.hadoop.mapreduce.security;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * get/set, store/load security keys
- * key's value - byte[]
- * store/load from DataInput/DataOuptut
- * List of currently store keys:
- * jobToken for secure shuffle HTTP Get
- *
- */
-@InterfaceAudience.Private
-public class JobTokens implements Writable {
- /**
- * file name used on HDFS for generated keys
- */
- public static final String JOB_TOKEN_FILENAME = "jobTokens";
-
- private byte [] shuffleJobToken = null; // jobtoken for shuffle (map output)
-
-
- /**
- * returns the key value for the alias
- * @return key for this alias
- */
- public byte[] getShuffleJobToken() {
- return shuffleJobToken;
- }
-
- /**
- * sets the jobToken
- * @param key
- */
- public void setShuffleJobToken(byte[] key) {
- shuffleJobToken = key;
- }
-
- /**
- * stores all the keys to DataOutput
- * @param out
- * @throws IOException
- */
- @Override
- public void write(DataOutput out) throws IOException {
- WritableUtils.writeCompressedByteArray(out, shuffleJobToken);
- }
-
- /**
- * loads all the keys
- * @param in
- * @throws IOException
- */
- @Override
- public void readFields(DataInput in) throws IOException {
- shuffleJobToken = WritableUtils.readCompressedByteArray(in);
- }
-}
diff --git a/src/java/org/apache/hadoop/mapreduce/security/SecureShuffleUtils.java b/src/java/org/apache/hadoop/mapreduce/security/SecureShuffleUtils.java
index 401455f..589c193 100644
--- a/src/java/org/apache/hadoop/mapreduce/security/SecureShuffleUtils.java
+++ b/src/java/org/apache/hadoop/mapreduce/security/SecureShuffleUtils.java
@@ -22,16 +22,13 @@
import java.io.IOException;
import java.io.PrintStream;
import java.net.URL;
-import java.security.InvalidKeyException;
-import java.security.NoSuchAlgorithmException;
-import javax.crypto.KeyGenerator;
-import javax.crypto.Mac;
-import javax.crypto.spec.SecretKeySpec;
+import javax.crypto.SecretKey;
import javax.servlet.http.HttpServletRequest;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.apache.hadoop.record.Utils;
/**
@@ -43,62 +40,17 @@
public class SecureShuffleUtils {
public static final String HTTP_HEADER_URL_HASH = "UrlHash";
public static final String HTTP_HEADER_REPLY_URL_HASH = "ReplyHash";
- public static KeyGenerator kg = null;
- public static String DEFAULT_ALG="HmacSHA1";
-
- private SecretKeySpec secretKey;
- private Mac mac;
-
/**
- * static generate keys
- * @return new encoded key
- * @throws NoSuchAlgorithmException
+ * file name used on HDFS for generated job token
*/
- public static byte[] getNewEncodedKey() throws NoSuchAlgorithmException{
- SecretKeySpec key = generateKey(DEFAULT_ALG);
- return key.getEncoded();
- }
-
- private static SecretKeySpec generateKey(String alg) throws NoSuchAlgorithmException {
- if(kg==null) {
- kg = KeyGenerator.getInstance(alg);
- }
- return (SecretKeySpec) kg.generateKey();
- }
-
- /**
- * Create a util object with alg and key
- * @param sKeyEncoded
- * @throws NoSuchAlgorithmException
- * @throws InvalidKeyException
- */
- public SecureShuffleUtils(byte [] sKeyEncoded)
- throws IOException{
- secretKey = new SecretKeySpec(sKeyEncoded, SecureShuffleUtils.DEFAULT_ALG);
- try {
- mac = Mac.getInstance(DEFAULT_ALG);
- mac.init(secretKey);
- } catch (NoSuchAlgorithmException nae) {
- throw new IOException(nae);
- } catch( InvalidKeyException ie) {
- throw new IOException(ie);
- }
- }
-
- /**
- * get key as byte[]
- * @return encoded key
- */
- public byte [] getEncodedKey() {
- return secretKey.getEncoded();
- }
+ public static final String JOB_TOKEN_FILENAME = "jobToken";
/**
* Base64 encoded hash of msg
* @param msg
*/
- public String generateHash(byte[] msg) {
- return new String(Base64.encodeBase64(generateByteHash(msg)));
+ public static String generateHash(byte[] msg, SecretKey key) {
+ return new String(Base64.encodeBase64(generateByteHash(msg, key)));
}
/**
@@ -106,8 +58,8 @@
* @param msg
* @return
*/
- private byte[] generateByteHash(byte[] msg) {
- return mac.doFinal(msg);
+ private static byte[] generateByteHash(byte[] msg, SecretKey key) {
+ return JobTokenSecretManager.computeHash(msg, key);
}
/**
@@ -115,20 +67,21 @@
* @param newHash
* @return true if is the same
*/
- private boolean verifyHash(byte[] hash, byte[] msg) {
- byte[] msg_hash = generateByteHash(msg);
+ private static boolean verifyHash(byte[] hash, byte[] msg, SecretKey key) {
+ byte[] msg_hash = generateByteHash(msg, key);
return Utils.compareBytes(msg_hash, 0, msg_hash.length, hash, 0, hash.length) == 0;
}
/**
* Aux util to calculate hash of a String
* @param enc_str
+ * @param key
* @return Base64 encodedHash
* @throws IOException
*/
- public String hashFromString(String enc_str)
+ public static String hashFromString(String enc_str, SecretKey key)
throws IOException {
- return generateHash(enc_str.getBytes());
+ return generateHash(enc_str.getBytes(), key);
}
/**
@@ -137,11 +90,11 @@
* @param msg
* @throws IOException if not the same
*/
- public void verifyReply(String base64Hash, String msg)
+ public static void verifyReply(String base64Hash, String msg, SecretKey key)
throws IOException {
byte[] hash = Base64.decodeBase64(base64Hash.getBytes());
- boolean res = verifyHash(hash, msg.getBytes());
+ boolean res = verifyHash(hash, msg.getBytes(), key);
if(res != true) {
throw new IOException("Verification of the hashReply failed");
diff --git a/src/java/org/apache/hadoop/mapreduce/security/TokenCache.java b/src/java/org/apache/hadoop/mapreduce/security/TokenCache.java
new file mode 100644
index 0000000..a2bdc82
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/security/TokenCache.java
@@ -0,0 +1,147 @@
+/**
+ * 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.mapreduce.security;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+
+/**
+ * this class keeps static references to TokenStorage object
+ * also it provides auxiliary methods for setting and getting secret keys
+ */
+@InterfaceStability.Evolving
+public class TokenCache {
+
+ private static final Log LOG = LogFactory.getLog(TokenCache.class);
+
+ private static TokenStorage tokenStorage;
+
+ /**
+ * auxiliary method to get user's secret keys..
+ * @param alias
+ * @return secret key from the storage
+ */
+ public static byte[] getSecretKey(Text alias) {
+ if(tokenStorage == null)
+ return null;
+ return tokenStorage.getSecretKey(alias);
+ }
+
+ /**
+ * auxiliary methods to store user' s secret keys
+ * @param alias
+ * @param key
+ */
+ public static void setSecretKey(Text alias, byte[] key) {
+ getTokenStorage().addSecretKey(alias, key);
+ }
+
+ /**
+ * auxiliary method to add a delegation token
+ */
+ public static void addDelegationToken(
+ String namenode, Token<? extends TokenIdentifier> t) {
+ getTokenStorage().setToken(new Text(namenode), t);
+ }
+
+ /**
+ * auxiliary method
+ * @return all the available tokens
+ */
+ public static Collection<Token<? extends TokenIdentifier>> getAllTokens() {
+ return getTokenStorage().getAllTokens();
+ }
+
+
+ /**
+ * @return TokenStore object
+ */
+ @InterfaceAudience.Private
+ public static TokenStorage getTokenStorage() {
+ if(tokenStorage==null)
+ tokenStorage = new TokenStorage();
+
+ return tokenStorage;
+ }
+
+ /**
+ * sets TokenStorage
+ * @param ts
+ */
+ @InterfaceAudience.Private
+ public static void setTokenStorage(TokenStorage ts) {
+ if(tokenStorage != null)
+ LOG.warn("Overwriting existing token storage with # keys=" +
+ tokenStorage.numberOfSecretKeys());
+ tokenStorage = ts;
+ }
+
+ /**
+ * load token storage and stores it
+ * @param conf
+ * @return Loaded TokenStorage object
+ * @throws IOException
+ */
+ @InterfaceAudience.Private
+ public static TokenStorage loadTaskTokenStorage(JobConf conf)
+ throws IOException {
+ if(tokenStorage != null)
+ return tokenStorage;
+
+ tokenStorage = loadTokens(conf);
+
+ return tokenStorage;
+ }
+
+ /**
+ * load job token from a file
+ * @param conf
+ * @throws IOException
+ */
+ @InterfaceAudience.Private
+ public static TokenStorage loadTokens(JobConf conf)
+ throws IOException {
+ String jobTokenFile = conf.get(JobContext.JOB_TOKEN_FILE);
+ Path localJobTokenFile = new Path (jobTokenFile);
+ FileSystem localFS = FileSystem.getLocal(conf);
+ FSDataInputStream in = localFS.open(localJobTokenFile);
+
+ TokenStorage ts = new TokenStorage();
+ ts.readFields(in);
+
+ LOG.info("Task: Loaded jobTokenFile from: "+localJobTokenFile.toUri().getPath()
+ +"; num of sec keys = " + ts.numberOfSecretKeys());
+ in.close();
+ return ts;
+ }
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/security/TokenStorage.java b/src/java/org/apache/hadoop/mapreduce/security/TokenStorage.java
new file mode 100644
index 0000000..bfda311
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/security/TokenStorage.java
@@ -0,0 +1,168 @@
+/**
+ * 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.mapreduce.security;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * get/set, store/load security keys
+ * key's value - byte[]
+ * store/load from DataInput/DataOuptut
+ *
+ */
+@InterfaceAudience.Private
+public class TokenStorage implements Writable {
+
+ private static final Text SHUFFLE_JOB_TOKEN = new Text("ShuffleJobToken");
+
+ private Map<Text, byte[]> secretKeysMap = new HashMap<Text, byte[]>();
+ private Map<Text, Token<? extends TokenIdentifier>> tokenMap =
+ new HashMap<Text, Token<? extends TokenIdentifier>>();
+
+ /**
+ * returns the key value for the alias
+ * @param alias
+ * @return key for this alias
+ */
+ byte[] getSecretKey(Text alias) {
+ return secretKeysMap.get(alias);
+ }
+
+ /**
+ * returns the key value for the alias
+ * @param alias
+ * @return token for this alias
+ */
+ Token<? extends TokenIdentifier> getToken(Text alias) {
+ return tokenMap.get(alias);
+ }
+
+ void setToken(Text alias, Token<? extends TokenIdentifier> t) {
+ tokenMap.put(alias, t);
+ }
+
+ /**
+ * store job token
+ * @param t
+ */
+ @InterfaceAudience.Private
+ public void setJobToken(Token<? extends TokenIdentifier> t) {
+ setToken(SHUFFLE_JOB_TOKEN, t);
+ }
+ /**
+ *
+ * @return job token
+ */
+ @InterfaceAudience.Private
+ public Token<? extends TokenIdentifier> getJobToken() {
+ return getToken(SHUFFLE_JOB_TOKEN);
+ }
+
+ /**
+ *
+ * @return all the tokens in the storage
+ */
+ Collection<Token<? extends TokenIdentifier>> getAllTokens() {
+ return tokenMap.values();
+ }
+
+
+
+ /**
+ *
+ * @return number of keys
+ */
+ public int numberOfSecretKeys() {
+ return secretKeysMap.size();
+ }
+
+
+ /**
+ * set the key for an alias
+ * @param alias
+ * @param key
+ */
+ public void addSecretKey(Text alias, byte[] key) {
+ secretKeysMap.put(alias, key);
+ }
+
+ /**
+ * stores all the keys to DataOutput
+ * @param out
+ * @throws IOException
+ */
+ @Override
+ public void write(DataOutput out) throws IOException {
+ // write out tokens first
+ System.out.println("about to write out: token = " + tokenMap.size() +
+ "; sec = " + secretKeysMap.size());
+ WritableUtils.writeVInt(out, tokenMap.size());
+ for(Map.Entry<Text, Token<? extends TokenIdentifier>> e: tokenMap.entrySet()) {
+ e.getKey().write(out);
+ e.getValue().write(out);
+ }
+
+ // now write out secret keys
+ WritableUtils.writeVInt(out, secretKeysMap.size());
+ for(Map.Entry<Text, byte[]> e : secretKeysMap.entrySet()) {
+ e.getKey().write(out);
+ WritableUtils.writeCompressedByteArray(out, e.getValue());
+ }
+ }
+
+ /**
+ * loads all the keys
+ * @param in
+ * @throws IOException
+ */
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ secretKeysMap.clear();
+ tokenMap.clear();
+
+ int size = WritableUtils.readVInt(in);
+ for(int i=0; i<size; i++) {
+ Text alias = new Text();
+ alias.readFields(in);
+ Token<? extends TokenIdentifier> t = new Token<TokenIdentifier>();
+ t.readFields(in);
+ tokenMap.put(alias, t);
+ }
+
+ size = WritableUtils.readVInt(in);
+ for(int i=0; i<size; i++) {
+ Text alias = new Text();
+ alias.readFields(in);
+ byte[] key = WritableUtils.readCompressedByteArray(in);
+ secretKeysMap.put(alias, key);
+ }
+ }
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/security/token/JobTokenIdentifier.java b/src/java/org/apache/hadoop/mapreduce/security/token/JobTokenIdentifier.java
new file mode 100644
index 0000000..a75c383
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/security/token/JobTokenIdentifier.java
@@ -0,0 +1,70 @@
+/**
+ * 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.mapreduce.security.token;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * The token identifier for job token
+ */
+@InterfaceAudience.Private
+public class JobTokenIdentifier extends TokenIdentifier {
+ private Text jobid;
+ final static Text KIND_NAME = new Text("mapreduce.job");
+
+ /**
+ * Create a job token identifier from a jobid
+ * @param jobid the jobid to use
+ */
+ public JobTokenIdentifier(Text jobid) {
+ this.jobid = jobid;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public Text getKind() {
+ return KIND_NAME;
+ }
+
+ /**
+ * Get the jobid
+ * @return the jobid
+ */
+ public Text getJobId() {
+ return jobid;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ jobid.readFields(in);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public void write(DataOutput out) throws IOException {
+ jobid.write(out);
+ }
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/security/token/JobTokenSecretManager.java b/src/java/org/apache/hadoop/mapreduce/security/token/JobTokenSecretManager.java
new file mode 100644
index 0000000..cba2688
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/security/token/JobTokenSecretManager.java
@@ -0,0 +1,127 @@
+/**
+ * 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.mapreduce.security.token;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import javax.crypto.SecretKey;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * SecretManager for job token. It can be used to cache generated job tokens.
+ */
+@InterfaceAudience.Private
+public class JobTokenSecretManager extends SecretManager<JobTokenIdentifier> {
+ private final SecretKey masterKey;
+ private final Map<String, SecretKey> currentJobTokens;
+
+ /**
+ * Convert the byte[] to a secret key
+ * @param key the byte[] to create the secret key from
+ * @return the secret key
+ */
+ public static SecretKey createSecretKey(byte[] key) {
+ return SecretManager.createSecretKey(key);
+ }
+
+ /**
+ * Compute the HMAC hash of the message using the key
+ * @param msg the message to hash
+ * @param key the key to use
+ * @return the computed hash
+ */
+ public static byte[] computeHash(byte[] msg, SecretKey key) {
+ return createPassword(msg, key);
+ }
+
+ /**
+ * Default constructor
+ */
+ public JobTokenSecretManager() {
+ this.masterKey = generateSecret();
+ this.currentJobTokens = new TreeMap<String, SecretKey>();
+ }
+
+ /**
+ * Create a new password/secret for the given job token identifier.
+ * @param identifier the job token identifier
+ * @return token password/secret
+ */
+ @Override
+ public byte[] createPassword(JobTokenIdentifier identifier) {
+ byte[] result = createPassword(identifier.getBytes(), masterKey);
+ return result;
+ }
+
+ /**
+ * Add the job token of a job to cache
+ * @param jobId the job that owns the token
+ * @param token the job token
+ */
+ public void addTokenForJob(String jobId, Token<JobTokenIdentifier> token) {
+ SecretKey tokenSecret = createSecretKey(token.getPassword());
+ synchronized (currentJobTokens) {
+ currentJobTokens.put(jobId, tokenSecret);
+ }
+ }
+
+ /**
+ * Remove the cached job token of a job from cache
+ * @param jobId the job whose token is to be removed
+ */
+ public void removeTokenForJob(String jobId) {
+ synchronized (currentJobTokens) {
+ currentJobTokens.remove(jobId);
+ }
+ }
+
+ /**
+ * Look up the token password/secret for the given jobId.
+ * @param jobId the jobId to look up
+ * @return token password/secret as SecretKey
+ * @throws InvalidToken
+ */
+ public SecretKey retrieveTokenSecret(String jobId) throws InvalidToken {
+ SecretKey tokenSecret = null;
+ synchronized (currentJobTokens) {
+ tokenSecret = currentJobTokens.get(jobId);
+ }
+ if (tokenSecret == null) {
+ throw new InvalidToken("Can't find job token for job " + jobId + " !!");
+ }
+ return tokenSecret;
+ }
+
+ /**
+ * Look up the token password/secret for the given job token identifier.
+ * @param identifier the job token identifier to look up
+ * @return token password/secret as byte[]
+ * @throws InvalidToken
+ */
+ @Override
+ public byte[] retrievePassword(JobTokenIdentifier identifier)
+ throws InvalidToken {
+ return retrieveTokenSecret(identifier.getJobId().toString()).getEncoded();
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/server/jobtracker/JTConfig.java b/src/java/org/apache/hadoop/mapreduce/server/jobtracker/JTConfig.java
index a6ea640..cbc40f5 100644
--- a/src/java/org/apache/hadoop/mapreduce/server/jobtracker/JTConfig.java
+++ b/src/java/org/apache/hadoop/mapreduce/server/jobtracker/JTConfig.java
@@ -80,6 +80,8 @@
public static final String JT_AVG_BLACKLIST_THRESHOLD =
"mapreduce.jobtracker.blacklist.average.threshold";
public static final String JT_SYSTEM_DIR = "mapreduce.jobtracker.system.dir";
+ public static final String JT_STAGING_AREA_ROOT =
+ "mapreduce.jobtracker.staging.root.dir";
public static final String JT_MAX_TRACKER_BLACKLISTS =
"mapreduce.jobtracker.tasktracker.maxblacklists";
public static final String JT_JOBHISTORY_MAXAGE =
@@ -88,4 +90,6 @@
"mapreduce.jobtracker.maxmapmemory.mb";
public static final String JT_MAX_REDUCEMEMORY_MB =
"mapreduce.jobtracker.maxreducememory.mb";
+ public static final String MAX_JOB_SPLIT_METAINFO_SIZE =
+ "mapreduce.job.split.metainfo.maxsize";
}
diff --git a/src/java/org/apache/hadoop/mapreduce/server/tasktracker/Localizer.java b/src/java/org/apache/hadoop/mapreduce/server/tasktracker/Localizer.java
index b959e08..6a54503 100644
--- a/src/java/org/apache/hadoop/mapreduce/server/tasktracker/Localizer.java
+++ b/src/java/org/apache/hadoop/mapreduce/server/tasktracker/Localizer.java
@@ -193,7 +193,7 @@
synchronized (localizedUser) {
if (localizedUser.get()) {
- // User-directories are already localized for his user.
+ // User-directories are already localized for this user.
LOG.info("User-directories for the user " + user
+ " are already initialized on this TT. Not doing anything.");
return;
@@ -233,7 +233,7 @@
// Set up the cache directory used for distributed cache files
File distributedCacheDir =
- new File(localDir, TaskTracker.getDistributedCacheDir(user));
+ new File(localDir, TaskTracker.getPrivateDistributedCacheDir(user));
if (distributedCacheDir.exists() || distributedCacheDir.mkdirs()) {
// Set permissions on the distcache-directory
PermissionsHandler.setPermissions(distributedCacheDir,
diff --git a/src/java/org/apache/hadoop/mapreduce/server/tasktracker/TTConfig.java b/src/java/org/apache/hadoop/mapreduce/server/tasktracker/TTConfig.java
index 849bca8..de058dc 100644
--- a/src/java/org/apache/hadoop/mapreduce/server/tasktracker/TTConfig.java
+++ b/src/java/org/apache/hadoop/mapreduce/server/tasktracker/TTConfig.java
@@ -67,8 +67,14 @@
"mapreduce.tasktracker.instrumentation";
public static final String TT_MAP_SLOTS =
"mapreduce.tasktracker.map.tasks.maximum";
+ /**
+ * @deprecated Use {@link #TT_RESOURCE_CALCULATOR_PLUGIN} instead
+ */
+ @Deprecated
public static final String TT_MEMORY_CALCULATOR_PLUGIN =
"mapreduce.tasktracker.memorycalculatorplugin";
+ public static final String TT_RESOURCE_CALCULATOR_PLUGIN =
+ "mapreduce.tasktracker.resourcecalculatorplugin";
public static final String TT_REDUCE_SLOTS =
"mapreduce.tasktracker.reduce.tasks.maximum";
public static final String TT_MEMORY_MANAGER_MONITORING_INTERVAL =
diff --git a/src/java/org/apache/hadoop/mapreduce/split/JobSplit.java b/src/java/org/apache/hadoop/mapreduce/split/JobSplit.java
new file mode 100644
index 0000000..764b7ce
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/split/JobSplit.java
@@ -0,0 +1,217 @@
+/**
+ * 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.mapreduce.split;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * This class groups the fundamental classes associated with
+ * reading/writing splits. The split information is divided into
+ * two parts based on the consumer of the information. The two
+ * parts are the split meta information, and the raw split
+ * information. The first part is consumed by the JobTracker to
+ * create the tasks' locality data structures. The second part is
+ * used by the maps at runtime to know what to do!
+ * These pieces of information are written to two separate files.
+ * The metainformation file is slurped by the JobTracker during
+ * job initialization. A map task gets the meta information during
+ * the launch and it reads the raw split bytes directly from the
+ * file.
+ */
+@InterfaceAudience.Private
+public class JobSplit {
+ static final int META_SPLIT_VERSION = 1;
+ static final byte[] META_SPLIT_FILE_HEADER;
+ static {
+ try {
+ META_SPLIT_FILE_HEADER = "META-SPL".getBytes("UTF-8");
+ } catch (UnsupportedEncodingException u) {
+ throw new RuntimeException(u);
+ }
+ }
+ public static final TaskSplitMetaInfo EMPTY_TASK_SPLIT =
+ new TaskSplitMetaInfo();
+
+ /**
+ * This represents the meta information about the task split.
+ * The main fields are
+ * - start offset in actual split
+ * - data length that will be processed in this split
+ * - hosts on which this split is local
+ */
+ public static class SplitMetaInfo implements Writable {
+ private long startOffset;
+ private long inputDataLength;
+ private String[] locations;
+
+ public SplitMetaInfo() {}
+
+ public SplitMetaInfo(String[] locations, long startOffset,
+ long inputDataLength) {
+ this.locations = locations;
+ this.startOffset = startOffset;
+ this.inputDataLength = inputDataLength;
+ }
+
+ public SplitMetaInfo(InputSplit split, long startOffset) throws IOException {
+ try {
+ this.locations = split.getLocations();
+ this.inputDataLength = split.getLength();
+ this.startOffset = startOffset;
+ } catch (InterruptedException ie) {
+ throw new IOException(ie);
+ }
+ }
+
+ public String[] getLocations() {
+ return locations;
+ }
+
+ public long getStartOffset() {
+ return startOffset;
+ }
+
+ public long getInputDataLength() {
+ return inputDataLength;
+ }
+
+ public void setInputDataLocations(String[] locations) {
+ this.locations = locations;
+ }
+
+ public void setInputDataLength(long length) {
+ this.inputDataLength = length;
+ }
+
+ public void readFields(DataInput in) throws IOException {
+ int len = WritableUtils.readVInt(in);
+ locations = new String[len];
+ for (int i = 0; i < locations.length; i++) {
+ locations[i] = Text.readString(in);
+ }
+ startOffset = WritableUtils.readVLong(in);
+ inputDataLength = WritableUtils.readVLong(in);
+ }
+
+ public void write(DataOutput out) throws IOException {
+ WritableUtils.writeVInt(out, locations.length);
+ for (int i = 0; i < locations.length; i++) {
+ Text.writeString(out, locations[i]);
+ }
+ WritableUtils.writeVLong(out, startOffset);
+ WritableUtils.writeVLong(out, inputDataLength);
+ }
+
+ @Override
+ public String toString() {
+ StringBuffer buf = new StringBuffer();
+ buf.append("data-size : " + inputDataLength + "\n");
+ buf.append("start-offset : " + startOffset + "\n");
+ buf.append("locations : " + "\n");
+ for (String loc : locations) {
+ buf.append(" " + loc + "\n");
+ }
+ return buf.toString();
+ }
+ }
+ /**
+ * This represents the meta information about the task split that the
+ * JobTracker creates
+ */
+ public static class TaskSplitMetaInfo {
+ private TaskSplitIndex splitIndex;
+ private long inputDataLength;
+ private String[] locations;
+ public TaskSplitMetaInfo(){
+ this.splitIndex = new TaskSplitIndex();
+ this.locations = new String[0];
+ }
+ public TaskSplitMetaInfo(TaskSplitIndex splitIndex, String[] locations,
+ long inputDataLength) {
+ this.splitIndex = splitIndex;
+ this.locations = locations;
+ this.inputDataLength = inputDataLength;
+ }
+ public TaskSplitMetaInfo(InputSplit split, long startOffset)
+ throws InterruptedException, IOException {
+ this(new TaskSplitIndex("", startOffset), split.getLocations(),
+ split.getLength());
+ }
+
+ public TaskSplitMetaInfo(String[] locations, long startOffset,
+ long inputDataLength) {
+ this(new TaskSplitIndex("",startOffset), locations, inputDataLength);
+ }
+
+ public TaskSplitIndex getSplitIndex() {
+ return splitIndex;
+ }
+
+ public String getSplitLocation() {
+ return splitIndex.getSplitLocation();
+ }
+ public long getInputDataLength() {
+ return inputDataLength;
+ }
+ public String[] getLocations() {
+ return locations;
+ }
+ public long getStartOffset() {
+ return splitIndex.getStartOffset();
+ }
+ }
+
+ /**
+ * This represents the meta information about the task split that the
+ * task gets
+ */
+ public static class TaskSplitIndex {
+ private String splitLocation;
+ private long startOffset;
+ public TaskSplitIndex(){
+ this("", 0);
+ }
+ public TaskSplitIndex(String splitLocation, long startOffset) {
+ this.splitLocation = splitLocation;
+ this.startOffset = startOffset;
+ }
+ public long getStartOffset() {
+ return startOffset;
+ }
+ public String getSplitLocation() {
+ return splitLocation;
+ }
+ public void readFields(DataInput in) throws IOException {
+ splitLocation = Text.readString(in);
+ startOffset = WritableUtils.readVLong(in);
+ }
+ public void write(DataOutput out) throws IOException {
+ Text.writeString(out, splitLocation);
+ WritableUtils.writeVLong(out, startOffset);
+ }
+ }
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/split/JobSplitWriter.java b/src/java/org/apache/hadoop/mapreduce/split/JobSplitWriter.java
new file mode 100644
index 0000000..51d35bf
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/split/JobSplitWriter.java
@@ -0,0 +1,172 @@
+/**
+ * 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.mapreduce.split;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobSubmissionFiles;
+import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+/**
+ * The class that is used by the Job clients to write splits (both the meta
+ * and the raw bytes parts)
+ */
+public class JobSplitWriter {
+
+ private static final int splitVersion = JobSplit.META_SPLIT_VERSION;
+ private static final byte[] SPLIT_FILE_HEADER;
+ static {
+ try {
+ SPLIT_FILE_HEADER = "SPL".getBytes("UTF-8");
+ } catch (UnsupportedEncodingException u) {
+ throw new RuntimeException(u);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public static <T extends InputSplit> void createSplitFiles(Path jobSubmitDir,
+ Configuration conf, List<InputSplit> splits)
+ throws IOException, InterruptedException {
+ T[] array = (T[]) splits.toArray(new InputSplit[splits.size()]);
+ createSplitFiles(jobSubmitDir, conf, array);
+ }
+
+ public static <T extends InputSplit> void createSplitFiles(Path jobSubmitDir,
+ Configuration conf,T[] splits)
+ throws IOException, InterruptedException {
+ FileSystem fs = jobSubmitDir.getFileSystem(conf);
+ FSDataOutputStream out = createFile(fs,
+ JobSubmissionFiles.getJobSplitFile(jobSubmitDir), conf);
+ SplitMetaInfo[] info = writeNewSplits(conf, splits, out);
+ out.close();
+ writeJobSplitMetaInfo(fs,JobSubmissionFiles.getJobSplitMetaFile(jobSubmitDir),
+ new FsPermission(JobSubmissionFiles.JOB_FILE_PERMISSION), splitVersion,
+ info);
+ }
+
+ public static void createSplitFiles(Path jobSubmitDir,
+ Configuration conf, org.apache.hadoop.mapred.InputSplit[] splits)
+ throws IOException {
+ FileSystem fs = jobSubmitDir.getFileSystem(conf);
+ FSDataOutputStream out = createFile(fs,
+ JobSubmissionFiles.getJobSplitFile(jobSubmitDir), conf);
+ SplitMetaInfo[] info = writeOldSplits(splits, out);
+ out.close();
+ writeJobSplitMetaInfo(fs,JobSubmissionFiles.getJobSplitMetaFile(jobSubmitDir),
+ new FsPermission(JobSubmissionFiles.JOB_FILE_PERMISSION), splitVersion,
+ info);
+ }
+
+ private static FSDataOutputStream createFile(FileSystem fs, Path splitFile,
+ Configuration job) throws IOException {
+ FSDataOutputStream out = FileSystem.create(fs, splitFile,
+ new FsPermission(JobSubmissionFiles.JOB_FILE_PERMISSION));
+ int replication = job.getInt(Job.SUBMIT_REPLICATION, 10);
+ fs.setReplication(splitFile, (short)replication);
+ writeSplitHeader(out);
+ return out;
+ }
+ private static void writeSplitHeader(FSDataOutputStream out)
+ throws IOException {
+ out.write(SPLIT_FILE_HEADER);
+ out.writeInt(splitVersion);
+ }
+
+ @SuppressWarnings("unchecked")
+ private static <T extends InputSplit>
+ SplitMetaInfo[] writeNewSplits(Configuration conf,
+ T[] array, FSDataOutputStream out)
+ throws IOException, InterruptedException {
+
+ SplitMetaInfo[] info = new SplitMetaInfo[array.length];
+ if (array.length != 0) {
+ SerializationFactory factory = new SerializationFactory(conf);
+ int i = 0;
+ long offset = out.size();
+ for(T split: array) {
+ int prevCount = out.size();
+ Text.writeString(out, split.getClass().getName());
+ Serializer<T> serializer =
+ factory.getSerializer((Class<T>) split.getClass());
+ serializer.open(out);
+ serializer.serialize(split);
+ int currCount = out.size();
+ info[i++] =
+ new JobSplit.SplitMetaInfo(
+ split.getLocations(), offset,
+ split.getLength());
+ offset += currCount - prevCount;
+ }
+ }
+ return info;
+ }
+
+ private static SplitMetaInfo[] writeOldSplits(
+ org.apache.hadoop.mapred.InputSplit[] splits,
+ FSDataOutputStream out) throws IOException {
+ SplitMetaInfo[] info = new SplitMetaInfo[splits.length];
+ if (splits.length != 0) {
+ int i = 0;
+ long offset = out.size();
+ for(org.apache.hadoop.mapred.InputSplit split: splits) {
+ int prevLen = out.size();
+ Text.writeString(out, split.getClass().getName());
+ split.write(out);
+ int currLen = out.size();
+ info[i++] = new JobSplit.SplitMetaInfo(
+ split.getLocations(), offset,
+ split.getLength());
+ offset += currLen - prevLen;
+ }
+ }
+ return info;
+ }
+
+ private static void writeJobSplitMetaInfo(FileSystem fs, Path filename,
+ FsPermission p, int splitMetaInfoVersion,
+ JobSplit.SplitMetaInfo[] allSplitMetaInfo)
+ throws IOException {
+ // write the splits meta-info to a file for the job tracker
+ FSDataOutputStream out =
+ FileSystem.create(fs, filename, p);
+ out.write(JobSplit.META_SPLIT_FILE_HEADER);
+ WritableUtils.writeVInt(out, splitMetaInfoVersion);
+ WritableUtils.writeVInt(out, allSplitMetaInfo.length);
+ for (JobSplit.SplitMetaInfo splitMetaInfo : allSplitMetaInfo) {
+ splitMetaInfo.write(out);
+ }
+ out.close();
+ }
+}
+
diff --git a/src/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReader.java b/src/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReader.java
new file mode 100644
index 0000000..0bb05b2
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReader.java
@@ -0,0 +1,82 @@
+/**
+ * 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.mapreduce.split;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.JobSubmissionFiles;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+/**
+ * A utility that reads the split meta info and creates
+ * split meta info objects
+ */
+
+public class SplitMetaInfoReader {
+
+ public static JobSplit.TaskSplitMetaInfo[] readSplitMetaInfo(
+ JobID jobId, FileSystem fs, Configuration conf, Path jobSubmitDir)
+ throws IOException {
+ long maxMetaInfoSize = conf.getLong(JTConfig.MAX_JOB_SPLIT_METAINFO_SIZE,
+ 10000000L);
+ Path metaSplitFile = JobSubmissionFiles.getJobSplitMetaFile(jobSubmitDir);
+ FileStatus fStatus = fs.getFileStatus(metaSplitFile);
+ if (maxMetaInfoSize > 0 && fStatus.getLen() > maxMetaInfoSize) {
+ throw new IOException("Split metadata size exceeded " +
+ maxMetaInfoSize +". Aborting job " + jobId);
+ }
+ FSDataInputStream in = fs.open(metaSplitFile);
+ byte[] header = new byte[JobSplit.META_SPLIT_FILE_HEADER.length];
+ in.readFully(header);
+ if (!Arrays.equals(JobSplit.META_SPLIT_FILE_HEADER, header)) {
+ throw new IOException("Invalid header on split file");
+ }
+ int vers = WritableUtils.readVInt(in);
+ if (vers != JobSplit.META_SPLIT_VERSION) {
+ in.close();
+ throw new IOException("Unsupported split version " + vers);
+ }
+ int numSplits = WritableUtils.readVInt(in); //TODO: check for insane values
+ JobSplit.TaskSplitMetaInfo[] allSplitMetaInfo =
+ new JobSplit.TaskSplitMetaInfo[numSplits];
+ for (int i = 0; i < numSplits; i++) {
+ JobSplit.SplitMetaInfo splitMetaInfo = new JobSplit.SplitMetaInfo();
+ splitMetaInfo.readFields(in);
+ JobSplit.TaskSplitIndex splitIndex = new JobSplit.TaskSplitIndex(
+ JobSubmissionFiles.getJobSplitFile(jobSubmitDir).toString(),
+ splitMetaInfo.getStartOffset());
+ allSplitMetaInfo[i] = new JobSplit.TaskSplitMetaInfo(splitIndex,
+ splitMetaInfo.getLocations(),
+ splitMetaInfo.getInputDataLength());
+ }
+ in.close();
+ return allSplitMetaInfo;
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java b/src/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
index 4a0beb9..a785241 100644
--- a/src/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
+++ b/src/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
@@ -28,6 +28,8 @@
import java.util.List;
import java.util.Set;
+import javax.crypto.SecretKey;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.IOUtils;
@@ -41,15 +43,11 @@
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.security.JobTokens;
import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
import org.apache.hadoop.mapreduce.task.reduce.MapOutput.Type;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.commons.codec.binary.Base64;
-import java.security.GeneralSecurityException;
-
class Fetcher<K,V> extends Thread {
private static final Log LOG = LogFactory.getLog(Fetcher.class);
@@ -88,12 +86,12 @@
// Decompression of map-outputs
private final CompressionCodec codec;
private final Decompressor decompressor;
- private final byte[] shuffleJobToken;
+ private final SecretKey jobTokenSecret;
public Fetcher(JobConf job, TaskAttemptID reduceId,
ShuffleScheduler<K,V> scheduler, MergeManager<K,V> merger,
Reporter reporter, ShuffleClientMetrics metrics,
- ExceptionReporter exceptionReporter, byte [] shuffleJobToken) {
+ ExceptionReporter exceptionReporter, SecretKey jobTokenSecret) {
this.reporter = reporter;
this.scheduler = scheduler;
this.merger = merger;
@@ -101,7 +99,7 @@
this.exceptionReporter = exceptionReporter;
this.id = ++nextId;
this.reduce = reduceId.getTaskID().getId();
- this.shuffleJobToken = shuffleJobToken;
+ this.jobTokenSecret = jobTokenSecret;
ioErrs = reporter.getCounter(SHUFFLE_ERR_GRP_NAME,
ShuffleErrors.IO_ERROR.toString());
wrongLengthErrs = reporter.getCounter(SHUFFLE_ERR_GRP_NAME,
@@ -196,9 +194,8 @@
URLConnection connection = url.openConnection();
// generate hash of the url
- SecureShuffleUtils ssutil = new SecureShuffleUtils(shuffleJobToken);
String msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
- String encHash = ssutil.hashFromString(msgToEncode);
+ String encHash = SecureShuffleUtils.hashFromString(msgToEncode, jobTokenSecret);
// put url hash into http header
connection.addRequestProperty(
@@ -215,7 +212,7 @@
}
LOG.debug("url="+msgToEncode+";encHash="+encHash+";replyHash="+replyHash);
// verify that replyHash is HMac of encHash
- ssutil.verifyReply(replyHash, encHash);
+ SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecret);
LOG.info("for url="+msgToEncode+" sent hash and receievd reply");
} catch (IOException ie) {
ioErrs.increment(1);
diff --git a/src/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java b/src/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java
index 7480fad..c2e2616 100644
--- a/src/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java
+++ b/src/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java
@@ -107,7 +107,7 @@
for (int i=0; i < numFetchers; ++i) {
fetchers[i] = new Fetcher<K,V>(jobConf, reduceId, scheduler, merger,
reporter, metrics, this,
- reduceTask.getJobTokens().getShuffleJobToken());
+ reduceTask.getJobTokenSecret());
fetchers[i].start();
}
diff --git a/src/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleScheduler.java b/src/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleScheduler.java
index 560b80e..0f6d52d 100644
--- a/src/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleScheduler.java
+++ b/src/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleScheduler.java
@@ -37,6 +37,7 @@
import org.apache.hadoop.mapred.Counters;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.TaskStatus;
+import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.TaskID;
import org.apache.hadoop.mapreduce.task.reduce.MapHost.State;
@@ -82,10 +83,12 @@
private int maxMapRuntime = 0;
private int maxFailedUniqueFetches = 5;
+ private int maxFetchFailuresBeforeReporting;
private long totalBytesShuffledTillNow = 0;
private DecimalFormat mbpsFormat = new DecimalFormat("0.00");
+ private boolean reportReadErrorImmediately = true;
public ShuffleScheduler(JobConf job, TaskStatus status,
ExceptionReporter reporter,
@@ -108,6 +111,10 @@
referee.start();
this.maxFailedUniqueFetches = Math.min(totalMaps,
this.maxFailedUniqueFetches);
+ this.maxFetchFailuresBeforeReporting = job.getInt(
+ JobContext.SHUFFLE_FETCH_FAILURES, REPORT_FAILURE_LIMIT);
+ this.reportReadErrorImmediately = job.getBoolean(
+ JobContext.SHUFFLE_NOTIFY_READERROR, true);
}
public synchronized void copySucceeded(TaskAttemptID mapId,
@@ -175,7 +182,6 @@
}
}
- // Notify the JobTracker after every 'reportFailureLimit' failures
checkAndInformJobTracker(failures, mapId, readError);
checkReducerHealth();
@@ -188,9 +194,14 @@
failedShuffleCounter.increment(1);
}
+ // Notify the JobTracker
+ // after every read error, if 'reportReadErrorImmediately' is true or
+ // after every 'maxFetchFailuresBeforeReporting' failures
private void checkAndInformJobTracker(
int failures, TaskAttemptID mapId, boolean readError) {
- if (readError || ((failures % REPORT_FAILURE_LIMIT) == 0)) {
+ if ((reportReadErrorImmediately && readError)
+ || ((failures % maxFetchFailuresBeforeReporting) == 0)) {
+ LOG.info("Reporting fetch failure for " + mapId + " to jobtracker.");
status.addFetchFailedMap((org.apache.hadoop.mapred.TaskAttemptID) mapId);
}
}
diff --git a/src/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java b/src/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java
index d600eb3..361ec84 100644
--- a/src/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java
+++ b/src/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java
@@ -144,7 +144,9 @@
Configuration.addDeprecation("mapred.tasktracker.map.tasks.maximum",
new String[] {TTConfig.TT_MAP_SLOTS});
Configuration.addDeprecation("mapred.tasktracker.memory_calculator_plugin",
- new String[] {TTConfig.TT_MEMORY_CALCULATOR_PLUGIN});
+ new String[] {TTConfig.TT_RESOURCE_CALCULATOR_PLUGIN});
+ Configuration.addDeprecation("mapred.tasktracker.memorycalculatorplugin",
+ new String[] {TTConfig.TT_RESOURCE_CALCULATOR_PLUGIN});
Configuration.addDeprecation("mapred.tasktracker.reduce.tasks.maximum",
new String[] {TTConfig.TT_REDUCE_SLOTS});
Configuration.addDeprecation(
@@ -185,8 +187,6 @@
new String[] {JobContext.QUEUE_NAME});
Configuration.addDeprecation("mapred.job.reuse.jvm.num.tasks",
new String[] {JobContext.JVM_NUMTASKS_TORUN});
- Configuration.addDeprecation("mapred.job.split.file",
- new String[] {JobContext.SPLIT_FILE});
Configuration.addDeprecation("mapred.map.tasks",
new String[] {JobContext.NUM_MAPS});
Configuration.addDeprecation("mapred.max.tracker.failures",
@@ -324,7 +324,7 @@
Configuration.addDeprecation("mapred.skip.map.max.skip.records",
new String[] {JobContext.MAP_SKIP_MAX_RECORDS});
Configuration.addDeprecation("min.num.spills.for.combine",
- new String[] {JobContext.MAP_COMBINE_MIN_SPISS});
+ new String[] {JobContext.MAP_COMBINE_MIN_SPILLS});
Configuration.addDeprecation("mapred.compress.map.output",
new String[] {JobContext.MAP_OUTPUT_COMPRESS});
Configuration.addDeprecation("mapred.map.output.compression.codec",
diff --git a/src/java/org/apache/hadoop/mapreduce/util/LinuxMemoryCalculatorPlugin.java b/src/java/org/apache/hadoop/mapreduce/util/LinuxMemoryCalculatorPlugin.java
index 8da9487..69c4b0a 100644
--- a/src/java/org/apache/hadoop/mapreduce/util/LinuxMemoryCalculatorPlugin.java
+++ b/src/java/org/apache/hadoop/mapreduce/util/LinuxMemoryCalculatorPlugin.java
@@ -18,115 +18,29 @@
package org.apache.hadoop.mapreduce.util;
-import java.io.BufferedReader;
-import java.io.FileNotFoundException;
-import java.io.FileReader;
-import java.io.IOException;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
/**
* Plugin to calculate virtual and physical memories on Linux systems.
+ * @deprecated
+ * Use {@link org.apache.hadoop.mapreduce.util.LinuxResourceCalculatorPlugin}
+ * instead
*/
+@Deprecated
public class LinuxMemoryCalculatorPlugin extends MemoryCalculatorPlugin {
- private static final Log LOG =
- LogFactory.getLog(LinuxMemoryCalculatorPlugin.class);
-
- /**
- * proc's meminfo virtual file has keys-values in the format
- * "key:[ \t]*value[ \t]kB".
- */
- private static final String PROCFS_MEMFILE = "/proc/meminfo";
- private static final Pattern PROCFS_MEMFILE_FORMAT =
- Pattern.compile("^([a-zA-Z]*):[ \t]*([0-9]*)[ \t]kB");
-
- // We just need the values for the keys MemTotal and SwapTotal
- private static final String MEMTOTAL_STRING = "MemTotal";
- private static final String SWAPTOTAL_STRING = "SwapTotal";
-
- private long ramSize = 0;
- private long swapSize = 0;
-
- boolean readMemInfoFile = false;
-
- private void readProcMemInfoFile() {
-
- if (readMemInfoFile) {
- return;
- }
-
- // Read "/proc/memInfo" file
- BufferedReader in = null;
- FileReader fReader = null;
- try {
- fReader = new FileReader(PROCFS_MEMFILE);
- in = new BufferedReader(fReader);
- } catch (FileNotFoundException f) {
- // shouldn't happen....
- return;
- }
-
- Matcher mat = null;
-
- try {
- String str = in.readLine();
- while (str != null) {
- mat = PROCFS_MEMFILE_FORMAT.matcher(str);
- if (mat.find()) {
- if (mat.group(1).equals(MEMTOTAL_STRING)) {
- ramSize = Long.parseLong(mat.group(2));
- } else if (mat.group(1).equals(SWAPTOTAL_STRING)) {
- swapSize = Long.parseLong(mat.group(2));
- }
- }
- str = in.readLine();
- }
- } catch (IOException io) {
- LOG.warn("Error reading the stream " + io);
- } finally {
- // Close the streams
- try {
- fReader.close();
- try {
- in.close();
- } catch (IOException i) {
- LOG.warn("Error closing the stream " + in);
- }
- } catch (IOException i) {
- LOG.warn("Error closing the stream " + fReader);
- }
- }
-
- readMemInfoFile = true;
+ private LinuxResourceCalculatorPlugin resourceCalculatorPlugin;
+ // Use everything from LinuxResourceCalculatorPlugin
+ public LinuxMemoryCalculatorPlugin() {
+ resourceCalculatorPlugin = new LinuxResourceCalculatorPlugin();
}
-
+
/** {@inheritDoc} */
@Override
public long getPhysicalMemorySize() {
- readProcMemInfoFile();
- return ramSize * 1024;
+ return resourceCalculatorPlugin.getPhysicalMemorySize();
}
-
+
/** {@inheritDoc} */
@Override
public long getVirtualMemorySize() {
- readProcMemInfoFile();
- return (ramSize + swapSize) * 1024;
- }
-
- /**
- * Test the {@link LinuxMemoryCalculatorPlugin}
- *
- * @param args
- */
- public static void main(String[] args) {
- LinuxMemoryCalculatorPlugin plugin = new LinuxMemoryCalculatorPlugin();
- System.out.println("Physical memory Size(bytes) : "
- + plugin.getPhysicalMemorySize());
- System.out.println("Total Virtual memory Size(bytes) : "
- + plugin.getVirtualMemorySize());
+ return resourceCalculatorPlugin.getVirtualMemorySize();
}
}
\ No newline at end of file
diff --git a/src/java/org/apache/hadoop/mapreduce/util/LinuxResourceCalculatorPlugin.java b/src/java/org/apache/hadoop/mapreduce/util/LinuxResourceCalculatorPlugin.java
new file mode 100644
index 0000000..0b7d771
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/util/LinuxResourceCalculatorPlugin.java
@@ -0,0 +1,394 @@
+/**
+ * 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.mapreduce.util;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.TaskTrackerStatus;
+
+/**
+ * Plugin to calculate resource information on Linux systems.
+ */
+public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
+ private static final Log LOG =
+ LogFactory.getLog(LinuxResourceCalculatorPlugin.class);
+
+ /**
+ * proc's meminfo virtual file has keys-values in the format
+ * "key:[ \t]*value[ \t]kB".
+ */
+ private static final String PROCFS_MEMFILE = "/proc/meminfo";
+ private static final Pattern PROCFS_MEMFILE_FORMAT =
+ Pattern.compile("^([a-zA-Z]*):[ \t]*([0-9]*)[ \t]kB");
+
+ // We need the values for the following keys in meminfo
+ private static final String MEMTOTAL_STRING = "MemTotal";
+ private static final String SWAPTOTAL_STRING = "SwapTotal";
+ private static final String MEMFREE_STRING = "MemFree";
+ private static final String SWAPFREE_STRING = "SwapFree";
+ private static final String INACTIVE_STRING = "Inactive";
+
+ /**
+ * Patterns for parsing /proc/cpuinfo
+ */
+ private static final String PROCFS_CPUINFO = "/proc/cpuinfo";
+ private static final Pattern PROCESSOR_FORMAT =
+ Pattern.compile("^processor[ \t]:[ \t]*([0-9]*)");
+ private static final Pattern FREQUENCY_FORMAT =
+ Pattern.compile("^cpu MHz[ \t]*:[ \t]*([0-9.]*)");
+
+ /**
+ * Pattern for parsing /proc/stat
+ */
+ private static final String PROCFS_STAT = "/proc/stat";
+ private static final Pattern CPU_TIME_FORMAT =
+ Pattern.compile("^cpu[ \t]*([0-9]*)" +
+ "[ \t]*([0-9]*)[ \t]*([0-9]*)[ \t].*");
+
+ private String procfsMemFile;
+ private String procfsCpuFile;
+ private String procfsStatFile;
+ long jiffyLengthInMillis;
+
+ private long ramSize = 0;
+ private long swapSize = 0;
+ private long ramSizeFree = 0; // free ram space on the machine (kB)
+ private long swapSizeFree = 0; // free swap space on the machine (kB)
+ private long inactiveSize = 0; // inactive cache memory (kB)
+ private int numProcessors = 0; // number of processors on the system
+ private long cpuFrequency = 0L; // CPU frequency on the system (kHz)
+ private long cumulativeCpuTime = 0L; // CPU used time since system is on (ms)
+ private long lastCumulativeCpuTime = 0L; // CPU used time read last time (ms)
+ // Unix timestamp while reading the CPU time (ms)
+ private float cpuUsage = TaskTrackerStatus.UNAVAILABLE;
+ private long sampleTime = TaskTrackerStatus.UNAVAILABLE;
+ private long lastSampleTime = TaskTrackerStatus.UNAVAILABLE;
+
+ boolean readMemInfoFile = false;
+ boolean readCpuInfoFile = false;
+
+ /**
+ * Get current time
+ * @return Unix time stamp in millisecond
+ */
+ long getCurrentTime() {
+ return System.currentTimeMillis();
+ }
+
+ public LinuxResourceCalculatorPlugin() {
+ procfsMemFile = PROCFS_MEMFILE;
+ procfsCpuFile = PROCFS_CPUINFO;
+ procfsStatFile = PROCFS_STAT;
+ jiffyLengthInMillis = ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS;
+
+ }
+
+ /**
+ * Constructor which allows assigning the /proc/ directories. This will be
+ * used only in unit tests
+ * @param procfsMemFile fake file for /proc/meminfo
+ * @param procfsCpuFile fake file for /proc/cpuinfo
+ * @param procfsStatFile fake file for /proc/stat
+ * @param jiffyLengthInMillis fake jiffy length value
+ */
+ public LinuxResourceCalculatorPlugin(String procfsMemFile,
+ String procfsCpuFile,
+ String procfsStatFile,
+ long jiffyLengthInMillis) {
+ this.procfsMemFile = procfsMemFile;
+ this.procfsCpuFile = procfsCpuFile;
+ this.procfsStatFile = procfsStatFile;
+ this.jiffyLengthInMillis = jiffyLengthInMillis;
+ }
+
+ /**
+ * Read /proc/meminfo, parse and compute memory information only once
+ */
+ private void readProcMemInfoFile() {
+ readProcMemInfoFile(false);
+ }
+
+ /**
+ * Read /proc/meminfo, parse and compute memory information
+ * @param readAgain if false, read only on the first time
+ */
+ private void readProcMemInfoFile(boolean readAgain) {
+
+ if (readMemInfoFile && !readAgain) {
+ return;
+ }
+
+ // Read "/proc/memInfo" file
+ BufferedReader in = null;
+ FileReader fReader = null;
+ try {
+ fReader = new FileReader(procfsMemFile);
+ in = new BufferedReader(fReader);
+ } catch (FileNotFoundException f) {
+ // shouldn't happen....
+ return;
+ }
+
+ Matcher mat = null;
+
+ try {
+ String str = in.readLine();
+ while (str != null) {
+ mat = PROCFS_MEMFILE_FORMAT.matcher(str);
+ if (mat.find()) {
+ if (mat.group(1).equals(MEMTOTAL_STRING)) {
+ ramSize = Long.parseLong(mat.group(2));
+ } else if (mat.group(1).equals(SWAPTOTAL_STRING)) {
+ swapSize = Long.parseLong(mat.group(2));
+ } else if (mat.group(1).equals(MEMFREE_STRING)) {
+ ramSizeFree = Long.parseLong(mat.group(2));
+ } else if (mat.group(1).equals(SWAPFREE_STRING)) {
+ swapSizeFree = Long.parseLong(mat.group(2));
+ } else if (mat.group(1).equals(INACTIVE_STRING)) {
+ inactiveSize = Long.parseLong(mat.group(2));
+ }
+ }
+ str = in.readLine();
+ }
+ } catch (IOException io) {
+ LOG.warn("Error reading the stream " + io);
+ } finally {
+ // Close the streams
+ try {
+ fReader.close();
+ try {
+ in.close();
+ } catch (IOException i) {
+ LOG.warn("Error closing the stream " + in);
+ }
+ } catch (IOException i) {
+ LOG.warn("Error closing the stream " + fReader);
+ }
+ }
+
+ readMemInfoFile = true;
+ }
+
+ /**
+ * Read /proc/cpuinfo, parse and calculate CPU information
+ */
+ private void readProcCpuInfoFile() {
+ // This directory needs to be read only once
+ if (readCpuInfoFile) {
+ return;
+ }
+ // Read "/proc/cpuinfo" file
+ BufferedReader in = null;
+ FileReader fReader = null;
+ try {
+ fReader = new FileReader(procfsCpuFile);
+ in = new BufferedReader(fReader);
+ } catch (FileNotFoundException f) {
+ // shouldn't happen....
+ return;
+ }
+ Matcher mat = null;
+ try {
+ numProcessors = 0;
+ String str = in.readLine();
+ while (str != null) {
+ mat = PROCESSOR_FORMAT.matcher(str);
+ if (mat.find()) {
+ numProcessors++;
+ }
+ mat = FREQUENCY_FORMAT.matcher(str);
+ if (mat.find()) {
+ cpuFrequency = (long)(Double.parseDouble(mat.group(1)) * 1000); // kHz
+ }
+ str = in.readLine();
+ }
+ } catch (IOException io) {
+ LOG.warn("Error reading the stream " + io);
+ } finally {
+ // Close the streams
+ try {
+ fReader.close();
+ try {
+ in.close();
+ } catch (IOException i) {
+ LOG.warn("Error closing the stream " + in);
+ }
+ } catch (IOException i) {
+ LOG.warn("Error closing the stream " + fReader);
+ }
+ }
+ readCpuInfoFile = true;
+ }
+
+ /**
+ * Read /proc/stat file, parse and calculate cumulative CPU
+ */
+ private void readProcStatFile() {
+ // Read "/proc/stat" file
+ BufferedReader in = null;
+ FileReader fReader = null;
+ try {
+ fReader = new FileReader(procfsStatFile);
+ in = new BufferedReader(fReader);
+ } catch (FileNotFoundException f) {
+ // shouldn't happen....
+ return;
+ }
+
+ Matcher mat = null;
+ try {
+ String str = in.readLine();
+ while (str != null) {
+ mat = CPU_TIME_FORMAT.matcher(str);
+ if (mat.find()) {
+ long uTime = Long.parseLong(mat.group(1));
+ long nTime = Long.parseLong(mat.group(2));
+ long sTime = Long.parseLong(mat.group(3));
+ cumulativeCpuTime = uTime + nTime + sTime; // milliseconds
+ break;
+ }
+ str = in.readLine();
+ }
+ cumulativeCpuTime *= jiffyLengthInMillis;
+ } catch (IOException io) {
+ LOG.warn("Error reading the stream " + io);
+ } finally {
+ // Close the streams
+ try {
+ fReader.close();
+ try {
+ in.close();
+ } catch (IOException i) {
+ LOG.warn("Error closing the stream " + in);
+ }
+ } catch (IOException i) {
+ LOG.warn("Error closing the stream " + fReader);
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getPhysicalMemorySize() {
+ readProcMemInfoFile();
+ return ramSize * 1024;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getVirtualMemorySize() {
+ readProcMemInfoFile();
+ return (ramSize + swapSize) * 1024;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getAvailablePhysicalMemorySize() {
+ readProcMemInfoFile(true);
+ return (ramSizeFree + inactiveSize) * 1024;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getAvailableVirtualMemorySize() {
+ readProcMemInfoFile(true);
+ return (ramSizeFree + swapSizeFree + inactiveSize) * 1024;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public int getNumProcessors() {
+ readProcCpuInfoFile();
+ return numProcessors;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getCpuFrequency() {
+ readProcCpuInfoFile();
+ return cpuFrequency;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getCumulativeCpuTime() {
+ readProcStatFile();
+ return cumulativeCpuTime;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public float getCpuUsage() {
+ readProcStatFile();
+ sampleTime = getCurrentTime();
+ if (lastSampleTime == TaskTrackerStatus.UNAVAILABLE ||
+ lastSampleTime > sampleTime) {
+ // lastSampleTime > sampleTime may happen when the system time is changed
+ lastSampleTime = sampleTime;
+ lastCumulativeCpuTime = cumulativeCpuTime;
+ return cpuUsage;
+ }
+ // When lastSampleTime is sufficiently old, update cpuUsage.
+ // Also take a sample of the current time and cumulative CPU time for the
+ // use of the next calculation.
+ final long MINIMUM_UPDATE_INTERVAL = 10 * jiffyLengthInMillis;
+ if (sampleTime > lastSampleTime + MINIMUM_UPDATE_INTERVAL) {
+ cpuUsage = (float)(cumulativeCpuTime - lastCumulativeCpuTime) * 100F /
+ ((float)(sampleTime - lastSampleTime) * getNumProcessors());
+ lastSampleTime = sampleTime;
+ lastCumulativeCpuTime = cumulativeCpuTime;
+ }
+ return cpuUsage;
+ }
+
+ /**
+ * Test the {@link LinuxResourceCalculatorPlugin}
+ *
+ * @param args
+ */
+ public static void main(String[] args) {
+ LinuxResourceCalculatorPlugin plugin = new LinuxResourceCalculatorPlugin();
+ System.out.println("Physical memory Size (bytes) : "
+ + plugin.getPhysicalMemorySize());
+ System.out.println("Total Virtual memory Size (bytes) : "
+ + plugin.getVirtualMemorySize());
+ System.out.println("Available Physical memory Size (bytes) : "
+ + plugin.getAvailablePhysicalMemorySize());
+ System.out.println("Total Available Virtual memory Size (bytes) : "
+ + plugin.getAvailableVirtualMemorySize());
+ System.out.println("Number of Processors : " + plugin.getNumProcessors());
+ System.out.println("CPU frequency (kHz) : " + plugin.getCpuFrequency());
+ System.out.println("Cumulative CPU time (ms) : " +
+ plugin.getCumulativeCpuTime());
+ try {
+ // Sleep so we can compute the CPU usage
+ Thread.sleep(500L);
+ } catch (InterruptedException e) {
+ // do nothing
+ }
+ System.out.println("CPU usage % : " + plugin.getCpuUsage());
+ }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java b/src/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java
new file mode 100644
index 0000000..85e633c
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java
@@ -0,0 +1,367 @@
+/**
+ * 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.mapreduce.util;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.util.AsyncDiskService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * This class is a container of multiple thread pools, each for a volume,
+ * so that we can schedule async disk operations easily.
+ *
+ * Examples of async disk operations are deletion of files.
+ * We can move the files to a "toBeDeleted" folder before asychronously
+ * deleting it, to make sure the caller can run it faster.
+ *
+ * Users should not write files into the "toBeDeleted" folder, otherwise
+ * the files can be gone any time we restart the MRAsyncDiskService.
+ *
+ * This class also contains all operations that will be performed by the
+ * thread pools.
+ */
+@InterfaceAudience.Private
+public class MRAsyncDiskService {
+
+ public static final Log LOG = LogFactory.getLog(MRAsyncDiskService.class);
+
+ AsyncDiskService asyncDiskService;
+
+ public static final String TOBEDELETED = "toBeDeleted";
+
+ /**
+ * Create a AsyncDiskServices with a set of volumes (specified by their
+ * root directories).
+ *
+ * The AsyncDiskServices uses one ThreadPool per volume to do the async
+ * disk operations.
+ *
+ * @param localFileSystem The localFileSystem used for deletions.
+ * @param volumes The roots of the file system volumes.
+ */
+ public MRAsyncDiskService(FileSystem localFileSystem, String[] volumes)
+ throws IOException {
+
+ this.volumes = new String[volumes.length];
+ for (int v = 0; v < volumes.length; v++) {
+ this.volumes[v] = normalizePath(volumes[v]);
+ }
+ this.localFileSystem = localFileSystem;
+
+ asyncDiskService = new AsyncDiskService(this.volumes);
+
+ // Create one ThreadPool per volume
+ for (int v = 0 ; v < volumes.length; v++) {
+ // Create the root for file deletion
+ Path absoluteSubdir = new Path(volumes[v], TOBEDELETED);
+ if (!localFileSystem.mkdirs(absoluteSubdir)) {
+ throw new IOException("Cannot create " + TOBEDELETED + " in " + volumes[v]);
+ }
+ }
+
+ // Create tasks to delete the paths inside the volumes
+ for (int v = 0 ; v < volumes.length; v++) {
+ Path absoluteSubdir = new Path(volumes[v], TOBEDELETED);
+ // List all files inside the volumes
+ FileStatus[] files = localFileSystem.listStatus(absoluteSubdir);
+ for (int f = 0; f < files.length; f++) {
+ // Get the relative file name to the root of the volume
+ String absoluteFilename = files[f].getPath().toUri().getPath();
+ String relative = getRelativePathName(absoluteFilename, volumes[v]);
+ if (relative == null) {
+ // This should never happen
+ throw new IOException("Cannot delete " + absoluteFilename
+ + " because it's outside of " + volumes[v]);
+ }
+ DeleteTask task = new DeleteTask(volumes[v], absoluteFilename,
+ relative);
+ execute(volumes[v], task);
+ }
+ }
+ }
+
+ /**
+ * Initialize MRAsyncDiskService based on conf.
+ * @param conf local file system and local dirs will be read from conf
+ */
+ public MRAsyncDiskService(JobConf conf) throws IOException {
+ this(FileSystem.getLocal(conf), conf.getLocalDirs());
+ }
+
+ /**
+ * Execute the task sometime in the future, using ThreadPools.
+ */
+ synchronized void execute(String root, Runnable task) {
+ asyncDiskService.execute(root, task);
+ }
+
+ /**
+ * Gracefully start the shut down of all ThreadPools.
+ */
+ public synchronized void shutdown() {
+ asyncDiskService.shutdown();
+ }
+
+ /**
+ * Shut down all ThreadPools immediately.
+ */
+ public synchronized List<Runnable> shutdownNow() {
+ return asyncDiskService.shutdownNow();
+ }
+
+ /**
+ * Wait for the termination of the thread pools.
+ *
+ * @param milliseconds The number of milliseconds to wait
+ * @return true if all thread pools are terminated within time limit
+ * @throws InterruptedException
+ */
+ public synchronized boolean awaitTermination(long milliseconds)
+ throws InterruptedException {
+ return asyncDiskService.awaitTermination(milliseconds);
+ }
+
+ private SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss.SSS");
+
+ private FileSystem localFileSystem;
+
+ private String[] volumes;
+
+ private static AtomicLong uniqueId = new AtomicLong(0);
+
+ /** A task for deleting a pathName from a volume.
+ */
+ class DeleteTask implements Runnable {
+
+ /** The volume that the file is on*/
+ String volume;
+ /** The file name before the move */
+ String originalPath;
+ /** The file name after the move */
+ String pathToBeDeleted;
+
+ /**
+ * Delete a file/directory (recursively if needed).
+ * @param volume The volume that the file/dir is in.
+ * @param originalPath The original name, relative to volume root.
+ * @param pathToBeDeleted The name after the move, relative to volume root,
+ * containing TOBEDELETED.
+ */
+ DeleteTask(String volume, String originalPath, String pathToBeDeleted) {
+ this.volume = volume;
+ this.originalPath = originalPath;
+ this.pathToBeDeleted = pathToBeDeleted;
+ }
+
+ @Override
+ public String toString() {
+ // Called in AsyncDiskService.execute for displaying error messages.
+ return "deletion of " + pathToBeDeleted + " on " + volume
+ + " with original name " + originalPath;
+ }
+
+ @Override
+ public void run() {
+ boolean success = false;
+ Exception e = null;
+ try {
+ Path absolutePathToBeDeleted = new Path(volume, pathToBeDeleted);
+ success = localFileSystem.delete(absolutePathToBeDeleted, true);
+ } catch (Exception ex) {
+ e = ex;
+ }
+
+ if (!success) {
+ if (e != null) {
+ LOG.warn("Failure in " + this + " with exception "
+ + StringUtils.stringifyException(e));
+ } else {
+ LOG.warn("Failure in " + this);
+ }
+ } else {
+ LOG.debug("Successfully did " + this.toString());
+ }
+ }
+ };
+
+
+ /**
+ * Move the path name on one volume to a temporary location and then
+ * delete them.
+ *
+ * This functions returns when the moves are done, but not necessarily all
+ * deletions are done. This is usually good enough because applications
+ * won't see the path name under the old name anyway after the move.
+ *
+ * @param volume The disk volume
+ * @param pathName The path name relative to volume root.
+ * @throws IOException If the move failed
+ * @return false if the file is not found
+ */
+ public boolean moveAndDeleteRelativePath(String volume, String pathName)
+ throws IOException {
+
+ volume = normalizePath(volume);
+
+ // Move the file right now, so that it can be deleted later
+ String newPathName =
+ format.format(new Date()) + "_" + uniqueId.getAndIncrement();
+ newPathName = TOBEDELETED + Path.SEPARATOR_CHAR + newPathName;
+
+ Path source = new Path(volume, pathName);
+ Path target = new Path(volume, newPathName);
+ try {
+ if (!localFileSystem.rename(source, target)) {
+ // Try to recreate the parent directory just in case it gets deleted.
+ if (!localFileSystem.mkdirs(new Path(volume, TOBEDELETED))) {
+ throw new IOException("Cannot create " + TOBEDELETED + " under "
+ + volume);
+ }
+ // Try rename again. If it fails, return false.
+ if (!localFileSystem.rename(source, target)) {
+ throw new IOException("Cannot rename " + source + " to "
+ + target);
+ }
+ }
+ } catch (FileNotFoundException e) {
+ // Return false in case that the file is not found.
+ return false;
+ }
+
+ DeleteTask task = new DeleteTask(volume, pathName, newPathName);
+ execute(volume, task);
+ return true;
+ }
+
+ /**
+ * Move the path name on each volume to a temporary location and then
+ * delete them.
+ *
+ * This functions returns when the moves are done, but not necessarily all
+ * deletions are done. This is usually good enough because applications
+ * won't see the path name under the old name anyway after the move.
+ *
+ * @param pathName The path name relative to each volume root
+ * @throws IOException If any of the move failed
+ * @return false If any of the target pathName did not exist,
+ * note that the operation is still done on all volumes.
+ */
+ public boolean moveAndDeleteFromEachVolume(String pathName) throws IOException {
+ boolean result = true;
+ for (int i = 0; i < volumes.length; i++) {
+ result = result && moveAndDeleteRelativePath(volumes[i], pathName);
+ }
+ return result;
+ }
+
+ /**
+ * Move all files/directories inside volume into TOBEDELETED, and then
+ * delete them. The TOBEDELETED directory itself is ignored.
+ */
+ public void cleanupAllVolumes() throws IOException {
+ for (int v = 0; v < volumes.length; v++) {
+ // List all files inside the volumes
+ FileStatus[] files = localFileSystem.listStatus(new Path(volumes[v]));
+ for (int f = 0; f < files.length; f++) {
+ // Get the relative file name to the root of the volume
+ String absoluteFilename = files[f].getPath().toUri().getPath();
+ String relative = getRelativePathName(absoluteFilename, volumes[v]);
+ if (relative == null) {
+ // This should never happen
+ throw new IOException("Cannot delete " + absoluteFilename
+ + " because it's outside of " + volumes[v]);
+ }
+ // Do not delete the current TOBEDELETED
+ if (!TOBEDELETED.equals(relative)) {
+ moveAndDeleteRelativePath(volumes[v], relative);
+ }
+ }
+ }
+ }
+
+ /**
+ * Returns the normalized path of a path.
+ */
+ private static String normalizePath(String path) {
+ return (new Path(path)).toUri().getPath();
+ }
+
+ /**
+ * Get the relative path name with respect to the root of the volume.
+ * @param absolutePathName The absolute path name
+ * @param volume Root of the volume.
+ * @return null if the absolute path name is outside of the volume.
+ */
+ private static String getRelativePathName(String absolutePathName,
+ String volume) {
+
+ absolutePathName = normalizePath(absolutePathName);
+ // Get the file names
+ if (!absolutePathName.startsWith(volume)) {
+ return null;
+ }
+ // Get rid of the volume prefix
+ String fileName = absolutePathName.substring(volume.length());
+ if (fileName.charAt(0) == Path.SEPARATOR_CHAR) {
+ fileName = fileName.substring(1);
+ }
+ return fileName;
+ }
+
+ /**
+ * Move the path name to a temporary location and then delete it.
+ *
+ * Note that if there is no volume that contains this path, the path
+ * will stay as it is, and the function will return false.
+ *
+ * This functions returns when the moves are done, but not necessarily all
+ * deletions are done. This is usually good enough because applications
+ * won't see the path name under the old name anyway after the move.
+ *
+ * @param absolutePathName The path name from root "/"
+ * @throws IOException If the move failed
+ * @return false if we are unable to move the path name
+ */
+ public boolean moveAndDeleteAbsolutePath(String absolutePathName)
+ throws IOException {
+
+ for (int v = 0; v < volumes.length; v++) {
+ String relative = getRelativePathName(absolutePathName, volumes[v]);
+ if (relative != null) {
+ return moveAndDeleteRelativePath(volumes[v], relative);
+ }
+ }
+
+ throw new IOException("Cannot delete " + absolutePathName
+ + " because it's outside of all volumes.");
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/mapreduce/util/MemoryCalculatorPlugin.java b/src/java/org/apache/hadoop/mapreduce/util/MemoryCalculatorPlugin.java
index 200059f..8f2ce68 100644
--- a/src/java/org/apache/hadoop/mapreduce/util/MemoryCalculatorPlugin.java
+++ b/src/java/org/apache/hadoop/mapreduce/util/MemoryCalculatorPlugin.java
@@ -24,8 +24,11 @@
/**
* Plugin to calculate virtual and physical memories on the system.
- *
+ * @deprecated Use
+ * {@link org.apache.hadoop.mapreduce.util.ResourceCalculatorPlugin}
+ * instead
*/
+@Deprecated
public abstract class MemoryCalculatorPlugin extends Configured {
/**
diff --git a/src/java/org/apache/hadoop/mapreduce/util/ProcessTree.java b/src/java/org/apache/hadoop/mapreduce/util/ProcessTree.java
index a43fd0a..21e8e42 100644
--- a/src/java/org/apache/hadoop/mapreduce/util/ProcessTree.java
+++ b/src/java/org/apache/hadoop/mapreduce/util/ProcessTree.java
@@ -37,6 +37,15 @@
public static final long DEFAULT_SLEEPTIME_BEFORE_SIGKILL = 5000L;
+ private static final int SIGQUIT = 3;
+ private static final int SIGTERM = 15;
+ private static final int SIGKILL = 9;
+
+ private static final String SIGQUIT_STR = "SIGQUIT";
+ private static final String SIGTERM_STR = "SIGTERM";
+ private static final String SIGKILL_STR = "SIGKILL";
+
+
public static final boolean isSetsidAvailable = isSetsidSupported();
private static boolean isSetsidSupported() {
ShellCommandExecutor shexec = null;
@@ -102,23 +111,68 @@
sigKill(pgrpId, true, sleeptimeBeforeSigkill, inBackground);
}
+
+ /**
+ * Send a specified signal to the specified pid
+ *
+ * @param pid the pid of the process [group] to signal.
+ * @param signalNum the signal to send.
+ * @param signalName the human-readable description of the signal
+ * (for logging).
+ */
+ private static void sendSignal(String pid, int signalNum, String signalName) {
+ ShellCommandExecutor shexec = null;
+ try {
+ String[] args = { "kill", "-" + signalNum, pid };
+ shexec = new ShellCommandExecutor(args);
+ shexec.execute();
+ } catch (IOException ioe) {
+ LOG.warn("Error executing shell command " + ioe);
+ } finally {
+ if (pid.startsWith("-")) {
+ LOG.info("Sending signal to all members of process group " + pid
+ + ": " + signalName + ". Exit code " + shexec.getExitCode());
+ } else {
+ LOG.info("Signaling process " + pid
+ + " with " + signalName + ". Exit code " + shexec.getExitCode());
+ }
+ }
+ }
+
+ /**
+ * Send a specified signal to the process, if it is alive.
+ *
+ * @param pid the pid of the process to signal.
+ * @param signalNum the signal to send.
+ * @param signalName the human-readable description of the signal
+ * (for logging).
+ * @param alwaysSignal if true then send signal even if isAlive(pid) is false
+ */
+ private static void maybeSignalProcess(String pid, int signalNum,
+ String signalName, boolean alwaysSignal) {
+ // If process tree is not alive then don't signal, unless alwaysSignal
+ // forces it so.
+ if (alwaysSignal || ProcessTree.isAlive(pid)) {
+ sendSignal(pid, signalNum, signalName);
+ }
+ }
+
+ private static void maybeSignalProcessGroup(String pgrpId, int signalNum,
+ String signalName, boolean alwaysSignal) {
+
+ if (alwaysSignal || ProcessTree.isProcessGroupAlive(pgrpId)) {
+ // signaling a process group means using a negative pid.
+ sendSignal("-" + pgrpId, signalNum, signalName);
+ }
+ }
+
/**
* Sends terminate signal to the process, allowing it to gracefully exit.
*
* @param pid pid of the process to be sent SIGTERM
*/
public static void terminateProcess(String pid) {
- ShellCommandExecutor shexec = null;
- try {
- String[] args = { "kill", pid };
- shexec = new ShellCommandExecutor(args);
- shexec.execute();
- } catch (IOException ioe) {
- LOG.warn("Error executing shell command " + ioe);
- } finally {
- LOG.info("Killing process " + pid +
- " with SIGTERM. Exit code " + shexec.getExitCode());
- }
+ maybeSignalProcess(pid, SIGTERM, SIGTERM_STR, true);
}
/**
@@ -128,17 +182,7 @@
* @param pgrpId process group id
*/
public static void terminateProcessGroup(String pgrpId) {
- ShellCommandExecutor shexec = null;
- try {
- String[] args = { "kill", "--", "-" + pgrpId };
- shexec = new ShellCommandExecutor(args);
- shexec.execute();
- } catch (IOException ioe) {
- LOG.warn("Error executing shell command " + ioe);
- } finally {
- LOG.info("Killing all processes in the process group " + pgrpId +
- " with SIGTERM. Exit code " + shexec.getExitCode());
- }
+ maybeSignalProcessGroup(pgrpId, SIGTERM, SIGTERM_STR, true);
}
/**
@@ -197,22 +241,17 @@
* @param pid process id
*/
public static void killProcess(String pid) {
+ maybeSignalProcess(pid, SIGKILL, SIGKILL_STR, false);
+ }
- //If process tree is not alive then return immediately.
- if(!ProcessTree.isAlive(pid)) {
- return;
- }
- String[] args = { "kill", "-9", pid };
- ShellCommandExecutor shexec = new ShellCommandExecutor(args);
- try {
- shexec.execute();
- } catch (IOException e) {
- LOG.warn("Error sending SIGKILL to process "+ pid + " ."+
- StringUtils.stringifyException(e));
- } finally {
- LOG.info("Killing process " + pid + " with SIGKILL. Exit code "
- + shexec.getExitCode());
- }
+ /**
+ * Sends SIGQUIT to process; Java programs will dump their stack to
+ * stdout.
+ *
+ * @param pid process id
+ */
+ public static void sigQuitProcess(String pid) {
+ maybeSignalProcess(pid, SIGQUIT, SIGQUIT_STR, false);
}
/**
@@ -222,25 +261,20 @@
* @param pgrpId process group id
*/
public static void killProcessGroup(String pgrpId) {
-
- //If process tree is not alive then return immediately.
- if(!ProcessTree.isProcessGroupAlive(pgrpId)) {
- return;
- }
-
- String[] args = { "kill", "-9", "-"+pgrpId };
- ShellCommandExecutor shexec = new ShellCommandExecutor(args);
- try {
- shexec.execute();
- } catch (IOException e) {
- LOG.warn("Error sending SIGKILL to process group "+ pgrpId + " ."+
- StringUtils.stringifyException(e));
- } finally {
- LOG.info("Killing process group" + pgrpId + " with SIGKILL. Exit code "
- + shexec.getExitCode());
- }
+ maybeSignalProcessGroup(pgrpId, SIGKILL, SIGKILL_STR, false);
}
-
+
+ /**
+ * Sends SIGQUIT to all processes belonging to the same process group,
+ * ordering all processes in the group to send their stack dump to
+ * stdout.
+ *
+ * @param pgrpId process group id
+ */
+ public static void sigQuitProcessGroup(String pgrpId) {
+ maybeSignalProcessGroup(pgrpId, SIGQUIT, SIGQUIT_STR, false);
+ }
+
/**
* Is the process with PID pid still alive?
* This method assumes that isAlive is called on a pid that was alive not
diff --git a/src/java/org/apache/hadoop/mapreduce/util/ProcfsBasedProcessTree.java b/src/java/org/apache/hadoop/mapreduce/util/ProcfsBasedProcessTree.java
index e31e57e..45dda2a 100644
--- a/src/java/org/apache/hadoop/mapreduce/util/ProcfsBasedProcessTree.java
+++ b/src/java/org/apache/hadoop/mapreduce/util/ProcfsBasedProcessTree.java
@@ -46,8 +46,10 @@
private static final String PROCFS = "/proc/";
- private static final Pattern PROCFS_STAT_FILE_FORMAT = Pattern
- .compile("^([0-9-]+)\\s([^\\s]+)\\s[^\\s]\\s([0-9-]+)\\s([0-9-]+)\\s([0-9-]+)\\s([0-9-]+\\s){16}([0-9]+)\\s([0-9]+)(\\s[0-9-]+){15}");
+ private static final Pattern PROCFS_STAT_FILE_FORMAT = Pattern .compile(
+ "^([0-9-]+)\\s([^\\s]+)\\s[^\\s]\\s([0-9-]+)\\s([0-9-]+)\\s([0-9-]+)\\s" +
+ "([0-9-]+\\s){7}([0-9]+)\\s([0-9]+)\\s([0-9-]+\\s){7}([0-9]+)\\s([0-9]+)" +
+ "(\\s[0-9-]+){15}");
static final String PROCFS_STAT_FILE = "stat";
static final String PROCFS_CMDLINE_FILE = "cmdline";
@@ -65,12 +67,28 @@
PAGE_SIZE = pageSize;
}
}
+ public static final long JIFFY_LENGTH_IN_MILLIS; // in millisecond
+ static {
+ ShellCommandExecutor shellExecutor =
+ new ShellCommandExecutor(new String[]{"getconf", "CLK_TCK"});
+ long jiffiesPerSecond = -1;
+ try {
+ shellExecutor.execute();
+ jiffiesPerSecond = Long.parseLong(shellExecutor.getOutput().replace("\n", ""));
+ } catch (IOException e) {
+ LOG.error(StringUtils.stringifyException(e));
+ } finally {
+ JIFFY_LENGTH_IN_MILLIS = jiffiesPerSecond != -1 ?
+ Math.round(1000D / jiffiesPerSecond) : -1;
+ }
+ }
// to enable testing, using this variable which can be configured
// to a test directory.
private String procfsDir;
private Integer pid = -1;
+ private Long cpuTime = 0L;
private boolean setsidUsed = false;
private long sleeptimeBeforeSigkill = DEFAULT_SLEEPTIME_BEFORE_SIGKILL;
@@ -195,11 +213,12 @@
pInfoQueue.addAll(pInfo.getChildren());
}
- // update age values.
+ // update age values and compute the number of jiffies since last update
for (Map.Entry<Integer, ProcessInfo> procs : processTree.entrySet()) {
ProcessInfo oldInfo = oldProcs.get(procs.getKey());
- if (oldInfo != null) {
- if (procs.getValue() != null) {
+ if (procs.getValue() != null) {
+ procs.getValue().updateJiffy(oldInfo);
+ if (oldInfo != null) {
procs.getValue().updateAge(oldInfo);
}
}
@@ -324,7 +343,7 @@
}
private static final String PROCESSTREE_DUMP_FORMAT =
- "\t|- %d %d %d %d %s %d %d %s\n";
+ "\t|- %d %d %d %d %s %d %d %d %d %s\n";
/**
* Get a dump of the process-tree.
@@ -336,12 +355,14 @@
StringBuilder ret = new StringBuilder();
// The header.
ret.append(String.format("\t|- PID PPID PGRPID SESSID CMD_NAME "
- + "VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
+ + "USER_MODE_TIME(MILLIS) SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) "
+ + "RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
for (ProcessInfo p : processTree.values()) {
if (p != null) {
ret.append(String.format(PROCESSTREE_DUMP_FORMAT, p.getPid(), p
.getPpid(), p.getPgrpId(), p.getSessionId(), p.getName(), p
- .getVmem(), p.getRssmemPage(), p.getCmdLine(procfsDir)));
+ .getUtime(), p.getStime(), p.getVmem(), p.getRssmemPage(), p
+ .getCmdLine(procfsDir)));
}
}
return ret.toString();
@@ -412,6 +433,27 @@
return totalPages * PAGE_SIZE; // convert # pages to byte
}
+ /**
+ * Get the CPU time in millisecond used by all the processes in the
+ * process-tree since the process-tree created
+ *
+ * @return cumulative CPU time in millisecond since the process-tree created
+ * return 0 if it cannot be calculated
+ */
+ public long getCumulativeCpuTime() {
+ if (JIFFY_LENGTH_IN_MILLIS < 0) {
+ return 0;
+ }
+ long incJiffies = 0;
+ for (ProcessInfo p : processTree.values()) {
+ if (p != null) {
+ incJiffies += p.dtime;
+ }
+ }
+ cpuTime += incJiffies * JIFFY_LENGTH_IN_MILLIS;
+ return cpuTime;
+ }
+
private static Integer getValidPID(String pid) {
Integer retPid = -1;
try {
@@ -481,12 +523,12 @@
Matcher m = PROCFS_STAT_FILE_FORMAT.matcher(str);
boolean mat = m.find();
if (mat) {
- // Set ( name ) ( ppid ) ( pgrpId ) (session ) (vsize ) (rss)
- pinfo.updateProcessInfo(m.group(2), Integer.parseInt(m.group(3)), Integer
- .parseInt(m.group(4)), Integer.parseInt(m.group(5)), Long
- .parseLong(m.group(7)), Long.parseLong(m.group(8)));
- }
- else {
+ // Set (name) (ppid) (pgrpId) (session) (utime) (stime) (vsize) (rss)
+ pinfo.updateProcessInfo(m.group(2), Integer.parseInt(m.group(3)),
+ Integer.parseInt(m.group(4)), Integer.parseInt(m.group(5)),
+ Long.parseLong(m.group(7)), Long.parseLong(m.group(8)),
+ Long.parseLong(m.group(10)), Long.parseLong(m.group(11)));
+ } else {
LOG.warn("Unexpected: procfs stat file is not in the expected format"
+ " for process with pid " + pinfo.getPid());
ret = null;
@@ -536,8 +578,17 @@
private Integer sessionId; // session-id
private Long vmem; // virtual memory usage
private Long rssmemPage; // rss memory usage in # of pages
+ private Long utime = 0L; // # of jiffies in user mode
+ private Long stime = 0L; // # of jiffies in kernel mode
// how many times has this process been seen alive
private int age;
+
+ // # of jiffies used since last update:
+ private Long dtime = 0L;
+ // dtime = (utime + stime) - (utimeOld + stimeOld)
+ // We need this to compute the cumulative CPU time
+ // because the subprocess may finish earlier than root process
+
private List<ProcessInfo> children = new ArrayList<ProcessInfo>(); // list of children
public ProcessInfo(int pid) {
@@ -570,6 +621,18 @@
return vmem;
}
+ public Long getUtime() {
+ return utime;
+ }
+
+ public Long getStime() {
+ return stime;
+ }
+
+ public Long getDtime() {
+ return dtime;
+ }
+
public Long getRssmemPage() { // get rss # of pages
return rssmemPage;
}
@@ -586,15 +649,22 @@
}
public void updateProcessInfo(String name, Integer ppid, Integer pgrpId,
- Integer sessionId, Long vmem, Long rssmem) {
+ Integer sessionId, Long utime, Long stime, Long vmem, Long rssmem) {
this.name = name;
this.ppid = ppid;
this.pgrpId = pgrpId;
this.sessionId = sessionId;
+ this.utime = utime;
+ this.stime = stime;
this.vmem = vmem;
this.rssmemPage = rssmem;
}
+ public void updateJiffy(ProcessInfo oldInfo) {
+ this.dtime = (oldInfo == null ? this.utime + this.stime
+ : (this.utime + this.stime) - (oldInfo.utime + oldInfo.stime));
+ }
+
public void updateAge(ProcessInfo oldInfo) {
this.age = oldInfo.age + 1;
}
diff --git a/src/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java b/src/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java
new file mode 100644
index 0000000..53d5bcf
--- /dev/null
+++ b/src/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java
@@ -0,0 +1,119 @@
+/**
+ * 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.mapreduce.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Plugin to calculate resource information on the system.
+ *
+ */
+public abstract class ResourceCalculatorPlugin extends Configured {
+
+ /**
+ * Obtain the total size of the virtual memory present in the system.
+ *
+ * @return virtual memory size in bytes.
+ */
+ public abstract long getVirtualMemorySize();
+
+ /**
+ * Obtain the total size of the physical memory present in the system.
+ *
+ * @return physical memory size bytes.
+ */
+ public abstract long getPhysicalMemorySize();
+
+ /**
+ * Obtain the total size of the available virtual memory present
+ * in the system.
+ *
+ * @return available virtual memory size in bytes.
+ */
+ public abstract long getAvailableVirtualMemorySize();
+
+ /**
+ * Obtain the total size of the available physical memory present
+ * in the system.
+ *
+ * @return available physical memory size bytes.
+ */
+ public abstract long getAvailablePhysicalMemorySize();
+
+ /**
+ * Obtain the total number of processors present on the system.
+ *
+ * @return number of processors
+ */
+ public abstract int getNumProcessors();
+
+ /**
+ * Obtain the CPU frequency of on the system.
+ *
+ * @return CPU frequency in kHz
+ */
+ public abstract long getCpuFrequency();
+
+ /**
+ * Obtain the cumulative CPU time since the system is on.
+ *
+ * @return cumulative CPU time in milliseconds
+ */
+ public abstract long getCumulativeCpuTime();
+
+ /**
+ * Obtain the CPU usage % of the machine. Return -1 if it is unavailable
+ *
+ * @return CPU usage in %
+ */
+ public abstract float getCpuUsage();
+
+ /**
+ * Get the ResourceCalculatorPlugin from the class name and configure it. If
+ * class name is null, this method will try and return a memory calculator
+ * plugin available for this system.
+ *
+ * @param clazz class-name
+ * @param conf configure the plugin with this.
+ * @return ResourceCalculatorPlugin
+ */
+ public static ResourceCalculatorPlugin getResourceCalculatorPlugin(
+ Class<? extends ResourceCalculatorPlugin> clazz, Configuration conf) {
+
+ if (clazz != null) {
+ return ReflectionUtils.newInstance(clazz, conf);
+ }
+
+ // No class given, try a os specific class
+ try {
+ String osName = System.getProperty("os.name");
+ if (osName.startsWith("Linux")) {
+ return new LinuxResourceCalculatorPlugin();
+ }
+ } catch (SecurityException se) {
+ // Failed to get Operating System name.
+ return null;
+ }
+
+ // Not supported on this system.
+ return null;
+ }
+}
\ No newline at end of file
diff --git a/src/test/aop/build/aop.xml b/src/test/aop/build/aop.xml
new file mode 100644
index 0000000..d236258
--- /dev/null
+++ b/src/test/aop/build/aop.xml
@@ -0,0 +1,132 @@
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project name="aspects">
+ <property name="build-fi.dir" value="${basedir}/build-fi"/>
+ <property name="hadoop-fi.jar" location="${build.dir}/${final.name}-fi.jar" />
+ <property name="compile-inject.output" value="${build-fi.dir}/compile-fi.log"/>
+ <property name="aspectversion" value="1.6.5"/>
+ <property file="${basedir}/build.properties"/>
+
+ <!--All Fault Injection (FI) related targets are located in this session -->
+
+ <target name="clean-fi">
+ <delete dir="${build-fi.dir}"/>
+ </target>
+
+ <!-- Weaving aspects in place
+ Later on one can run 'ant jar-fault-inject' to create
+ Hadoop jar file with instrumented classes
+ -->
+ <!-- Target -classes-compilation has to be defined in build.xml and
+ needs to depend on classes compilation and test classes compilation
+ targets. This is a poor man parametrization for targets -->
+ <target name="compile-fault-inject" depends="-classes-compilation" >
+ <!-- AspectJ task definition -->
+ <taskdef
+ resource="org/aspectj/tools/ant/taskdefs/aspectjTaskdefs.properties">
+ <classpath>
+ <pathelement
+ location="${common.ivy.lib.dir}/aspectjtools-${aspectversion}.jar"/>
+ </classpath>
+ </taskdef>
+ <echo message="Start weaving aspects in place"/>
+ <iajc
+ encoding="${build.encoding}"
+ srcdir="${java.src.dir};${build.src};${test.src.dir}/aop"
+ includes="org/apache/hadoop/**/*.java, org/apache/hadoop/**/*.aj"
+ excludes="org/apache/hadoop/record/**/*"
+ destDir="${build.classes}"
+ debug="${javac.debug}"
+ target="${javac.version}"
+ source="${javac.version}"
+ deprecation="${javac.deprecation}">
+ <classpath refid="test.classpath"/>
+ </iajc>
+ <loadfile property="injection.failure" srcfile="${compile-inject.output}">
+ <filterchain>
+ <linecontainsregexp>
+ <regexp pattern='iajc.*warning'/>
+ </linecontainsregexp>
+ </filterchain>
+ </loadfile>
+ <fail if="injection.failure">
+ Broken binding of advises: ${line.separator}${injection.failure}
+ </fail>
+ <echo message="Weaving of aspects is finished"/>
+ </target>
+
+ <target name="injectfaults"
+ description="Instrument classes with faults and other AOP advices">
+ <!--mkdir to prevent <subant> failure in case the folder has been removed-->
+ <mkdir dir="${build-fi.dir}"/>
+ <delete file="${compile-inject.output}"/>
+ <subant buildpath="${basedir}" target="compile-fault-inject"
+ output="${compile-inject.output}">
+ <property name="build.dir" value="${build-fi.dir}"/>
+ </subant>
+ </target>
+
+ <macrodef name="macro-run-tests-fault-inject">
+ <attribute name="target.name" />
+ <attribute name="testcasesonly" />
+ <sequential>
+ <subant buildpath="build.xml" target="@{target.name}">
+ <property name="build.dir" value="${build-fi.dir}"/>
+ <property name="test.fault.inject" value="yes"/>
+ <property name="test.include" value="TestFi*"/>
+ <!-- This one is needed for the special "regression" target only -->
+ <property name="special.fi.testcasesonly" value="@{testcasesonly}"/>
+ </subant>
+ </sequential>
+ </macrodef>
+
+ <!-- ================================================================== -->
+ <!-- Make hadoop-fi.jar including all Fault injected artifacts -->
+ <!-- ================================================================== -->
+ <macrodef name="macro-jar-fault-inject">
+ <attribute name="target.name" />
+ <attribute name="jar.final.name" />
+ <attribute name="jar.final.value" />
+ <sequential>
+ <subant buildpath="build.xml" target="@{target.name}">
+ <property name="build.dir" value="${build-fi.dir}"/>
+ <property name="@{jar.final.name}" value="@{jar.final.value}"/>
+ <property name="jar.extra.properties.list"
+ value="${test.src.dir}/fi-site.xml" />
+ </subant>
+ </sequential>
+ </macrodef>
+
+ <!-- ================================================================== -->
+ <!-- Make test jar files including all Fault Injected artifacts -->
+ <!-- ================================================================== -->
+
+ <macrodef name="macro-jar-test-fault-inject">
+ <attribute name="target.name" />
+ <attribute name="jar.final.name" />
+ <attribute name="jar.final.value" />
+ <sequential>
+ <subant buildpath="build.xml" target="@{target.name}">
+ <property name="build.dir" value="${build-fi.dir}"/>
+ <property name="@{jar.final.name}"
+ value="@{jar.final.value}"/>
+ </subant>
+ </sequential>
+ </macrodef>
+
+ <!--End of Fault Injection (FI) related session-->
+</project>
diff --git a/src/test/aop/org/apache/hadoop/fi/FiConfig.java b/src/test/aop/org/apache/hadoop/fi/FiConfig.java
new file mode 100644
index 0000000..b1f5ff9
--- /dev/null
+++ b/src/test/aop/org/apache/hadoop/fi/FiConfig.java
@@ -0,0 +1,55 @@
+/*
+ * 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.fi;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+
+/**
+ * This class wraps the logic around fault injection configuration file
+ * Default file is expected to be found in src/test/fi-site.xml
+ * This default file should be copied by JUnit Ant's tasks to
+ * build/test/extraconf folder before tests are ran
+ * An alternative location can be set through
+ * -Dfi.config=<file_name>
+ */
+public class FiConfig {
+ private static final String CONFIG_PARAMETER = ProbabilityModel.FPROB_NAME + "config";
+ private static final String DEFAULT_CONFIG = "fi-site.xml";
+ private static Configuration conf;
+ static {
+ init();
+ }
+
+ protected static void init () {
+ if (conf == null) {
+ conf = new HdfsConfiguration(false);
+ String configName = System.getProperty(CONFIG_PARAMETER, DEFAULT_CONFIG);
+ conf.addResource(configName);
+ }
+ }
+
+ /**
+ * Method provides access to local Configuration
+ *
+ * @return Configuration initialized with fault injection's parameters
+ */
+ public static Configuration getConfig() {
+ return conf;
+ }
+}
diff --git a/src/test/aop/org/apache/hadoop/fi/ProbabilityModel.java b/src/test/aop/org/apache/hadoop/fi/ProbabilityModel.java
new file mode 100644
index 0000000..48e57f5
--- /dev/null
+++ b/src/test/aop/org/apache/hadoop/fi/ProbabilityModel.java
@@ -0,0 +1,105 @@
+/*
+ * 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.fi;
+
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * This class is responsible for the decision of when a fault
+ * has to be triggered within a class of Hadoop
+ *
+ * Default probability of injection is set to 0%. To change it
+ * one can set the sys. prop. -Dfi.*=<new probability level>
+ * Another way to do so is to set this level through FI config file,
+ * located under src/test/fi-site.conf
+ *
+ * To change the level one has to specify the following sys,prop.:
+ * -Dfi.<name of fault location>=<probability level> in the runtime
+ * Probability level is specified by a float between 0.0 and 1.0
+ *
+ * <name of fault location> might be represented by a short classname
+ * or otherwise. This decision is left up to the discretion of aspects
+ * developer, but has to be consistent through the code
+ */
+public class ProbabilityModel {
+ private static Random generator = new Random();
+ private static final Log LOG = LogFactory.getLog(ProbabilityModel.class);
+
+ static final String FPROB_NAME = "fi.";
+ private static final String ALL_PROBABILITIES = FPROB_NAME + "*";
+ private static final float DEFAULT_PROB = 0.00f; //Default probability is 0%
+ private static final float MAX_PROB = 1.00f; // Max probability is 100%
+
+ private static Configuration conf = FiConfig.getConfig();
+
+ static {
+ // Set new default probability if specified through a system.property
+ // If neither is specified set default probability to DEFAULT_PROB
+ conf.set(ALL_PROBABILITIES,
+ System.getProperty(ALL_PROBABILITIES,
+ conf.get(ALL_PROBABILITIES, Float.toString(DEFAULT_PROB))));
+
+ LOG.info(ALL_PROBABILITIES + "=" + conf.get(ALL_PROBABILITIES));
+ }
+
+ /**
+ * Simplistic method to check if we have reached the point of injection
+ * @param klassName is the name of the probability level to check.
+ * If a configuration has been set for "fi.myClass" then you can check if the
+ * inject criteria has been reached by calling this method with "myClass"
+ * string as its parameter
+ * @return true if the probability threshold has been reached; false otherwise
+ */
+ public static boolean injectCriteria(String klassName) {
+ boolean trigger = false;
+ // TODO fix this: make it more sophisticated!!!
+ if (generator.nextFloat() < getProbability(klassName)) {
+ trigger = true;
+ }
+ return trigger;
+ }
+
+ /**
+ * This primitive checks for arbitrary set of desired probability. If the
+ * level hasn't been set method will return default setting.
+ * The probability expected to be set as an float between 0.0 and 1.0
+ * @param klass is the name of the resource
+ * @return float representation of configured probability level of
+ * the requested resource or default value if hasn't been set
+ */
+ protected static float getProbability(final String klass) {
+ String newProbName = FPROB_NAME + klass;
+
+ String newValue = System.getProperty(newProbName, conf.get(ALL_PROBABILITIES));
+ if (newValue != null && !newValue.equals(conf.get(newProbName)))
+ conf.set(newProbName, newValue);
+
+ float ret = conf.getFloat(newProbName,
+ conf.getFloat(ALL_PROBABILITIES, DEFAULT_PROB));
+ LOG.debug("Request for " + newProbName + " returns=" + ret);
+ // Make sure that probability level is valid.
+ if (ret < DEFAULT_PROB || ret > MAX_PROB)
+ ret = conf.getFloat(ALL_PROBABILITIES, DEFAULT_PROB);
+
+ return ret;
+ }
+}
diff --git a/src/test/checkstyle-noframes-sorted.xsl b/src/test/checkstyle-noframes-sorted.xsl
index 5f9e93b..9c0ac30 100644
--- a/src/test/checkstyle-noframes-sorted.xsl
+++ b/src/test/checkstyle-noframes-sorted.xsl
@@ -1,4 +1,21 @@
<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<xsl:output method="html" indent="yes"/>
<xsl:decimal-format decimal-separator="." grouping-separator="," />
diff --git a/src/test/checkstyle.xml b/src/test/checkstyle.xml
index 5e3b894..5fb0c47 100644
--- a/src/test/checkstyle.xml
+++ b/src/test/checkstyle.xml
@@ -1,4 +1,21 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<!DOCTYPE module PUBLIC
"-//Puppy Crawl//DTD Check Configuration 1.2//EN"
"http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
diff --git a/src/test/fi-site.xml b/src/test/fi-site.xml
new file mode 100644
index 0000000..42bae52
--- /dev/null
+++ b/src/test/fi-site.xml
@@ -0,0 +1,31 @@
+<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!-- Put fault injection specific property overrides in this file. -->
+
+<configuration>
+ <property>
+ <name>fi.*</name>
+ <value>0.00</value>
+ <description>
+ Default probability level for all injected faults specified
+ as a floating number between 0 and 1.00
+ </description>
+ </property>
+</configuration>
diff --git a/src/test/findbugsExcludeFile.xml b/src/test/findbugsExcludeFile.xml
index 0d20d04..9e8ea7c 100644
--- a/src/test/findbugsExcludeFile.xml
+++ b/src/test/findbugsExcludeFile.xml
@@ -1,3 +1,20 @@
+<!--
+ 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.
+-->
+
<FindBugsFilter>
<Match>
<Package name="org.apache.hadoop.record.compiler.generated" />
@@ -43,6 +60,10 @@
<Class name="org.apache.hadoop.mapred.taskdetails_jsp" />
<Bug code="HRS" />
</Match>
+ <Match>
+ <Class name="org.apache.hadoop.mapred.jobdetails_jsp"/>
+ <Bug pattern="HRS_REQUEST_PARAMETER_TO_HTTP_HEADER"/>
+ </Match>
<!--
Ignore warnings where child class has the same name as
super class. Classes based on Old API shadow names from
diff --git a/src/test/hadoop-policy.xml b/src/test/hadoop-policy.xml
index f57800f..0935274 100644
--- a/src/test/hadoop-policy.xml
+++ b/src/test/hadoop-policy.xml
@@ -1,4 +1,21 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!-- Put site-specific property overrides in this file. -->
diff --git a/src/test/log4j.properties b/src/test/log4j.properties
index 1a2122a..c557d8f 100644
--- a/src/test/log4j.properties
+++ b/src/test/log4j.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
# log4j configuration used during build and unit tests
log4j.rootLogger=info,stdout
diff --git a/src/test/mapred-site.xml b/src/test/mapred-site.xml
index 95541c9..1596ebc 100644
--- a/src/test/mapred-site.xml
+++ b/src/test/mapred-site.xml
@@ -1,4 +1,21 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!-- Put site-specific property overrides in this file. -->
@@ -19,4 +36,9 @@
<value>false</value>
<description></description>
</property>
+<property>
+ <name>mapreduce.jobtracker.staging.root.dir</name>
+ <value>${hadoop.tmp.dir}/staging</value>
+ <description></description>
+</property>
</configuration>
diff --git a/src/test/mapred/org/apache/hadoop/mapred/ClusterWithLinuxTaskController.java b/src/test/mapred/org/apache/hadoop/mapred/ClusterWithLinuxTaskController.java
index a469dd6..2c7d70b 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/ClusterWithLinuxTaskController.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/ClusterWithLinuxTaskController.java
@@ -28,11 +28,14 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
import org.apache.hadoop.security.UnixUserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
import junit.framework.TestCase;
@@ -48,7 +51,10 @@
* <li>Make the built binary to setuid executable</li>
* <li>Execute following targets:
* <code>ant test -Dcompile.c++=true -Dtaskcontroller-path=<em>path to built binary</em>
- * -Dtaskcontroller-ugi=<em>user,group</em></code></li>
+ * -Dtaskcontroller-ugi=<em>user,group</em></code>
+ * <br/>(Note that "path to built binary" means the directory containing task-controller -
+ * not the actual complete path of the binary itself. This path must end in ".../bin")
+ * </li>
* </ol>
*
*/
@@ -72,6 +78,24 @@
void setTaskControllerExe(String execPath) {
this.taskControllerExePath = execPath;
}
+
+ volatile static int attemptedSigQuits = 0;
+ volatile static int failedSigQuits = 0;
+
+ /** Work like LinuxTaskController, but also count the number of
+ * attempted and failed SIGQUIT sends via the task-controller
+ * executable.
+ */
+ @Override
+ void dumpTaskStack(TaskControllerContext context) {
+ attemptedSigQuits++;
+ try {
+ signalTask(context, TaskCommands.SIGQUIT_TASK_JVM);
+ } catch (Exception e) {
+ LOG.warn("Execution sending SIGQUIT: " + StringUtils.stringifyException(e));
+ failedSigQuits++;
+ }
+ }
}
// cluster instances which sub classes can use
@@ -120,10 +144,10 @@
String[] splits = ugi.split(",");
taskControllerUser = new UnixUserGroupInformation(splits);
clusterConf.set(UnixUserGroupInformation.UGI_PROPERTY_NAME, ugi);
- createHomeDirectory(clusterConf);
+ createHomeAndStagingDirectory(clusterConf);
}
- private void createHomeDirectory(JobConf conf)
+ private void createHomeAndStagingDirectory(JobConf conf)
throws IOException {
FileSystem fs = dfsCluster.getFileSystem();
String path = "/user/" + taskControllerUser.getUserName();
@@ -131,6 +155,10 @@
LOG.info("Creating Home directory : " + homeDirectory);
fs.mkdirs(homeDirectory);
changePermission(conf, homeDirectory);
+ Path stagingArea = new Path(conf.get(JTConfig.JT_STAGING_AREA_ROOT));
+ LOG.info("Creating Staging root directory : " + stagingArea);
+ fs.mkdirs(stagingArea);
+ fs.setPermission(stagingArea, new FsPermission((short)0777));
}
private void changePermission(JobConf conf, Path p)
diff --git a/src/test/mapred/org/apache/hadoop/mapred/DummyResourceCalculatorPlugin.java b/src/test/mapred/org/apache/hadoop/mapred/DummyResourceCalculatorPlugin.java
new file mode 100644
index 0000000..0354d5d
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapred/DummyResourceCalculatorPlugin.java
@@ -0,0 +1,101 @@
+/**
+ * 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.mapred;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.mapreduce.util.ResourceCalculatorPlugin;
+
+/**
+ * Plugin class to test resource information reported by TT. Use
+ * configuration items {@link #MAXVMEM_TESTING_PROPERTY} and
+ * {@link #MAXPMEM_TESTING_PROPERTY} to tell TT the total vmem and the total
+ * pmem. Use configuration items {@link #NUM_PROCESSORS},
+ * {@link #CPU_FREQUENCY}, {@link #CUMULATIVE_CPU_TIME} and {@link #CPU_USAGE}
+ * to tell TT the CPU information.
+ */
+@InterfaceAudience.Private
+public class DummyResourceCalculatorPlugin extends ResourceCalculatorPlugin {
+
+ /** max vmem on the TT */
+ public static final String MAXVMEM_TESTING_PROPERTY =
+ "mapred.tasktracker.maxvmem.testing";
+ /** max pmem on the TT */
+ public static final String MAXPMEM_TESTING_PROPERTY =
+ "mapred.tasktracker.maxpmem.testing";
+ /** number of processors for testing */
+ public static final String NUM_PROCESSORS =
+ "mapred.tasktracker.numprocessors.testing";
+ /** CPU frequency for testing */
+ public static final String CPU_FREQUENCY =
+ "mapred.tasktracker.cpufrequency.testing";
+ /** cumulative CPU usage time for testing */
+ public static final String CUMULATIVE_CPU_TIME =
+ "mapred.tasktracker.cumulativecputime.testing";
+ /** CPU usage percentage for testing */
+ public static final String CPU_USAGE =
+ "mapred.tasktracker.cpuusage.testing";
+
+ /** {@inheritDoc} */
+ @Override
+ public long getVirtualMemorySize() {
+ return getConf().getLong(MAXVMEM_TESTING_PROPERTY, -1);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getPhysicalMemorySize() {
+ return getConf().getLong(MAXPMEM_TESTING_PROPERTY, -1);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getAvailableVirtualMemorySize() {
+ return getConf().getLong(MAXVMEM_TESTING_PROPERTY, -1);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getAvailablePhysicalMemorySize() {
+ return getConf().getLong(MAXPMEM_TESTING_PROPERTY, -1);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public int getNumProcessors() {
+ return getConf().getInt(NUM_PROCESSORS, -1);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getCpuFrequency() {
+ return getConf().getLong(CPU_FREQUENCY, -1);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public long getCumulativeCpuTime() {
+ return getConf().getLong(CUMULATIVE_CPU_TIME, -1);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public float getCpuUsage() {
+ return getConf().getFloat(CPU_USAGE, -1);
+ }
+}
\ No newline at end of file
diff --git a/src/test/mapred/org/apache/hadoop/mapred/FakeObjectUtilities.java b/src/test/mapred/org/apache/hadoop/mapred/FakeObjectUtilities.java
index 024dc04..fe72c20 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/FakeObjectUtilities.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/FakeObjectUtilities.java
@@ -29,11 +29,11 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.TaskStatus.Phase;
-import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.Job.RawSplit;
import org.apache.hadoop.mapreduce.jobhistory.HistoryEvent;
import org.apache.hadoop.mapreduce.jobhistory.JobHistory;
+import org.apache.hadoop.mapreduce.split.JobSplit;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
/**
* Utilities used in unit test.
@@ -67,8 +67,10 @@
}
@Override
public ClusterStatus getClusterStatus(boolean detailed) {
- return new ClusterStatus(trackers.length,
- 0, 0, 0, 0, totalSlots/2, totalSlots/2, JobTracker.State.RUNNING, 0);
+ return new ClusterStatus(
+ taskTrackers().size() - getBlacklistedTrackerCount(),
+ getBlacklistedTrackerCount(), 0, 0, 0, totalSlots/2, totalSlots/2,
+ JobTracker.State.RUNNING, 0);
}
public void setNumSlots(int totalSlots) {
@@ -77,7 +79,6 @@
}
static class FakeJobInProgress extends JobInProgress {
- Job.RawSplit[] rawSplits;
@SuppressWarnings("deprecation")
FakeJobInProgress(JobConf jobConf, JobTracker tracker) throws IOException {
super(new JobID(jtIdentifier, ++jobCounter), jobConf, tracker);
@@ -91,27 +92,27 @@
@Override
public synchronized void initTasks() throws IOException {
- Job.RawSplit[] splits = createSplits();
- numMapTasks = splits.length;
- createMapTasks(null, splits);
- nonRunningMapCache = createCache(splits, maxLevel);
+ TaskSplitMetaInfo[] taskSplitMetaInfo = createSplits(jobId);
+ numMapTasks = taskSplitMetaInfo.length;
+ createMapTasks(null, taskSplitMetaInfo);
+ nonRunningMapCache = createCache(taskSplitMetaInfo, maxLevel);
createReduceTasks(null);
tasksInited.set(true);
this.status.setRunState(JobStatus.RUNNING);
}
@Override
- Job.RawSplit[] createSplits(){
- Job.RawSplit[] splits = new Job.RawSplit[numMapTasks];
+ TaskSplitMetaInfo [] createSplits(org.apache.hadoop.mapreduce.JobID jobId){
+ TaskSplitMetaInfo[] splits =
+ new TaskSplitMetaInfo[numMapTasks];
for (int i = 0; i < numMapTasks; i++) {
- splits[i] = new Job.RawSplit();
- splits[i].setLocations(new String[0]);
+ splits[i] = JobSplit.EMPTY_TASK_SPLIT;
}
return splits;
}
@Override
- protected void createMapTasks(String ignored, Job.RawSplit[] splits) {
+ protected void createMapTasks(String ignored, TaskSplitMetaInfo[] splits) {
maps = new TaskInProgress[numMapTasks];
for (int i = 0; i < numMapTasks; i++) {
maps[i] = new TaskInProgress(getJobID(), "test",
@@ -218,6 +219,15 @@
updateTaskStatus(tip, status);
}
+ public void killTask(TaskAttemptID taskId) {
+ TaskInProgress tip = jobtracker.taskidToTIPMap.get(taskId);
+ TaskStatus status = TaskStatus.createTaskStatus(tip.isMapTask(), taskId,
+ 1.0f, 1, TaskStatus.State.KILLED, "", "", tip
+ .machineWhereTaskRan(taskId), tip.isMapTask() ? Phase.MAP
+ : Phase.REDUCE, new Counters());
+ updateTaskStatus(tip, status);
+ }
+
public void cleanUpMetrics() {
}
@@ -253,7 +263,7 @@
numSlotsRequired);
}
- public FakeTaskInProgress(JobID jobId, String jobFile, RawSplit emptySplit,
+ public FakeTaskInProgress(JobID jobId, String jobFile, TaskSplitMetaInfo emptySplit,
JobTracker jobTracker, JobConf jobConf,
JobInProgress job, int partition, int numSlotsRequired) {
super(jobId, jobFile, emptySplit, jobTracker, jobConf, job,
diff --git a/src/test/mapred/org/apache/hadoop/mapred/GenericMRLoadGenerator.java b/src/test/mapred/org/apache/hadoop/mapred/GenericMRLoadGenerator.java
index 03dd48a..e33bdf8 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/GenericMRLoadGenerator.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/GenericMRLoadGenerator.java
@@ -141,16 +141,16 @@
org.apache.hadoop.mapreduce.GenericMRLoadGenerator.INDIRECT_INPUT_FORMAT,
null)) {
// specified IndirectInputFormat? Build src list
- JobClient jClient = new JobClient(job);
- Path sysdir = jClient.getSystemDir();
+ JobClient jClient = new JobClient(job);
+ Path tmpDir = new Path(jClient.getFs().getHomeDirectory(), ".staging");
Random r = new Random();
- Path indirInputFile = new Path(sysdir,
+ Path indirInputFile = new Path(tmpDir,
Integer.toString(r.nextInt(Integer.MAX_VALUE), 36) + "_files");
job.set(
org.apache.hadoop.mapreduce.GenericMRLoadGenerator.INDIRECT_INPUT_FILE,
indirInputFile.toString());
SequenceFile.Writer writer = SequenceFile.createWriter(
- sysdir.getFileSystem(job), job, indirInputFile,
+ tmpDir.getFileSystem(job), job, indirInputFile,
LongWritable.class, Text.class,
SequenceFile.CompressionType.NONE);
try {
diff --git a/src/test/mapred/org/apache/hadoop/mapred/MiniMRCluster.java b/src/test/mapred/org/apache/hadoop/mapred/MiniMRCluster.java
index e1d60b3..251a3d4 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/MiniMRCluster.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/MiniMRCluster.java
@@ -189,7 +189,7 @@
conf.set(MRConfig.LOCAL_DIR, localPath.toString());
LOG.info(MRConfig.LOCAL_DIR + " is " + localPath);
try {
- tt = new TaskTracker(conf);
+ tt = createTaskTracker(conf);
isInitialized = true;
} catch (Throwable e) {
isDead = true;
@@ -199,6 +199,13 @@
}
/**
+ * Creates a default {@link TaskTracker} using the conf passed.
+ */
+ TaskTracker createTaskTracker(JobConf conf) throws IOException {
+ return new TaskTracker(conf);
+ }
+
+ /**
* Create and run the task tracker.
*/
public void run() {
@@ -268,7 +275,18 @@
public int getNumTaskTrackers() {
return taskTrackerList.size();
}
-
+
+ /**
+ * Sets inline cleanup threads to all task trackers sothat deletion of
+ * temporary files/dirs happen inline
+ */
+ public void setInlineCleanupThreads() {
+ for (int i = 0; i < getNumTaskTrackers(); i++) {
+ getTaskTrackerRunner(i).getTaskTracker().setCleanupThread(
+ new UtilsForTests.InlineCleanupQueue());
+ }
+ }
+
/**
* Wait until the system is idle.
*/
@@ -671,6 +689,13 @@
TaskTrackerRunner taskTracker;
taskTracker = new TaskTrackerRunner(idx, numDir, host, conf);
+ addTaskTracker(taskTracker);
+ }
+
+ /**
+ * Add a task-tracker to the Mini-MR cluster.
+ */
+ void addTaskTracker(TaskTrackerRunner taskTracker) {
Thread taskTrackerThread = new Thread(taskTracker);
taskTrackerList.add(taskTracker);
taskTrackerThreadList.add(taskTrackerThread);
diff --git a/src/test/mapred/org/apache/hadoop/mapred/NotificationTestCase.java b/src/test/mapred/org/apache/hadoop/mapred/NotificationTestCase.java
index 2290c3c..dcc411a 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/NotificationTestCase.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/NotificationTestCase.java
@@ -32,7 +32,6 @@
import javax.servlet.ServletException;
import java.io.IOException;
import java.io.DataOutputStream;
-import java.util.Date;
/**
* Base class to test Job end notification in local and cluster mode.
@@ -51,17 +50,12 @@
*/
public abstract class NotificationTestCase extends HadoopTestCase {
- private static void stdPrintln(String s) {
- //System.out.println(s);
- }
-
protected NotificationTestCase(int mode) throws IOException {
super(mode, HadoopTestCase.LOCAL_FS, 1, 1);
}
private int port;
private String contextPath = "/notification";
- private Class servletClass = NotificationServlet.class;
private String servletPath = "/mapred";
private Server webServer;
@@ -118,15 +112,9 @@
break;
}
if (counter % 2 == 0) {
- stdPrintln((new Date()).toString() +
- "Receiving First notification for [" + req.getQueryString() +
- "], returning error");
res.sendError(HttpServletResponse.SC_BAD_REQUEST, "forcing error");
}
else {
- stdPrintln((new Date()).toString() +
- "Receiving Second notification for [" + req.getQueryString() +
- "], returning OK");
res.setStatus(HttpServletResponse.SC_OK);
}
counter++;
@@ -160,10 +148,7 @@
public void testMR() throws Exception {
System.out.println(launchWordCount(this.createJobConf(),
"a b c d e f g h", 1, 1));
- synchronized(Thread.currentThread()) {
- stdPrintln("Sleeping for 2 seconds to give time for retry");
- Thread.currentThread().sleep(2000);
- }
+ Thread.sleep(2000);
assertEquals(2, NotificationServlet.counter);
Path inDir = new Path("notificationjob/input");
@@ -180,19 +165,13 @@
// run a job with KILLED status
System.out.println(UtilsForTests.runJobKill(this.createJobConf(), inDir,
outDir).getID());
- synchronized(Thread.currentThread()) {
- stdPrintln("Sleeping for 2 seconds to give time for retry");
- Thread.currentThread().sleep(2000);
- }
+ Thread.sleep(2000);
assertEquals(4, NotificationServlet.counter);
// run a job with FAILED status
System.out.println(UtilsForTests.runJobFail(this.createJobConf(), inDir,
outDir).getID());
- synchronized(Thread.currentThread()) {
- stdPrintln("Sleeping for 2 seconds to give time for retry");
- Thread.currentThread().sleep(2000);
- }
+ Thread.sleep(2000);
assertEquals(6, NotificationServlet.counter);
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestFieldSelection.java b/src/test/mapred/org/apache/hadoop/mapred/TestFieldSelection.java
index e6b8162..b86905e 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestFieldSelection.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestFieldSelection.java
@@ -20,6 +20,7 @@
import org.apache.hadoop.fs.*;
import org.apache.hadoop.io.*;
import org.apache.hadoop.mapred.lib.*;
+import org.apache.hadoop.mapreduce.lib.fieldsel.FieldSelectionHelper;
import org.apache.hadoop.mapreduce.lib.fieldsel.TestMRFieldSelection;
import junit.framework.TestCase;
@@ -72,9 +73,9 @@
job.setOutputFormat(TextOutputFormat.class);
job.setNumReduceTasks(1);
- job.set("mapreduce.fieldsel.data.field.separator", "-");
- job.set("mapreduce.fieldsel.mapreduce.fieldsel.map.output.key.value.fields.spec", "6,5,1-3:0-");
- job.set("mapreduce.fieldsel.mapreduce.fieldsel.reduce.output.key.value.fields.spec", ":4,3,2,1,0,0-");
+ job.set(FieldSelectionHelper.DATA_FIELD_SEPERATOR, "-");
+ job.set(FieldSelectionHelper.MAP_OUTPUT_KEY_VALUE_SPEC, "6,5,1-3:0-");
+ job.set(FieldSelectionHelper.REDUCE_OUTPUT_KEY_VALUE_SPEC, ":4,3,2,1,0,0-");
JobClient.runJob(job);
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestJobDirCleanup.java b/src/test/mapred/org/apache/hadoop/mapred/TestJobDirCleanup.java
index e50a6a0..8f30357 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestJobDirCleanup.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestJobDirCleanup.java
@@ -58,6 +58,9 @@
namenode = fileSys.getUri().toString();
mr = new MiniMRCluster(10, namenode, 3,
null, null, mrConf);
+ // make cleanup inline sothat validation of existence of these directories
+ // can be done
+ mr.setInlineCleanupThreads();
final String jobTrackerName = "localhost:" + mr.getJobTrackerPort();
JobConf jobConf = mr.createJobConf();
runSleepJob(jobConf);
@@ -78,19 +81,8 @@
"/taskTracker/jobcache";
File jobDir = new File(jobDirStr);
String[] contents = jobDir.list();
- if (contents == null || contents.length == 0) {
- return;
- }
- while (contents.length > 0) {
- try {
- Thread.sleep(1000);
- LOG.warn(jobDir +" not empty yet, contents are");
- for (String s: contents) {
- LOG.info(s);
- }
- contents = jobDir.list();
- } catch (InterruptedException ie){}
- }
+ assertTrue("Contents of " + jobDir + " not cleanup.",
+ (contents == null || contents.length == 0));
}
}
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestJobExecutionAsDifferentUser.java b/src/test/mapred/org/apache/hadoop/mapred/TestJobExecutionAsDifferentUser.java
index 3c90b44..366ba29 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestJobExecutionAsDifferentUser.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestJobExecutionAsDifferentUser.java
@@ -22,6 +22,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.SleepJob;
import org.apache.hadoop.util.ToolRunner;
@@ -92,4 +93,30 @@
}
}
}
+
+ /** Ensure that SIGQUIT can be properly sent by the LinuxTaskController
+ * if a task times out.
+ */
+ public void testTimeoutStackTrace() throws Exception {
+ if (!shouldRun()) {
+ return;
+ }
+
+ // Run a job that should timeout and trigger a SIGQUIT.
+ startCluster();
+ JobConf conf = getClusterConf();
+ conf.setInt(JobContext.TASK_TIMEOUT, 10000);
+ conf.setInt(Job.COMPLETION_POLL_INTERVAL_KEY, 50);
+ SleepJob sleepJob = new SleepJob();
+ sleepJob.setConf(conf);
+ Job job = sleepJob.createJob(1, 0, 30000, 1, 0, 0);
+ job.setMaxMapAttempts(1);
+ int prevNumSigQuits = MyLinuxTaskController.attemptedSigQuits;
+ job.waitForCompletion(true);
+ assertTrue("Did not detect a new SIGQUIT!",
+ prevNumSigQuits < MyLinuxTaskController.attemptedSigQuits);
+ assertEquals("A SIGQUIT attempt failed!", 0,
+ MyLinuxTaskController.failedSigQuits);
+
+ }
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestJobInProgress.java b/src/test/mapred/org/apache/hadoop/mapred/TestJobInProgress.java
index bda34b4..dc9c40e 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestJobInProgress.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestJobInProgress.java
@@ -40,10 +40,9 @@
import org.apache.hadoop.mapred.FakeObjectUtilities.FakeJobTracker;
import org.apache.hadoop.mapred.TaskStatus.Phase;
import org.apache.hadoop.mapred.UtilsForTests.FakeClock;
-import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.mapreduce.Job.RawSplit;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.net.StaticMapping;
@@ -101,16 +100,14 @@
}
@Override
- Job.RawSplit[] createSplits() {
+ TaskSplitMetaInfo[] createSplits(org.apache.hadoop.mapreduce.JobID jobId) {
// Set all splits to reside on one host. This will ensure that
// one tracker gets data local, one gets rack local and two others
// get non-local maps
- Job.RawSplit[] splits = new Job.RawSplit[numMapTasks];
+ TaskSplitMetaInfo[] splits = new TaskSplitMetaInfo[numMapTasks];
String[] splitHosts0 = new String[] { hosts[0] };
for (int i = 0; i < numMapTasks; i++) {
- splits[i] = new Job.RawSplit();
- splits[i].setDataLength(0);
- splits[i].setLocations(splitHosts0);
+ splits[i] = new TaskSplitMetaInfo(splitHosts0, 0, 0);
}
return splits;
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestJobKillAndFail.java b/src/test/mapred/org/apache/hadoop/mapred/TestJobKillAndFail.java
index 1d2c41f..410207e 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestJobKillAndFail.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestJobKillAndFail.java
@@ -18,12 +18,20 @@
package org.apache.hadoop.mapred;
+import java.io.BufferedReader;
+import java.io.FileInputStream;
import java.io.File;
+import java.io.InputStreamReader;
import java.io.IOException;
import junit.framework.TestCase;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.SleepJob;
/**
* A JUnit test to test Kill Job & Fail Job functionality with local file
@@ -31,38 +39,96 @@
*/
public class TestJobKillAndFail extends TestCase {
+ static final Log LOG = LogFactory.getLog(TestJobKillAndFail.class);
+
private static String TEST_ROOT_DIR = new File(System.getProperty(
"test.build.data", "/tmp")).toURI().toString().replace(' ', '+');
- public void testJobFailAndKill() throws IOException {
+ /**
+ * TaskController instance that just sets a flag when a stack dump
+ * is performed in a child thread.
+ */
+ static class MockStackDumpTaskController extends DefaultTaskController {
+
+ static volatile int numStackDumps = 0;
+
+ static final Log LOG = LogFactory.getLog(TestJobKillAndFail.class);
+
+ public MockStackDumpTaskController() {
+ LOG.info("Instantiated MockStackDumpTC");
+ }
+
+ @Override
+ void dumpTaskStack(TaskControllerContext context) {
+ LOG.info("Got stack-dump request in TaskController");
+ MockStackDumpTaskController.numStackDumps++;
+ super.dumpTaskStack(context);
+ }
+
+ }
+
+ /** If a task was killed, then dumpTaskStack() should have been
+ * called. Test whether or not the counter was incremented
+ * and succeed/fail based on this. */
+ private void checkForStackDump(boolean expectDump, int lastNumDumps) {
+ int curNumDumps = MockStackDumpTaskController.numStackDumps;
+
+ LOG.info("curNumDumps=" + curNumDumps + "; lastNumDumps=" + lastNumDumps
+ + "; expect=" + expectDump);
+
+ if (expectDump) {
+ assertTrue("No stack dump recorded!", lastNumDumps < curNumDumps);
+ } else {
+ assertTrue("Stack dump happened anyway!", lastNumDumps == curNumDumps);
+ }
+ }
+
+ public void testJobFailAndKill() throws Exception {
MiniMRCluster mr = null;
try {
JobConf jtConf = new JobConf();
jtConf.set("mapred.jobtracker.instrumentation",
JTInstrumentation.class.getName());
+ jtConf.set("mapreduce.tasktracker.taskcontroller",
+ MockStackDumpTaskController.class.getName());
mr = new MiniMRCluster(2, "file:///", 3, null, null, jtConf);
JTInstrumentation instr = (JTInstrumentation)
mr.getJobTrackerRunner().getJobTracker().getInstrumentation();
// run the TCs
JobConf conf = mr.createJobConf();
+ conf.setInt(Job.COMPLETION_POLL_INTERVAL_KEY, 50);
Path inDir = new Path(TEST_ROOT_DIR + "/failkilljob/input");
Path outDir = new Path(TEST_ROOT_DIR + "/failkilljob/output");
- RunningJob job = UtilsForTests.runJobFail(conf, inDir, outDir);
+ RunningJob runningJob = UtilsForTests.runJobFail(conf, inDir, outDir);
// Checking that the Job got failed
- assertEquals(job.getJobState(), JobStatus.FAILED);
+ assertEquals(runningJob.getJobState(), JobStatus.FAILED);
assertTrue(instr.verifyJob());
assertEquals(1, instr.failed);
instr.reset();
-
- job = UtilsForTests.runJobKill(conf, inDir, outDir);
+ int prevNumDumps = MockStackDumpTaskController.numStackDumps;
+ runningJob = UtilsForTests.runJobKill(conf, inDir, outDir);
// Checking that the Job got killed
- assertTrue(job.isComplete());
- assertEquals(job.getJobState(), JobStatus.KILLED);
+ assertTrue(runningJob.isComplete());
+ assertEquals(runningJob.getJobState(), JobStatus.KILLED);
assertTrue(instr.verifyJob());
assertEquals(1, instr.killed);
+ // check that job kill does not put a stacktrace in task logs.
+ checkForStackDump(false, prevNumDumps);
+
+ // Test that a task that times out does have a stack trace
+ conf = mr.createJobConf();
+ conf.setInt(JobContext.TASK_TIMEOUT, 10000);
+ conf.setInt(Job.COMPLETION_POLL_INTERVAL_KEY, 50);
+ SleepJob sleepJob = new SleepJob();
+ sleepJob.setConf(conf);
+ Job job = sleepJob.createJob(1, 0, 30000, 1,0, 0);
+ job.setMaxMapAttempts(1);
+ prevNumDumps = MockStackDumpTaskController.numStackDumps;
+ job.waitForCompletion(true);
+ checkForStackDump(true, prevNumDumps);
} finally {
if (mr != null) {
mr.shutdown();
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueClient.java b/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueClient.java
index 3dc9ff2..94735fe 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueClient.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueClient.java
@@ -17,20 +17,33 @@
*/
package org.apache.hadoop.mapred;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.CONFIG;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.checkForConfigFile;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.createDocument;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.createSimpleDocumentWithAcls;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.miniMRCluster;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.setUpCluster;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.writeToFile;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.File;
+import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
import java.util.List;
-import org.junit.After;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.QueueInfo;
import org.junit.Test;
+import org.w3c.dom.Document;
public class TestJobQueueClient {
@Test
public void testQueueOrdering() throws Exception {
- System.out.println("in test queue ordering");
// create some sample queues in a hierarchy..
JobQueueInfo[] roots = new JobQueueInfo[2];
roots[0] = new JobQueueInfo("q1", "q1 scheduling info");
@@ -53,7 +66,6 @@
@Test
public void testQueueInfoPrinting() throws Exception {
- System.out.println("in test queue info printing");
// create a test queue with children.
// create some sample queues in a hierarchy..
JobQueueInfo root = new JobQueueInfo("q1", "q1 scheduling info");
@@ -76,4 +88,24 @@
assertEquals(sb.toString(), writer.toString());
}
-}
\ No newline at end of file
+
+ @Test
+ public void testGetQueue() throws Exception {
+ checkForConfigFile();
+ Document doc = createDocument();
+ createSimpleDocumentWithAcls(doc, "true");
+ writeToFile(doc, CONFIG);
+ Configuration conf = new Configuration();
+ conf.addResource(CONFIG);
+ setUpCluster(conf);
+ JobClient jc = new JobClient(miniMRCluster.createJobConf());
+ // test for existing queue
+ QueueInfo queueInfo = jc.getQueueInfo("q1");
+ assertEquals("q1",queueInfo.getQueueName());
+ // try getting a non-existing queue
+ queueInfo = jc.getQueueInfo("queue");
+ assertNull(queueInfo);
+
+ new File(CONFIG).delete();
+ }
+}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueTaskScheduler.java b/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueTaskScheduler.java
index ee9c92a..d126abe 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueTaskScheduler.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueTaskScheduler.java
@@ -30,6 +30,7 @@
import org.apache.hadoop.mapred.JobStatusChangeEvent.EventType;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
+import org.apache.hadoop.mapreduce.split.JobSplit;
public class TestJobQueueTaskScheduler extends TestCase {
@@ -81,7 +82,7 @@
public Task obtainNewMapTask(final TaskTrackerStatus tts, int clusterSize,
int ignored) throws IOException {
TaskAttemptID attemptId = getTaskAttemptID(TaskType.MAP);
- Task task = new MapTask("", attemptId, 0, "", new BytesWritable(), 1) {
+ Task task = new MapTask("", attemptId, 0, new JobSplit.TaskSplitIndex(), 1) {
@Override
public String toString() {
return String.format("%s on %s", getTaskID(), tts.getTrackerName());
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestJobRetire.java b/src/test/mapred/org/apache/hadoop/mapred/TestJobRetire.java
index dda5bd2..2068306 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestJobRetire.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestJobRetire.java
@@ -20,28 +20,46 @@
import java.io.File;
import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
import junit.framework.TestCase;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.MiniMRCluster.TaskTrackerRunner;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.SleepJob;
+import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
+import org.apache.hadoop.mapreduce.split.JobSplit;
/**
* Test if the job retire works fine.
*/
public class TestJobRetire extends TestCase {
+ static final Log LOG = LogFactory.getLog(TestJobRetire.class);
static final Path testDir =
new Path(System.getProperty("test.build.data","/tmp"),
"job-expiry-testing");
+ private MiniMRCluster startCluster(JobConf conf, int numTrackers)
+ throws IOException {
+ conf.setBoolean(JTConfig.JT_RETIREJOBS, true);
+ conf.setLong(JTConfig.JT_RETIREJOB_CACHE_SIZE, 1);
+ return new MiniMRCluster(0, 0, numTrackers, "file:///", 1, null, null, null,
+ conf, 0);
+ }
+
public void testJobRetire() throws Exception {
MiniMRCluster mr = null;
try {
JobConf conf = new JobConf();
-
- conf.setBoolean(JTConfig.JT_RETIREJOBS, true);
- conf.setLong(JTConfig.JT_RETIREJOB_CACHE_SIZE, 1);
- mr = new MiniMRCluster(0, 0, 1, "file:///", 1, null, null, null, conf, 0);
+ mr = startCluster(conf, 1);
+
JobConf jobConf = mr.createJobConf();
JobTracker jobtracker = mr.getJobTrackerRunner().getJobTracker();
@@ -59,6 +77,7 @@
1, jobtracker.getAllJobs().length);
} finally {
if (mr != null) { mr.shutdown();}
+ FileUtil.fullyDelete(new File(testDir.toString()));
}
}
@@ -70,13 +89,9 @@
assertTrue(rj.isSuccessful());
JobID id = rj.getID();
- JobInProgress job = jobtracker.getJob(id);
//wait for job to get retired
- for (int i = 0; i < 10 && job != null; i++) {
- UtilsForTests.waitFor(1000);
- job = jobtracker.getJob(id);
- }
- assertNull("Job did not retire", job);
+ waitTillRetire(id, jobtracker);
+
assertTrue("History url not set", rj.getHistoryUrl() != null &&
rj.getHistoryUrl().length() > 0);
assertNotNull("Job is not in cache", jobtracker.getJobStatus(id));
@@ -86,7 +101,262 @@
File file = new File(name);
assertFalse("JobConf file not deleted", file.exists());
+
+ //test redirection
+ URL jobUrl = new URL(rj.getTrackingURL());
+ HttpURLConnection conn = (HttpURLConnection) jobUrl.openConnection();
+ conn.setInstanceFollowRedirects(false);
+ conn.connect();
+ assertEquals(HttpURLConnection.HTTP_MOVED_TEMP, conn.getResponseCode());
+ conn.disconnect();
+
+ URL redirectedUrl = new URL(conn.getHeaderField("Location"));
+ conn = (HttpURLConnection) redirectedUrl.openConnection();
+ conn.connect();
+ assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+ conn.disconnect();
return id;
}
+ // wait till the job retires
+ private void waitTillRetire(JobID id, JobTracker jobtracker) {
+ JobInProgress job = jobtracker.getJob(id);
+ //wait for job to get retired
+ for (int i = 0; i < 10 && job != null; i++) {
+ UtilsForTests.waitFor(1000);
+ job = jobtracker.getJob(id);
+ }
+ assertNull("Job did not retire", job);
+ }
+
+ /**
+ * Custom TaskTracker which waits forever after a successful contact to
+ * the JobTracker.
+ */
+ class WaitingTaskTracker extends TaskTracker {
+
+ WaitingTaskTracker(JobConf conf) throws IOException {
+ super(conf);
+ }
+
+ @Override
+ HeartbeatResponse transmitHeartBeat(long now) throws IOException {
+ HeartbeatResponse response = super.transmitHeartBeat(now);
+ LOG.info("WaitingTaskTracker waiting");
+ // wait forever
+ UtilsForTests.waitFor(Long.MAX_VALUE);
+ throw new IOException ("WaitingTaskTracker interrupted. Bailing out");
+ }
+ }
+
+ /**
+ * Test job retire with tasks that report their *first* status only after the
+ * job retires.
+ * Steps :
+ * - Start a mini-mr cluster with 1 task-tracker having only map slots.
+ * Note that this task-tracker will take care of setup/cleanup and map
+ * tasks.
+ * - Submit a job with 1 map task and 1 reduce task
+ * - Wait for the job to finish the map task
+ * - Start a 2nd tracker that waits for a long time after contacting the JT.
+ * - Wait for the 2nd tracker to get stuck
+ * - Kill the job
+ * - Wait for the job to retire
+ * - Check if the tip mappings are cleaned up.
+ */
+ public void testJobRetireWithUnreportedTasks() throws Exception {
+ MiniMRCluster mr = null;
+ try {
+ JobConf conf = new JobConf();
+ conf.setInt(TTConfig.TT_MAP_SLOTS, 1);
+ conf.setInt(TTConfig.TT_REDUCE_SLOTS, 0);
+ mr = startCluster(conf, 1);
+ JobTracker jobtracker = mr.getJobTrackerRunner().getJobTracker();
+
+ // submit a job
+ Path inDir = new Path(testDir, "in-1");
+ Path outDir = new Path(testDir, "out-1");
+ JobConf jConf = mr.createJobConf();
+ FileInputFormat.setInputPaths(jConf, new Path[] {inDir});
+ FileOutputFormat.setOutputPath(jConf, outDir);
+ SleepJob sleepJob = new SleepJob();
+ sleepJob.setConf(jConf);
+ Job job = sleepJob.createJob(1, 1, 0, 1, 0, 1);
+
+ job.submit();
+ JobID id = JobID.downgrade(job.getStatus().getJobID());
+ JobInProgress jip = jobtracker.getJob(id);
+
+ // wait 100 secs for the map to complete
+ for (int i = 0; i < 100 && (jip.finishedMaps() < 1); i++) {
+ UtilsForTests.waitFor(1000);
+ }
+ assertEquals(jip.finishedMaps(), 1);
+
+ // start a tracker that will wait
+ LOG.info("Adding a waiting tracker");
+ TaskTrackerRunner testTrackerRunner =
+ mr.new TaskTrackerRunner(1, 1, null, mr.createJobConf()) {
+ @Override
+ TaskTracker createTaskTracker(JobConf conf) throws IOException {
+ return new WaitingTaskTracker(conf);
+ }
+ };
+ mr.addTaskTracker(testTrackerRunner);
+ LOG.info("Waiting tracker added");
+
+ WaitingTaskTracker testTT =
+ (WaitingTaskTracker)testTrackerRunner.getTaskTracker();
+
+ // wait 100 secs for the newly started task-tracker to join
+ for (int i = 0; i < 1000 && (jobtracker.taskTrackers().size() < 2); i++) {
+ UtilsForTests.waitFor(100);
+ }
+ assertEquals(jobtracker.taskTrackers().size(), 2);
+ LOG.info("Cluster is now ready");
+
+ // stop the test-tt as its no longer required
+ mr.stopTaskTracker(mr.getTaskTrackerID(testTT.getName()));
+
+ // check if a reduce task got scheduled or not
+ assertEquals("Waiting tracker joined but no reduce task got scheduled",
+ 1, jip.runningReduces());
+
+ // kill the job
+ job.killJob();
+
+ // check if the reduce task attempt status is missing
+ TaskInProgress tip = jip.getTasks(TaskType.REDUCE)[0];
+ assertNull(tip.getTaskStatus(tip.getAllTaskAttemptIDs()[0]));
+
+ // wait for the job to retire
+ waitTillRetire(id, jobtracker);
+
+ // check the taskidToTIPMap
+ for (TaskAttemptID tid : jobtracker.taskidToTIPMap.keySet()) {
+ LOG.info("TaskidToTIP mapping left over : " + tid);
+ }
+ assertEquals("'taskid' to TIP mapping still exists",
+ 0, jobtracker.taskidToTIPMap.size());
+ } finally {
+ if (mr != null) { mr.shutdown(); }
+ FileUtil.fullyDelete(new File(testDir.toString()));
+ }
+ }
+
+ /**
+ * (Mock)Test JobTracker.removeJobTasks() which is called only when the job
+ * retires.
+ */
+ public void testJobRemoval() throws Exception {
+ MiniMRCluster mr = null;
+ try {
+ JobConf conf = new JobConf();
+ mr = startCluster(conf, 0);
+ JobTracker jobtracker = mr.getJobTrackerRunner().getJobTracker();
+
+ // test map task removal
+ testRemoveJobTasks(jobtracker, conf, TaskType.MAP);
+ // test reduce task removal
+ testRemoveJobTasks(jobtracker, conf, TaskType.REDUCE);
+ // test job setup removal
+ testRemoveJobTasks(jobtracker, conf, TaskType.JOB_SETUP);
+ // test job cleanup removal
+ testRemoveJobTasks(jobtracker, conf, TaskType.JOB_CLEANUP);
+ } finally {
+ if (mr != null) { mr.shutdown();}
+ // cleanup
+ FileUtil.fullyDelete(new File(testDir.toString()));
+ }
+ }
+
+ // create a new job and add it to the jobtracker
+ private JobInProgress createAndAddJob(JobTracker jobtracker, JobConf conf) {
+ // submit a job in a fake manner
+ // get the new job-id
+ JobID id =
+ new JobID(jobtracker.getTrackerIdentifier(), jobtracker.jobs.size() + 1);
+ // create a JobInProgress for this fake job
+ JobInProgress jip = new JobInProgress(id, conf, jobtracker);
+
+ // insert this fake completed job in the jobtracker
+ jobtracker.jobs.put(id, jip);
+
+ return jip;
+ }
+
+ // create a new TaskInProgress and make it running by adding it to jobtracker
+ private TaskInProgress createAndAddTIP(JobTracker jobtracker,
+ JobInProgress jip, TaskType type) {
+ JobConf conf = jip.getJobConf();
+ JobID id = jip.getJobID();
+ // now create a fake tip for this fake job
+ TaskInProgress tip = null;
+ if (type == TaskType.MAP) {
+ tip = new TaskInProgress(id, "dummy", JobSplit.EMPTY_TASK_SPLIT,
+ jobtracker, conf, jip, 0, 1);
+ jip.maps = new TaskInProgress[] {tip};
+ } else if (type == TaskType.REDUCE) {
+ tip = new TaskInProgress(id, "dummy", jip.desiredMaps(), 0,
+ jobtracker, conf, jip, 1);
+ jip.reduces = new TaskInProgress[] {tip};
+ } else if (type == TaskType.JOB_SETUP) {
+ tip =
+ new TaskInProgress(id, "dummy", JobSplit.EMPTY_TASK_SPLIT,
+ jobtracker, conf, jip, 0, 1);
+ jip.setup = new TaskInProgress[] {tip};
+ } else if (type == TaskType.JOB_CLEANUP) {
+ tip =
+ new TaskInProgress(id, "dummy", JobSplit.EMPTY_TASK_SPLIT,
+ jobtracker, conf, jip, 0, 1);
+ jip.cleanup = new TaskInProgress[] {tip};
+ }
+ return tip;
+ }
+
+ // create a new Task for the given tip and make it running
+ private TaskAttemptID createAndAddAttempt(TaskInProgress tip, int attemptId) {
+ // create a fake attempt for this fake task
+ TaskAttemptID taskid = new TaskAttemptID(tip.getTIPId(), attemptId);
+
+ // insert this fake task into the jobtracker by making it running
+ tip.addRunningTask(taskid, "test-tt");
+
+ return taskid;
+ }
+
+ // Mock a job run such that the jobtracker is in a state similar to that
+ // resulting from an actual job run.
+ // Steps :
+ // - generate a new job-id
+ // - create and add a JobInProgress object using the fake job-id
+ // - create and add a fake tip of the passed type 't' under the fake job
+ // Note that t can be a MAP or a REDUCE or a JOB_SETUP or a JOB_CLEANUP.
+ // - create and add a fake attempt under the fake tip
+ // - remove the job from the jobtracker
+ // - check if the fake attempt is removed from the jobtracker
+ private void testRemoveJobTasks(JobTracker jobtracker, JobConf conf,
+ TaskType type) {
+ // create and submit a job
+ JobInProgress jip = createAndAddJob(jobtracker, conf);
+ // create and add a tip
+ TaskInProgress tip = createAndAddTIP(jobtracker, jip, type);
+ // create and add an attempt
+ TaskAttemptID taskid = createAndAddAttempt(tip, 0);
+
+ // this fake attempt should not have any status
+ assertNull(tip.getTaskStatus(taskid));
+
+ // remove the job tasks for this fake job from the jobtracker
+ jobtracker.removeJobTasks(jip);
+
+ // check the taskidToTIPMap
+ for (TaskAttemptID tid : jobtracker.taskidToTIPMap.keySet()) {
+ LOG.info("TaskidToTIP : " + tid);
+ }
+
+ // check if the fake attempt is removed from the jobtracker
+ assertEquals("'taskid' to TIP mapping still exists",
+ 0, jobtracker.taskidToTIPMap.size());
+ }
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java b/src/test/mapred/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java
index fb663f7..6146574 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java
@@ -56,7 +56,8 @@
Path outDir,
String input,
int numMaps,
- int numReduces) throws IOException {
+ int numReduces,
+ String sysDir) throws IOException {
FileSystem inFs = inDir.getFileSystem(conf);
FileSystem outFs = outDir.getFileSystem(conf);
outFs.delete(outDir, true);
@@ -90,14 +91,13 @@
assertFalse(FileSystem.get(conf).exists(
new Path(conf.get(JTConfig.JT_SYSTEM_DIR))));
// Check if the Job Tracker system dir is propogated to client
- String sysDir = jobClient.getSystemDir().toString();
- System.out.println("Job sys dir -->" + sysDir);
assertFalse(sysDir.contains("/tmp/subru/mapred/system"));
assertTrue(sysDir.contains("custom"));
return new TestResult(job, TestMiniMRWithDFS.readOutput(outDir, conf));
}
- static void runWordCount(MiniMRCluster mr, JobConf jobConf) throws IOException {
+ static void runWordCount(MiniMRCluster mr, JobConf jobConf, String sysDir)
+ throws IOException {
LOG.info("runWordCount");
// Run a word count example
// Keeping tasks that match this pattern
@@ -107,7 +107,7 @@
result = launchWordCount(jobConf, inDir, outDir,
"The quick brown fox\nhas many silly\n" +
"red fox sox\n",
- 3, 1);
+ 3, 1, sysDir);
assertEquals("The\t1\nbrown\t1\nfox\t2\nhas\t1\nmany\t1\n" +
"quick\t1\nred\t1\nsilly\t1\nsox\t1\n", result.output);
// Checking if the Job ran successfully in spite of different system dir config
@@ -128,7 +128,7 @@
fileSys = dfs.getFileSystem();
mr = new MiniMRCluster(taskTrackers, fileSys.getUri().toString(), 1, null, null, conf);
- runWordCount(mr, mr.createJobConf());
+ runWordCount(mr, mr.createJobConf(), conf.get("mapred.system.dir"));
} finally {
if (dfs != null) { dfs.shutdown(); }
if (mr != null) { mr.shutdown();
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestJobTrackerInstrumentation.java b/src/test/mapred/org/apache/hadoop/mapred/TestJobTrackerInstrumentation.java
index d9e9de6..15d11d8 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestJobTrackerInstrumentation.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestJobTrackerInstrumentation.java
@@ -186,8 +186,13 @@
job1.finishTask(taskAttemptID[2]);
jobTracker.finalizeJob(job1);
+ assertEquals("Mismatch in number of failed map tasks",
+ 1, mi.numMapTasksFailed);
+ assertEquals("Mismatch in number of failed reduce tasks",
+ 1, mi.numReduceTasksFailed);
+
assertEquals("Mismatch in number of blacklisted trackers",
- mi.numTrackersBlackListed, 1);
+ 1, mi.numTrackersBlackListed);
assertEquals("Mismatch in blacklisted map slots",
mi.numBlackListedMapSlots,
@@ -321,6 +326,41 @@
trackers.length - 1, mi.numTrackers);
}
+ public void testKillTasks() throws IOException {
+ int numMaps, numReds;
+ JobConf conf = new JobConf();
+ conf.setSpeculativeExecution(false);
+ conf.setMaxTaskFailuresPerTracker(1);
+ conf.setBoolean(JobContext.SETUP_CLEANUP_NEEDED, false);
+ TaskAttemptID[] taskAttemptID = new TaskAttemptID[2];
+
+ numMaps = 1;
+ numReds = 1;
+ conf.setNumMapTasks(numMaps);
+ conf.setNumReduceTasks(numReds);
+ conf.setBoolean(JobContext.SETUP_CLEANUP_NEEDED, false);
+
+ assertEquals("Mismatch in number of killed map tasks",
+ 0, mi.numMapTasksKilled);
+ assertEquals("Mismatch in number of killed reduce tasks",
+ 0, mi.numReduceTasksKilled);
+
+ FakeJobInProgress job1 = new FakeJobInProgress(conf, jobTracker);
+ job1.setClusterSize(trackers.length);
+ job1.initTasks();
+ jobTracker.addJob(job1.getJobID(), job1);
+ taskAttemptID[0] = job1.findMapTask(trackers[0]);
+ job1.killTask(taskAttemptID[0]);
+ taskAttemptID[1] = job1.findReduceTask(trackers[0]);
+ job1.killTask(taskAttemptID[1]);
+ jobTracker.finalizeJob(job1);
+
+ assertEquals("Mismatch in number of killed map tasks",
+ 1, mi.numMapTasksKilled);
+ assertEquals("Mismatch in number of killed reduce tasks",
+ 1, mi.numReduceTasksKilled);
+ }
+
static class FakeTaskScheduler extends JobQueueTaskScheduler {
public FakeTaskScheduler() {
super();
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestKillCompletedJob.java b/src/test/mapred/org/apache/hadoop/mapred/TestKillCompletedJob.java
index b668fab..8ce2283 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestKillCompletedJob.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestKillCompletedJob.java
@@ -52,7 +52,7 @@
@SuppressWarnings("deprecation")
public void testKillCompletedJob() throws IOException, InterruptedException {
job = new MyFakeJobInProgress(new JobConf(), jobTracker);
- jobTracker.addJob(job.getJobID(), job);
+ jobTracker.addJob(job.getJobID(), (JobInProgress)job);
job.status.setRunState(JobStatus.SUCCEEDED);
jobTracker.killJob(job.getJobID());
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestLocalizationWithLinuxTaskController.java b/src/test/mapred/org/apache/hadoop/mapred/TestLocalizationWithLinuxTaskController.java
index b5a0f1c..bd6a728 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestLocalizationWithLinuxTaskController.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestLocalizationWithLinuxTaskController.java
@@ -45,10 +45,15 @@
private static String taskTrackerSpecialGroup;
@Override
+ protected boolean canRun() {
+ return ClusterWithLinuxTaskController.shouldRun();
+ }
+
+ @Override
protected void setUp()
throws Exception {
- if (!ClusterWithLinuxTaskController.shouldRun()) {
+ if (!canRun()) {
return;
}
@@ -66,7 +71,8 @@
taskController.setConf(trackerFConf);
taskController.setup();
- tracker.setLocalizer(new Localizer(tracker.localFs, localDirs,
+ tracker.setTaskController(taskController);
+ tracker.setLocalizer(new Localizer(tracker.getLocalFileSystem(), localDirs,
taskController));
// Rewrite conf so as to reflect task's correct user name.
@@ -81,7 +87,7 @@
@Override
protected void tearDown()
throws Exception {
- if (!ClusterWithLinuxTaskController.shouldRun()) {
+ if (!canRun()) {
return;
}
super.tearDown();
@@ -96,21 +102,6 @@
// Do nothing.
}
- /**
- * Test the localization of a user on the TT when {@link LinuxTaskController}
- * is in use.
- */
- @Override
- public void testUserLocalization()
- throws IOException {
-
- if (!ClusterWithLinuxTaskController.shouldRun()) {
- return;
- }
-
- super.testJobLocalization();
- }
-
@Override
protected void checkUserLocalization()
throws IOException {
@@ -140,7 +131,7 @@
// Verify the distributed cache dir.
File distributedCacheDir =
new File(localDir, TaskTracker
- .getDistributedCacheDir(task.getUser()));
+ .getPrivateDistributedCacheDir(task.getUser()));
assertTrue("distributed cache dir " + distributedCacheDir
+ " doesn't exists!", distributedCacheDir.exists());
checkFilePermissions(distributedCacheDir.getAbsolutePath(),
@@ -148,21 +139,6 @@
}
}
- /**
- * Test job localization with {@link LinuxTaskController}. Also check the
- * permissions and file ownership of the job related files.
- */
- @Override
- public void testJobLocalization()
- throws IOException {
-
- if (!ClusterWithLinuxTaskController.shouldRun()) {
- return;
- }
-
- super.testJobLocalization();
- }
-
@Override
protected void checkJobLocalization()
throws IOException {
@@ -208,21 +184,6 @@
}
}
- /**
- * Test task localization with {@link LinuxTaskController}. Also check the
- * permissions and file ownership of task related files.
- */
- @Override
- public void testTaskLocalization()
- throws IOException {
-
- if (!ClusterWithLinuxTaskController.shouldRun()) {
- return;
- }
-
- super.testTaskLocalization();
- }
-
@Override
protected void checkTaskLocalization()
throws IOException {
@@ -248,16 +209,4 @@
.getUser(), taskTrackerSpecialGroup);
}
}
-
- /**
- * Test cleanup of task files with {@link LinuxTaskController}.
- */
- @Override
- public void testTaskCleanup()
- throws IOException {
- if (!ClusterWithLinuxTaskController.shouldRun()) {
- return;
- }
- super.testTaskCleanup();
- }
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestLostTracker.java b/src/test/mapred/org/apache/hadoop/mapred/TestLostTracker.java
index 9bfa818..ef38fee 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestLostTracker.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestLostTracker.java
@@ -24,6 +24,7 @@
import org.apache.hadoop.mapred.FakeObjectUtilities.FakeJobInProgress;
import org.apache.hadoop.mapred.FakeObjectUtilities.FakeJobTracker;
import org.apache.hadoop.mapred.UtilsForTests.FakeClock;
+import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
/**
@@ -47,6 +48,7 @@
conf.set(JTConfig.JT_IPC_ADDRESS, "localhost:0");
conf.set(JTConfig.JT_HTTP_ADDRESS, "0.0.0.0:0");
conf.setLong(JTConfig.JT_TRACKER_EXPIRY_INTERVAL, 1000);
+ conf.set(JTConfig.JT_MAX_TRACKER_BLACKLISTS, "1");
jobTracker = new FakeJobTracker(conf, (clock = new FakeClock()), trackers);
jobTracker.startExpireTrackersThread();
}
@@ -91,4 +93,139 @@
job.finishTask(tid[1]);
}
+
+ /**
+ * Test whether the tracker gets blacklisted after its lost.
+ */
+ public void testLostTrackerBeforeBlacklisting() throws Exception {
+ FakeObjectUtilities.establishFirstContact(jobTracker, trackers[0]);
+ TaskAttemptID[] tid = new TaskAttemptID[3];
+ JobConf conf = new JobConf();
+ conf.setNumMapTasks(1);
+ conf.setNumReduceTasks(1);
+ conf.set(JobContext.MAX_TASK_FAILURES_PER_TRACKER, "1");
+ conf.set(JobContext.SETUP_CLEANUP_NEEDED, "false");
+ FakeJobInProgress job = new FakeJobInProgress(conf, jobTracker);
+ job.initTasks();
+ job.setClusterSize(4);
+
+ // Tracker 0 gets the map task
+ tid[0] = job.findMapTask(trackers[0]);
+
+ job.finishTask(tid[0]);
+
+ // validate the total tracker count
+ assertEquals("Active tracker count mismatch",
+ 1, jobTracker.getClusterStatus(false).getTaskTrackers());
+
+ // lose the tracker
+ clock.advance(1100);
+ jobTracker.checkExpiredTrackers();
+ assertFalse("Tracker 0 not lost",
+ jobTracker.getClusterStatus(false).getActiveTrackerNames()
+ .contains(trackers[0]));
+
+ // validate the total tracker count
+ assertEquals("Active tracker count mismatch",
+ 0, jobTracker.getClusterStatus(false).getTaskTrackers());
+
+ // Tracker 1 establishes contact with JT
+ FakeObjectUtilities.establishFirstContact(jobTracker, trackers[1]);
+
+ // Tracker1 should get assigned the lost map task
+ tid[1] = job.findMapTask(trackers[1]);
+
+ assertNotNull("Map Task from Lost Tracker did not get reassigned", tid[1]);
+
+ assertEquals("Task ID of reassigned map task does not match",
+ tid[0].getTaskID().toString(), tid[1].getTaskID().toString());
+
+ // finish the map task
+ job.finishTask(tid[1]);
+
+ // finish the reduce task
+ tid[2] = job.findReduceTask(trackers[1]);
+ job.finishTask(tid[2]);
+
+ // check if job is successful
+ assertEquals("Job not successful",
+ JobStatus.SUCCEEDED, job.getStatus().getRunState());
+
+ // check if the tracker is lost
+ // validate the total tracker count
+ assertEquals("Active tracker count mismatch",
+ 1, jobTracker.getClusterStatus(false).getTaskTrackers());
+ // validate blacklisted count .. since we lost one blacklisted tracker
+ assertEquals("Blacklisted tracker count mismatch",
+ 0, jobTracker.getClusterStatus(false).getBlacklistedTrackers());
+ }
+
+ /**
+ * Test whether the tracker gets lost after its blacklisted.
+ */
+ public void testLostTrackerAfterBlacklisting() throws Exception {
+ FakeObjectUtilities.establishFirstContact(jobTracker, trackers[0]);
+ clock.advance(600);
+ TaskAttemptID[] tid = new TaskAttemptID[2];
+ JobConf conf = new JobConf();
+ conf.setNumMapTasks(1);
+ conf.setNumReduceTasks(0);
+ conf.set(JobContext.MAX_TASK_FAILURES_PER_TRACKER, "1");
+ conf.set(JobContext.SETUP_CLEANUP_NEEDED, "false");
+ FakeJobInProgress job = new FakeJobInProgress(conf, jobTracker);
+ job.initTasks();
+ job.setClusterSize(4);
+
+ // check if the tracker count is correct
+ assertEquals("Active tracker count mismatch",
+ 1, jobTracker.taskTrackers().size());
+
+ // Tracker 0 gets the map task
+ tid[0] = job.findMapTask(trackers[0]);
+ // Fail the task
+ job.failTask(tid[0]);
+
+ // Tracker 1 establishes contact with JT
+ FakeObjectUtilities.establishFirstContact(jobTracker, trackers[1]);
+ // check if the tracker count is correct
+ assertEquals("Active tracker count mismatch",
+ 2, jobTracker.taskTrackers().size());
+
+ // Tracker 1 gets the map task
+ tid[1] = job.findMapTask(trackers[1]);
+ // Finish the task and also the job
+ job.finishTask(tid[1]);
+
+ // check if job is successful
+ assertEquals("Job not successful",
+ JobStatus.SUCCEEDED, job.getStatus().getRunState());
+
+ // check if the trackers 1 got blacklisted
+ assertTrue("Tracker 0 not blacklisted",
+ jobTracker.getBlacklistedTrackers()[0].getTaskTrackerName()
+ .equals(trackers[0]));
+ // check if the tracker count is correct
+ assertEquals("Active tracker count mismatch",
+ 2, jobTracker.taskTrackers().size());
+ // validate blacklisted count
+ assertEquals("Blacklisted tracker count mismatch",
+ 1, jobTracker.getClusterStatus(false).getBlacklistedTrackers());
+
+ // Advance clock. Tracker 0 should be lost
+ clock.advance(500);
+ jobTracker.checkExpiredTrackers();
+
+ // check if the task tracker is lost
+ assertFalse("Tracker 0 not lost",
+ jobTracker.getClusterStatus(false).getActiveTrackerNames()
+ .contains(trackers[0]));
+
+ // check if the lost tracker has removed from the jobtracker
+ assertEquals("Active tracker count mismatch",
+ 1, jobTracker.taskTrackers().size());
+ // validate blacklisted count
+ assertEquals("Blacklisted tracker count mismatch",
+ 0, jobTracker.getClusterStatus(false).getBlacklistedTrackers());
+
+ }
}
\ No newline at end of file
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestMapProgress.java b/src/test/mapred/org/apache/hadoop/mapred/TestMapProgress.java
index 719b403..8580763 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestMapProgress.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestMapProgress.java
@@ -20,6 +20,9 @@
import java.io.DataOutputStream;
import java.io.File;
import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
@@ -27,10 +30,19 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.mapreduce.split.JobSplitWriter;
+import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
+import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
+import org.apache.hadoop.util.ReflectionUtils;
/**
* Validates map phase progress.
@@ -92,9 +104,9 @@
*/
class TestMapTask extends MapTask {
public TestMapTask(String jobFile, TaskAttemptID taskId,
- int partition, String splitClass, BytesWritable split,
+ int partition, TaskSplitIndex splitIndex,
int numSlotsRequired) {
- super(jobFile, taskId, partition, splitClass, split, numSlotsRequired);
+ super(jobFile, taskId, partition, splitIndex, numSlotsRequired);
}
/**
@@ -141,16 +153,20 @@
jobId = taskId.getJobID();
JobContext jContext = new JobContextImpl(job, jobId);
- Job.RawSplit[] rawSplits = LocalJobRunner.getRawSplits(jContext, job);
+ InputFormat<?, ?> input =
+ ReflectionUtils.newInstance(jContext.getInputFormatClass(), job);
- job.setUseNewMapper(true); // use new api
- for (int i = 0; i < rawSplits.length; i++) {// rawSplits.length is 1
+ List<InputSplit> splits = input.getSplits(jContext);
+ JobSplitWriter.createSplitFiles(new Path(TEST_ROOT_DIR), job, splits);
+ TaskSplitMetaInfo[] splitMetaInfo =
+ SplitMetaInfoReader.readSplitMetaInfo(jobId, fs, job, new Path(TEST_ROOT_DIR));
+ job.setUseNewMapper(true); // use new api
+ for (int i = 0; i < splitMetaInfo.length; i++) {// rawSplits.length is 1
map = new TestMapTask(
job.get(JTConfig.JT_SYSTEM_DIR, "/tmp/hadoop/mapred/system") +
jobId + "job.xml",
taskId, i,
- rawSplits[i].getClassName(),
- rawSplits[i].getBytes(), 1);
+ splitMetaInfo[i].getSplitIndex(), 1);
JobConf localConf = new JobConf(job);
map.localizeConfiguration(localConf);
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestMapRed.java b/src/test/mapred/org/apache/hadoop/mapred/TestMapRed.java
index 4da1119..805dde9 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestMapRed.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestMapRed.java
@@ -34,7 +34,6 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
@@ -308,56 +307,6 @@
}
- private static class BadPartitioner
- implements Partitioner<LongWritable,Text> {
- boolean low;
- public void configure(JobConf conf) {
- low = conf.getBoolean("test.testmapred.badpartition", true);
- }
- public int getPartition(LongWritable k, Text v, int numPartitions) {
- return low ? -1 : numPartitions;
- }
- }
-
- @Test
- public void testPartitioner() throws Exception {
- JobConf conf = new JobConf(TestMapRed.class);
- conf.setPartitionerClass(BadPartitioner.class);
- FileSystem fs = FileSystem.getLocal(conf);
- Path testdir = new Path(
- System.getProperty("test.build.data","/tmp")).makeQualified(fs);
- Path inFile = new Path(testdir, "blah/blah");
- DataOutputStream f = fs.create(inFile);
- f.writeBytes("blah blah blah\n");
- f.close();
- FileInputFormat.setInputPaths(conf, inFile);
- FileOutputFormat.setOutputPath(conf, new Path(testdir, "out"));
- conf.setMapperClass(IdentityMapper.class);
- conf.setReducerClass(IdentityReducer.class);
- conf.setOutputKeyClass(LongWritable.class);
- conf.setOutputValueClass(Text.class);
-
- // partition too low
- conf.setBoolean("test.testmapred.badpartition", true);
- boolean pass = true;
- try {
- JobClient.runJob(conf);
- } catch (IOException e) {
- pass = false;
- }
- assertFalse("should fail for partition < 0", pass);
-
- // partition too high
- conf.setBoolean("test.testmapred.badpartition", false);
- pass = true;
- try {
- JobClient.runJob(conf);
- } catch (IOException e) {
- pass = false;
- }
- assertFalse("should fail for partition >= numPartitions", pass);
- }
-
public static class NullMapper
implements Mapper<NullWritable,Text,NullWritable,Text> {
public void map(NullWritable key, Text val,
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRClasspath.java b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRClasspath.java
index e4b641b..911aa2c 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRClasspath.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRClasspath.java
@@ -38,15 +38,13 @@
public class TestMiniMRClasspath extends TestCase {
- static String launchWordCount(URI fileSys,
+ static void configureWordCount(FileSystem fs,
String jobTracker,
JobConf conf,
String input,
int numMaps,
- int numReduces) throws IOException {
- final Path inDir = new Path("/testing/wc/input");
- final Path outDir = new Path("/testing/wc/output");
- FileSystem fs = FileSystem.get(fileSys, conf);
+ int numReduces,
+ Path inDir, Path outDir) throws IOException {
fs.delete(outDir, true);
if (!fs.mkdirs(inDir)) {
throw new IOException("Mkdirs failed to create " + inDir.toString());
@@ -56,7 +54,7 @@
file.writeBytes(input);
file.close();
}
- FileSystem.setDefaultUri(conf, fileSys);
+ FileSystem.setDefaultUri(conf, fs.getUri());
conf.set(JTConfig.JT_IPC_ADDRESS, jobTracker);
conf.setJobName("wordcount");
conf.setInputFormat(TextInputFormat.class);
@@ -75,6 +73,16 @@
conf.setNumReduceTasks(numReduces);
//pass a job.jar already included in the hadoop build
conf.setJar("build/test/mapred/testjar/testjob.jar");
+ }
+
+ static String launchWordCount(URI fileSys, String jobTracker, JobConf conf,
+ String input, int numMaps, int numReduces)
+ throws IOException {
+ final Path inDir = new Path("/testing/wc/input");
+ final Path outDir = new Path("/testing/wc/output");
+ FileSystem fs = FileSystem.get(fileSys, conf);
+ configureWordCount(fs, jobTracker, conf, input, numMaps, numReduces, inDir,
+ outDir);
JobClient.runJob(conf);
StringBuffer result = new StringBuffer();
{
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRDFSSort.java b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRDFSSort.java
index 0d08e87..5058410 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRDFSSort.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRDFSSort.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.mapred;
+import java.io.DataOutputStream;
import java.io.IOException;
import junit.extensions.TestSetup;
@@ -28,7 +29,10 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
import org.apache.hadoop.mapred.lib.NullOutputFormat;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -178,4 +182,56 @@
public void testNoJvmReuse() throws Exception {
runJvmReuseTest(mrCluster.createJobConf(), false);
}
+
+ private static class BadPartitioner
+ implements Partitioner<LongWritable,Text> {
+ boolean low;
+ public void configure(JobConf conf) {
+ low = conf.getBoolean("test.testmapred.badpartition", true);
+ }
+ public int getPartition(LongWritable k, Text v, int numPartitions) {
+ return low ? -1 : numPartitions;
+ }
+ }
+
+ public void testPartitioner() throws Exception {
+ JobConf conf = mrCluster.createJobConf();
+ conf.setPartitionerClass(BadPartitioner.class);
+ conf.setNumReduceTasks(3);
+ FileSystem fs = FileSystem.get(conf);
+ Path testdir =
+ new Path("blah").makeQualified(fs.getUri(), fs.getWorkingDirectory());
+ Path inFile = new Path(testdir, "blah");
+ DataOutputStream f = fs.create(inFile);
+ f.writeBytes("blah blah blah\n");
+ f.close();
+ FileInputFormat.setInputPaths(conf, inFile);
+ FileOutputFormat.setOutputPath(conf, new Path(testdir, "out"));
+ conf.setMapperClass(IdentityMapper.class);
+ conf.setReducerClass(IdentityReducer.class);
+ conf.setOutputKeyClass(LongWritable.class);
+ conf.setOutputValueClass(Text.class);
+ conf.setMaxMapAttempts(1);
+
+ // partition too low
+ conf.setBoolean("test.testmapred.badpartition", true);
+ boolean pass = true;
+ try {
+ JobClient.runJob(conf);
+ } catch (IOException e) {
+ pass = false;
+ }
+ assertFalse("should fail for partition < 0", pass);
+
+ // partition too high
+ conf.setBoolean("test.testmapred.badpartition", false);
+ pass = true;
+ try {
+ JobClient.runJob(conf);
+ } catch (IOException e) {
+ pass = false;
+ }
+ assertFalse("should fail for partition >= numPartitions", pass);
+ }
+
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRLocalFS.java b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRLocalFS.java
index 4614d36..4b711fe 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRLocalFS.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRLocalFS.java
@@ -61,6 +61,10 @@
MiniMRCluster mr = null;
try {
mr = new MiniMRCluster(2, "file:///", 3);
+ // make cleanup inline sothat validation of existence of these directories
+ // can be done
+ mr.setInlineCleanupThreads();
+
TestMiniMRWithDFS.runPI(mr, mr.createJobConf());
// run the wordcount example with caching
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRWithDFS.java b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRWithDFS.java
index 6f5bf81..396b6bf 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRWithDFS.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRWithDFS.java
@@ -36,12 +36,14 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.security.UnixUserGroupInformation;
import org.apache.hadoop.util.StringUtils;
@@ -289,8 +291,11 @@
long hdfsWrite =
counters.findCounter(Task.FILESYSTEM_COUNTER_GROUP,
Task.getFileSystemCounterNames("hdfs")[1]).getCounter();
+ long rawSplitBytesRead =
+ counters.findCounter(TaskCounter.SPLIT_RAW_BYTES).getCounter();
assertEquals(result.output.length(), hdfsWrite);
- assertEquals(input.length(), hdfsRead);
+ // add the correction factor of 234 as the input split is also streamed
+ assertEquals(input.length() + rawSplitBytesRead, hdfsRead);
// Run a job with input and output going to localfs even though the
// default fs is hdfs.
@@ -324,6 +329,9 @@
dfs = new MiniDFSCluster(conf, 4, true, null);
fileSys = dfs.getFileSystem();
mr = new MiniMRCluster(taskTrackers, fileSys.getUri().toString(), 1);
+ // make cleanup inline sothat validation of existence of these directories
+ // can be done
+ mr.setInlineCleanupThreads();
runPI(mr, mr.createJobConf());
runWordCount(mr, mr.createJobConf());
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRWithDFSWithDistinctUsers.java b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRWithDFSWithDistinctUsers.java
index 15f0813..1f52a45 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRWithDFSWithDistinctUsers.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRWithDFSWithDistinctUsers.java
@@ -23,9 +23,16 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
+import org.apache.hadoop.mapreduce.JobSubmissionFiles;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.mapreduce.split.JobSplitWriter;
+import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo;
import org.apache.hadoop.security.*;
/**
@@ -43,7 +50,11 @@
}
static JobConf createJobConf(MiniMRCluster mr, UnixUserGroupInformation ugi) {
- JobConf jobconf = mr.createJobConf();
+ return createJobConf(mr.createJobConf(), ugi);
+ }
+
+ static JobConf createJobConf(JobConf conf, UnixUserGroupInformation ugi) {
+ JobConf jobconf = new JobConf(conf);
UnixUserGroupInformation.saveToConf(jobconf,
UnixUserGroupInformation.UGI_PROPERTY_NAME, ugi);
return jobconf;
@@ -55,6 +66,50 @@
fs.setPermission(p, new FsPermission((short)0777));
}
+ // runs a sample job as a user (ugi)
+ RunningJob runJobAsUser(JobConf job, UserGroupInformation ugi)
+ throws Exception {
+ ClientProtocol jobSubmitClient =
+ TestSubmitJob.getJobSubmitClient(job, ugi);
+ org.apache.hadoop.mapreduce.JobID id = jobSubmitClient.getNewJobID();
+
+ InputSplit[] splits = computeJobSplit(JobID.downgrade(id), job);
+ Path jobSubmitDir = new Path(id.toString());
+ FileSystem fs = jobSubmitDir.getFileSystem(job);
+ jobSubmitDir = jobSubmitDir.makeQualified(fs);
+ uploadJobFiles(JobID.downgrade(id), splits, jobSubmitDir, job);
+
+ jobSubmitClient.submitJob(id, jobSubmitDir.toString(), null);
+
+ JobClient jc = new JobClient(job);
+ return jc.getJob(JobID.downgrade(id));
+ }
+
+ // a helper api for split computation
+ private InputSplit[] computeJobSplit(JobID id, JobConf conf)
+ throws IOException {
+ InputSplit[] splits =
+ conf.getInputFormat().getSplits(conf, conf.getNumMapTasks());
+ conf.setNumMapTasks(splits.length);
+ return splits;
+ }
+
+
+ // a helper api for split submission
+ private void uploadJobFiles(JobID id, InputSplit[] splits,
+ Path jobSubmitDir, JobConf conf)
+ throws IOException {
+ Path confLocation = JobSubmissionFiles.getJobConfPath(jobSubmitDir);
+ JobSplitWriter.createSplitFiles(jobSubmitDir, conf, splits);
+ FileSystem fs = confLocation.getFileSystem(conf);
+ FsPermission perm = new FsPermission((short)0700);
+
+ // localize conf
+ DataOutputStream confOut = FileSystem.create(fs, confLocation, perm);
+ conf.writeXml(confOut);
+ confOut.close();
+ }
+
public void testDistinctUsers() throws Exception {
MiniDFSCluster dfs = null;
MiniMRCluster mr = null;
@@ -71,15 +126,32 @@
UnixUserGroupInformation.login().getUserName(), false);
mr = new MiniMRCluster(0, 0, 4, dfs.getFileSystem().getUri().toString(),
1, null, null, MR_UGI);
+ String jobTrackerName = "localhost:" + mr.getJobTrackerPort();
- JobConf pi = createJobConf(mr, PI_UGI);
- TestMiniMRWithDFS.runPI(mr, pi);
+ JobConf job1 = mr.createJobConf();
+ String input = "The quick brown fox\nhas many silly\n"
+ + "red fox sox\n";
+ Path inDir = new Path("/testing/distinct/input");
+ Path outDir = new Path("/testing/distinct/output");
+ TestMiniMRClasspath.configureWordCount(fs, jobTrackerName, job1,
+ input, 2, 1, inDir, outDir);
+ job1 = createJobConf(job1, PI_UGI);
+ runJobAsUser(job1, PI_UGI);
- JobConf wc = createJobConf(mr, WC_UGI);
- TestMiniMRWithDFS.runWordCount(mr, wc);
+ JobConf job2 = mr.createJobConf();
+ Path inDir2 = new Path("/testing/distinct/input2");
+ Path outDir2 = new Path("/testing/distinct/output2");
+ TestMiniMRClasspath.configureWordCount(fs, jobTrackerName, job2,
+ input, 2, 1, inDir2, outDir2);
+ job2 = createJobConf(job2, WC_UGI);
+ runJobAsUser(job2, WC_UGI);
} finally {
if (dfs != null) { dfs.shutdown(); }
if (mr != null) { mr.shutdown();}
}
}
+
+ public void testRestartWithDistinctUsers() {
+
+ }
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestNodeRefresh.java b/src/test/mapred/org/apache/hadoop/mapred/TestNodeRefresh.java
index 38f36ce..06a7b29 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestNodeRefresh.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestNodeRefresh.java
@@ -22,7 +22,9 @@
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStreamWriter;
+import java.util.HashMap;
import java.util.HashSet;
+import java.util.Map;
import java.util.Set;
import junit.framework.TestCase;
@@ -31,6 +33,7 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
@@ -61,6 +64,14 @@
int numExcluded, Configuration conf)
throws IOException {
try {
+ // create fake mapping for the groups
+ Map<String, String[]> u2g_map = new HashMap<String, String[]> (1);
+ u2g_map.put("user1", new String[] {"user1" });
+ u2g_map.put("user2", new String[] {"user2" });
+ u2g_map.put("user3", new String[] {"abc" });
+ u2g_map.put("user4", new String[] {"supergroup" });
+ DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
+
conf.setBoolean("dfs.replication.considerLoad", false);
// prepare hosts info
@@ -168,7 +179,7 @@
// refresh with super user
success = false;
UserGroupInformation ugi_super =
- TestMiniMRWithDFSWithDistinctUsers.createUGI("user2", true);
+ TestMiniMRWithDFSWithDistinctUsers.createUGI("user4", true);
client = getClient(conf, ugi_super);
try {
client.refreshNodes();
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestQueueManager.java b/src/test/mapred/org/apache/hadoop/mapred/TestQueueManager.java
index 841ab28..11612ee 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestQueueManager.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestQueueManager.java
@@ -250,6 +250,28 @@
}
@Test
+ public void testMissingConfigFile() throws Exception {
+ checkForConfigFile(); // deletes file
+
+ try {
+ new QueueManager(CONFIG);
+ fail("Should throw an exception for missing file when " +
+ "explicitly passed.");
+ } catch (RuntimeException re) {
+ }
+
+ // If we just want to pick up the queues from the class loader
+ // it should fall through to the default. The class loader is set to
+ // load CONFIG for the "mapred-queues.xml" resource, but it's missing
+ // so should fall through to mapred-queues-default.xml
+ QueueManager qm = new QueueManager();
+ List<JobQueueInfo> rootQueues =
+ qm.getRoot().getJobQueueInfo().getChildren();
+ assertEquals(1, rootQueues.size());
+ assertEquals("default", rootQueues.get(0).getQueueName());
+ }
+
+ @Test
public void testEmptyProperties() throws Exception {
checkForConfigFile();
Document doc = createDocument();
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestRackAwareTaskPlacement.java b/src/test/mapred/org/apache/hadoop/mapred/TestRackAwareTaskPlacement.java
index 3f7cf01..dc80ab1 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestRackAwareTaskPlacement.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestRackAwareTaskPlacement.java
@@ -31,8 +31,10 @@
import org.apache.hadoop.mapred.UtilsForTests.FakeClock;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobCounter;
+import org.apache.hadoop.mapreduce.JobSubmissionFiles;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.mapreduce.Job.RawSplit;
+import org.apache.hadoop.mapreduce.split.JobSplit;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.apache.hadoop.net.StaticMapping;
@@ -93,38 +95,36 @@
@Override
public void initTasks() throws IOException {
- Job.RawSplit[] splits = createSplits();
- numMapTasks = splits.length;
- createMapTasks(null, splits);
- nonRunningMapCache = createCache(splits, maxLevel);
+ TaskSplitMetaInfo[] taskSplitMetaInfo = createSplits(jobId);
+ numMapTasks = taskSplitMetaInfo.length;
+ createMapTasks(null, taskSplitMetaInfo);
+ nonRunningMapCache = createCache(taskSplitMetaInfo, maxLevel);
tasksInited.set(true);
this.status.setRunState(JobStatus.RUNNING);
}
-
- protected Job.RawSplit[] createSplits() throws IOException {
- Job.RawSplit[] splits = new Job.RawSplit[numMaps];
+ @Override
+ protected TaskSplitMetaInfo [] createSplits(
+ org.apache.hadoop.mapreduce.JobID jobId) throws IOException {
+ TaskSplitMetaInfo[] splits = new TaskSplitMetaInfo[numMaps];
// Hand code for now.
// M0,2,3 reside in Host1
// M1 resides in Host3
// M4 resides in Host4
String[] splitHosts0 = new String[] { allHosts[0] };
- for (int i = 0; i < numMaps; i++) {
- splits[i] = new Job.RawSplit();
- splits[i].setDataLength(0);
- }
-
- splits[0].setLocations(splitHosts0);
- splits[2].setLocations(splitHosts0);
- splits[3].setLocations(splitHosts0);
-
String[] splitHosts1 = new String[] { allHosts[2] };
- splits[1].setLocations(splitHosts1);
-
String[] splitHosts2 = new String[] { allHosts[3] };
- splits[4].setLocations(splitHosts2);
+ for (int i = 0; i < numMaps; i++) {
+ if (i == 0 || i == 2 || i == 3) {
+ splits[i] = new TaskSplitMetaInfo(splitHosts0, 0, 0);
+ } else if (i == 1) {
+ splits[i] = new TaskSplitMetaInfo(splitHosts1, 0, 0);
+ } else if (i == 4) {
+ splits[i] = new TaskSplitMetaInfo(splitHosts2, 0, 0);
+ }
+ }
return splits;
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestRecoveryManager.java b/src/test/mapred/org/apache/hadoop/mapred/TestRecoveryManager.java
index 4851ab2..a8ced9f 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestRecoveryManager.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestRecoveryManager.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.mapred;
+import java.io.File;
import java.io.IOException;
import junit.framework.TestCase;
@@ -48,10 +49,11 @@
/**
* Tests the {@link JobTracker} against the exceptions thrown in
* {@link JobTracker.RecoveryManager}. It does the following :
- * - submits 2 jobs
+ * - submits 3 jobs
* - kills the jobtracker
* - Garble job.xml for one job causing it to fail in constructor
* and job.split for another causing it to fail in init.
+ * - delete the job temp/submit dir
* - restarts the jobtracker
* - checks if the jobtraker starts normally
*/
@@ -79,7 +81,7 @@
// wait for 50%
UtilsForTests.waitForJobHalfDone(rJob1);
-
+
JobConf job2 = mr.createJobConf();
UtilsForTests.configureWaitingJobConf(job2,
@@ -101,26 +103,15 @@
// delete the job.xml of job #1 causing the job to fail in submit Job
//while recovery itself
Path jobFile =
- new Path(sysDir, rJob1.getID().toString() + Path.SEPARATOR + "job.xml");
- LOG.info("Deleting job.xml file : " + jobFile.toString());
+ new Path(sysDir, rJob1.getID().toString() + "/" + JobTracker.JOB_INFO_FILE);
+ LOG.info("Deleting job token file : " + jobFile.toString());
fs.delete(jobFile, false); // delete the job.xml file
- // create the job.xml file with 0 bytes
+ // create the job token file with 1 byte
FSDataOutputStream out = fs.create(jobFile);
out.write(1);
out.close();
-
- // delete the job.split of job #2 causing the job to fail in initTasks
- Path jobSplitFile =
- new Path(sysDir, rJob2.getID().toString() + Path.SEPARATOR + "job.split");
- LOG.info("Deleting job.split file : " + jobSplitFile.toString());
- fs.delete(jobSplitFile, false); // delete the job.split file
- // create the job.split file with 0 bytes
- out = fs.create(jobSplitFile);
- out.write(1);
- out.close();
-
// make sure that the jobtracker is in recovery mode
mr.getJobTrackerConf().setBoolean(JTConfig.JT_RESTART_ENABLED, true);
// start the jobtracker
@@ -290,7 +281,7 @@
conf.set(JTConfig.JT_IPC_ADDRESS, "localhost:0");
conf.set(JTConfig.JT_HTTP_ADDRESS, "127.0.0.1:0");
- JobTracker jobtracker = new JobTracker(conf);
+ JobTracker jobtracker = JobTracker.startTracker(conf);
// now check if the update restart count works fine or not
boolean failed = false;
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestResourceEstimation.java b/src/test/mapred/org/apache/hadoop/mapred/TestResourceEstimation.java
index 1099320..2b47a38 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestResourceEstimation.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestResourceEstimation.java
@@ -17,10 +17,9 @@
*/
package org.apache.hadoop.mapred;
-import junit.framework.TestCase;
+import org.apache.hadoop.mapreduce.split.JobSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Job.RawSplit;
+import junit.framework.TestCase;
public class TestResourceEstimation extends TestCase {
@@ -47,8 +46,8 @@
TaskStatus ts = new MapTaskStatus();
ts.setOutputSize(singleMapOutputSize);
- Job.RawSplit split = new Job.RawSplit();
- split.setDataLength(0);
+ JobSplit.TaskSplitMetaInfo split =
+ new JobSplit.TaskSplitMetaInfo(new String[0], 0, 0);
TaskInProgress tip = new TaskInProgress(jid, "", split, null, jc, jip, 0, 1);
re.updateWithCompletedTask(ts, tip);
}
@@ -83,8 +82,9 @@
TaskStatus ts = new MapTaskStatus();
ts.setOutputSize(singleMapOutputSize);
- Job.RawSplit split = new Job.RawSplit();
- split.setDataLength(singleMapInputSize);
+ JobSplit.TaskSplitMetaInfo split =
+ new JobSplit.TaskSplitMetaInfo(new String[0], 0,
+ singleMapInputSize);
TaskInProgress tip = new TaskInProgress(jid, "", split, null, jc, jip, 0, 1);
re.updateWithCompletedTask(ts, tip);
}
@@ -95,8 +95,8 @@
//add one more map task with input size as 0
TaskStatus ts = new MapTaskStatus();
ts.setOutputSize(singleMapOutputSize);
- Job.RawSplit split = new Job.RawSplit();
- split.setDataLength(0);
+ JobSplit.TaskSplitMetaInfo split =
+ new JobSplit.TaskSplitMetaInfo(new String[0], 0, 0);
TaskInProgress tip = new TaskInProgress(jid, "", split, null, jc, jip, 0, 1);
re.updateWithCompletedTask(ts, tip);
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestSetupAndCleanupFailure.java b/src/test/mapred/org/apache/hadoop/mapred/TestSetupAndCleanupFailure.java
index f6c4122..07706b1 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestSetupAndCleanupFailure.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestSetupAndCleanupFailure.java
@@ -27,6 +27,7 @@
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.mapred.lib.IdentityMapper;
import org.apache.hadoop.mapred.lib.IdentityReducer;
+import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
@@ -155,7 +156,7 @@
JobTracker jt = mr.getJobTrackerRunner().getJobTracker();
JobInProgress jip = jt.getJob(job.getID());
// get the running setup task id
- TaskAttemptID setupID = getRunningTaskID(jip.getSetupTasks());
+ TaskAttemptID setupID = getRunningTaskID(jip.getTasks(TaskType.JOB_SETUP));
if (commandLineKill) {
killTaskFromCommandLine(job, setupID, jt);
} else {
@@ -172,7 +173,8 @@
} catch (InterruptedException ie) {}
}
// get the running cleanup task id
- TaskAttemptID cleanupID = getRunningTaskID(jip.getCleanupTasks());
+ TaskAttemptID cleanupID =
+ getRunningTaskID(jip.getTasks(TaskType.JOB_CLEANUP));
if (commandLineKill) {
killTaskFromCommandLine(job, cleanupID, jt);
} else {
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestSetupTaskScheduling.java b/src/test/mapred/org/apache/hadoop/mapred/TestSetupTaskScheduling.java
index f041e22..01cfdc1 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestSetupTaskScheduling.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestSetupTaskScheduling.java
@@ -25,6 +25,7 @@
import org.apache.hadoop.mapred.FakeObjectUtilities.FakeTaskInProgress;
import org.apache.hadoop.mapred.FakeObjectUtilities.FakeJobTracker;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.mapreduce.split.JobSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.TaskType;
@@ -60,7 +61,7 @@
@Override
public synchronized void initTasks() throws IOException {
super.initTasks();
- Job.RawSplit emptySplit = new Job.RawSplit();
+ JobSplit.TaskSplitMetaInfo emptySplit = new JobSplit.TaskSplitMetaInfo();
setup = new TaskInProgress[2];
setup[0] = new TaskInProgress(getJobID(), "test", emptySplit,
jobtracker, getJobConf(), this, numMapTasks + 1, 1);
@@ -109,12 +110,13 @@
@Override
public synchronized void initTasks() throws IOException {
super.initTasks();
- Job.RawSplit emptySplit = new Job.RawSplit();
+
final int numSlotsPerTask = 2;
maps = new TaskInProgress[1];
reduces = new TaskInProgress[1];
- maps[0] = new FakeTaskInProgress(getJobID(), "test", emptySplit,
+ maps[0] = new FakeTaskInProgress(getJobID(), "test",
+ JobSplit.EMPTY_TASK_SPLIT,
jobtracker, getJobConf(), this, 0, numSlotsPerTask);
TaskAttemptID attemptId = new TaskAttemptID(maps[0].getTIPId(), 0);
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestSetupWorkDir.java b/src/test/mapred/org/apache/hadoop/mapred/TestSetupWorkDir.java
new file mode 100644
index 0000000..607867d
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestSetupWorkDir.java
@@ -0,0 +1,88 @@
+/**
+ * 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.mapred;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+
+public class TestSetupWorkDir extends TestCase {
+ private static final Log LOG =
+ LogFactory.getLog(TestSetupWorkDir.class);
+
+ /**
+ * Create a file in the given dir and set permissions r_xr_xr_x sothat no one
+ * can delete it directly(without doing chmod).
+ * Creates dir/subDir and dir/subDir/file
+ */
+ static void createFileAndSetPermissions(JobConf jobConf, Path dir)
+ throws IOException {
+ Path subDir = new Path(dir, "subDir");
+ FileSystem fs = FileSystem.getLocal(jobConf);
+ fs.mkdirs(subDir);
+ Path p = new Path(subDir, "file");
+ DataOutputStream out = fs.create(p);
+ out.writeBytes("dummy input");
+ out.close();
+ // no write permission for subDir and subDir/file
+ try {
+ int ret = 0;
+ if((ret = FileUtil.chmod(subDir.toUri().getPath(), "a=rx", true)) != 0) {
+ LOG.warn("chmod failed for " + subDir + ";retVal=" + ret);
+ }
+ } catch(InterruptedException e) {
+ LOG.warn("Interrupted while doing chmod for " + subDir);
+ }
+ }
+
+ /**
+ * Validates if setupWorkDir is properly cleaning up contents of workDir.
+ */
+ public void testSetupWorkDir() throws IOException {
+ Path rootDir = new Path(System.getProperty("test.build.data", "/tmp"),
+ "testSetupWorkDir");
+ Path myWorkDir = new Path(rootDir, "./work");
+ JobConf jConf = new JobConf();
+ FileSystem fs = FileSystem.getLocal(jConf);
+ if (fs.exists(myWorkDir)) {
+ fs.delete(myWorkDir, true);
+ }
+ if (!fs.mkdirs(myWorkDir)) {
+ throw new IOException("Unable to create workDir " + myWorkDir);
+ }
+
+ // create {myWorkDir}/subDir/file and set 555 perms for subDir and file
+ createFileAndSetPermissions(jConf, myWorkDir);
+
+ TaskRunner.deleteDirContents(jConf, new File(myWorkDir.toUri().getPath()));
+
+ assertTrue("Contents of " + myWorkDir + " are not cleaned up properly.",
+ fs.listStatus(myWorkDir).length == 0);
+
+ // cleanup
+ fs.delete(rootDir, true);
+ }
+}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestSeveral.java b/src/test/mapred/org/apache/hadoop/mapred/TestSeveral.java
index 859553c..3d7ae89 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestSeveral.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestSeveral.java
@@ -92,6 +92,8 @@
TestMiniMRWithDFSWithDistinctUsers.mkdir(fs, "/user");
TestMiniMRWithDFSWithDistinctUsers.mkdir(fs, "/mapred");
+ TestMiniMRWithDFSWithDistinctUsers.mkdir(fs,
+ conf.get(JTConfig.JT_STAGING_AREA_ROOT));
UnixUserGroupInformation MR_UGI =
TestMiniMRWithDFSWithDistinctUsers.createUGI(
@@ -105,6 +107,10 @@
mrCluster = new MiniMRCluster(0, 0,
numTT, dfs.getFileSystem().getUri().toString(),
1, null, null, MR_UGI, new JobConf());
+ // make cleanup inline sothat validation of existence of these directories
+ // can be done
+ mrCluster.setInlineCleanupThreads();
+
mrCluster.getJobTrackerRunner().getJobTracker()
.addJobInProgressListener(myListener);
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestShuffleJobToken.java b/src/test/mapred/org/apache/hadoop/mapred/TestShuffleJobToken.java
index 1c5f31b..70722dc 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestShuffleJobToken.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestShuffleJobToken.java
@@ -27,9 +27,14 @@
import java.net.URLConnection;
import java.security.GeneralSecurityException;
+import javax.crypto.SecretKey;
+
import org.apache.hadoop.http.HttpServer;
-import org.apache.hadoop.mapreduce.security.JobTokens;
+import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
+import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
+import org.apache.hadoop.security.token.Token;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -40,6 +45,7 @@
private static URL baseUrl;
private static File dir;
private static final String JOB_ID = "job_20091117075357176_0001";
+ private static final String BAD_JOB_ID = "job_20091117075357176_0002";
// create fake url
private URL getMapOutputURL(String host) throws MalformedURLException {
@@ -86,25 +92,26 @@
URL url = getMapOutputURL(baseUrl.toString());
String enc_str = SecureShuffleUtils.buildMsgFrom(url);
URLConnection connectionGood = url.openConnection();
-
- // create key
- byte [] key= SecureShuffleUtils.getNewEncodedKey();
- // create fake TaskTracker - needed for keys storage
- JobTokens jt = new JobTokens();
- jt.setShuffleJobToken(key);
TaskTracker tt = new TaskTracker();
+ JobTokenSecretManager jtSecretManager = new JobTokenSecretManager();
+ // create fake TaskTracker - needed for keys storage
+ JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(JOB_ID));
+ Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>(identifier,
+ jtSecretManager);
+ SecretKey tokenSecret = JobTokenSecretManager.createSecretKey(jt.getPassword());
addJobToken(tt, JOB_ID, jt); // fake id
server.setAttribute("task.tracker", tt);
// encode the url
- SecureShuffleUtils mac = new SecureShuffleUtils(key);
- String urlHashGood = mac.generateHash(enc_str.getBytes()); // valid hash
+ String urlHashGood = SecureShuffleUtils.generateHash(enc_str.getBytes(), tokenSecret); // valid hash
// another the key
- byte [] badKey= SecureShuffleUtils.getNewEncodedKey();
- mac = new SecureShuffleUtils(badKey);
- String urlHashBad = mac.generateHash(enc_str.getBytes()); // invalid hash
+ JobTokenIdentifier badIdentifier = new JobTokenIdentifier(new Text(BAD_JOB_ID));
+ Token<JobTokenIdentifier> badToken = new Token<JobTokenIdentifier>(badIdentifier,
+ jtSecretManager);
+ SecretKey badSecret = JobTokenSecretManager.createSecretKey(badToken.getPassword());
+ String urlHashBad = SecureShuffleUtils.generateHash(enc_str.getBytes(), badSecret); // invalid hash
// put url hash into http header
connectionGood.addRequestProperty(SecureShuffleUtils.HTTP_HEADER_URL_HASH, urlHashGood);
@@ -135,13 +142,13 @@
}
}
/*Note that this method is there for a unit testcase (TestShuffleJobToken)*/
- void addJobToken(TaskTracker tt, String jobIdStr, JobTokens jt) {
+ void addJobToken(TaskTracker tt, String jobIdStr, Token<JobTokenIdentifier> token) {
JobID jobId = JobID.forName(jobIdStr);
TaskTracker.RunningJob rJob = new TaskTracker.RunningJob(jobId);
- rJob.jobTokens = jt;
synchronized (tt.runningJobs) {
tt.runningJobs.put(jobId, rJob);
}
+ tt.getJobTokenSecretManager().addTokenForJob(jobIdStr, token);
}
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestSpeculativeExecution.java b/src/test/mapred/org/apache/hadoop/mapred/TestSpeculativeExecution.java
index 0d9ae5c..e9fc156 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestSpeculativeExecution.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestSpeculativeExecution.java
@@ -23,6 +23,8 @@
import junit.framework.Test;
import junit.framework.TestCase;
import junit.framework.TestSuite;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapred.FakeObjectUtilities.FakeJobInProgress;
import org.apache.hadoop.mapred.FakeObjectUtilities.FakeJobTracker;
@@ -40,6 +42,8 @@
}
};
static SpecFakeClock clock;
+ static final Log LOG = LogFactory.getLog(TestSpeculativeExecution.class);
+
static String trackers[] = new String[] {"tracker_tracker1:1000",
"tracker_tracker2:1000", "tracker_tracker3:1000",
@@ -65,6 +69,68 @@
return setup;
}
+ public void testRunningTaskCountWithSpeculation() throws IOException {
+ TaskAttemptID[] taskAttemptID = new TaskAttemptID[8];
+ JobConf conf = new JobConf();
+ conf.setSpeculativeExecution(true);
+ conf.setNumMapTasks(3);
+ conf.setNumReduceTasks(3);
+ conf.setFloat(JobContext.SPECULATIVE_SLOWTASK_THRESHOLD, 0.5f);
+ FakeJobInProgress job = new FakeJobInProgress(conf, jobTracker);
+ job.initTasks();
+
+ //Check for runningMap counts first
+ //schedule maps
+ taskAttemptID[0] = job.findMapTask(trackers[0]);
+ taskAttemptID[1] = job.findMapTask(trackers[1]);
+ taskAttemptID[2] = job.findMapTask(trackers[2]);
+
+ clock.advance(5000);
+ job.finishTask(taskAttemptID[0]);
+ clock.advance(1000);
+ job.finishTask(taskAttemptID[1]);
+ clock.advanceBySpeculativeLag();
+
+ //we should get a speculative task now
+ taskAttemptID[3] = job.findMapTask(trackers[3]);
+ int oldRunningMap = job.runningMaps();
+ LOG.info("No of running maps before fail was " + oldRunningMap);
+ job.failTask(taskAttemptID[2]);
+ assertEquals(
+ "Running maps count should be updated from " + oldRunningMap + " to " +
+ (oldRunningMap - 1), job.runningMaps(), oldRunningMap - 1);
+ LOG.info(" Job running maps after fail " + job.runningMaps());
+
+ clock.advance(5000);
+ job.finishTask(taskAttemptID[3]);
+
+ //check for runningReduce count.
+ taskAttemptID[4] = job.findReduceTask(trackers[0]);
+ taskAttemptID[5] = job.findReduceTask(trackers[1]);
+ taskAttemptID[6] = job.findReduceTask(trackers[2]);
+
+ clock.advance(5000);
+ job.finishTask(taskAttemptID[4]);
+ clock.advance(1000);
+ job.finishTask(taskAttemptID[5]);
+
+ clock.advanceBySpeculativeLag();
+ taskAttemptID[7] = job.findReduceTask(trackers[4]);
+
+ int oldRunningReduces = job.runningReduces();
+ job.failTask(taskAttemptID[6]);
+ LOG.info(
+ " No of running Reduces before fail " + oldRunningReduces);
+ LOG.info(
+ " No of runing reduces after fail " + job.runningReduces());
+ assertEquals(
+ "Running reduces count should be updated from " + oldRunningReduces +
+ " to " + (oldRunningReduces - 1), job.runningReduces(),
+ oldRunningReduces - 1);
+
+ job.finishTask(taskAttemptID[7]);
+ }
+
public void testIsSlowTracker() throws IOException {
TaskAttemptID[] taskAttemptID = new TaskAttemptID[20];
JobConf conf = new JobConf();
@@ -171,7 +237,7 @@
taskAttemptID[5] = job.findMapTask(trackers[4]);
assertEquals(taskAttemptID[5].getTaskID().getId(),4);
}
-
+
/*
* Tests the fact that we only launch a limited number of speculative tasks,
* even though we have a lot of tasks in RUNNING state
@@ -219,7 +285,7 @@
taskAttemptID[i] = job.findMapTask(trackers[1]);
clock.advance(2000);
if (taskAttemptID[i] != null) {
- //add some good progress constantly for the different
+ //add some good progress constantly for the different
//task-attempts so that
//the tasktracker doesn't get into the slow trackers category
job.progressMade(taskAttemptID[i], 0.99f);
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java b/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java
index 065b1a1..ec153d1 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java
@@ -17,25 +17,81 @@
*/
package org.apache.hadoop.mapred;
+import java.io.DataOutputStream;
import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.SleepJob;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo;
import org.apache.hadoop.util.ToolRunner;
import junit.framework.TestCase;
+/**
+ * Test job submission. This test checks if
+ * - basic : job submission via jobclient
+ * - cleanup : job client crashes while submitting
+ * - invalid job config
+ * - invalid memory config
+ *
+ */
public class TestSubmitJob extends TestCase {
- private MiniMRCluster miniMRCluster;
+ static final Log LOG = LogFactory.getLog(TestSubmitJob.class);
+
+ private MiniMRCluster mrCluster;
- @Override
- protected void tearDown()
- throws Exception {
- if (miniMRCluster != null) {
- miniMRCluster.shutdown();
- }
+ private MiniDFSCluster dfsCluster;
+ private JobTracker jt;
+ private FileSystem fs;
+ private static Path TEST_DIR =
+ new Path(System.getProperty("test.build.data","/tmp"),
+ "job-submission-testing");
+ private static int numSlaves = 1;
+
+ private void startCluster() throws Exception {
+ super.setUp();
+ Configuration conf = new Configuration();
+ dfsCluster = new MiniDFSCluster(conf, numSlaves, true, null);
+ JobConf jConf = new JobConf(conf);
+ jConf.setLong("mapred.job.submission.expiry.interval", 6 * 1000);
+ mrCluster = new MiniMRCluster(0, 0, numSlaves,
+ dfsCluster.getFileSystem().getUri().toString(), 1, null, null, null,
+ jConf);
+ jt = mrCluster.getJobTrackerRunner().getJobTracker();
+ fs = FileSystem.get(mrCluster.createJobConf());
+ }
+
+ private void stopCluster() throws Exception {
+ mrCluster.shutdown();
+ mrCluster = null;
+ dfsCluster.shutdown();
+ dfsCluster = null;
+ jt = null;
+ fs = null;
}
/**
@@ -56,9 +112,9 @@
jtConf.setLong(JTConfig.JT_MAX_REDUCEMEMORY_MB,
4 * 1024L);
- miniMRCluster = new MiniMRCluster(0, "file:///", 0, null, null, jtConf);
+ mrCluster = new MiniMRCluster(0, "file:///", 0, null, null, jtConf);
- JobConf clusterConf = miniMRCluster.createJobConf();
+ JobConf clusterConf = mrCluster.createJobConf();
// No map-memory configuration
JobConf jobConf = new JobConf(clusterConf);
@@ -85,6 +141,9 @@
jobConf.setMemoryForReduceTask(5 * 1024L);
runJobAndVerifyFailure(jobConf, 1 * 1024L, 5 * 1024L,
"Exceeds the cluster's max-memory-limit.");
+
+ mrCluster.shutdown();
+ mrCluster = null;
}
private void runJobAndVerifyFailure(JobConf jobConf, long memForMapTasks,
@@ -110,4 +169,127 @@
+ " - doesn't contain expected message - " + overallExpectedMsg, msg
.contains(overallExpectedMsg));
}
-}
+
+ static ClientProtocol getJobSubmitClient(JobConf conf,
+ UserGroupInformation ugi)
+ throws IOException {
+ return (ClientProtocol) RPC.getProxy(ClientProtocol.class,
+ ClientProtocol.versionID, JobTracker.getAddress(conf), ugi,
+ conf, NetUtils.getSocketFactory(conf, ClientProtocol.class));
+ }
+
+ static org.apache.hadoop.hdfs.protocol.ClientProtocol getDFSClient(
+ Configuration conf, UserGroupInformation ugi)
+ throws IOException {
+ return (org.apache.hadoop.hdfs.protocol.ClientProtocol)
+ RPC.getProxy(org.apache.hadoop.hdfs.protocol.ClientProtocol.class,
+ org.apache.hadoop.hdfs.protocol.ClientProtocol.versionID,
+ NameNode.getAddress(conf), ugi,
+ conf,
+ NetUtils.getSocketFactory(conf,
+ org.apache.hadoop.hdfs.protocol.ClientProtocol.class));
+ }
+
+ /**
+ * Submit a job and check if the files are accessible to other users.
+ */
+ public void testSecureJobExecution() throws Exception {
+ LOG.info("Testing secure job submission/execution");
+ MiniDFSCluster dfs = null;
+ MiniMRCluster mr = null;
+ try {
+ Configuration conf = new Configuration();
+ UnixUserGroupInformation.saveToConf(conf,
+ UnixUserGroupInformation.UGI_PROPERTY_NAME,
+ TestMiniMRWithDFSWithDistinctUsers.DFS_UGI);
+ dfs = new MiniDFSCluster(conf, 1, true, null);
+ FileSystem fs = dfs.getFileSystem();
+ TestMiniMRWithDFSWithDistinctUsers.mkdir(fs, "/user");
+ TestMiniMRWithDFSWithDistinctUsers.mkdir(fs, "/mapred");
+ TestMiniMRWithDFSWithDistinctUsers.mkdir(fs,
+ conf.get(JTConfig.JT_STAGING_AREA_ROOT));
+ UnixUserGroupInformation MR_UGI =
+ TestMiniMRWithDFSWithDistinctUsers.createUGI(
+ UnixUserGroupInformation.login().getUserName(), false);
+ mr = new MiniMRCluster(0, 0, 1, dfs.getFileSystem().getUri().toString(),
+ 1, null, null, MR_UGI);
+ JobTracker jt = mr.getJobTrackerRunner().getJobTracker();
+ String jobTrackerName = "localhost:" + mr.getJobTrackerPort();
+
+ // cleanup
+ dfs.getFileSystem().delete(TEST_DIR, true);
+
+ final Path mapSignalFile = new Path(TEST_DIR, "map-signal");
+ final Path reduceSignalFile = new Path(TEST_DIR, "reduce-signal");
+
+ // create a ugi for user 1
+ UnixUserGroupInformation user1 =
+ TestMiniMRWithDFSWithDistinctUsers.createUGI("user1", false);
+ Path inDir = new Path("/user/input");
+ Path outDir = new Path("/user/output");
+ JobConf job =
+ TestMiniMRWithDFSWithDistinctUsers.createJobConf(mr, user1);
+
+ UtilsForTests.configureWaitingJobConf(job, inDir, outDir, 2, 0,
+ "test-submit-job", mapSignalFile.toString(),
+ reduceSignalFile.toString());
+ job.set(UtilsForTests.getTaskSignalParameter(true),
+ mapSignalFile.toString());
+ job.set(UtilsForTests.getTaskSignalParameter(false),
+ reduceSignalFile.toString());
+ LOG.info("Submit job as the actual user (" + user1.getUserName() + ")");
+ JobClient jClient = new JobClient(job);
+ RunningJob rJob = jClient.submitJob(job);
+ JobID id = rJob.getID();
+ LOG.info("Running job " + id);
+
+ // create user2
+ UnixUserGroupInformation user2 =
+ TestMiniMRWithDFSWithDistinctUsers.createUGI("user2", false);
+ JobConf conf_other =
+ TestMiniMRWithDFSWithDistinctUsers.createJobConf(mr, user2);
+ org.apache.hadoop.hdfs.protocol.ClientProtocol client =
+ getDFSClient(conf_other, user2);
+
+ // try accessing mapred.system.dir/jobid/*
+ boolean failed = false;
+ try {
+ Path path = new Path(new URI(jt.getSystemDir()).getPath());
+ LOG.info("Try listing the mapred-system-dir as the user ("
+ + user2.getUserName() + ")");
+ client.getListing(path.toString());
+ } catch (IOException ioe) {
+ failed = true;
+ }
+ assertTrue("JobTracker system dir is accessible to others", failed);
+ // try accessing ~/.staging/jobid/*
+ failed = false;
+ JobInProgress jip = jt.getJob(id);
+ Path jobSubmitDirpath =
+ new Path(jip.getJobConf().get("mapreduce.job.dir"));
+ try {
+ LOG.info("Try accessing the job folder for job " + id + " as the user ("
+ + user2.getUserName() + ")");
+ client.getListing(jobSubmitDirpath.toString());
+ } catch (IOException ioe) {
+ failed = true;
+ }
+ assertTrue("User's staging folder is accessible to others", failed);
+ UtilsForTests.signalTasks(dfs, fs, true, mapSignalFile.toString(),
+ reduceSignalFile.toString());
+ // wait for job to be done
+ UtilsForTests.waitTillDone(jClient);
+
+ // check if the staging area is cleaned up
+ LOG.info("Check if job submit dir is cleanup or not");
+ assertFalse(fs.exists(jobSubmitDirpath));
+ } finally {
+ if (mr != null) {
+ mr.shutdown();
+ }
+ if (dfs != null) {
+ dfs.shutdown();
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestTTMemoryReporting.java b/src/test/mapred/org/apache/hadoop/mapred/TestTTMemoryReporting.java
deleted file mode 100644
index 9fe77ad..0000000
--- a/src/test/mapred/org/apache/hadoop/mapred/TestTTMemoryReporting.java
+++ /dev/null
@@ -1,229 +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.
- */
-package org.apache.hadoop.mapred;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.mapreduce.SleepJob;
-import org.apache.hadoop.util.LinuxMemoryCalculatorPlugin;
-import org.apache.hadoop.util.MemoryCalculatorPlugin;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
-
-import junit.framework.TestCase;
-
-/**
- * This test class tests the functionality related to configuring, reporting
- * and computing memory related parameters in a Map/Reduce cluster.
- *
- * Each test sets up a {@link MiniMRCluster} with a locally defined
- * {@link org.apache.hadoop.mapred.TaskScheduler}. This scheduler validates
- * the memory related configuration is correctly computed and reported from
- * the tasktracker in
- * {@link org.apache.hadoop.mapred.TaskScheduler#assignTasks(TaskTrackerStatus)}.
- */
-public class TestTTMemoryReporting extends TestCase {
-
- static final Log LOG = LogFactory.getLog(TestTTMemoryReporting.class);
-
- private MiniMRCluster miniMRCluster;
-
- /**
- * Fake scheduler to test the proper reporting of memory values by TT
- */
- public static class FakeTaskScheduler extends JobQueueTaskScheduler {
-
- private boolean hasPassed = true;
- private String message;
-
- public FakeTaskScheduler() {
- super();
- }
-
- public boolean hasTestPassed() {
- return hasPassed;
- }
-
- public String getFailureMessage() {
- return message;
- }
-
- @Override
- public List<Task> assignTasks(TaskTracker taskTracker)
- throws IOException {
- TaskTrackerStatus status = taskTracker.getStatus();
- long totalVirtualMemoryOnTT =
- getConf().getLong("totalVmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
- long totalPhysicalMemoryOnTT =
- getConf().getLong("totalPmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
- long mapSlotMemorySize =
- getConf().getLong("mapSlotMemorySize", JobConf.DISABLED_MEMORY_LIMIT);
- long reduceSlotMemorySize =
- getConf()
- .getLong("reduceSlotMemorySize", JobConf.DISABLED_MEMORY_LIMIT);
-
- long reportedTotalVirtualMemoryOnTT =
- status.getResourceStatus().getTotalVirtualMemory();
- long reportedTotalPhysicalMemoryOnTT =
- status.getResourceStatus().getTotalPhysicalMemory();
- long reportedMapSlotMemorySize =
- status.getResourceStatus().getMapSlotMemorySizeOnTT();
- long reportedReduceSlotMemorySize =
- status.getResourceStatus().getReduceSlotMemorySizeOnTT();
-
- message =
- "expected memory values : (totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
- + "mapSlotMemSize, reduceSlotMemorySize) = ("
- + totalVirtualMemoryOnTT + ", " + totalPhysicalMemoryOnTT + ","
- + mapSlotMemorySize + "," + reduceSlotMemorySize + ")";
- message +=
- "\nreported memory values : (totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
- + "reportedMapSlotMemorySize, reportedReduceSlotMemorySize) = ("
- + reportedTotalVirtualMemoryOnTT
- + ", "
- + reportedTotalPhysicalMemoryOnTT
- + ","
- + reportedMapSlotMemorySize
- + ","
- + reportedReduceSlotMemorySize
- + ")";
- LOG.info(message);
- if (totalVirtualMemoryOnTT != reportedTotalVirtualMemoryOnTT
- || totalPhysicalMemoryOnTT != reportedTotalPhysicalMemoryOnTT
- || mapSlotMemorySize != reportedMapSlotMemorySize
- || reduceSlotMemorySize != reportedReduceSlotMemorySize) {
- hasPassed = false;
- }
- return super.assignTasks(taskTracker);
- }
- }
-
- /**
- * Test that verifies default values are configured and reported correctly.
- *
- * @throws Exception
- */
- public void testDefaultMemoryValues()
- throws Exception {
- JobConf conf = new JobConf();
- try {
- // Memory values are disabled by default.
- conf.setClass(
- org.apache.hadoop.mapred.TaskTracker.MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
- DummyMemoryCalculatorPlugin.class, MemoryCalculatorPlugin.class);
- setUpCluster(conf);
- runSleepJob(miniMRCluster.createJobConf());
- verifyTestResults();
- } finally {
- tearDownCluster();
- }
- }
-
- /**
- * Test that verifies that configured values are reported correctly.
- *
- * @throws Exception
- */
- public void testConfiguredMemoryValues()
- throws Exception {
- JobConf conf = new JobConf();
- conf.setLong("totalVmemOnTT", 4 * 1024 * 1024 * 1024L);
- conf.setLong("totalPmemOnTT", 2 * 1024 * 1024 * 1024L);
- conf.setLong("mapSlotMemorySize", 1 * 512L);
- conf.setLong("reduceSlotMemorySize", 1 * 1024L);
-
- conf.setClass(
- org.apache.hadoop.mapred.TaskTracker.MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
- DummyMemoryCalculatorPlugin.class, MemoryCalculatorPlugin.class);
- conf.setLong(DummyMemoryCalculatorPlugin.MAXVMEM_TESTING_PROPERTY,
- 4 * 1024 * 1024 * 1024L);
- conf.setLong(DummyMemoryCalculatorPlugin.MAXPMEM_TESTING_PROPERTY,
- 2 * 1024 * 1024 * 1024L);
- conf.setLong(MRConfig.MAPMEMORY_MB, 512L);
- conf.setLong(MRConfig.REDUCEMEMORY_MB, 1024L);
-
- try {
- setUpCluster(conf);
- JobConf jobConf = miniMRCluster.createJobConf();
- jobConf.setMemoryForMapTask(1 * 1024L);
- jobConf.setMemoryForReduceTask(2 * 1024L);
- runSleepJob(jobConf);
- verifyTestResults();
- } finally {
- tearDownCluster();
- }
- }
-
- /**
- * Test that verifies that total memory values are calculated and reported
- * correctly.
- *
- * @throws Exception
- */
- public void testMemoryValuesOnLinux()
- throws Exception {
- if (!System.getProperty("os.name").startsWith("Linux")) {
- return;
- }
-
- JobConf conf = new JobConf();
- LinuxMemoryCalculatorPlugin plugin = new LinuxMemoryCalculatorPlugin();
- conf.setLong("totalVmemOnTT", plugin.getVirtualMemorySize());
- conf.setLong("totalPmemOnTT", plugin.getPhysicalMemorySize());
-
- try {
- setUpCluster(conf);
- runSleepJob(miniMRCluster.createJobConf());
- verifyTestResults();
- } finally {
- tearDownCluster();
- }
- }
-
- private void setUpCluster(JobConf conf)
- throws Exception {
- conf.setClass(JTConfig.JT_TASK_SCHEDULER,
- TestTTMemoryReporting.FakeTaskScheduler.class, TaskScheduler.class);
- conf.set(JTConfig.JT_IPC_HANDLER_COUNT, "1");
- miniMRCluster = new MiniMRCluster(1, "file:///", 3, null, null, conf);
- }
-
- private void runSleepJob(JobConf conf) throws Exception {
- String[] args = { "-m", "1", "-r", "1",
- "-mt", "10", "-rt", "10" };
- ToolRunner.run(conf, new SleepJob(), args);
- }
-
- private void verifyTestResults() {
- FakeTaskScheduler scheduler =
- (FakeTaskScheduler)miniMRCluster.getJobTrackerRunner().
- getJobTracker().getTaskScheduler();
- assertTrue(scheduler.getFailureMessage(), scheduler.hasTestPassed());
- }
-
- private void tearDownCluster() {
- if (miniMRCluster != null) {
- miniMRCluster.shutdown();
- }
- }
-}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestTTResourceReporting.java b/src/test/mapred/org/apache/hadoop/mapred/TestTTResourceReporting.java
new file mode 100644
index 0000000..cdc7d32
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestTTResourceReporting.java
@@ -0,0 +1,313 @@
+/**
+ * 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.mapred;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapreduce.SleepJob;
+import org.apache.hadoop.mapreduce.util.LinuxResourceCalculatorPlugin;
+import org.apache.hadoop.mapreduce.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
+
+import junit.framework.TestCase;
+import org.junit.Test;
+import org.junit.After;
+
+/**
+ * This test class tests the functionality related to configuring, reporting
+ * and computing memory related parameters in a Map/Reduce cluster.
+ *
+ * Each test sets up a {@link MiniMRCluster} with a locally defined
+ * {@link org.apache.hadoop.mapred.TaskScheduler}. This scheduler validates
+ * the memory related configuration is correctly computed and reported from
+ * the tasktracker in
+ * {@link org.apache.hadoop.mapred.TaskScheduler#assignTasks(TaskTrackerStatus)}.
+ */
+public class TestTTResourceReporting extends TestCase {
+
+ static final Log LOG = LogFactory.getLog(TestTTResourceReporting.class);
+
+ private MiniMRCluster miniMRCluster;
+
+ /**
+ * Fake scheduler to test the proper reporting of memory values by TT
+ */
+ public static class FakeTaskScheduler extends JobQueueTaskScheduler {
+
+ private boolean hasPassed = true;
+ private boolean hasDynamicValuePassed = true;
+ private String message;
+
+ public FakeTaskScheduler() {
+ super();
+ }
+
+ public boolean hasTestPassed() {
+ return hasPassed;
+ }
+
+ public boolean hasDynamicTestPassed() {
+ return hasDynamicValuePassed;
+ }
+
+ public String getFailureMessage() {
+ return message;
+ }
+
+ @Override
+ public List<Task> assignTasks(TaskTracker taskTracker)
+ throws IOException {
+ TaskTrackerStatus status = taskTracker.getStatus();
+ long totalVirtualMemoryOnTT =
+ getConf().getLong("totalVmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
+ long totalPhysicalMemoryOnTT =
+ getConf().getLong("totalPmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
+ long mapSlotMemorySize =
+ getConf().getLong("mapSlotMemorySize", JobConf.DISABLED_MEMORY_LIMIT);
+ long reduceSlotMemorySize =
+ getConf()
+ .getLong("reduceSlotMemorySize", JobConf.DISABLED_MEMORY_LIMIT);
+ long availableVirtualMemoryOnTT =
+ getConf().getLong("availableVmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
+ long availablePhysicalMemoryOnTT =
+ getConf().getLong("availablePmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
+ long cumulativeCpuTime =
+ getConf().getLong("cumulativeCpuTime", TaskTrackerStatus.UNAVAILABLE);
+ long cpuFrequency =
+ getConf().getLong("cpuFrequency", TaskTrackerStatus.UNAVAILABLE);
+ int numProcessors =
+ getConf().getInt("numProcessors", TaskTrackerStatus.UNAVAILABLE);
+ float cpuUsage =
+ getConf().getFloat("cpuUsage", TaskTrackerStatus.UNAVAILABLE);
+
+ long reportedTotalVirtualMemoryOnTT =
+ status.getResourceStatus().getTotalVirtualMemory();
+ long reportedTotalPhysicalMemoryOnTT =
+ status.getResourceStatus().getTotalPhysicalMemory();
+ long reportedMapSlotMemorySize =
+ status.getResourceStatus().getMapSlotMemorySizeOnTT();
+ long reportedReduceSlotMemorySize =
+ status.getResourceStatus().getReduceSlotMemorySizeOnTT();
+ long reportedAvailableVirtualMemoryOnTT =
+ status.getResourceStatus().getAvailabelVirtualMemory();
+ long reportedAvailablePhysicalMemoryOnTT =
+ status.getResourceStatus().getAvailablePhysicalMemory();
+ long reportedCumulativeCpuTime =
+ status.getResourceStatus().getCumulativeCpuTime();
+ long reportedCpuFrequency = status.getResourceStatus().getCpuFrequency();
+ int reportedNumProcessors = status.getResourceStatus().getNumProcessors();
+ float reportedCpuUsage = status.getResourceStatus().getCpuUsage();
+
+ message =
+ "expected memory values : "
+ + "(totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
+ + "availableVirtualMemoryOnTT, availablePhysicalMemoryOnTT, "
+ + "mapSlotMemSize, reduceSlotMemorySize, cumulativeCpuTime, "
+ + "cpuFrequency, numProcessors) = ("
+ + totalVirtualMemoryOnTT + ", "
+ + totalPhysicalMemoryOnTT + ","
+ + availableVirtualMemoryOnTT + ", "
+ + availablePhysicalMemoryOnTT + ","
+ + mapSlotMemorySize + ","
+ + reduceSlotMemorySize + ","
+ + cumulativeCpuTime + ","
+ + cpuFrequency + ","
+ + numProcessors + ","
+ + cpuUsage
+ +")";
+ message +=
+ "\nreported memory values : "
+ + "(totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
+ + "availableVirtualMemoryOnTT, availablePhysicalMemoryOnTT, "
+ + "reportedMapSlotMemorySize, reportedReduceSlotMemorySize, "
+ + "reportedCumulativeCpuTime, reportedCpuFrequency, "
+ + "reportedNumProcessors) = ("
+ + reportedTotalVirtualMemoryOnTT + ", "
+ + reportedTotalPhysicalMemoryOnTT + ","
+ + reportedAvailableVirtualMemoryOnTT + ", "
+ + reportedAvailablePhysicalMemoryOnTT + ","
+ + reportedMapSlotMemorySize + ","
+ + reportedReduceSlotMemorySize + ","
+ + reportedCumulativeCpuTime + ","
+ + reportedCpuFrequency + ","
+ + reportedNumProcessors + ","
+ + reportedCpuUsage
+ + ")";
+ hasPassed = true;
+ hasDynamicValuePassed = true;
+ LOG.info(message);
+ if (totalVirtualMemoryOnTT != reportedTotalVirtualMemoryOnTT
+ || totalPhysicalMemoryOnTT != reportedTotalPhysicalMemoryOnTT
+ || mapSlotMemorySize != reportedMapSlotMemorySize
+ || reduceSlotMemorySize != reportedReduceSlotMemorySize
+ || cpuFrequency != reportedCpuFrequency
+ || numProcessors != reportedNumProcessors) {
+ hasPassed = false;
+ }
+ // These values changes every moment on the node so it can only be
+ // tested by DummyMemoryCalculatorPlugin. Need to check them separately
+ if (availableVirtualMemoryOnTT != reportedAvailableVirtualMemoryOnTT
+ || availablePhysicalMemoryOnTT != reportedAvailablePhysicalMemoryOnTT
+ || cumulativeCpuTime != reportedCumulativeCpuTime
+ || cpuUsage != reportedCpuUsage) {
+ hasDynamicValuePassed = false;
+ }
+ return super.assignTasks(taskTracker);
+ }
+ }
+
+ /**
+ * Test that verifies default values are configured and reported correctly.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testDefaultResourceValues()
+ throws Exception {
+ JobConf conf = new JobConf();
+ try {
+ // Memory values are disabled by default.
+ conf.setClass(
+ org.apache.hadoop.mapreduce.server.tasktracker.TTConfig.TT_RESOURCE_CALCULATOR_PLUGIN,
+ DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
+ setUpCluster(conf);
+ runSleepJob(miniMRCluster.createJobConf());
+ verifyTestResults();
+ } finally {
+ tearDownCluster();
+ }
+ }
+
+ /**
+ * Test that verifies that configured values are reported correctly.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testConfiguredResourceValues()
+ throws Exception {
+ JobConf conf = new JobConf();
+ conf.setLong("totalVmemOnTT", 4 * 1024 * 1024 * 1024L);
+ conf.setLong("totalPmemOnTT", 2 * 1024 * 1024 * 1024L);
+ conf.setLong("mapSlotMemorySize", 1 * 512L);
+ conf.setLong("reduceSlotMemorySize", 1 * 1024L);
+ conf.setLong("availableVmemOnTT", 4 * 1024 * 1024 * 1024L);
+ conf.setLong("availablePmemOnTT", 2 * 1024 * 1024 * 1024L);
+ conf.setLong("cumulativeCpuTime", 10000L);
+ conf.setLong("cpuFrequency", 2000000L);
+ conf.setInt("numProcessors", 8);
+ conf.setFloat("cpuUsage", 15.5F);
+
+ conf.setClass(
+ org.apache.hadoop.mapreduce.server.tasktracker.TTConfig.TT_RESOURCE_CALCULATOR_PLUGIN,
+ DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
+ conf.setLong(DummyResourceCalculatorPlugin.MAXVMEM_TESTING_PROPERTY,
+ 4 * 1024 * 1024 * 1024L);
+ conf.setLong(DummyResourceCalculatorPlugin.MAXPMEM_TESTING_PROPERTY,
+ 2 * 1024 * 1024 * 1024L);
+ conf.setLong(MRConfig.MAPMEMORY_MB, 512L);
+ conf.setLong(MRConfig.REDUCEMEMORY_MB, 1024L);
+ conf.setLong(DummyResourceCalculatorPlugin.CUMULATIVE_CPU_TIME, 10000L);
+ conf.setLong(DummyResourceCalculatorPlugin.CPU_FREQUENCY, 2000000L);
+ conf.setInt(DummyResourceCalculatorPlugin.NUM_PROCESSORS, 8);
+ conf.setFloat(DummyResourceCalculatorPlugin.CPU_USAGE, 15.5F);
+
+ try {
+ setUpCluster(conf);
+ JobConf jobConf = miniMRCluster.createJobConf();
+ jobConf.setMemoryForMapTask(1 * 1024L);
+ jobConf.setMemoryForReduceTask(2 * 1024L);
+ runSleepJob(jobConf);
+ verifyTestResults();
+ } finally {
+ tearDownCluster();
+ }
+ }
+
+ /**
+ * Test that verifies that total memory values are calculated and reported
+ * correctly.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testResourceValuesOnLinux()
+ throws Exception {
+ if (!System.getProperty("os.name").startsWith("Linux")) {
+ return;
+ }
+
+ JobConf conf = new JobConf();
+ LinuxResourceCalculatorPlugin plugin = new LinuxResourceCalculatorPlugin();
+ // In this case, we only check these four fields because they are static
+ conf.setLong("totalVmemOnTT", plugin.getVirtualMemorySize());
+ conf.setLong("totalPmemOnTT", plugin.getPhysicalMemorySize());
+ conf.setLong("cpuFrequency", plugin.getCpuFrequency());
+ conf.setLong("numProcessors", plugin.getNumProcessors());
+
+ try {
+ setUpCluster(conf);
+ runSleepJob(miniMRCluster.createJobConf());
+ verifyTestResults(true);
+ } finally {
+ tearDownCluster();
+ }
+ }
+
+ private void setUpCluster(JobConf conf)
+ throws Exception {
+ conf.setClass(JTConfig.JT_TASK_SCHEDULER,
+ TestTTResourceReporting.FakeTaskScheduler.class, TaskScheduler.class);
+ conf.set(JTConfig.JT_IPC_HANDLER_COUNT, "1");
+ miniMRCluster = new MiniMRCluster(1, "file:///", 3, null, null, conf);
+ }
+
+ private void runSleepJob(JobConf conf) throws Exception {
+ String[] args = { "-m", "1", "-r", "1",
+ "-mt", "10", "-rt", "10" };
+ ToolRunner.run(conf, new SleepJob(), args);
+ }
+
+ private void verifyTestResults() {
+ verifyTestResults(false);
+ }
+
+ private void verifyTestResults(boolean excludeDynamic) {
+ FakeTaskScheduler scheduler =
+ (FakeTaskScheduler)miniMRCluster.getJobTrackerRunner().
+ getJobTracker().getTaskScheduler();
+ assertTrue(scheduler.getFailureMessage(), scheduler.hasTestPassed());
+ if (!excludeDynamic) {
+ assertTrue(scheduler.getFailureMessage(),
+ scheduler.hasDynamicTestPassed());
+ }
+ }
+
+ @After
+ private void tearDownCluster() {
+ if (miniMRCluster != null) {
+ miniMRCluster.shutdown();
+ }
+ }
+}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerBlacklisting.java b/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerBlacklisting.java
index 1b1b6df..52774d4 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerBlacklisting.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerBlacklisting.java
@@ -179,7 +179,7 @@
responseId++;
}
- public void AtestTrackerBlacklistingForJobFailures() throws Exception {
+ public void testTrackerBlacklistingForJobFailures() throws Exception {
runBlackListingJob(jobTracker, trackers);
assertEquals("Tracker 1 not blacklisted", jobTracker
.getBlacklistedTrackerCount(), 1);
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java b/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java
index c170aa6..f32c40b 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java
@@ -21,6 +21,7 @@
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
+import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.List;
import java.util.jar.JarOutputStream;
@@ -36,15 +37,18 @@
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.security.JobTokens;
+import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
+import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
import org.apache.hadoop.mapreduce.server.tasktracker.Localizer;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.mapred.JvmManager.JvmEnv;
import org.apache.hadoop.mapred.TaskController.JobInitializationContext;
import org.apache.hadoop.mapred.TaskController.TaskControllerContext;
import org.apache.hadoop.mapred.TaskTracker.TaskInProgress;
+import org.apache.hadoop.mapred.UtilsForTests.InlineCleanupQueue;
import junit.framework.TestCase;
@@ -55,9 +59,14 @@
*/
public class TestTaskTrackerLocalization extends TestCase {
- private File TEST_ROOT_DIR;
+ private static File TEST_ROOT_DIR =
+ new File(System.getProperty("test.build.data", "/tmp"));
private File ROOT_MAPRED_LOCAL_DIR;
private File HADOOP_LOG_DIR;
+ private static File PERMISSION_SCRIPT_DIR;
+ private static File PERMISSION_SCRIPT_FILE;
+ private static final String PERMISSION_SCRIPT_CONTENT = "ls -l -d $1 | " +
+ "awk '{print $1\":\"$3\":\"$4}'";
private int numLocalDirs = 6;
private static final Log LOG =
@@ -78,36 +87,20 @@
protected File[] attemptLogFiles;
protected JobConf localizedTaskConf;
- class InlineCleanupQueue extends CleanupQueue {
- List<Path> stalePaths = new ArrayList<Path>();
-
- public InlineCleanupQueue() {
- // do nothing
- }
-
- @Override
- public void addToQueue(FileSystem fs, Path... paths) {
- // delete in-line
- for (Path p : paths) {
- try {
- LOG.info("Trying to delete the path " + p);
- if (!fs.delete(p, true)) {
- LOG.warn("Stale path " + p.toUri().getPath());
- stalePaths.add(p);
- }
- } catch (IOException e) {
- LOG.warn("Caught exception while deleting path "
- + p.toUri().getPath());
- LOG.info(StringUtils.stringifyException(e));
- stalePaths.add(p);
- }
- }
- }
+ /**
+ * Dummy method in this base class. Only derived classes will define this
+ * method for checking if a test can be run.
+ */
+ protected boolean canRun() {
+ return true;
}
@Override
protected void setUp()
throws Exception {
+ if (!canRun()) {
+ return;
+ }
TEST_ROOT_DIR =
new File(System.getProperty("test.build.data", "/tmp"), getClass()
.getSimpleName());
@@ -147,7 +140,8 @@
tracker.setConf(trackerFConf);
// for test case system FS is the local FS
- tracker.localFs = tracker.systemFS = FileSystem.getLocal(trackerFConf);
+ tracker.systemFS = FileSystem.getLocal(trackerFConf);
+ tracker.setLocalFileSystem(tracker.systemFS);
tracker.systemDirectory = new Path(TEST_ROOT_DIR.getAbsolutePath());
taskTrackerUGI = UserGroupInformation.login(trackerFConf);
@@ -158,7 +152,7 @@
taskId =
new TaskAttemptID(jtIdentifier, jobId.getId(), TaskType.MAP, 1, 0);
task =
- new MapTask(jobConfFile.toURI().toString(), taskId, 1, null, null, 1);
+ new MapTask(jobConfFile.toURI().toString(), taskId, 1, null, 1);
task.setConf(job.getConfiguration()); // Set conf. Set user name in particular.
// create jobTokens file
@@ -169,11 +163,40 @@
taskController.setConf(trackerFConf);
taskController.setup();
- tracker.setLocalizer(new Localizer(tracker.localFs, localDirs,
+ tracker.setTaskController(taskController);
+ tracker.setLocalizer(new Localizer(tracker.getLocalFileSystem(), localDirs,
taskController));
}
/**
+ * static block setting up the permission script which would be used by the
+ * checkFilePermissions
+ */
+ static {
+ PERMISSION_SCRIPT_DIR = new File(TEST_ROOT_DIR, "permission_script_dir");
+ PERMISSION_SCRIPT_FILE = new File(PERMISSION_SCRIPT_DIR, "getperms.sh");
+
+ if(PERMISSION_SCRIPT_FILE.exists()) {
+ PERMISSION_SCRIPT_FILE.delete();
+ }
+
+ if(PERMISSION_SCRIPT_DIR.exists()) {
+ PERMISSION_SCRIPT_DIR.delete();
+ }
+
+ PERMISSION_SCRIPT_DIR.mkdir();
+
+ try {
+ PrintWriter writer = new PrintWriter(PERMISSION_SCRIPT_FILE);
+ writer.write(PERMISSION_SCRIPT_CONTENT);
+ writer.close();
+ } catch (FileNotFoundException fe) {
+ fail();
+ }
+ PERMISSION_SCRIPT_FILE.setExecutable(true, true);
+ }
+
+ /**
* @param job
* @throws IOException
* @throws FileNotFoundException
@@ -222,10 +245,10 @@
if(!dir.exists())
assertTrue("faild to create dir="+dir.getAbsolutePath(), dir.mkdirs());
- File jobTokenFile = new File(dir, JobTokens.JOB_TOKEN_FILENAME);
+ File jobTokenFile = new File(dir, SecureShuffleUtils.JOB_TOKEN_FILENAME);
FileOutputStream fos = new FileOutputStream(jobTokenFile);
java.io.DataOutputStream out = new java.io.DataOutputStream(fos);
- JobTokens jt = new JobTokens();
+ Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
jt.write(out); // writing empty file, we don't the keys for this test
out.close();
}
@@ -233,15 +256,31 @@
@Override
protected void tearDown()
throws Exception {
+ if (!canRun()) {
+ return;
+ }
FileUtil.fullyDelete(TEST_ROOT_DIR);
}
protected static String[] getFilePermissionAttrs(String path)
throws IOException {
- String output = Shell.execCommand("stat", path, "-c", "%A:%U:%G");
+ String[] command = {"bash",PERMISSION_SCRIPT_FILE.getAbsolutePath(), path};
+ String output=Shell.execCommand(command);
return output.split(":|\n");
}
+
+ /**
+ * Utility method to check permission of a given path. Requires the permission
+ * script directory to be setup in order to call.
+ *
+ *
+ * @param path
+ * @param expectedPermissions
+ * @param expectedOwnerUser
+ * @param expectedOwnerGroup
+ * @throws IOException
+ */
static void checkFilePermissions(String path, String expectedPermissions,
String expectedOwnerUser, String expectedOwnerGroup)
throws IOException {
@@ -264,6 +303,9 @@
*/
public void testTaskControllerSetup()
throws IOException {
+ if (!canRun()) {
+ return;
+ }
// Task-controller is already set up in the test's setup method. Now verify.
for (String localDir : localDirs) {
@@ -287,7 +329,9 @@
*/
public void testUserLocalization()
throws IOException {
-
+ if (!canRun()) {
+ return;
+ }
// /////////// The main method being tested
tracker.getLocalizer().initializeUserDirs(task.getUser());
// ///////////
@@ -341,7 +385,7 @@
// Verify the distributed cache dir.
File distributedCacheDir =
new File(localDir, TaskTracker
- .getDistributedCacheDir(task.getUser()));
+ .getPrivateDistributedCacheDir(task.getUser()));
assertTrue("distributed cache dir " + distributedCacheDir
+ " doesn't exists!", distributedCacheDir.exists());
checkFilePermissions(distributedCacheDir.getAbsolutePath(),
@@ -358,7 +402,9 @@
*/
public void testJobLocalization()
throws IOException {
-
+ if (!canRun()) {
+ return;
+ }
tracker.getLocalizer().initializeUserDirs(task.getUser());
// /////////// The main method being tested
@@ -452,7 +498,9 @@
*/
public void testTaskLocalization()
throws IOException {
-
+ if (!canRun()) {
+ return;
+ }
tracker.getLocalizer().initializeUserDirs(task.getUser());
localizedJobConf = tracker.localizeJobFiles(task);
@@ -568,14 +616,102 @@
}
/**
+ * Validates the removal of $taskid and $tasid/work under mapred-local-dir
+ * in cases where those directories cannot be deleted without adding
+ * write permission to the newly created directories under $taskid and
+ * $taskid/work
+ * Also see TestSetupWorkDir.createFileAndSetPermissions for details
+ */
+ void validateRemoveFiles(boolean needCleanup, boolean jvmReuse,
+ TaskInProgress tip) throws IOException {
+ // create files and set permissions 555. Verify if task controller sets
+ // the permissions for TT to delete the taskDir or workDir
+ String dir = (!needCleanup || jvmReuse) ?
+ TaskTracker.getTaskWorkDir(task.getUser(), task.getJobID().toString(),
+ taskId.toString(), task.isTaskCleanupTask())
+ : TaskTracker.getLocalTaskDir(task.getUser(), task.getJobID().toString(),
+ taskId.toString(), task.isTaskCleanupTask());
+
+ Path[] paths = tracker.getLocalFiles(localizedJobConf, dir);
+ for (Path p : paths) {
+ if (tracker.getLocalFileSystem().exists(p)) {
+ TestSetupWorkDir.createFileAndSetPermissions(localizedJobConf, p);
+ }
+ }
+
+ InlineCleanupQueue cleanupQueue = new InlineCleanupQueue();
+ tracker.setCleanupThread(cleanupQueue);
+
+ tip.removeTaskFiles(needCleanup, taskId);
+
+ if (jvmReuse) {
+ // work dir should still exist and cleanup queue should be empty
+ assertTrue("cleanup queue is not empty after removeTaskFiles() in case "
+ + "of jvm reuse.", cleanupQueue.isQueueEmpty());
+ boolean workDirExists = false;
+ for (Path p : paths) {
+ if (tracker.getLocalFileSystem().exists(p)) {
+ workDirExists = true;
+ }
+ }
+ assertTrue("work dir does not exist in case of jvm reuse", workDirExists);
+
+ // now try to delete the work dir and verify that there are no stale paths
+ JvmManager.deleteWorkDir(tracker, task);
+ }
+ tracker.removeJobFiles(task.getUser(), jobId.toString());
+
+ assertTrue("Some task files are not deleted!! Number of stale paths is "
+ + cleanupQueue.stalePaths.size(), cleanupQueue.stalePaths.size() == 0);
+ }
+
+ /**
+ * Validates if task cleanup is done properly for a succeeded task
* @throws IOException
*/
public void testTaskCleanup()
throws IOException {
+ if (!canRun()) {
+ return;
+ }
+ testTaskCleanup(false, false);// no needCleanup; no jvmReuse
+ }
+ /**
+ * Validates if task cleanup is done properly for a task that is not succeeded
+ * @throws IOException
+ */
+ public void testFailedTaskCleanup()
+ throws IOException {
+ if (!canRun()) {
+ return;
+ }
+ testTaskCleanup(true, false);// needCleanup; no jvmReuse
+ }
+
+ /**
+ * Validates if task cleanup is done properly for a succeeded task
+ * @throws IOException
+ */
+ public void testTaskCleanupWithJvmUse()
+ throws IOException {
+ if (!canRun()) {
+ return;
+ }
+ testTaskCleanup(false, true);// no needCleanup; jvmReuse
+ }
+
+ /**
+ * Validates if task cleanup is done properly
+ */
+ private void testTaskCleanup(boolean needCleanup, boolean jvmReuse)
+ throws IOException {
// Localize job and localize task.
tracker.getLocalizer().initializeUserDirs(task.getUser());
localizedJobConf = tracker.localizeJobFiles(task);
+ if (jvmReuse) {
+ localizedJobConf.setNumTasksToExecutePerJvm(2);
+ }
// Now initialize the job via task-controller so as to set
// ownership/permissions of jars, job-work-dir
JobInitializationContext jobContext = new JobInitializationContext();
@@ -614,18 +750,9 @@
// TODO: Let the task run and create files.
- InlineCleanupQueue cleanupQueue = new InlineCleanupQueue();
- tracker.directoryCleanupThread = cleanupQueue;
-
- // ////////// The central methods being tested
- tip.removeTaskFiles(true, taskId);
- tracker.removeJobFiles(task.getUser(), jobId.toString());
- // //////////
-
- // TODO: make sure that all files intended to be deleted are deleted.
-
- assertTrue("Some task files are not deleted!! Number of stale paths is "
- + cleanupQueue.stalePaths.size(), cleanupQueue.stalePaths.size() == 0);
+ // create files and set permissions 555. Verify if task controller sets
+ // the permissions for TT to delete the task dir or work dir properly
+ validateRemoveFiles(needCleanup, jvmReuse, tip);
// Check that the empty $mapreduce.cluster.local.dir/taskTracker/$user dirs are still
// there.
@@ -633,7 +760,7 @@
Path userDir =
new Path(localDir, TaskTracker.getUserDir(task.getUser()));
assertTrue("User directory " + userDir + " is not present!!",
- tracker.localFs.exists(userDir));
+ tracker.getLocalFileSystem().exists(userDir));
}
// Test userlogs cleanup.
@@ -653,7 +780,7 @@
// Logs should be there before cleanup.
assertTrue("Userlogs dir " + logDir + " is not presen as expected!!",
- tracker.localFs.exists(logDir));
+ tracker.getLocalFileSystem().exists(logDir));
// ////////// Another being tested
TaskLog.cleanup(-1); // -1 so as to move purgeTimeStamp to future and file
@@ -662,6 +789,6 @@
// Logs should be gone after cleanup.
assertFalse("Userlogs dir " + logDir + " is not deleted as expected!!",
- tracker.localFs.exists(logDir));
+ tracker.getLocalFileSystem().exists(logDir));
}
}
diff --git a/src/test/mapred/org/apache/hadoop/mapred/TestTrackerDistributedCacheManagerWithLinuxTaskController.java b/src/test/mapred/org/apache/hadoop/mapred/TestTrackerDistributedCacheManagerWithLinuxTaskController.java
index 43c6345..68b0ade 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/TestTrackerDistributedCacheManagerWithLinuxTaskController.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/TestTrackerDistributedCacheManagerWithLinuxTaskController.java
@@ -24,6 +24,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.ClusterWithLinuxTaskController.MyLinuxTaskController;
import org.apache.hadoop.mapreduce.filecache.TestTrackerDistributedCacheManager;
@@ -36,7 +37,6 @@
TestTrackerDistributedCacheManager {
private File configFile;
- private MyLinuxTaskController taskController;
private String taskTrackerSpecialGroup;
private static final Log LOG =
@@ -45,7 +45,7 @@
@Override
protected void setUp()
- throws IOException {
+ throws IOException, InterruptedException {
if (!ClusterWithLinuxTaskController.shouldRun()) {
return;
@@ -57,7 +57,7 @@
.getSimpleName()).getAbsolutePath();
super.setUp();
-
+
taskController = new MyLinuxTaskController();
String path =
System.getProperty(ClusterWithLinuxTaskController.TASKCONTROLLER_PATH);
@@ -65,7 +65,7 @@
ClusterWithLinuxTaskController.createTaskControllerConf(path, conf
.getStrings(JobConf.MAPRED_LOCAL_DIR_PROPERTY));
String execPath = path + "/task-controller";
- taskController.setTaskControllerExe(execPath);
+ ((MyLinuxTaskController)taskController).setTaskControllerExe(execPath);
taskController.setConf(conf);
taskController.setup();
@@ -74,6 +74,17 @@
}
@Override
+ protected void refreshConf(Configuration conf) throws IOException {
+ super.refreshConf(conf);
+ String path =
+ System.getProperty(ClusterWithLinuxTaskController.TASKCONTROLLER_PATH);
+ configFile =
+ ClusterWithLinuxTaskController.createTaskControllerConf(path, conf
+ .getStrings(JobConf.MAPRED_LOCAL_DIR_PROPERTY));
+
+ }
+
+ @Override
protected void tearDown()
throws IOException {
if (!ClusterWithLinuxTaskController.shouldRun()) {
@@ -99,27 +110,19 @@
}
@Override
- protected TaskController getTaskController() {
- return taskController;
- }
-
- @Override
protected void checkFilePermissions(Path[] localCacheFiles)
throws IOException {
- String cachedFirstFile = localCacheFiles[0].toUri().getPath();
- String cachedSecondFile = localCacheFiles[1].toUri().getPath();
String userName = getJobOwnerName();
- // First make sure that the cache files have proper permissions.
- TestTaskTrackerLocalization.checkFilePermissions(cachedFirstFile,
- "-r-xrwx---", userName, taskTrackerSpecialGroup);
- TestTaskTrackerLocalization.checkFilePermissions(cachedSecondFile,
- "-r-xrwx---", userName, taskTrackerSpecialGroup);
+ for (Path p : localCacheFiles) {
+ // First make sure that the cache file has proper permissions.
+ TestTaskTrackerLocalization.checkFilePermissions(p.toUri().getPath(),
+ "-r-xrwx---", userName, taskTrackerSpecialGroup);
+ // Now. make sure that all the path components also have proper
+ // permissions.
+ checkPermissionOnPathComponents(p.toUri().getPath(), userName);
+ }
- // Now. make sure that all the path components also have proper
- // permissions.
- checkPermissionOnPathComponents(cachedFirstFile, userName);
- checkPermissionOnPathComponents(cachedSecondFile, userName);
}
/**
@@ -134,9 +137,9 @@
String trailingStringForFirstFile =
cachedFilePath.replaceFirst(ROOT_MAPRED_LOCAL_DIR.getAbsolutePath()
+ Path.SEPARATOR + "0_[0-" + (numLocalDirs - 1) + "]"
- + Path.SEPARATOR + TaskTracker.getDistributedCacheDir(userName),
+ + Path.SEPARATOR + TaskTracker.getPrivateDistributedCacheDir(userName),
"");
- LOG.info("Leading path for cacheFirstFile is : "
+ LOG.info("Trailing path for cacheFirstFile is : "
+ trailingStringForFirstFile);
// The leading mapreduce.cluster.local.dir/0_[0-n]/taskTracker/$user string.
String leadingStringForFirstFile =
diff --git a/src/test/mapred/org/apache/hadoop/mapred/UtilsForTests.java b/src/test/mapred/org/apache/hadoop/mapred/UtilsForTests.java
index 689dad2..7c98849 100644
--- a/src/test/mapred/org/apache/hadoop/mapred/UtilsForTests.java
+++ b/src/test/mapred/org/apache/hadoop/mapred/UtilsForTests.java
@@ -25,9 +25,11 @@
import java.io.IOException;
import java.io.InputStream;
import java.text.DecimalFormat;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Enumeration;
import java.util.Iterator;
+import java.util.List;
import java.util.Properties;
import org.apache.commons.logging.LogFactory;
@@ -48,6 +50,7 @@
import org.apache.hadoop.mapred.lib.IdentityMapper;
import org.apache.hadoop.mapred.lib.IdentityReducer;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.util.StringUtils;
import org.apache.commons.logging.Log;
@@ -272,7 +275,9 @@
while (true) {
boolean shouldWait = false;
for (JobStatus jobStatuses : jobClient.getAllJobs()) {
- if (jobStatuses.getRunState() == JobStatus.RUNNING) {
+ if (jobStatuses.getRunState() != JobStatus.SUCCEEDED
+ && jobStatuses.getRunState() != JobStatus.FAILED
+ && jobStatuses.getRunState() != JobStatus.KILLED) {
shouldWait = true;
break;
}
@@ -620,6 +625,7 @@
conf.setJobName("test-job-fail");
conf.setMapperClass(FailMapper.class);
conf.setReducerClass(IdentityReducer.class);
+ conf.setMaxMapAttempts(1);
RunningJob job = UtilsForTests.runJob(conf, inDir, outDir);
while (!job.isComplete()) {
@@ -660,6 +666,37 @@
return job;
}
+
+ /**
+ * Cleans up files/dirs inline. CleanupQueue deletes in a separate thread
+ * asynchronously.
+ */
+ public static class InlineCleanupQueue extends CleanupQueue {
+ List<String> stalePaths = new ArrayList<String>();
+
+ public InlineCleanupQueue() {
+ // do nothing
+ }
+
+ @Override
+ public void addToQueue(PathDeletionContext... contexts) {
+ // delete paths in-line
+ for (PathDeletionContext context : contexts) {
+ try {
+ if (!deletePath(context)) {
+ LOG.warn("Stale path " + context.fullPath);
+ stalePaths.add(context.fullPath);
+ }
+ } catch (IOException e) {
+ LOG.warn("Caught exception while deleting path "
+ + context.fullPath);
+ LOG.info(StringUtils.stringifyException(e));
+ stalePaths.add(context.fullPath);
+ }
+ }
+ }
+ }
+
static class FakeClock extends Clock {
long time = 0;
@@ -722,7 +759,7 @@
conf.set(JTConfig.JT_HTTP_ADDRESS, "0.0.0.0:0");
JobTracker jt;
try {
- jt = new JobTracker(conf);
+ jt = JobTracker.startTracker(conf);
return jt;
} catch (Exception e) {
throw new RuntimeException("Could not start jt", e);
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/GenericMRLoadGenerator.java b/src/test/mapred/org/apache/hadoop/mapreduce/GenericMRLoadGenerator.java
index e726f03..30d58d5 100644
--- a/src/test/mapred/org/apache/hadoop/mapreduce/GenericMRLoadGenerator.java
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/GenericMRLoadGenerator.java
@@ -150,13 +150,13 @@
} else if (null != conf.getClass(INDIRECT_INPUT_FORMAT, null)) {
// specified IndirectInputFormat? Build src list
JobClient jClient = new JobClient(conf);
- Path sysdir = jClient.getSystemDir();
+ Path tmpDir = new Path("/tmp");
Random r = new Random();
- Path indirInputFile = new Path(sysdir,
+ Path indirInputFile = new Path(tmpDir,
Integer.toString(r.nextInt(Integer.MAX_VALUE), 36) + "_files");
conf.set(INDIRECT_INPUT_FILE, indirInputFile.toString());
SequenceFile.Writer writer = SequenceFile.createWriter(
- sysdir.getFileSystem(conf), conf, indirInputFile,
+ tmpDir.getFileSystem(conf), conf, indirInputFile,
LongWritable.class, Text.class,
SequenceFile.CompressionType.NONE);
try {
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/TestMRJobClient.java b/src/test/mapred/org/apache/hadoop/mapreduce/TestMRJobClient.java
index 2e781e7..c5962ee 100644
--- a/src/test/mapred/org/apache/hadoop/mapreduce/TestMRJobClient.java
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/TestMRJobClient.java
@@ -19,6 +19,7 @@
import java.io.BufferedReader;
import java.io.ByteArrayOutputStream;
+import java.io.File;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.OutputStream;
@@ -34,6 +35,8 @@
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+
public class TestMRJobClient extends ClusterMapReduceTestCase {
private static final Log LOG = LogFactory.getLog(TestMRJobClient.class);
@@ -61,6 +64,7 @@
}
}
+ @Test
public void testJobClient() throws Exception {
Configuration conf = createJobConf();
Job job = runJob(conf);
@@ -69,7 +73,8 @@
testJobList(jobId, conf);
testChangingJobPriority(jobId, conf);
}
-
+
+ @Test
public void testGetCounter(String jobId,
Configuration conf) throws Exception {
ByteArrayOutputStream out = new ByteArrayOutputStream();
@@ -81,6 +86,7 @@
assertEquals("Counter", "3", out.toString().trim());
}
+ @Test
public void testJobList(String jobId,
Configuration conf) throws Exception {
verifyJobPriority(jobId, "HIGH", conf, createJobClient());
@@ -106,7 +112,8 @@
}
pis.close();
}
-
+
+ @Test
public void testChangingJobPriority(String jobId, Configuration conf)
throws Exception {
int exitCode = runTool(conf, createJobClient(),
@@ -115,7 +122,56 @@
assertEquals("Exit code", 0, exitCode);
verifyJobPriority(jobId, "VERY_LOW", conf, createJobClient());
}
-
+
+ @Test
+ public void testMissingProfileOutput() throws Exception {
+ Configuration conf = createJobConf();
+ final String input = "hello1\n";
+
+ // Set a job to be profiled with an empty agentlib parameter.
+ // This will fail to create profile.out files for tasks.
+ // This will succeed by skipping the HTTP fetch of the
+ // profiler output.
+ Job job = MapReduceTestUtil.createJob(conf,
+ getInputDir(), getOutputDir(), 1, 1, input);
+ job.setJobName("disable-profile-fetch");
+ job.setProfileEnabled(true);
+ job.setProfileParams("-agentlib:,verbose=n,file=%s");
+ job.setMaxMapAttempts(1);
+ job.setMaxReduceAttempts(1);
+ job.setJobSetupCleanupNeeded(false);
+ job.waitForCompletion(true);
+
+ // Run another job with an hprof agentlib param; verify
+ // that the HTTP fetch works here.
+ Job job2 = MapReduceTestUtil.createJob(conf,
+ getInputDir(), getOutputDir(), 1, 1, input);
+ job2.setJobName("enable-profile-fetch");
+ job2.setProfileEnabled(true);
+ job2.setProfileParams(
+ "-agentlib:hprof=cpu=samples,heap=sites,force=n,"
+ + "thread=y,verbose=n,file=%s");
+ job2.setProfileTaskRange(true, "0-1");
+ job2.setProfileTaskRange(false, "");
+ job2.setMaxMapAttempts(1);
+ job2.setMaxReduceAttempts(1);
+ job2.setJobSetupCleanupNeeded(false);
+ job2.waitForCompletion(true);
+
+ // Find the first map task, verify that we got its profile output file.
+ TaskReport [] reports = job2.getTaskReports(TaskType.MAP);
+ assertTrue("No task reports found!", reports.length > 0);
+ TaskReport report = reports[0];
+ TaskID id = report.getTaskId();
+ assertTrue(TaskType.MAP == id.getTaskType());
+ System.out.println("Using task id: " + id);
+ TaskAttemptID attemptId = new TaskAttemptID(id, 0);
+
+ File profileOutFile = new File(attemptId.toString() + ".profile");
+ assertTrue("Couldn't find profiler output", profileOutFile.exists());
+ assertTrue("Couldn't remove profiler output", profileOutFile.delete());
+ }
+
protected CLI createJobClient() throws IOException {
return new CLI();
}
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/TestMapReduceLocal.java b/src/test/mapred/org/apache/hadoop/mapreduce/TestMapReduceLocal.java
index 29d8fb7..ad447ae 100644
--- a/src/test/mapred/org/apache/hadoop/mapreduce/TestMapReduceLocal.java
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/TestMapReduceLocal.java
@@ -162,10 +162,16 @@
"REDUCE_OUTPUT_RECORDS").getValue();
long reduceGrps = ctrs.findCounter(COUNTER_GROUP,
"REDUCE_INPUT_GROUPS").getValue();
+ long mergedMapOutputs = ctrs.findCounter(COUNTER_GROUP,
+ "MERGED_MAP_OUTPUTS").getValue();
+ long shuffledMaps = ctrs.findCounter(COUNTER_GROUP,
+ "SHUFFLED_MAPS").getValue();
assertEquals("map out = combine in", mapOut, combineIn);
assertEquals("combine out = reduce in", combineOut, reduceIn);
assertTrue("combine in > combine out", combineIn > combineOut);
assertEquals("reduce groups = reduce out", reduceGrps, reduceOut);
+ assertEquals("Mismatch in mergedMapOutputs", mergedMapOutputs, 2);
+ assertEquals("Mismatch in shuffledMaps", shuffledMaps, 2);
String group = "Random Group";
CounterGroup ctrGrp = ctrs.getGroup(group);
assertEquals(0, ctrGrp.size());
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/filecache/TestTrackerDistributedCacheManager.java b/src/test/mapred/org/apache/hadoop/mapreduce/filecache/TestTrackerDistributedCacheManager.java
index b3e66bd..1df5103 100644
--- a/src/test/mapred/org/apache/hadoop/mapreduce/filecache/TestTrackerDistributedCacheManager.java
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/filecache/TestTrackerDistributedCacheManager.java
@@ -32,12 +32,12 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.DefaultTaskController;
-import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.TaskController;
import org.apache.hadoop.mapred.TaskTracker;
-import org.apache.hadoop.mapred.TaskController.InitializationContext;
import org.apache.hadoop.mapreduce.Cluster;
import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -45,10 +45,13 @@
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.mapreduce.filecache.TaskDistributedCacheManager;
import org.apache.hadoop.mapreduce.filecache.TrackerDistributedCacheManager;
import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ReflectionUtils;
import org.mortbay.log.Log;
public class TestTrackerDistributedCacheManager extends TestCase {
@@ -59,7 +62,6 @@
.getAbsolutePath();
protected File ROOT_MAPRED_LOCAL_DIR;
- private static String TEST_CACHE_BASE_DIR = "cachebasedir";
protected int numLocalDirs = 6;
private static final int TEST_FILE_SIZE = 4 * 1024; // 4K
@@ -70,10 +72,11 @@
private FileSystem fs;
protected LocalDirAllocator localDirAllocator =
- new LocalDirAllocator(JobConf.MAPRED_LOCAL_DIR_PROPERTY);
+ new LocalDirAllocator(MRConfig.LOCAL_DIR);
+ protected TaskController taskController;
@Override
- protected void setUp() throws IOException {
+ protected void setUp() throws IOException,InterruptedException {
// Prepare the tests' root dir
File TEST_ROOT = new File(TEST_ROOT_DIR);
@@ -85,17 +88,36 @@
ROOT_MAPRED_LOCAL_DIR = new File(TEST_ROOT_DIR, "mapred/local");
ROOT_MAPRED_LOCAL_DIR.mkdirs();
+ String []localDirs = new String[numLocalDirs];
+ for (int i = 0; i < numLocalDirs; i++) {
+ File localDir = new File(ROOT_MAPRED_LOCAL_DIR, "0_" + i);
+ localDirs[i] = localDir.getPath();
+ localDir.mkdir();
+ }
+
conf = new Configuration();
- conf.setLong(TTConfig.TT_LOCAL_CACHE_SIZE, LOCAL_CACHE_LIMIT);
- conf.set(JobConf.MAPRED_LOCAL_DIR_PROPERTY, ROOT_MAPRED_LOCAL_DIR.toString());
+ conf.setStrings(MRConfig.LOCAL_DIR, localDirs);
conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "file:///");
fs = FileSystem.get(conf);
+ Class<? extends TaskController> taskControllerClass = conf.getClass(
+ TTConfig.TT_TASK_CONTROLLER, DefaultTaskController.class,
+ TaskController.class);
+ taskController = (TaskController) ReflectionUtils.newInstance(
+ taskControllerClass, conf);
+
+ // setup permissions for mapred local dir
+ taskController.setup();
// Create the temporary cache files to be used in the tests.
firstCacheFile = new Path(TEST_ROOT_DIR, "firstcachefile");
secondCacheFile = new Path(TEST_ROOT_DIR, "secondcachefile");
- createTempFile(firstCacheFile);
- createTempFile(secondCacheFile);
+ createPrivateTempFile(firstCacheFile);
+ createPrivateTempFile(secondCacheFile);
+ }
+
+ protected void refreshConf(Configuration conf) throws IOException {
+ taskController.setConf(conf);
+ taskController.setup();
}
/**
@@ -121,9 +143,12 @@
// ****** Imitate JobClient code
// Configures a task/job with both a regular file and a "classpath" file.
Configuration subConf = new Configuration(conf);
+ String userName = getJobOwnerName();
+ subConf.set(JobContext.USER_NAME, userName);
DistributedCache.addCacheFile(firstCacheFile.toUri(), subConf);
DistributedCache.addFileToClassPath(secondCacheFile, subConf);
TrackerDistributedCacheManager.determineTimestamps(subConf);
+ TrackerDistributedCacheManager.determineCacheVisibilities(subConf);
// ****** End of imitating JobClient code
Path jobFile = new Path(TEST_ROOT_DIR, "job.xml");
@@ -131,22 +156,16 @@
subConf.writeXml(os);
os.close();
- String userName = getJobOwnerName();
-
// ****** Imitate TaskRunner code.
TrackerDistributedCacheManager manager =
- new TrackerDistributedCacheManager(conf);
+ new TrackerDistributedCacheManager(conf, taskController);
TaskDistributedCacheManager handle =
manager.newTaskDistributedCacheManager(subConf);
assertNull(null, DistributedCache.getLocalCacheFiles(subConf));
File workDir = new File(new Path(TEST_ROOT_DIR, "workdir").toString());
handle.setup(localDirAllocator, workDir, TaskTracker
- .getDistributedCacheDir(userName));
-
- InitializationContext context = new InitializationContext();
- context.user = userName;
- context.workDir = workDir;
- getTaskController().initializeDistributedCache(context);
+ .getPrivateDistributedCacheDir(userName),
+ TaskTracker.getPublicDistributedCacheDir());
// ****** End of imitating TaskRunner code
Path[] localCacheFiles = DistributedCache.getLocalCacheFiles(subConf);
@@ -176,28 +195,26 @@
TrackerDistributedCacheManager {
public FakeTrackerDistributedCacheManager(Configuration conf)
throws IOException {
- super(conf);
+ super(conf, taskController);
}
@Override
Path localizeCache(Configuration conf, URI cache, long confFileStamp,
- CacheStatus cacheStatus, FileStatus fileStatus, boolean isArchive)
- throws IOException {
+ CacheStatus cacheStatus, FileStatus fileStatus, boolean isArchive,
+ boolean isPublic) throws IOException {
if (cache.equals(firstCacheFile.toUri())) {
throw new IOException("fake fail");
}
return super.localizeCache(conf, cache, confFileStamp, cacheStatus,
- fileStatus, isArchive);
+ fileStatus, isArchive, isPublic);
}
}
public void testReferenceCount() throws IOException, LoginException,
- URISyntaxException {
+ URISyntaxException, InterruptedException {
if (!canRun()) {
return;
}
- Configuration conf = new Configuration();
- conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "file:///");
TrackerDistributedCacheManager manager =
new FakeTrackerDistributedCacheManager(conf);
Cluster cluster = new Cluster(conf);
@@ -206,25 +223,29 @@
// Configures a job with a regular file
Job job1 = Job.getInstance(cluster, conf);
+ job1.setUser(userName);
job1.addCacheFile(secondCacheFile.toUri());
Configuration conf1 = job1.getConfiguration();
TrackerDistributedCacheManager.determineTimestamps(conf1);
+ TrackerDistributedCacheManager.determineCacheVisibilities(conf1);
// Task localizing for first job
TaskDistributedCacheManager handle = manager
.newTaskDistributedCacheManager(conf1);
handle.setup(localDirAllocator, workDir, TaskTracker
- .getDistributedCacheDir(userName));
+ .getPrivateDistributedCacheDir(userName),
+ TaskTracker.getPublicDistributedCacheDir());
handle.release();
for (TaskDistributedCacheManager.CacheFile c : handle.getCacheFiles()) {
assertEquals(0, manager.getReferenceCount(c.uri, conf1, c.timestamp));
}
Path thirdCacheFile = new Path(TEST_ROOT_DIR, "thirdcachefile");
- createTempFile(thirdCacheFile);
+ createPrivateTempFile(thirdCacheFile);
// Configures another job with three regular files.
Job job2 = Job.getInstance(cluster, conf);
+ job2.setUser(userName);
// add a file that would get failed to localize
job2.addCacheFile(firstCacheFile.toUri());
// add a file that is already localized by different job
@@ -233,6 +254,7 @@
job2.addCacheFile(thirdCacheFile.toUri());
Configuration conf2 = job2.getConfiguration();
TrackerDistributedCacheManager.determineTimestamps(conf2);
+ TrackerDistributedCacheManager.determineCacheVisibilities(conf2);
// Task localizing for second job
// localization for the "firstCacheFile" will fail.
@@ -240,7 +262,8 @@
Throwable th = null;
try {
handle.setup(localDirAllocator, workDir, TaskTracker
- .getDistributedCacheDir(userName));
+ .getPrivateDistributedCacheDir(userName),
+ TaskTracker.getPublicDistributedCacheDir());
} catch (IOException e) {
th = e;
Log.info("Exception during setup", e);
@@ -261,7 +284,73 @@
assertTrue(th.getMessage().contains(thirdCacheFile.getName()));
fs.delete(thirdCacheFile, false);
}
+
+ /**
+ * Tests that localization of distributed cache file happens in the desired
+ * directory
+ * @throws IOException
+ * @throws LoginException
+ */
+ public void testPublicPrivateCache()
+ throws IOException, LoginException, InterruptedException {
+ if (!canRun()) {
+ return;
+ }
+ checkLocalizedPath("true");
+ checkLocalizedPath("false");
+ }
+
+ private void checkLocalizedPath(String visibility)
+ throws IOException, LoginException, InterruptedException {
+ TrackerDistributedCacheManager manager =
+ new TrackerDistributedCacheManager(conf, taskController);
+ Cluster cluster = new Cluster(conf);
+ String userName = getJobOwnerName();
+ File workDir = new File(TEST_ROOT_DIR, "workdir");
+ Path cacheFile = new Path(TEST_ROOT_DIR, "fourthcachefile");
+ if ("true".equals(visibility)) {
+ createPublicTempFile(cacheFile);
+ } else {
+ createPrivateTempFile(cacheFile);
+ }
+
+ Job job1 = Job.getInstance(cluster, conf);
+ job1.setUser(userName);
+ job1.addCacheFile(cacheFile.toUri());
+ Configuration conf1 = job1.getConfiguration();
+ TrackerDistributedCacheManager.determineTimestamps(conf1);
+ TrackerDistributedCacheManager.determineCacheVisibilities(conf1);
+ // Task localizing for job
+ TaskDistributedCacheManager handle = manager
+ .newTaskDistributedCacheManager(conf1);
+ handle.setup(localDirAllocator, workDir, TaskTracker
+ .getPrivateDistributedCacheDir(userName),
+ TaskTracker.getPublicDistributedCacheDir());
+ TaskDistributedCacheManager.CacheFile c = handle.getCacheFiles().get(0);
+ String distCacheDir;
+ if ("true".equals(visibility)) {
+ distCacheDir = TaskTracker.getPublicDistributedCacheDir();
+ } else {
+ distCacheDir = TaskTracker.getPrivateDistributedCacheDir(userName);
+ }
+ Path localizedPath =
+ manager.getLocalCache(cacheFile.toUri(), conf1, distCacheDir,
+ fs.getFileStatus(cacheFile), false,
+ c.timestamp, new Path(TEST_ROOT_DIR), false,
+ Boolean.parseBoolean(visibility));
+ assertTrue("Cache file didn't get localized in the expected directory. " +
+ "Expected localization to happen within " +
+ ROOT_MAPRED_LOCAL_DIR + "/" + distCacheDir +
+ ", but was localized at " +
+ localizedPath, localizedPath.toString().contains(distCacheDir));
+ if ("true".equals(visibility)) {
+ checkPublicFilePermissions(new Path[]{localizedPath});
+ } else {
+ checkFilePermissions(new Path[]{localizedPath});
+ }
+ }
+
/**
* Check proper permissions on the cache files
*
@@ -270,17 +359,29 @@
*/
protected void checkFilePermissions(Path[] localCacheFiles)
throws IOException {
- Path cachedFirstFile = localCacheFiles[0];
- Path cachedSecondFile = localCacheFiles[1];
- // Both the files should have executable permissions on them.
- assertTrue("First cache file is not executable!", new File(cachedFirstFile
- .toUri().getPath()).canExecute());
- assertTrue("Second cache file is not executable!", new File(
- cachedSecondFile.toUri().getPath()).canExecute());
+ // All the files should have executable permissions on them.
+ for (Path p : localCacheFiles) {
+ assertTrue("Cache file is not executable!", new File(p
+ .toUri().getPath()).canExecute());
+ }
}
- protected TaskController getTaskController() {
- return new DefaultTaskController();
+ /**
+ * Check permissions on the public cache files
+ *
+ * @param localCacheFiles
+ * @throws IOException
+ */
+ private void checkPublicFilePermissions(Path[] localCacheFiles)
+ throws IOException {
+ // All the files should have read and executable permissions for others
+ for (Path p : localCacheFiles) {
+ FsPermission perm = fs.getFileStatus(p).getPermission();
+ assertTrue("cache file is not readable by others", perm.getOtherAction()
+ .implies(FsAction.READ));
+ assertTrue("cache file is not executable by others", perm
+ .getOtherAction().implies(FsAction.EXECUTE));
+ }
}
protected String getJobOwnerName() throws LoginException {
@@ -293,27 +394,39 @@
if (!canRun()) {
return;
}
+ // This test needs MRConfig.LOCAL_DIR to be single directory
+ // instead of four, because it assumes that both
+ // firstcachefile and secondcachefile will be localized on same directory
+ // so that second localization triggers deleteCache.
+ // If MRConfig.LOCAL_DIR is four directories, second localization might not
+ // trigger deleteCache, if it is localized in different directory.
+ Configuration conf2 = new Configuration(conf);
+ conf2.set(MRConfig.LOCAL_DIR, ROOT_MAPRED_LOCAL_DIR.toString());
+ conf2.setLong(TTConfig.TT_LOCAL_CACHE_SIZE, LOCAL_CACHE_LIMIT);
+ refreshConf(conf2);
TrackerDistributedCacheManager manager =
- new TrackerDistributedCacheManager(conf);
- FileSystem localfs = FileSystem.getLocal(conf);
+ new TrackerDistributedCacheManager(conf2, taskController);
+ FileSystem localfs = FileSystem.getLocal(conf2);
long now = System.currentTimeMillis();
+ String userName = getJobOwnerName();
+ conf2.set(JobContext.USER_NAME, userName);
- manager.getLocalCache(firstCacheFile.toUri(), conf,
- TEST_CACHE_BASE_DIR, fs.getFileStatus(firstCacheFile), false,
- now, new Path(TEST_ROOT_DIR), false);
- manager.releaseCache(firstCacheFile.toUri(), conf, now);
+ Path localCache = manager.getLocalCache(firstCacheFile.toUri(), conf2,
+ TaskTracker.getPrivateDistributedCacheDir(userName),
+ fs.getFileStatus(firstCacheFile), false,
+ now, new Path(TEST_ROOT_DIR), false, false);
+ manager.releaseCache(firstCacheFile.toUri(), conf2, now);
//in above code,localized a file of size 4K and then release the cache
// which will cause the cache be deleted when the limit goes out.
// The below code localize another cache which's designed to
//sweep away the first cache.
- manager.getLocalCache(secondCacheFile.toUri(), conf,
- TEST_CACHE_BASE_DIR, fs.getFileStatus(secondCacheFile), false,
- System.currentTimeMillis(), new Path(TEST_ROOT_DIR), false);
- FileStatus[] dirStatuses = localfs.listStatus(
- new Path(ROOT_MAPRED_LOCAL_DIR.toString()));
- assertTrue("DistributedCache failed deleting old" +
+ manager.getLocalCache(secondCacheFile.toUri(), conf2,
+ TaskTracker.getPrivateDistributedCacheDir(userName),
+ fs.getFileStatus(secondCacheFile), false,
+ System.currentTimeMillis(), new Path(TEST_ROOT_DIR), false, false);
+ assertFalse("DistributedCache failed deleting old" +
" cache when the cache store is full.",
- dirStatuses.length == 1);
+ localfs.exists(localCache));
}
public void testFileSystemOtherThanDefault() throws Exception {
@@ -321,14 +434,17 @@
return;
}
TrackerDistributedCacheManager manager =
- new TrackerDistributedCacheManager(conf);
+ new TrackerDistributedCacheManager(conf, taskController);
conf.set("fs.fakefile.impl", conf.get("fs.file.impl"));
+ String userName = getJobOwnerName();
+ conf.set(JobContext.USER_NAME, userName);
Path fileToCache = new Path("fakefile:///"
+ firstCacheFile.toUri().getPath());
Path result = manager.getLocalCache(fileToCache.toUri(), conf,
- TEST_CACHE_BASE_DIR, fs.getFileStatus(firstCacheFile), false,
+ TaskTracker.getPrivateDistributedCacheDir(userName),
+ fs.getFileStatus(firstCacheFile), false,
System.currentTimeMillis(),
- new Path(TEST_ROOT_DIR), false);
+ new Path(TEST_ROOT_DIR), false, false);
assertNotNull("DistributedCache cached file on non-default filesystem.",
result);
}
@@ -342,6 +458,18 @@
os.close();
FileSystem.LOG.info("created: " + p + ", size=" + TEST_FILE_SIZE);
}
+
+ static void createPublicTempFile(Path p)
+ throws IOException, InterruptedException {
+ createTempFile(p);
+ FileUtil.chmod(p.toString(), "0777",true);
+ }
+
+ static void createPrivateTempFile(Path p)
+ throws IOException, InterruptedException {
+ createTempFile(p);
+ FileUtil.chmod(p.toString(), "0770",true);
+ }
@Override
protected void tearDown() throws IOException {
@@ -382,26 +510,29 @@
return;
}
Configuration myConf = new Configuration(conf);
- myConf.set("fs.default.name", "refresh:///");
+ myConf.set(FileSystem.FS_DEFAULT_NAME_KEY, "refresh:///");
myConf.setClass("fs.refresh.impl", FakeFileSystem.class, FileSystem.class);
+ String userName = getJobOwnerName();
+
TrackerDistributedCacheManager manager =
- new TrackerDistributedCacheManager(myConf);
+ new TrackerDistributedCacheManager(myConf, taskController);
// ****** Imitate JobClient code
// Configures a task/job with both a regular file and a "classpath" file.
Configuration subConf = new Configuration(myConf);
+ subConf.set(JobContext.USER_NAME, userName);
DistributedCache.addCacheFile(firstCacheFile.toUri(), subConf);
TrackerDistributedCacheManager.determineTimestamps(subConf);
+ TrackerDistributedCacheManager.determineCacheVisibilities(subConf);
// ****** End of imitating JobClient code
- String userName = getJobOwnerName();
-
// ****** Imitate TaskRunner code.
TaskDistributedCacheManager handle =
manager.newTaskDistributedCacheManager(subConf);
assertNull(null, DistributedCache.getLocalCacheFiles(subConf));
File workDir = new File(new Path(TEST_ROOT_DIR, "workdir").toString());
handle.setup(localDirAllocator, workDir, TaskTracker
- .getDistributedCacheDir(userName));
+ .getPrivateDistributedCacheDir(userName),
+ TaskTracker.getPublicDistributedCacheDir());
// ****** End of imitating TaskRunner code
Path[] localCacheFiles = DistributedCache.getLocalCacheFiles(subConf);
@@ -422,7 +553,7 @@
Throwable th = null;
try {
handle.setup(localDirAllocator, workDir, TaskTracker
- .getDistributedCacheDir(userName));
+ .getPrivateDistributedCacheDir(userName), TaskTracker.getPublicDistributedCacheDir());
} catch (IOException ie) {
th = ie;
}
@@ -434,13 +565,15 @@
// submit another job
Configuration subConf2 = new Configuration(myConf);
+ subConf2.set(JobContext.USER_NAME, userName);
DistributedCache.addCacheFile(firstCacheFile.toUri(), subConf2);
TrackerDistributedCacheManager.determineTimestamps(subConf2);
+ TrackerDistributedCacheManager.determineCacheVisibilities(subConf2);
handle =
manager.newTaskDistributedCacheManager(subConf2);
handle.setup(localDirAllocator, workDir, TaskTracker
- .getDistributedCacheDir(userName));
+ .getPrivateDistributedCacheDir(userName), TaskTracker.getPublicDistributedCacheDir());
Path[] localCacheFiles2 = DistributedCache.getLocalCacheFiles(subConf2);
assertNotNull(null, localCacheFiles2);
assertEquals(1, localCacheFiles2.length);
@@ -456,4 +589,46 @@
handle.release();
}
+ /**
+ * Localize a file. After localization is complete, create a file, "myFile",
+ * under the directory where the file is localized and ensure that it has
+ * permissions different from what is set by default. Then, localize another
+ * file. Verify that "myFile" has the right permissions.
+ * @throws Exception
+ */
+ public void testCustomPermissions() throws Exception {
+ if (!canRun()) {
+ return;
+ }
+ String userName = getJobOwnerName();
+ conf.set(JobContext.USER_NAME, userName);
+ TrackerDistributedCacheManager manager =
+ new TrackerDistributedCacheManager(conf, taskController);
+ FileSystem localfs = FileSystem.getLocal(conf);
+ long now = System.currentTimeMillis();
+
+ Path[] localCache = new Path[2];
+ localCache[0] = manager.getLocalCache(firstCacheFile.toUri(), conf,
+ TaskTracker.getPrivateDistributedCacheDir(userName),
+ fs.getFileStatus(firstCacheFile), false,
+ now, new Path(TEST_ROOT_DIR), false, false);
+ FsPermission myPermission = new FsPermission((short)0600);
+ Path myFile = new Path(localCache[0].getParent(), "myfile.txt");
+ if (FileSystem.create(localfs, myFile, myPermission) == null) {
+ throw new IOException("Could not create " + myFile);
+ }
+ try {
+ localCache[1] = manager.getLocalCache(secondCacheFile.toUri(), conf,
+ TaskTracker.getPrivateDistributedCacheDir(userName),
+ fs.getFileStatus(secondCacheFile), false,
+ System.currentTimeMillis(), new Path(TEST_ROOT_DIR), false, false);
+ FileStatus stat = localfs.getFileStatus(myFile);
+ assertTrue(stat.getPermission().equals(myPermission));
+ // validate permissions of localized files.
+ checkFilePermissions(localCache);
+ } finally {
+ localfs.delete(myFile, false);
+ }
+ }
+
}
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java b/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
new file mode 100644
index 0000000..2dfcf41
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
@@ -0,0 +1,223 @@
+/**
+ * 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.mapreduce.lib.chain;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+
+/**
+ * Tests error conditions in ChainMapper/ChainReducer.
+ */
+public class TestChainErrors extends HadoopTestCase {
+
+ private static String localPathRoot = System.getProperty("test.build.data",
+ "/tmp");
+
+ public TestChainErrors() throws IOException {
+ super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
+ }
+
+ private Path inDir = new Path(localPathRoot, "testing/chain/input");
+ private Path outDir = new Path(localPathRoot, "testing/chain/output");
+ private String input = "a\nb\nc\nd\n";
+
+ /**
+ * Tests errors during submission.
+ *
+ * @throws Exception
+ */
+ public void testChainSubmission() throws Exception {
+
+ Configuration conf = createJobConf();
+
+ Job job = MapReduceTestUtil.createJob(conf, inDir, outDir, 0, 0, input);
+ job.setJobName("chain");
+
+ Throwable th = null;
+ // output key,value classes of first map are not same as that of second map
+ try {
+ ChainMapper.addMapper(job, Mapper.class, LongWritable.class, Text.class,
+ IntWritable.class, Text.class, null);
+ ChainMapper.addMapper(job, Mapper.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+ } catch (IllegalArgumentException iae) {
+ th = iae;
+ }
+ assertTrue(th != null);
+
+ th = null;
+ // output key,value classes of reducer are not
+ // same as that of mapper in the chain
+ try {
+ ChainReducer.setReducer(job, Reducer.class, LongWritable.class,
+ Text.class, IntWritable.class, Text.class, null);
+ ChainMapper.addMapper(job, Mapper.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+ } catch (IllegalArgumentException iae) {
+ th = iae;
+ }
+ assertTrue(th != null);
+ }
+
+ /**
+ * Tests one of the mappers throwing exception.
+ *
+ * @throws Exception
+ */
+ public void testChainFail() throws Exception {
+
+ Configuration conf = createJobConf();
+
+ Job job = MapReduceTestUtil.createJob(conf, inDir, outDir, 1, 0, input);
+ job.setJobName("chain");
+
+ ChainMapper.addMapper(job, Mapper.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ ChainMapper.addMapper(job, FailMap.class, LongWritable.class, Text.class,
+ IntWritable.class, Text.class, null);
+
+ ChainMapper.addMapper(job, Mapper.class, IntWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ job.waitForCompletion(true);
+ assertTrue("Job Not failed", !job.isSuccessful());
+ }
+
+ /**
+ * Tests Reducer throwing exception.
+ *
+ * @throws Exception
+ */
+ public void testReducerFail() throws Exception {
+
+ Configuration conf = createJobConf();
+
+ Job job = MapReduceTestUtil.createJob(conf, inDir, outDir, 1, 1, input);
+ job.setJobName("chain");
+
+ ChainMapper.addMapper(job, Mapper.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ ChainReducer.setReducer(job, FailReduce.class, LongWritable.class,
+ Text.class, LongWritable.class, Text.class, null);
+
+ ChainReducer.addMapper(job, Mapper.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ job.waitForCompletion(true);
+ assertTrue("Job Not failed", !job.isSuccessful());
+ }
+
+ /**
+ * Tests one of the maps consuming output.
+ *
+ * @throws Exception
+ */
+ public void testChainMapNoOuptut() throws Exception {
+ Configuration conf = createJobConf();
+ String expectedOutput = "";
+
+ Job job = MapReduceTestUtil.createJob(conf, inDir, outDir, 1, 0, input);
+ job.setJobName("chain");
+
+ ChainMapper.addMapper(job, ConsumeMap.class, IntWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ ChainMapper.addMapper(job, Mapper.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ job.waitForCompletion(true);
+ assertTrue("Job failed", job.isSuccessful());
+ assertEquals("Outputs doesn't match", expectedOutput, MapReduceTestUtil
+ .readOutput(outDir, conf));
+ }
+
+ /**
+ * Tests reducer consuming output.
+ *
+ * @throws Exception
+ */
+ public void testChainReduceNoOuptut() throws Exception {
+ Configuration conf = createJobConf();
+ String expectedOutput = "";
+
+ Job job = MapReduceTestUtil.createJob(conf, inDir, outDir, 1, 1, input);
+ job.setJobName("chain");
+
+ ChainMapper.addMapper(job, Mapper.class, IntWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ ChainReducer.setReducer(job, ConsumeReduce.class, LongWritable.class,
+ Text.class, LongWritable.class, Text.class, null);
+
+ ChainReducer.addMapper(job, Mapper.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ job.waitForCompletion(true);
+ assertTrue("Job failed", job.isSuccessful());
+ assertEquals("Outputs doesn't match", expectedOutput, MapReduceTestUtil
+ .readOutput(outDir, conf));
+ }
+
+ // this map consumes all the input and output nothing
+ public static class ConsumeMap extends
+ Mapper<LongWritable, Text, LongWritable, Text> {
+ public void map(LongWritable key, Text value, Context context)
+ throws IOException, InterruptedException {
+ }
+ }
+
+ // this reduce consumes all the input and output nothing
+ public static class ConsumeReduce extends
+ Reducer<LongWritable, Text, LongWritable, Text> {
+ public void reduce(LongWritable key, Iterable<Text> values, Context context)
+ throws IOException, InterruptedException {
+ }
+ }
+
+ // this map throws IOException for input value "b"
+ public static class FailMap extends
+ Mapper<LongWritable, Text, IntWritable, Text> {
+ protected void map(LongWritable key, Text value, Context context)
+ throws IOException, InterruptedException {
+ if (value.toString().equals("b")) {
+ throw new IOException();
+ }
+ }
+ }
+
+ // this reduce throws IOEexception for any input
+ public static class FailReduce extends
+ Reducer<LongWritable, Text, LongWritable, Text> {
+ public void reduce(LongWritable key, Iterable<Text> values, Context context)
+ throws IOException, InterruptedException {
+ throw new IOException();
+ }
+ }
+}
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java b/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
new file mode 100644
index 0000000..971ea68
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
@@ -0,0 +1,246 @@
+/**
+ * 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.mapreduce.lib.chain;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+
+public class TestMapReduceChain extends HadoopTestCase {
+
+ private static String localPathRoot = System.getProperty("test.build.data",
+ "/tmp");
+ private static Path flagDir = new Path(localPathRoot, "testing/chain/flags");
+
+ private static void cleanFlags(Configuration conf) throws IOException {
+ FileSystem fs = FileSystem.get(conf);
+ fs.delete(flagDir, true);
+ fs.mkdirs(flagDir);
+ }
+
+ private static void writeFlag(Configuration conf, String flag)
+ throws IOException {
+ FileSystem fs = FileSystem.get(conf);
+ if (getFlag(conf, flag)) {
+ fail("Flag " + flag + " already exists");
+ }
+ DataOutputStream file = fs.create(new Path(flagDir, flag));
+ file.close();
+ }
+
+ private static boolean getFlag(Configuration conf, String flag)
+ throws IOException {
+ FileSystem fs = FileSystem.get(conf);
+ return fs.exists(new Path(flagDir, flag));
+ }
+
+ public TestMapReduceChain() throws IOException {
+ super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
+ }
+
+ public void testChain() throws Exception {
+ Path inDir = new Path(localPathRoot, "testing/chain/input");
+ Path outDir = new Path(localPathRoot, "testing/chain/output");
+ String input = "1\n2\n";
+ String expectedOutput = "0\t1ABCRDEF\n2\t2ABCRDEF\n";
+
+ Configuration conf = createJobConf();
+ cleanFlags(conf);
+ conf.set("a", "X");
+
+ Job job = MapReduceTestUtil.createJob(conf, inDir, outDir, 1, 1, input);
+ job.setJobName("chain");
+
+ Configuration mapAConf = new Configuration(false);
+ mapAConf.set("a", "A");
+ ChainMapper.addMapper(job, AMap.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, mapAConf);
+
+ ChainMapper.addMapper(job, BMap.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ ChainMapper.addMapper(job, CMap.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ Configuration reduceConf = new Configuration(false);
+ reduceConf.set("a", "C");
+ ChainReducer.setReducer(job, RReduce.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, reduceConf);
+
+ ChainReducer.addMapper(job, DMap.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ Configuration mapEConf = new Configuration(false);
+ mapEConf.set("a", "E");
+ ChainReducer.addMapper(job, EMap.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, mapEConf);
+
+ ChainReducer.addMapper(job, FMap.class, LongWritable.class, Text.class,
+ LongWritable.class, Text.class, null);
+
+ job.waitForCompletion(true);
+ assertTrue("Job failed", job.isSuccessful());
+
+ String str = "flag not set";
+ assertTrue(str, getFlag(conf, "map.setup.A"));
+ assertTrue(str, getFlag(conf, "map.setup.B"));
+ assertTrue(str, getFlag(conf, "map.setup.C"));
+ assertTrue(str, getFlag(conf, "reduce.setup.R"));
+ assertTrue(str, getFlag(conf, "map.setup.D"));
+ assertTrue(str, getFlag(conf, "map.setup.E"));
+ assertTrue(str, getFlag(conf, "map.setup.F"));
+
+ assertTrue(str, getFlag(conf, "map.A.value.1"));
+ assertTrue(str, getFlag(conf, "map.A.value.2"));
+ assertTrue(str, getFlag(conf, "map.B.value.1A"));
+ assertTrue(str, getFlag(conf, "map.B.value.2A"));
+ assertTrue(str, getFlag(conf, "map.C.value.1AB"));
+ assertTrue(str, getFlag(conf, "map.C.value.2AB"));
+ assertTrue(str, getFlag(conf, "reduce.R.value.1ABC"));
+ assertTrue(str, getFlag(conf, "reduce.R.value.2ABC"));
+ assertTrue(str, getFlag(conf, "map.D.value.1ABCR"));
+ assertTrue(str, getFlag(conf, "map.D.value.2ABCR"));
+ assertTrue(str, getFlag(conf, "map.E.value.1ABCRD"));
+ assertTrue(str, getFlag(conf, "map.E.value.2ABCRD"));
+ assertTrue(str, getFlag(conf, "map.F.value.1ABCRDE"));
+ assertTrue(str, getFlag(conf, "map.F.value.2ABCRDE"));
+
+ assertTrue(getFlag(conf, "map.cleanup.A"));
+ assertTrue(getFlag(conf, "map.cleanup.B"));
+ assertTrue(getFlag(conf, "map.cleanup.C"));
+ assertTrue(getFlag(conf, "reduce.cleanup.R"));
+ assertTrue(getFlag(conf, "map.cleanup.D"));
+ assertTrue(getFlag(conf, "map.cleanup.E"));
+ assertTrue(getFlag(conf, "map.cleanup.F"));
+
+ assertEquals("Outputs doesn't match", expectedOutput, MapReduceTestUtil
+ .readOutput(outDir, conf));
+ }
+
+ public static class AMap extends IDMap {
+ public AMap() {
+ super("A", "A");
+ }
+ }
+
+ public static class BMap extends IDMap {
+ public BMap() {
+ super("B", "X");
+ }
+ }
+
+ public static class CMap extends IDMap {
+ public CMap() {
+ super("C", "X");
+ }
+ }
+
+ public static class RReduce extends IDReduce {
+ public RReduce() {
+ super("R", "C");
+ }
+ }
+
+ public static class DMap extends IDMap {
+ public DMap() {
+ super("D", "X");
+ }
+ }
+
+ public static class EMap extends IDMap {
+ public EMap() {
+ super("E", "E");
+ }
+ }
+
+ public static class FMap extends IDMap {
+ public FMap() {
+ super("F", "X");
+ }
+ }
+
+ public static class IDMap extends
+ Mapper<LongWritable, Text, LongWritable, Text> {
+ private String name;
+ private String prop;
+
+ public IDMap(String name, String prop) {
+ this.name = name;
+ this.prop = prop;
+ }
+
+ public void setup(Context context) throws IOException, InterruptedException {
+ Configuration conf = context.getConfiguration();
+ assertEquals(prop, conf.get("a"));
+ writeFlag(conf, "map.setup." + name);
+ }
+
+ public void map(LongWritable key, Text value, Context context)
+ throws IOException, InterruptedException {
+ writeFlag(context.getConfiguration(), "map." + name + ".value." + value);
+ context.write(key, new Text(value + name));
+ }
+
+ public void cleanup(Context context) throws IOException,
+ InterruptedException {
+ writeFlag(context.getConfiguration(), "map.cleanup." + name);
+ }
+ }
+
+ public static class IDReduce extends
+ Reducer<LongWritable, Text, LongWritable, Text> {
+
+ private String name;
+ private String prop;
+
+ public IDReduce(String name, String prop) {
+ this.name = name;
+ this.prop = prop;
+ }
+
+ public void setup(Context context) throws IOException, InterruptedException {
+ Configuration conf = context.getConfiguration();
+ assertEquals(prop, conf.get("a"));
+ writeFlag(conf, "reduce.setup." + name);
+ }
+
+ public void reduce(LongWritable key, Iterable<Text> values, Context context)
+ throws IOException, InterruptedException {
+ for (Text value : values) {
+ writeFlag(context.getConfiguration(), "reduce." + name + ".value."
+ + value);
+ context.write(key, new Text(value + name));
+ }
+ }
+
+ public void cleanup(Context context) throws IOException,
+ InterruptedException {
+ writeFlag(context.getConfiguration(), "reduce.cleanup." + name);
+ }
+ }
+}
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java b/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
new file mode 100644
index 0000000..06cfe1c
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
@@ -0,0 +1,68 @@
+/**
+ * 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.mapreduce.lib.chain;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper;
+import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
+
+import java.io.IOException;
+
+/**
+ * Runs wordcount by adding single mapper and single reducer to chain
+ */
+public class TestSingleElementChain extends HadoopTestCase {
+
+ private static String localPathRoot = System.getProperty("test.build.data",
+ "/tmp");
+
+ public TestSingleElementChain() throws IOException {
+ super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
+ }
+
+ // test chain mapper and reducer by adding single mapper and reducer to chain
+ public void testNoChain() throws Exception {
+ Path inDir = new Path(localPathRoot, "testing/chain/input");
+ Path outDir = new Path(localPathRoot, "testing/chain/output");
+ String input = "a\nb\na\n";
+ String expectedOutput = "a\t2\nb\t1\n";
+
+ Configuration conf = createJobConf();
+
+ Job job = MapReduceTestUtil.createJob(conf, inDir, outDir, 1, 1, input);
+ job.setJobName("chain");
+
+ ChainMapper.addMapper(job, TokenCounterMapper.class, Object.class,
+ Text.class, Text.class, IntWritable.class, null);
+
+ ChainReducer.setReducer(job, IntSumReducer.class, Text.class,
+ IntWritable.class, Text.class, IntWritable.class, null);
+
+ job.waitForCompletion(true);
+ assertTrue("Job failed", job.isSuccessful());
+ assertEquals("Outputs doesn't match", expectedOutput, MapReduceTestUtil
+ .readOutput(outDir, conf));
+ }
+
+}
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/security/TestTokenCache.java b/src/test/mapred/org/apache/hadoop/mapreduce/security/TestTokenCache.java
new file mode 100644
index 0000000..9810217
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/security/TestTokenCache.java
@@ -0,0 +1,213 @@
+/** 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.mapreduce.security;
+
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.crypto.KeyGenerator;
+import javax.crypto.spec.SecretKeySpec;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.SleepJob;
+import org.apache.hadoop.util.ToolRunner;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestTokenCache {
+ private static final int NUM_OF_KEYS = 10;
+
+ // my sleep class - adds check for tokenCache
+ static class MySleepMapper extends SleepJob.SleepMapper {
+ /**
+ * attempts to access tokenCache as from client
+ */
+ @Override
+ public void map(IntWritable key, IntWritable value, Context context)
+ throws IOException, InterruptedException {
+ // get token storage and a key
+ TokenStorage ts = TokenCache.getTokenStorage();
+ byte[] key1 = TokenCache.getSecretKey(new Text("alias1"));
+
+ System.out.println("inside MAP: ts==NULL?=" + (ts==null) +
+ "; #keys = " + (ts==null? 0:ts.numberOfSecretKeys()) +
+ ";jobToken = " + (ts==null? "n/a":ts.getJobToken()) +
+ "; alias1 key=" + new String(key1));
+
+ if(key1 == null || ts == null || ts.numberOfSecretKeys() != NUM_OF_KEYS) {
+ throw new RuntimeException("tokens are not available"); // fail the test
+ }
+ super.map(key, value, context);
+ }
+ }
+
+ class MySleepJob extends SleepJob {
+ @Override
+ public Job createJob(int numMapper, int numReducer,
+ long mapSleepTime, int mapSleepCount,
+ long reduceSleepTime, int reduceSleepCount)
+ throws IOException {
+ Job job = super.createJob(numMapper, numReducer,
+ mapSleepTime, mapSleepCount,
+ reduceSleepTime, reduceSleepCount);
+
+ job.setMapperClass(MySleepMapper.class);
+ return job;
+ }
+ }
+
+ private static MiniMRCluster mrCluster;
+ private static MiniDFSCluster dfsCluster;
+ private static final Path TEST_DIR =
+ new Path(System.getProperty("test.build.data","/tmp"), "sleepTest");
+ private static final Path tokenFileName = new Path(TEST_DIR, "tokenFile.json");
+ private static int numSlaves = 1;
+ private static JobConf jConf;
+ private static ObjectMapper mapper = new ObjectMapper();
+
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ Configuration conf = new Configuration();
+ dfsCluster = new MiniDFSCluster(conf, numSlaves, true, null);
+ jConf = new JobConf(conf);
+ mrCluster = new MiniMRCluster(0, 0, numSlaves,
+ dfsCluster.getFileSystem().getUri().toString(), 1, null, null, null,
+ jConf);
+
+ createTokenFileJson();
+ verifySecretKeysInJSONFile();
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ if(mrCluster != null)
+ mrCluster.shutdown();
+ mrCluster = null;
+ if(dfsCluster != null)
+ dfsCluster.shutdown();
+ dfsCluster = null;
+ }
+
+ // create jason file and put some keys into it..
+ private static void createTokenFileJson() throws IOException {
+ Map<String, String> map = new HashMap<String, String>();
+
+ try {
+ KeyGenerator kg = KeyGenerator.getInstance("HmacSHA1");
+ for(int i=0; i<NUM_OF_KEYS; i++) {
+ SecretKeySpec key = (SecretKeySpec) kg.generateKey();
+ byte [] enc_key = key.getEncoded();
+ map.put("alias"+i, new String(Base64.encodeBase64(enc_key)));
+
+ }
+ } catch (NoSuchAlgorithmException e) {
+ throw new IOException(e);
+ }
+
+ System.out.println("writing secret keys into " + tokenFileName);
+ try {
+ File p = new File(tokenFileName.getParent().toString());
+ p.mkdirs();
+ // convert to JSON and save to the file
+ mapper.writeValue(new File(tokenFileName.toString()), map);
+
+ } catch (Exception e) {
+ System.out.println("failed with :" + e.getLocalizedMessage());
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private static void verifySecretKeysInJSONFile() throws IOException {
+ Map<String, String> map;
+ map = mapper.readValue(new File(tokenFileName.toString()), Map.class);
+ assertEquals("didn't read JSON correctly", map.size(), NUM_OF_KEYS);
+
+ System.out.println("file " + tokenFileName + " verified; size="+ map.size());
+ }
+
+ /**
+ * run a distributed job and verify that TokenCache is available
+ * @throws IOException
+ */
+ @Test
+ public void testTokenCache() throws IOException {
+
+ System.out.println("running dist job");
+
+ // make sure JT starts
+ jConf = mrCluster.createJobConf();
+
+ // using argument to pass the file name
+ String[] args = {
+ "-tokenCacheFile", tokenFileName.toString(),
+ "-m", "1", "-r", "1", "-mt", "1", "-rt", "1"
+ };
+
+ int res = -1;
+ try {
+ res = ToolRunner.run(jConf, new MySleepJob(), args);
+ } catch (Exception e) {
+ System.out.println("Job failed with" + e.getLocalizedMessage());
+ e.printStackTrace(System.out);
+ fail("Job failed");
+ }
+ assertEquals("dist job res is not 0", res, 0);
+ }
+
+ /**
+ * run a local job and verify that TokenCache is available
+ * @throws NoSuchAlgorithmException
+ * @throws IOException
+ */
+ @Test
+ public void testLocalJobTokenCache() throws NoSuchAlgorithmException, IOException {
+
+ System.out.println("running local job");
+ // this is local job
+ String[] args = {"-m", "1", "-r", "1", "-mt", "1", "-rt", "1"};
+ jConf.set("tokenCacheFile", tokenFileName.toString());
+
+ int res = -1;
+ try {
+ res = ToolRunner.run(jConf, new MySleepJob(), args);
+ } catch (Exception e) {
+ System.out.println("Job failed with" + e.getLocalizedMessage());
+ e.printStackTrace(System.out);
+ fail("local Job failed");
+ }
+ assertEquals("local job res is not 0", res, 0);
+ }
+}
\ No newline at end of file
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/security/TestTokenStorage.java b/src/test/mapred/org/apache/hadoop/mapreduce/security/TestTokenStorage.java
new file mode 100644
index 0000000..074d57b
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/security/TestTokenStorage.java
@@ -0,0 +1,109 @@
+/**
+ * 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.mapreduce.security;
+
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.security.Key;
+import java.security.NoSuchAlgorithmException;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.crypto.KeyGenerator;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
+import org.apache.hadoop.record.Utils;
+import org.apache.hadoop.security.token.Token;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestTokenStorage {
+ private static final String DEFAULT_HMAC_ALGORITHM = "HmacSHA1";
+ private static final File tmpDir =
+ new File(System.getProperty("test.build.data", "/tmp"), "mapred");
+
+ @Before
+ public void setUp() {
+ tmpDir.mkdir();
+ }
+
+ @Test
+ public void testReadWriteStorage() throws IOException, NoSuchAlgorithmException{
+ // create tokenStorage Object
+ TokenStorage ts = new TokenStorage();
+
+ // create a token
+ JobTokenSecretManager jtSecretManager = new JobTokenSecretManager();
+ JobTokenIdentifier identifier = new JobTokenIdentifier(new Text("fakeJobId"));
+ Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>(identifier,
+ jtSecretManager);
+ // store it
+ ts.setJobToken(jt);
+
+ // create keys and put it in
+ final KeyGenerator kg = KeyGenerator.getInstance(DEFAULT_HMAC_ALGORITHM);
+ String alias = "alias";
+ Map<Text, byte[]> m = new HashMap<Text, byte[]>(10);
+ for(int i=0; i<10; i++) {
+ Key key = kg.generateKey();
+ m.put(new Text(alias+i), key.getEncoded());
+ ts.addSecretKey(new Text(alias+i), key.getEncoded());
+ }
+
+ // create file to store
+ File tmpFileName = new File(tmpDir, "tokenStorageTest");
+ DataOutputStream dos = new DataOutputStream(new FileOutputStream(tmpFileName));
+ ts.write(dos);
+ dos.close();
+
+ // open and read it back
+ DataInputStream dis = new DataInputStream(new FileInputStream(tmpFileName));
+ ts = new TokenStorage();
+ ts.readFields(dis);
+ dis.close();
+
+ // get the token and compare the passwords
+ byte[] tp1 = ts.getJobToken().getPassword();
+ byte[] tp2 = jt.getPassword();
+ int comp = Utils.compareBytes(tp1, 0, tp1.length, tp2, 0, tp2.length);
+ assertTrue("shuffleToken doesn't match", comp==0);
+
+ // compare secret keys
+ int mapLen = m.size();
+ assertEquals("wrong number of keys in the Storage", mapLen, ts.numberOfSecretKeys());
+ for(Text a : m.keySet()) {
+ byte [] kTS = ts.getSecretKey(a);
+ byte [] kLocal = m.get(a);
+ assertTrue("keys don't match for " + a,
+ Utils.compareBytes(kTS, 0, kTS.length, kLocal, 0, kLocal.length)==0);
+ }
+
+ assertEquals("All tokens should return collection of size 1",
+ ts.getAllTokens().size(), 1);
+ }
+ }
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/util/TestLinuxResourceCalculatorPlugin.java b/src/test/mapred/org/apache/hadoop/mapreduce/util/TestLinuxResourceCalculatorPlugin.java
new file mode 100644
index 0000000..8f0010c
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/util/TestLinuxResourceCalculatorPlugin.java
@@ -0,0 +1,236 @@
+/**
+ * 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.mapreduce.util;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.TaskTrackerStatus;
+import org.junit.Test;
+
+/**
+ * A JUnit test to test {@link LinuxResourceCalculatorPlugin}
+ * Create the fake /proc/ information and verify the parsing and calculation
+ */
+public class TestLinuxResourceCalculatorPlugin extends TestCase {
+ /**
+ * LinuxResourceCalculatorPlugin with a fake timer
+ */
+ static class FakeLinuxResourceCalculatorPlugin extends
+ LinuxResourceCalculatorPlugin {
+
+ long currentTime = 0;
+ public FakeLinuxResourceCalculatorPlugin(String procfsMemFile,
+ String procfsCpuFile,
+ String procfsStatFile,
+ long jiffyLengthInMillis) {
+ super(procfsMemFile, procfsCpuFile, procfsStatFile, jiffyLengthInMillis);
+ }
+ @Override
+ long getCurrentTime() {
+ return currentTime;
+ }
+ public void advanceTime(long adv) {
+ currentTime += adv * jiffyLengthInMillis;
+ }
+ }
+ private static final FakeLinuxResourceCalculatorPlugin plugin;
+ private static String TEST_ROOT_DIR = new Path(System.getProperty(
+ "test.build.data", "/tmp")).toString().replace(' ', '+');
+ private static final String FAKE_MEMFILE;
+ private static final String FAKE_CPUFILE;
+ private static final String FAKE_STATFILE;
+ private static final long FAKE_JIFFY_LENGTH = 10L;
+ static {
+ int randomNum = (new Random()).nextInt(1000000000);
+ FAKE_MEMFILE = TEST_ROOT_DIR + File.separator + "MEMINFO_" + randomNum;
+ FAKE_CPUFILE = TEST_ROOT_DIR + File.separator + "CPUINFO_" + randomNum;
+ FAKE_STATFILE = TEST_ROOT_DIR + File.separator + "STATINFO_" + randomNum;
+ plugin = new FakeLinuxResourceCalculatorPlugin(FAKE_MEMFILE, FAKE_CPUFILE,
+ FAKE_STATFILE,
+ FAKE_JIFFY_LENGTH);
+ }
+ static final String MEMINFO_FORMAT =
+ "MemTotal: %d kB\n" +
+ "MemFree: %d kB\n" +
+ "Buffers: 138244 kB\n" +
+ "Cached: 947780 kB\n" +
+ "SwapCached: 142880 kB\n" +
+ "Active: 3229888 kB\n" +
+ "Inactive: %d kB\n" +
+ "SwapTotal: %d kB\n" +
+ "SwapFree: %d kB\n" +
+ "Dirty: 122012 kB\n" +
+ "Writeback: 0 kB\n" +
+ "AnonPages: 2710792 kB\n" +
+ "Mapped: 24740 kB\n" +
+ "Slab: 132528 kB\n" +
+ "SReclaimable: 105096 kB\n" +
+ "SUnreclaim: 27432 kB\n" +
+ "PageTables: 11448 kB\n" +
+ "NFS_Unstable: 0 kB\n" +
+ "Bounce: 0 kB\n" +
+ "CommitLimit: 4125904 kB\n" +
+ "Committed_AS: 4143556 kB\n" +
+ "VmallocTotal: 34359738367 kB\n" +
+ "VmallocUsed: 1632 kB\n" +
+ "VmallocChunk: 34359736375 kB\n" +
+ "HugePages_Total: 0\n" +
+ "HugePages_Free: 0\n" +
+ "HugePages_Rsvd: 0\n" +
+ "Hugepagesize: 2048 kB";
+
+ static final String CPUINFO_FORMAT =
+ "processor : %s\n" +
+ "vendor_id : AuthenticAMD\n" +
+ "cpu family : 15\n" +
+ "model : 33\n" +
+ "model name : Dual Core AMD Opteron(tm) Processor 280\n" +
+ "stepping : 2\n" +
+ "cpu MHz : %f\n" +
+ "cache size : 1024 KB\n" +
+ "physical id : 0\n" +
+ "siblings : 2\n" +
+ "core id : 0\n" +
+ "cpu cores : 2\n" +
+ "fpu : yes\n" +
+ "fpu_exception : yes\n" +
+ "cpuid level : 1\n" +
+ "wp : yes\n" +
+ "flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov " +
+ "pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt lm " +
+ "3dnowext 3dnow pni lahf_lm cmp_legacy\n" +
+ "bogomips : 4792.41\n" +
+ "TLB size : 1024 4K pages\n" +
+ "clflush size : 64\n" +
+ "cache_alignment : 64\n" +
+ "address sizes : 40 bits physical, 48 bits virtual\n" +
+ "power management: ts fid vid ttp";
+
+ static final String STAT_FILE_FORMAT =
+ "cpu %d %d %d 1646495089 831319 48713 164346 0\n" +
+ "cpu0 15096055 30805 3823005 411456015 206027 13 14269 0\n" +
+ "cpu1 14760561 89890 6432036 408707910 456857 48074 130857 0\n" +
+ "cpu2 12761169 20842 3758639 413976772 98028 411 10288 0\n" +
+ "cpu3 12355207 47322 5789691 412354390 70406 213 8931 0\n" +
+ "intr 114648668 20010764 2 0 945665 2 0 0 0 0 0 0 0 4 0 0 0 0 0 0\n" +
+ "ctxt 242017731764\n" +
+ "btime 1257808753\n" +
+ "processes 26414943\n" +
+ "procs_running 1\n" +
+ "procs_blocked 0\n";
+
+ /**
+ * Test parsing /proc/stat and /proc/cpuinfo
+ * @throws IOException
+ */
+ @Test
+ public void testParsingProcStatAndCpuFile() throws IOException {
+ // Write fake /proc/cpuinfo file.
+ long numProcessors = 8;
+ long cpuFrequencyKHz = 2392781;
+ String fileContent = "";
+ for (int i = 0; i < numProcessors; i++) {
+ fileContent += String.format(CPUINFO_FORMAT, i, cpuFrequencyKHz / 1000D) +
+ "\n";
+ }
+ File tempFile = new File(FAKE_CPUFILE);
+ tempFile.deleteOnExit();
+ FileWriter fWriter = new FileWriter(FAKE_CPUFILE);
+ fWriter.write(fileContent);
+ fWriter.close();
+ assertEquals(plugin.getNumProcessors(), numProcessors);
+ assertEquals(plugin.getCpuFrequency(), cpuFrequencyKHz);
+
+ // Write fake /proc/stat file.
+ long uTime = 54972994;
+ long nTime = 188860;
+ long sTime = 19803373;
+ tempFile = new File(FAKE_STATFILE);
+ tempFile.deleteOnExit();
+ updateStatFile(uTime, nTime, sTime);
+ assertEquals(plugin.getCumulativeCpuTime(),
+ FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
+ assertEquals(plugin.getCpuUsage(), (float)(TaskTrackerStatus.UNAVAILABLE));
+
+ // Advance the time and sample again to test the CPU usage calculation
+ uTime += 100L;
+ plugin.advanceTime(200L);
+ updateStatFile(uTime, nTime, sTime);
+ assertEquals(plugin.getCumulativeCpuTime(),
+ FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
+ assertEquals(plugin.getCpuUsage(), 6.25F);
+
+ // Advance the time and sample again. This time, we call getCpuUsage() only.
+ uTime += 600L;
+ plugin.advanceTime(300L);
+ updateStatFile(uTime, nTime, sTime);
+ assertEquals(plugin.getCpuUsage(), 25F);
+
+ // Advance very short period of time (one jiffy length).
+ // In this case, CPU usage should not be updated.
+ uTime += 1L;
+ plugin.advanceTime(1L);
+ updateStatFile(uTime, nTime, sTime);
+ assertEquals(plugin.getCumulativeCpuTime(),
+ FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
+ assertEquals(plugin.getCpuUsage(), 25F); // CPU usage is not updated.
+ }
+
+ /**
+ * Write information to fake /proc/stat file
+ */
+ private void updateStatFile(long uTime, long nTime, long sTime)
+ throws IOException {
+ FileWriter fWriter = new FileWriter(FAKE_STATFILE);
+ fWriter.write(String.format(STAT_FILE_FORMAT, uTime, nTime, sTime));
+ fWriter.close();
+ }
+
+ /**
+ * Test parsing /proc/meminfo
+ * @throws IOException
+ */
+ @Test
+ public void testParsingProcMemFile() throws IOException {
+ long memTotal = 4058864L;
+ long memFree = 99632L;
+ long inactive = 567732L;
+ long swapTotal = 2096472L;
+ long swapFree = 1818480L;
+ File tempFile = new File(FAKE_MEMFILE);
+ tempFile.deleteOnExit();
+ FileWriter fWriter = new FileWriter(FAKE_MEMFILE);
+ fWriter.write(String.format(MEMINFO_FORMAT,
+ memTotal, memFree, inactive, swapTotal, swapFree));
+
+ fWriter.close();
+ assertEquals(plugin.getAvailablePhysicalMemorySize(),
+ 1024L * (memFree + inactive));
+ assertEquals(plugin.getAvailableVirtualMemorySize(),
+ 1024L * (memFree + inactive + swapFree));
+ assertEquals(plugin.getPhysicalMemorySize(), 1024L * memTotal);
+ assertEquals(plugin.getVirtualMemorySize(), 1024L * (memTotal + swapTotal));
+ }
+}
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java b/src/test/mapred/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java
new file mode 100644
index 0000000..8a40cd1
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java
@@ -0,0 +1,208 @@
+/**
+ * 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.mapreduce.util;
+
+import java.io.File;
+import java.io.IOException;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.util.MRAsyncDiskService;
+import org.junit.Test;
+
+/**
+ * A test for MRAsyncDiskService.
+ */
+public class TestMRAsyncDiskService extends TestCase {
+
+ private static String TEST_ROOT_DIR = new Path(System.getProperty(
+ "test.build.data", "/tmp")).toString();
+
+ /**
+ * This test creates some directories and then removes them through
+ * MRAsyncDiskService.
+ */
+ @Test
+ public void testMRAsyncDiskService() throws Throwable {
+
+ FileSystem localFileSystem = FileSystem.getLocal(new Configuration());
+ String[] vols = new String[]{TEST_ROOT_DIR + "/0",
+ TEST_ROOT_DIR + "/1"};
+ MRAsyncDiskService service = new MRAsyncDiskService(
+ localFileSystem, vols);
+
+ String a = "a";
+ String b = "b";
+ String c = "b/c";
+ String d = "d";
+
+ File fa = new File(vols[0], a);
+ File fb = new File(vols[1], b);
+ File fc = new File(vols[1], c);
+ File fd = new File(vols[1], d);
+
+ // Create the directories
+ fa.mkdirs();
+ fb.mkdirs();
+ fc.mkdirs();
+ fd.mkdirs();
+
+ assertTrue(fa.exists());
+ assertTrue(fb.exists());
+ assertTrue(fc.exists());
+ assertTrue(fd.exists());
+
+ // Move and delete them
+ service.moveAndDeleteRelativePath(vols[0], a);
+ assertFalse(fa.exists());
+ service.moveAndDeleteRelativePath(vols[1], b);
+ assertFalse(fb.exists());
+ assertFalse(fc.exists());
+
+ // asyncDiskService is NOT able to delete files outside all volumes.
+ IOException ee = null;
+ try {
+ service.moveAndDeleteAbsolutePath(TEST_ROOT_DIR + "/2");
+ } catch (IOException e) {
+ ee = e;
+ }
+ assertNotNull("asyncDiskService should not be able to delete files "
+ + "outside all volumes", ee);
+ // asyncDiskService is able to automatically find the file in one
+ // of the volumes.
+ assertTrue(service.moveAndDeleteAbsolutePath(vols[1] + Path.SEPARATOR_CHAR + d));
+
+ // Make sure everything is cleaned up
+ makeSureCleanedUp(vols, service);
+ }
+
+ /**
+ * This test creates some directories inside the volume roots, and then
+ * call asyncDiskService.MoveAndDeleteAllVolumes.
+ * We should be able to delete all files/dirs inside the volumes except
+ * the toBeDeleted directory.
+ */
+ @Test
+ public void testMRAsyncDiskServiceMoveAndDeleteAllVolumes() throws Throwable {
+ FileSystem localFileSystem = FileSystem.getLocal(new Configuration());
+ String[] vols = new String[]{TEST_ROOT_DIR + "/0",
+ TEST_ROOT_DIR + "/1"};
+ MRAsyncDiskService service = new MRAsyncDiskService(
+ localFileSystem, vols);
+
+ String a = "a";
+ String b = "b";
+ String c = "b/c";
+ String d = "d";
+
+ File fa = new File(vols[0], a);
+ File fb = new File(vols[1], b);
+ File fc = new File(vols[1], c);
+ File fd = new File(vols[1], d);
+
+ // Create the directories
+ fa.mkdirs();
+ fb.mkdirs();
+ fc.mkdirs();
+ fd.mkdirs();
+
+ assertTrue(fa.exists());
+ assertTrue(fb.exists());
+ assertTrue(fc.exists());
+ assertTrue(fd.exists());
+
+ // Delete all of them
+ service.cleanupAllVolumes();
+
+ assertFalse(fa.exists());
+ assertFalse(fb.exists());
+ assertFalse(fc.exists());
+ assertFalse(fd.exists());
+
+ // Make sure everything is cleaned up
+ makeSureCleanedUp(vols, service);
+ }
+
+ /**
+ * This test creates some directories inside the toBeDeleted directory and
+ * then start the asyncDiskService.
+ * AsyncDiskService will create tasks to delete the content inside the
+ * toBeDeleted directories.
+ */
+ @Test
+ public void testMRAsyncDiskServiceStartupCleaning() throws Throwable {
+ FileSystem localFileSystem = FileSystem.getLocal(new Configuration());
+ String[] vols = new String[]{TEST_ROOT_DIR + "/0",
+ TEST_ROOT_DIR + "/1"};
+
+ String a = "a";
+ String b = "b";
+ String c = "b/c";
+ String d = "d";
+
+ // Create directories inside SUBDIR
+ File fa = new File(vols[0] + Path.SEPARATOR_CHAR + MRAsyncDiskService.TOBEDELETED, a);
+ File fb = new File(vols[1] + Path.SEPARATOR_CHAR + MRAsyncDiskService.TOBEDELETED, b);
+ File fc = new File(vols[1] + Path.SEPARATOR_CHAR + MRAsyncDiskService.TOBEDELETED, c);
+ File fd = new File(vols[1] + Path.SEPARATOR_CHAR + MRAsyncDiskService.TOBEDELETED, d);
+
+ // Create the directories
+ fa.mkdirs();
+ fb.mkdirs();
+ fc.mkdirs();
+ fd.mkdirs();
+
+ assertTrue(fa.exists());
+ assertTrue(fb.exists());
+ assertTrue(fc.exists());
+ assertTrue(fd.exists());
+
+ // Create the asyncDiskService which will delete all contents inside SUBDIR
+ MRAsyncDiskService service = new MRAsyncDiskService(
+ localFileSystem, vols);
+
+ // Make sure everything is cleaned up
+ makeSureCleanedUp(vols, service);
+ }
+
+ private void makeSureCleanedUp(String[] vols, MRAsyncDiskService service)
+ throws Throwable {
+ // Sleep at most 5 seconds to make sure the deleted items are all gone.
+ service.shutdown();
+ if (!service.awaitTermination(5000)) {
+ fail("MRAsyncDiskService is still not shutdown in 5 seconds!");
+ }
+
+ // All contents should be gone by now.
+ for (int i = 0; i < vols.length; i++) {
+ File subDir = new File(vols[0]);
+ String[] subDirContent = subDir.list();
+ assertEquals("Volume should contain a single child: "
+ + MRAsyncDiskService.TOBEDELETED, 1, subDirContent.length);
+
+ File toBeDeletedDir = new File(vols[0], MRAsyncDiskService.TOBEDELETED);
+ String[] content = toBeDeletedDir.list();
+ assertNotNull("Cannot find " + toBeDeletedDir, content);
+ assertEquals("" + toBeDeletedDir + " should be empty now.", 0,
+ content.length);
+ }
+ }
+
+}
diff --git a/src/test/mapred/org/apache/hadoop/mapreduce/util/TestProcfsBasedProcessTree.java b/src/test/mapred/org/apache/hadoop/mapreduce/util/TestProcfsBasedProcessTree.java
index 4049ebf..fb02e07 100644
--- a/src/test/mapred/org/apache/hadoop/mapreduce/util/TestProcfsBasedProcessTree.java
+++ b/src/test/mapred/org/apache/hadoop/mapreduce/util/TestProcfsBasedProcessTree.java
@@ -25,6 +25,7 @@
import java.io.FileReader;
import java.io.FileWriter;
import java.io.IOException;
+import java.util.Arrays;
import java.util.Random;
import java.util.Vector;
import java.util.regex.Matcher;
@@ -181,12 +182,12 @@
LOG.info("Process-tree dump follows: \n" + processTreeDump);
assertTrue("Process-tree dump doesn't start with a proper header",
- processTreeDump.startsWith("\t|- PID PPID PGRPID SESSID CMD_NAME "
- + "VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
+ processTreeDump.startsWith("\t|- PID PPID PGRPID SESSID CMD_NAME " +
+ "USER_MODE_TIME(MILLIS) SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) " +
+ "RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
for (int i = N; i >= 0; i--) {
- String cmdLineDump =
- "\\|- [0-9]+ [0-9]+ [0-9]+ [0-9]+ \\(sh\\) [0-9]+ [0-9]+ sh " + shellScript
- + " " + i;
+ String cmdLineDump = "\\|- [0-9]+ [0-9]+ [0-9]+ [0-9]+ \\(sh\\)" +
+ " [0-9]+ [0-9]+ [0-9]+ [0-9]+ sh " + shellScript + " " + i;
Pattern pat = Pattern.compile(cmdLineDump);
Matcher mat = pat.matcher(processTreeDump);
assertTrue("Process-tree dump doesn't contain the cmdLineDump of " + i
@@ -267,6 +268,8 @@
String session;
String vmem = "0";
String rssmemPage = "0";
+ String utime = "0";
+ String stime = "0";
public ProcessStatInfo(String[] statEntries) {
pid = statEntries[0];
@@ -278,27 +281,32 @@
if (statEntries.length > 6) {
rssmemPage = statEntries[6];
}
+ if (statEntries.length > 7) {
+ utime = statEntries[7];
+ stime = statEntries[8];
+ }
}
// construct a line that mimics the procfs stat file.
// all unused numerical entries are set to 0.
public String getStatLine() {
return String.format("%s (%s) S %s %s %s 0 0 0" +
- " 0 0 0 0 0 0 0 0 0 0 0 0 0 %s %s 0 0" +
+ " 0 0 0 0 %s %s 0 0 0 0 0 0 0 %s %s 0 0" +
" 0 0 0 0 0 0 0 0" +
" 0 0 0 0 0",
- pid, name, ppid, pgrpId, session, vmem, rssmemPage);
+ pid, name, ppid, pgrpId, session,
+ utime, stime, vmem, rssmemPage);
}
}
/**
* A basic test that creates a few process directories and writes
- * stat files. Verifies that the virtual and rss memory is correctly
+ * stat files. Verifies that the cpu time and memory is correctly
* computed.
* @throws IOException if there was a problem setting up the
* fake procfs directories or files.
*/
- public void testMemoryForProcessTree() throws IOException {
+ public void testCpuAndMemoryForProcessTree() throws IOException {
// test processes
String[] pids = { "100", "200", "300", "400" };
@@ -313,13 +321,13 @@
// assuming processes 100, 200, 300 are in tree and 400 is not.
ProcessStatInfo[] procInfos = new ProcessStatInfo[4];
procInfos[0] = new ProcessStatInfo(new String[]
- {"100", "proc1", "1", "100", "100", "100000", "100"});
+ {"100", "proc1", "1", "100", "100", "100000", "100", "1000", "200"});
procInfos[1] = new ProcessStatInfo(new String[]
- {"200", "proc2", "100", "100", "100", "200000", "200"});
+ {"200", "proc2", "100", "100", "100", "200000", "200", "2000", "400"});
procInfos[2] = new ProcessStatInfo(new String[]
- {"300", "proc3", "200", "100", "100", "300000", "300"});
+ {"300", "proc3", "200", "100", "100", "300000", "300", "3000", "600"});
procInfos[3] = new ProcessStatInfo(new String[]
- {"400", "proc4", "1", "400", "400", "400000", "400"});
+ {"400", "proc4", "1", "400", "400", "400000", "400", "4000", "800"});
writeStatFiles(procfsRootDir, pids, procInfos);
@@ -339,6 +347,28 @@
600L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
assertEquals("Cumulative rss memory does not match",
cumuRssMem, processTree.getCumulativeRssmem());
+
+ // verify cumulative cpu time
+ long cumuCpuTime = ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS > 0 ?
+ 7200L * ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS : 0L;
+ assertEquals("Cumulative cpu time does not match",
+ cumuCpuTime, processTree.getCumulativeCpuTime());
+
+ // test the cpu time again to see if it cumulates
+ procInfos[0] = new ProcessStatInfo(new String[]
+ {"100", "proc1", "1", "100", "100", "100000", "100", "2000", "300"});
+ procInfos[1] = new ProcessStatInfo(new String[]
+ {"200", "proc2", "100", "100", "100", "200000", "200", "3000", "500"});
+ writeStatFiles(procfsRootDir, pids, procInfos);
+
+ // build the process tree.
+ processTree.getProcessTree();
+
+ // verify cumulative cpu time again
+ cumuCpuTime = ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS > 0 ?
+ 9400L * ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS : 0L;
+ assertEquals("Cumulative cpu time does not match",
+ cumuCpuTime, processTree.getCumulativeCpuTime());
} finally {
FileUtil.fullyDelete(procfsRootDir);
}
@@ -498,17 +528,17 @@
// Processes 200, 300, 400 and 500 are descendants of 100. 600 is not.
ProcessStatInfo[] procInfos = new ProcessStatInfo[numProcesses];
procInfos[0] = new ProcessStatInfo(new String[] {
- "100", "proc1", "1", "100", "100", "100000", "100"});
+ "100", "proc1", "1", "100", "100", "100000", "100", "1000", "200"});
procInfos[1] = new ProcessStatInfo(new String[] {
- "200", "proc2", "100", "100", "100", "200000", "200"});
+ "200", "proc2", "100", "100", "100", "200000", "200", "2000", "400"});
procInfos[2] = new ProcessStatInfo(new String[] {
- "300", "proc3", "200", "100", "100", "300000", "300"});
+ "300", "proc3", "200", "100", "100", "300000", "300", "3000", "600"});
procInfos[3] = new ProcessStatInfo(new String[] {
- "400", "proc4", "200", "100", "100", "400000", "400"});
+ "400", "proc4", "200", "100", "100", "400000", "400", "4000", "800"});
procInfos[4] = new ProcessStatInfo(new String[] {
- "500", "proc5", "400", "100", "100", "400000", "400"});
+ "500", "proc5", "400", "100", "100", "400000", "400", "4000", "800"});
procInfos[5] = new ProcessStatInfo(new String[] {
- "600", "proc6", "1", "1", "1", "400000", "400"});
+ "600", "proc6", "1", "1", "1", "400000", "400", "4000", "800"});
String[] cmdLines = new String[numProcesses];
cmdLines[0] = "proc1 arg1 arg2";
@@ -532,15 +562,17 @@
LOG.info("Process-tree dump follows: \n" + processTreeDump);
assertTrue("Process-tree dump doesn't start with a proper header",
- processTreeDump.startsWith("\t|- PID PPID PGRPID SESSID CMD_NAME "
- + "VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
+ processTreeDump.startsWith("\t|- PID PPID PGRPID SESSID CMD_NAME " +
+ "USER_MODE_TIME(MILLIS) SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) " +
+ "RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
for (int i = 0; i < 5; i++) {
ProcessStatInfo p = procInfos[i];
assertTrue(
"Process-tree dump doesn't contain the cmdLineDump of process "
+ p.pid, processTreeDump.contains("\t|- " + p.pid + " "
+ p.ppid + " " + p.pgrpId + " " + p.session + " (" + p.name
- + ") " + p.vmem + " " + p.rssmemPage + " " + cmdLines[i]));
+ + ") " + p.utime + " " + p.stime + " " + p.vmem + " "
+ + p.rssmemPage + " " + cmdLines[i]));
}
// 600 should not be in the dump
@@ -549,7 +581,7 @@
"Process-tree dump shouldn't contain the cmdLineDump of process "
+ p.pid, processTreeDump.contains("\t|- " + p.pid + " " + p.ppid
+ " " + p.pgrpId + " " + p.session + " (" + p.name + ") "
- + p.vmem + " " + p.rssmemPage + " " + cmdLines[5]));
+ + p.utime + " " + p.stime + " " + p.vmem + " " + cmdLines[5]));
} finally {
FileUtil.fullyDelete(procfsRootDir);
}
diff --git a/src/test/mapred/org/apache/hadoop/security/TestMapredGroupMappingServiceRefresh.java b/src/test/mapred/org/apache/hadoop/security/TestMapredGroupMappingServiceRefresh.java
new file mode 100644
index 0000000..ba2acd9
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/security/TestMapredGroupMappingServiceRefresh.java
@@ -0,0 +1,144 @@
+/**
+ * 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.security;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.tools.MRAdmin;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestMapredGroupMappingServiceRefresh {
+ private MiniDFSCluster cluster;
+ JobConf config;
+ private static long groupRefreshTimeoutSec = 2;
+ private static final Log LOG = LogFactory
+ .getLog(TestMapredGroupMappingServiceRefresh.class);
+
+ public static class MockUnixGroupsMapping implements GroupMappingServiceProvider {
+ private int i=0;
+
+ @Override
+ public List<String> getGroups(String user) throws IOException {
+ String g1 = user + (10 * i + 1);
+ String g2 = user + (10 * i + 2);
+ List<String> l = new ArrayList<String>(2);
+ l.add(g1);
+ l.add(g2);
+ i++;
+ return l;
+ }
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ config = new JobConf(new Configuration());
+
+ config.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+ TestMapredGroupMappingServiceRefresh.MockUnixGroupsMapping.class,
+ GroupMappingServiceProvider.class);
+ config.setLong(CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS,
+ groupRefreshTimeoutSec);
+
+ LOG.info("GROUP MAPPING class name=" +
+ config.getClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+ ShellBasedUnixGroupsMapping.class,GroupMappingServiceProvider.class).
+ getName());
+
+ String namenodeUrl = "hdfs://localhost:" + "0";
+ FileSystem.setDefaultUri(config, namenodeUrl);
+
+ cluster = new MiniDFSCluster(0, config, 1, true, true, true, null, null,
+ null, null);
+ cluster.waitActive();
+ URI uri = cluster.getURI();
+
+ MiniMRCluster miniMRCluster = new MiniMRCluster(0, uri.toString() ,
+ 3, null, null, config);
+
+ config.set(JTConfig.JT_IPC_ADDRESS, "localhost:"+miniMRCluster.getJobTrackerPort());
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if(cluster!=null) {
+ cluster.shutdown();
+ }
+ }
+
+ @Test
+ public void testGroupMappingRefresh() throws Exception {
+ MRAdmin admin = new MRAdmin(config);
+ String [] args = new String[] { "-refreshUserToGroupsMappings" };
+
+ Groups groups = SecurityUtil.getUserToGroupsMappingService(config);
+ String user = UnixUserGroupInformation.getUnixUserName();
+ System.out.println("first attempt:");
+ List<String> g1 = groups.getGroups(user);
+ String [] str_groups = new String [g1.size()];
+ g1.toArray(str_groups);
+ System.out.println(Arrays.toString(str_groups));
+
+ System.out.println("second attempt, should be same:");
+ List<String> g2 = groups.getGroups(user);
+ g2.toArray(str_groups);
+ System.out.println(Arrays.toString(str_groups));
+ for(int i=0; i<g2.size(); i++) {
+ assertEquals("Should be same group ", g1.get(i), g2.get(i));
+ }
+ // run refresh command
+ admin.run(args);
+
+ System.out.println("third attempt(after refresh command), should be different:");
+ List<String> g3 = groups.getGroups(user);
+ g3.toArray(str_groups);
+ System.out.println(Arrays.toString(str_groups));
+ for(int i=0; i<g3.size(); i++) {
+ assertFalse("Should be different group ", g1.get(i).equals(g3.get(i)));
+ }
+ System.out.println("");
+
+ // test time out
+ Thread.sleep(groupRefreshTimeoutSec*1100);
+ System.out.println("fourth attempt(after timeout), should be different:");
+ List<String> g4 = groups.getGroups(user);
+ g4.toArray(str_groups);
+ System.out.println(Arrays.toString(str_groups));
+ for(int i=0; i<g4.size(); i++) {
+ assertFalse("Should be different group ", g3.get(i).equals(g4.get(i)));
+ }
+ }
+}
diff --git a/src/test/mapred/org/apache/hadoop/security/authorize/TestServiceLevelAuthorization.java b/src/test/mapred/org/apache/hadoop/security/authorize/TestServiceLevelAuthorization.java
index 947dc9d..42221a6 100644
--- a/src/test/mapred/org/apache/hadoop/security/authorize/TestServiceLevelAuthorization.java
+++ b/src/test/mapred/org/apache/hadoop/security/authorize/TestServiceLevelAuthorization.java
@@ -58,7 +58,10 @@
JobConf mrConf = new JobConf(conf);
mr = new MiniMRCluster(slaves, fileSys.getUri().toString(), 1,
null, null, mrConf);
-
+ // make cleanup inline sothat validation of existence of these directories
+ // can be done
+ mr.setInlineCleanupThreads();
+
// Run examples
TestMiniMRWithDFS.runPI(mr, mr.createJobConf(mrConf));
TestMiniMRWithDFS.runWordCount(mr, mr.createJobConf(mrConf));
diff --git a/src/test/mapred/org/apache/hadoop/tools/TestCopyFiles.java b/src/test/mapred/org/apache/hadoop/tools/TestCopyFiles.java
index f08c717..205b11e 100644
--- a/src/test/mapred/org/apache/hadoop/tools/TestCopyFiles.java
+++ b/src/test/mapred/org/apache/hadoop/tools/TestCopyFiles.java
@@ -995,6 +995,41 @@
}
}
+ /**
+ * verify that -delete option works for other {@link FileSystem}
+ * implementations. See MAPREDUCE-1285 */
+ public void testDeleteLocal() throws Exception {
+ MiniDFSCluster cluster = null;
+ try {
+ Configuration conf = new Configuration();
+ final FileSystem localfs = FileSystem.get(LOCAL_FS, conf);
+ cluster = new MiniDFSCluster(conf, 1, true, null);
+ final FileSystem hdfs = cluster.getFileSystem();
+ final String namenode = FileSystem.getDefaultUri(conf).toString();
+ if (namenode.startsWith("hdfs://")) {
+ MyFile[] files = createFiles(URI.create(namenode), "/srcdat");
+ String destdir = TEST_ROOT_DIR + "/destdat";
+ MyFile[] localFiles = createFiles(localfs, destdir);
+ ToolRunner.run(new DistCp(conf), new String[] {
+ "-delete",
+ "-update",
+ "-log",
+ "/logs",
+ namenode+"/srcdat",
+ "file:///"+TEST_ROOT_DIR+"/destdat"});
+ assertTrue("Source and destination directories do not match.",
+ checkFiles(localfs, destdir, files));
+ assertTrue("Log directory does not exist.",
+ hdfs.exists(new Path("/logs")));
+ deldir(localfs, destdir);
+ deldir(hdfs, "/logs");
+ deldir(hdfs, "/srcdat");
+ }
+ } finally {
+ if (cluster != null) { cluster.shutdown(); }
+ }
+ }
+
/** test globbing */
public void testGlobbing() throws Exception {
String namenode = null;
@@ -1057,4 +1092,4 @@
}
return results.toString();
}
-}
\ No newline at end of file
+}
diff --git a/src/test/mapred/org/apache/hadoop/tools/rumen/TestConcurrentRead.java b/src/test/mapred/org/apache/hadoop/tools/rumen/TestConcurrentRead.java
new file mode 100644
index 0000000..ff6f716
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/tools/rumen/TestConcurrentRead.java
@@ -0,0 +1,136 @@
+/**
+ * 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.tools.rumen;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestConcurrentRead {
+ static final List<LoggedJob> cachedTrace = new ArrayList<LoggedJob>();
+ static final String traceFile =
+ "rumen/small-trace-test/job-tracker-logs-trace-output.gz";
+
+ static Configuration conf;
+ static FileSystem lfs;
+ static Path path;
+
+ @BeforeClass
+ static public void globalSetUp() throws IOException {
+ conf = new Configuration();
+ lfs = FileSystem.getLocal(conf);
+ Path rootInputDir = new Path(System.getProperty("test.tools.input.dir", ""))
+ .makeQualified(lfs.getUri(), lfs.getWorkingDirectory());
+ path = new Path(rootInputDir, traceFile);
+ JobTraceReader reader = new JobTraceReader(path, conf);
+ try {
+ LoggedJob job;
+ while ((job = reader.getNext()) != null) {
+ cachedTrace.add(job);
+ }
+ } finally {
+ reader.close();
+ }
+ }
+
+ void readAndCompare() throws IOException {
+ JobTraceReader reader = new JobTraceReader(path, conf);
+ try {
+ for (Iterator<LoggedJob> it = cachedTrace.iterator(); it.hasNext();) {
+ LoggedJob jobExpected = it.next();
+ LoggedJob jobRead = reader.getNext();
+ assertNotNull(jobRead);
+ try {
+ jobRead.deepCompare(jobExpected, null);
+ } catch (DeepInequalityException e) {
+ fail(e.toString());
+ }
+ }
+ assertNull(reader.getNext());
+ } finally {
+ reader.close();
+ }
+ }
+
+ class TestThread extends Thread {
+ final int repeat;
+ final CountDownLatch startSignal, doneSignal;
+ final Map<String, Throwable> errors;
+
+ TestThread(int id, int repeat, CountDownLatch startSignal, CountDownLatch doneSignal, Map<String, Throwable> errors) {
+ super(String.format("TestThread-%d", id));
+ this.repeat = repeat;
+ this.startSignal = startSignal;
+ this.doneSignal = doneSignal;
+ this.errors = errors;
+ }
+
+ @Override
+ public void run() {
+ try {
+ startSignal.await();
+ for (int i = 0; i < repeat; ++i) {
+ try {
+ readAndCompare();
+ } catch (Throwable e) {
+ errors.put(getName(), e);
+ break;
+ }
+ }
+ doneSignal.countDown();
+ } catch (Throwable e) {
+ errors.put(getName(), e);
+ }
+ }
+ }
+
+ @Test
+ public void testConcurrentRead() throws InterruptedException {
+ int nThr = conf.getInt("test.rumen.concurrent-read.threads", 4);
+ int repeat = conf.getInt("test.rumen.concurrent-read.repeat", 10);
+ CountDownLatch startSignal = new CountDownLatch(1);
+ CountDownLatch doneSignal = new CountDownLatch(nThr);
+ Map<String, Throwable> errors = Collections
+ .synchronizedMap(new TreeMap<String, Throwable>());
+ for (int i = 0; i < nThr; ++i) {
+ new TestThread(i, repeat, startSignal, doneSignal, errors).start();
+ }
+ startSignal.countDown();
+ doneSignal.await();
+ if (!errors.isEmpty()) {
+ StringBuilder sb = new StringBuilder();
+ for (Map.Entry<String, Throwable> e : errors.entrySet()) {
+ sb.append(String.format("%s:\n%s\n", e.getKey(), e.getValue().toString()));
+ }
+ fail(sb.toString());
+ }
+ }
+}
diff --git a/src/test/mapred/org/apache/hadoop/tools/rumen/TestRumenFolder.java b/src/test/mapred/org/apache/hadoop/tools/rumen/TestRumenFolder.java
new file mode 100644
index 0000000..9d31ec5
--- /dev/null
+++ b/src/test/mapred/org/apache/hadoop/tools/rumen/TestRumenFolder.java
@@ -0,0 +1,104 @@
+package org.apache.hadoop.tools.rumen;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestRumenFolder {
+ @Test
+ public void testFoldingSmallTrace() throws Exception {
+ final Configuration conf = new Configuration();
+ final FileSystem lfs = FileSystem.getLocal(conf);
+
+ @SuppressWarnings("deprecation")
+ final Path rootInputDir =
+ new Path(System.getProperty("test.tools.input.dir", ""))
+ .makeQualified(lfs);
+ @SuppressWarnings("deprecation")
+ final Path rootTempDir =
+ new Path(System.getProperty("test.build.data", "/tmp"))
+ .makeQualified(lfs);
+
+ final Path rootInputFile = new Path(rootInputDir, "rumen/small-trace-test");
+ final Path tempDir = new Path(rootTempDir, "TestRumenJobTraces");
+ lfs.delete(tempDir, true);
+
+ final Path foldedTracePath = new Path(tempDir, "folded-trace.json");
+
+ final Path inputFile =
+ new Path(rootInputFile, "folder-input-trace.json.gz");
+
+ System.out.println("folded trace result path = " + foldedTracePath);
+
+ String[] args =
+ { "-input-cycle", "100S", "-output-duration", "300S",
+ "-skew-buffer-length", "1", "-seed", "100", "-concentration", "2",
+ inputFile.toString(), foldedTracePath.toString() };
+
+ final Path foldedGoldFile =
+ new Path(rootInputFile, "goldFoldedTrace.json.gz");
+
+ Folder folder = new Folder();
+ int result = ToolRunner.run(folder, args);
+ assertEquals("Non-zero exit", 0, result);
+
+ TestRumenFolder.<LoggedJob> jsonFileMatchesGold(conf, lfs, foldedTracePath,
+ foldedGoldFile, LoggedJob.class, "trace");
+ }
+
+ static private <T extends DeepCompare> void jsonFileMatchesGold(
+ Configuration conf, FileSystem lfs, Path result, Path gold,
+ Class<? extends T> clazz, String fileDescription) throws IOException {
+ JsonObjectMapperParser<T> goldParser =
+ new JsonObjectMapperParser<T>(gold, clazz, conf);
+ InputStream resultStream = lfs.open(result);
+ JsonObjectMapperParser<T> resultParser =
+ new JsonObjectMapperParser<T>(resultStream, clazz);
+ try {
+ while (true) {
+ DeepCompare goldJob = goldParser.getNext();
+ DeepCompare resultJob = resultParser.getNext();
+ if ((goldJob == null) || (resultJob == null)) {
+ assertTrue(goldJob == resultJob);
+ break;
+ }
+
+ try {
+ resultJob.deepCompare(goldJob, new TreePath(null, "<root>"));
+ } catch (DeepInequalityException e) {
+ String error = e.path.toString();
+
+ assertFalse(fileDescription + " mismatches: " + error, true);
+ }
+ }
+ } finally {
+ IOUtils.cleanup(null, goldParser, resultParser);
+ }
+ }
+}
diff --git a/src/test/mapred/org/apache/hadoop/util/TestReflectionUtils.java b/src/test/mapred/org/apache/hadoop/util/TestReflectionUtils.java
index 9561b7c..7b70979 100644
--- a/src/test/mapred/org/apache/hadoop/util/TestReflectionUtils.java
+++ b/src/test/mapred/org/apache/hadoop/util/TestReflectionUtils.java
@@ -16,109 +16,33 @@
* limitations under the License.
*/
-package org.apache.hadoop.util;
-
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.HashMap;
+package org.apache.hadoop.util;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.JobConfigurable;
-
-import junit.framework.TestCase;
-
-public class TestReflectionUtils extends TestCase {
-
- private static Class toConstruct[] = { String.class, TestReflectionUtils.class, HashMap.class };
- private Throwable failure = null;
-
- public void setUp() {
- ReflectionUtils.clearCache();
- }
-
- public void testCache() throws Exception {
- assertEquals(0, cacheSize());
- doTestCache();
- assertEquals(toConstruct.length, cacheSize());
- ReflectionUtils.clearCache();
- assertEquals(0, cacheSize());
- }
-
-
- @SuppressWarnings("unchecked")
- private void doTestCache() {
- for (int i=0; i<toConstruct.length; i++) {
- Class cl = toConstruct[i];
- Object x = ReflectionUtils.newInstance(cl, null);
- Object y = ReflectionUtils.newInstance(cl, null);
- assertEquals(cl, x.getClass());
- assertEquals(cl, y.getClass());
- }
- }
-
- public void testThreadSafe() throws Exception {
- Thread[] th = new Thread[32];
- for (int i=0; i<th.length; i++) {
- th[i] = new Thread() {
- public void run() {
- try {
- doTestCache();
- } catch (Throwable t) {
- failure = t;
- }
- }
- };
- th[i].start();
- }
- for (int i=0; i<th.length; i++) {
- th[i].join();
- }
- if (failure != null) {
- failure.printStackTrace();
- fail(failure.getMessage());
- }
- }
-
- private int cacheSize() throws Exception {
- return ReflectionUtils.getCacheSize();
- }
-
- public void testCantCreate() {
- try {
- ReflectionUtils.newInstance(NoDefaultCtor.class, null);
- fail("invalid call should fail");
- } catch (RuntimeException rte) {
- assertEquals(NoSuchMethodException.class, rte.getCause().getClass());
- }
- }
-
- @SuppressWarnings("unchecked")
- public void testCacheDoesntLeak() throws Exception {
- int iterations=9999; // very fast, but a bit less reliable - bigger numbers force GC
- for (int i=0; i<iterations; i++) {
- URLClassLoader loader = new URLClassLoader(new URL[0], getClass().getClassLoader());
- Class cl = Class.forName("org.apache.hadoop.util.TestReflectionUtils$LoadedInChild", false, loader);
- Object o = ReflectionUtils.newInstance(cl, null);
- assertEquals(cl, o.getClass());
- }
- System.gc();
- assertTrue(cacheSize()+" too big", cacheSize()<iterations);
- }
-
- private static class LoadedInChild {
- }
-
- public static class NoDefaultCtor {
- public NoDefaultCtor(int x) {}
+
+import static org.junit.Assert.*;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test for the JobConf-related parts of common's ReflectionUtils
+ * class.
+ */
+public class TestReflectionUtils {
+ @Before
+ public void setUp() {
+ ReflectionUtils.clearCache();
}
-
+
/**
* This is to test backward compatibility of ReflectionUtils for
* JobConfigurable objects.
* This should be made deprecated along with the mapred package HADOOP-1230.
* Should be removed when mapred package is removed.
*/
+ @Test
public void testSetConf() {
JobConfigurableOb ob = new JobConfigurableOb();
ReflectionUtils.setConf(ob, new Configuration());
@@ -132,5 +56,5 @@
public void configure(JobConf job) {
configured = true;
}
- }
-}
+ }
+}
diff --git a/src/test/tools/data/rumen/small-trace-test/folder-input-trace.json.gz b/src/test/tools/data/rumen/small-trace-test/folder-input-trace.json.gz
new file mode 100644
index 0000000..62cd24f
--- /dev/null
+++ b/src/test/tools/data/rumen/small-trace-test/folder-input-trace.json.gz
Binary files differ
diff --git a/src/test/tools/data/rumen/small-trace-test/goldFoldedTrace.json.gz b/src/test/tools/data/rumen/small-trace-test/goldFoldedTrace.json.gz
new file mode 100644
index 0000000..5c26718
--- /dev/null
+++ b/src/test/tools/data/rumen/small-trace-test/goldFoldedTrace.json.gz
Binary files differ
diff --git a/src/test/unit/org/apache/hadoop/mapred/TestLostTaskTracker.java b/src/test/unit/org/apache/hadoop/mapred/TestLostTaskTracker.java
new file mode 100644
index 0000000..99035b8
--- /dev/null
+++ b/src/test/unit/org/apache/hadoop/mapred/TestLostTaskTracker.java
@@ -0,0 +1,113 @@
+/**
+ * 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.mapred;
+
+import static org.mockito.Mockito.*;
+
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.mapred.UtilsForTests.FakeClock;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
+import org.hamcrest.Matcher;
+import org.mockito.ArgumentCaptor;
+import org.mockito.ArgumentMatcher;
+
+/**
+ * Tests that trackers that don't heartbeat within a given time are considered
+ * lost. Note that this test is not a direct replacement for
+ * {@link TestLostTracker} since it doesn't test that a task
+ * running on a lost tracker is retried on another tracker.
+ */
+@SuppressWarnings("deprecation")
+public class TestLostTaskTracker extends TestCase {
+
+ private JobTracker jobTracker;
+
+ private FakeClock clock;
+
+ @Override
+ protected void setUp() throws Exception {
+ JobConf conf = new JobConf();
+ conf.set(JTConfig.JT_IPC_ADDRESS, "localhost:0");
+ conf.set(JTConfig.JT_HTTP_ADDRESS, "0.0.0.0:0");
+ conf.setLong(JTConfig.JT_TRACKER_EXPIRY_INTERVAL, 1000);
+ clock = new FakeClock();
+ // We use a "partial mock" of JobTracker which lets us see when certain
+ // methods are called. If we were writing JobTracker from scratch then
+ // we would make it call another object which we would mock out instead
+ // (and use a real JobTracker) so we could perform assertions on the mock.
+ // See http://mockito.googlecode.com/svn/branches/1.8.0/javadoc/org/mockito/Mockito.html#16
+ jobTracker = spy(new JobTracker(conf, clock));
+ }
+
+ public void testLostTaskTrackerCalledAfterExpiryTime() throws IOException {
+
+ String tracker1 = "tracker_tracker1:1000";
+ String tracker2 = "tracker_tracker2:1000";
+
+ establishFirstContact(tracker1);
+
+ // Wait long enough for tracker1 to be considered lost
+ // We could have used a Mockito stub here, except we don't know how many
+ // times JobTracker calls getTime() on the clock, so a static mock
+ // is appropriate.
+ clock.advance(8 * 1000);
+
+ establishFirstContact(tracker2);
+
+ jobTracker.checkExpiredTrackers();
+
+ // Now we check that JobTracker's lostTaskTracker() was called for tracker1
+ // but not for tracker2.
+
+ // We use an ArgumentCaptor to capture the task tracker object
+ // in the lostTaskTracker() call, so we can perform an assertion on its
+ // name. (We could also have used a custom matcher, see below.)
+ // See http://mockito.googlecode.com/svn/branches/1.8.0/javadoc/org/mockito/Mockito.html#15
+ ArgumentCaptor<TaskTracker> argument =
+ ArgumentCaptor.forClass(TaskTracker.class);
+
+ verify(jobTracker).lostTaskTracker(argument.capture());
+ assertEquals(tracker1, argument.getValue().getTrackerName());
+
+ // Check tracker2 was not lost by using the never() construct
+ // We use a custom Hamcrest matcher to check that it was indeed tracker2
+ // that didn't match (since tracker1 did match).
+ // See http://mockito.googlecode.com/svn/branches/1.8.0/javadoc/org/mockito/Mockito.html#3
+ verify(jobTracker, never()).lostTaskTracker(
+ argThat(taskTrackerWithName(tracker2)));
+ }
+
+ private Matcher<TaskTracker> taskTrackerWithName(final String name) {
+ return new ArgumentMatcher<TaskTracker>() {
+ public boolean matches(Object taskTracker) {
+ return name.equals(((TaskTracker) taskTracker).getTrackerName());
+ }
+ };
+ }
+
+ private void establishFirstContact(String tracker) throws IOException {
+ TaskTrackerStatus status = new TaskTrackerStatus(tracker,
+ JobInProgress.convertTrackerNameToHostName(tracker));
+ jobTracker.heartbeat(status, false, true, false, (short) 0);
+ }
+
+}
\ No newline at end of file
diff --git a/src/test/unit/org/apache/hadoop/mapreduce/lib/jobcontrol/TestControlledJob.java b/src/test/unit/org/apache/hadoop/mapreduce/lib/jobcontrol/TestControlledJob.java
new file mode 100644
index 0000000..b893fe1
--- /dev/null
+++ b/src/test/unit/org/apache/hadoop/mapreduce/lib/jobcontrol/TestControlledJob.java
@@ -0,0 +1,46 @@
+/**
+ * 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.mapreduce.lib.jobcontrol;
+
+import static org.junit.Assert.assertFalse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+/**
+ */
+public class TestControlledJob {
+
+ @Test
+ public void testAddingDependingJobToRunningJobFails() throws Exception {
+ Configuration conf = new Configuration();
+ ControlledJob job1 = new ControlledJob(conf);
+ job1.setJobState(ControlledJob.State.RUNNING);
+ assertFalse(job1.addDependingJob(new ControlledJob(conf)));
+ }
+
+ @Test
+ public void testAddingDependingJobToCompletedJobFails() throws Exception {
+ Configuration conf = new Configuration();
+ ControlledJob job1 = new ControlledJob(conf);
+ job1.setJobState(ControlledJob.State.SUCCESS);
+ assertFalse(job1.addDependingJob(new ControlledJob(conf)));
+ }
+
+}
diff --git a/src/test/unit/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java b/src/test/unit/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java
new file mode 100644
index 0000000..cc72949
--- /dev/null
+++ b/src/test/unit/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java
@@ -0,0 +1,142 @@
+/**
+ * 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.mapreduce.lib.jobcontrol;
+
+import static junit.framework.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.junit.Test;
+
+/**
+ * Tests the JobControl API using mock and stub Job instances.
+ */
+public class TestMapReduceJobControlWithMocks {
+
+ @Test
+ public void testSuccessfulJobs() throws Exception {
+ JobControl jobControl = new JobControl("Test");
+
+ ControlledJob job1 = createSuccessfulControlledJob(jobControl);
+ ControlledJob job2 = createSuccessfulControlledJob(jobControl);
+ ControlledJob job3 = createSuccessfulControlledJob(jobControl, job1, job2);
+ ControlledJob job4 = createSuccessfulControlledJob(jobControl, job3);
+
+ runJobControl(jobControl);
+
+ assertEquals("Success list", 4, jobControl.getSuccessfulJobList().size());
+ assertEquals("Failed list", 0, jobControl.getFailedJobList().size());
+
+ assertTrue(job1.getJobState() == ControlledJob.State.SUCCESS);
+ assertTrue(job2.getJobState() == ControlledJob.State.SUCCESS);
+ assertTrue(job3.getJobState() == ControlledJob.State.SUCCESS);
+ assertTrue(job4.getJobState() == ControlledJob.State.SUCCESS);
+
+ jobControl.stop();
+ }
+
+ @Test
+ public void testFailedJob() throws Exception {
+ JobControl jobControl = new JobControl("Test");
+
+ ControlledJob job1 = createFailedControlledJob(jobControl);
+ ControlledJob job2 = createSuccessfulControlledJob(jobControl);
+ ControlledJob job3 = createSuccessfulControlledJob(jobControl, job1, job2);
+ ControlledJob job4 = createSuccessfulControlledJob(jobControl, job3);
+
+ runJobControl(jobControl);
+
+ assertEquals("Success list", 1, jobControl.getSuccessfulJobList().size());
+ assertEquals("Failed list", 3, jobControl.getFailedJobList().size());
+
+ assertTrue(job1.getJobState() == ControlledJob.State.FAILED);
+ assertTrue(job2.getJobState() == ControlledJob.State.SUCCESS);
+ assertTrue(job3.getJobState() == ControlledJob.State.DEPENDENT_FAILED);
+ assertTrue(job4.getJobState() == ControlledJob.State.DEPENDENT_FAILED);
+
+ jobControl.stop();
+ }
+
+ @Test
+ public void testKillJob() throws Exception {
+ JobControl jobControl = new JobControl("Test");
+
+ ControlledJob job = createFailedControlledJob(jobControl);
+
+ job.killJob();
+
+ // Verify that killJob() was called on the mock Job
+ verify(job.getJob()).killJob();
+ }
+
+ private Job createJob(boolean complete, boolean successful)
+ throws IOException, InterruptedException {
+ // Create a stub Job that responds in a controlled way
+ Job mockJob = mock(Job.class);
+ when(mockJob.getConfiguration()).thenReturn(new Configuration());
+ when(mockJob.isComplete()).thenReturn(complete);
+ when(mockJob.isSuccessful()).thenReturn(successful);
+ return mockJob;
+ }
+
+ private ControlledJob createControlledJob(JobControl jobControl,
+ boolean successful, ControlledJob... dependingJobs)
+ throws IOException, InterruptedException {
+ List<ControlledJob> dependingJobsList = dependingJobs == null ? null :
+ Arrays.asList(dependingJobs);
+ ControlledJob job = new ControlledJob(createJob(true, successful),
+ dependingJobsList);
+ jobControl.addJob(job);
+ return job;
+ }
+
+ private ControlledJob createSuccessfulControlledJob(JobControl jobControl,
+ ControlledJob... dependingJobs) throws IOException, InterruptedException {
+ return createControlledJob(jobControl, true, dependingJobs);
+ }
+
+ private ControlledJob createFailedControlledJob(JobControl jobControl,
+ ControlledJob... dependingJobs) throws IOException, InterruptedException {
+ return createControlledJob(jobControl, false, dependingJobs);
+ }
+
+ private void runJobControl(JobControl jobControl) {
+ Thread controller = new Thread(jobControl);
+ controller.start();
+ waitTillAllFinished(jobControl);
+ }
+
+ private void waitTillAllFinished(JobControl jobControl) {
+ while (!jobControl.allFinished()) {
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ // ignore
+ }
+ }
+ }
+}
diff --git a/src/tools/org/apache/hadoop/tools/DistCh.java b/src/tools/org/apache/hadoop/tools/DistCh.java
index 7a5f166..4fa6ea0 100644
--- a/src/tools/org/apache/hadoop/tools/DistCh.java
+++ b/src/tools/org/apache/hadoop/tools/DistCh.java
@@ -24,6 +24,8 @@
import java.util.List;
import java.util.Stack;
+import javax.security.auth.login.LoginException;
+
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -45,6 +47,10 @@
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.SequenceFileRecordReader;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobSubmissionFiles;
+import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ToolRunner;
@@ -420,10 +426,21 @@
return Math.max(numMaps, 1);
}
- private boolean setup(List<FileOperation> ops, Path log) throws IOException {
+ private boolean setup(List<FileOperation> ops, Path log)
+ throws IOException {
final String randomId = getRandomId();
JobClient jClient = new JobClient(jobconf);
- Path jobdir = new Path(jClient.getSystemDir(), NAME + "_" + randomId);
+ Path stagingArea;
+ try {
+ stagingArea = JobSubmissionFiles.getStagingDir(
+ jClient.getClusterHandle(), jobconf);
+ } catch (InterruptedException ie){
+ throw new IOException(ie);
+ }
+ Path jobdir = new Path(stagingArea + NAME + "_" + randomId);
+ FsPermission mapredSysPerms =
+ new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
+ FileSystem.mkdirs(jClient.getFs(), jobdir, mapredSysPerms);
LOG.info(JOB_DIR_LABEL + "=" + jobdir);
if (log == null) {
diff --git a/src/tools/org/apache/hadoop/tools/DistCp.java b/src/tools/org/apache/hadoop/tools/DistCp.java
index 770851a..c081142 100644
--- a/src/tools/org/apache/hadoop/tools/DistCp.java
+++ b/src/tools/org/apache/hadoop/tools/DistCp.java
@@ -34,6 +34,8 @@
import java.util.Stack;
import java.util.StringTokenizer;
+import javax.security.auth.login.LoginException;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -66,7 +68,10 @@
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.SequenceFileRecordReader;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobSubmissionFiles;
import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@@ -1196,9 +1201,22 @@
final String randomId = getRandomId();
JobClient jClient = new JobClient(jobConf);
- Path jobDirectory = new Path(jClient.getSystemDir(), NAME + "_" + randomId);
+ Path stagingArea;
+ try {
+ stagingArea =
+ JobSubmissionFiles.getStagingDir(jClient.getClusterHandle(), conf);
+ } catch (InterruptedException ie) {
+ throw new IOException(ie);
+ }
+
+ Path jobDirectory = new Path(stagingArea + NAME + "_" + randomId);
+ FsPermission mapredSysPerms =
+ new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
+ FileSystem.mkdirs(jClient.getFs(), jobDirectory, mapredSysPerms);
jobConf.set(JOB_DIR_LABEL, jobDirectory.toString());
+ long maxBytesPerMap = conf.getLong(BYTES_PER_MAP_LABEL, BYTES_PER_MAP);
+
FileSystem dstfs = args.dst.getFileSystem(conf);
boolean dstExists = dstfs.exists(args.dst);
boolean dstIsDir = false;
@@ -1365,7 +1383,7 @@
++cnsyncf;
cbsyncs += child.getLen();
- if (cnsyncf > SYNC_FILE_MAX || cbsyncs > BYTES_PER_MAP) {
+ if (cnsyncf > SYNC_FILE_MAX || cbsyncs > maxBytesPerMap) {
src_writer.sync();
dst_writer.sync();
cnsyncf = 0;
@@ -1540,7 +1558,7 @@
//write dst lsr results
final Path dstlsr = new Path(jobdir, "_distcp_dst_lsr");
final SequenceFile.Writer writer = SequenceFile.createWriter(jobfs, jobconf,
- dstlsr, Text.class, FileStatus.class,
+ dstlsr, Text.class, dstroot.getClass(),
SequenceFile.CompressionType.NONE);
try {
//do lsr to get all file statuses in dstroot
diff --git a/src/tools/org/apache/hadoop/tools/DistCp_Counter.properties b/src/tools/org/apache/hadoop/tools/DistCp_Counter.properties
index e6e21de..2234619 100644
--- a/src/tools/org/apache/hadoop/tools/DistCp_Counter.properties
+++ b/src/tools/org/apache/hadoop/tools/DistCp_Counter.properties
@@ -1,3 +1,15 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
# ResourceBundle properties file for distcp counters
CounterGroupName= distcp
diff --git a/src/tools/org/apache/hadoop/tools/HadoopArchives.java b/src/tools/org/apache/hadoop/tools/HadoopArchives.java
index 6108343..3a1663b 100644
--- a/src/tools/org/apache/hadoop/tools/HadoopArchives.java
+++ b/src/tools/org/apache/hadoop/tools/HadoopArchives.java
@@ -29,6 +29,8 @@
import java.util.Set;
import java.util.TreeMap;
+import javax.security.auth.login.LoginException;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -38,6 +40,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.HarFileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.SequenceFile;
@@ -56,7 +59,12 @@
import org.apache.hadoop.mapred.SequenceFileRecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.lib.NullOutputFormat;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobSubmissionFiles;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@@ -359,12 +367,22 @@
}
conf.set(DST_DIR_LABEL, outputPath.toString());
final String randomId = DistCp.getRandomId();
- Path jobDirectory = new Path(new JobClient(conf).getSystemDir(),
+ Path stagingArea;
+ try {
+ stagingArea = JobSubmissionFiles.getStagingDir(new Cluster(conf),
+ conf);
+ } catch (InterruptedException ie) {
+ throw new IOException(ie);
+ }
+ Path jobDirectory = new Path(stagingArea,
NAME + "_" + randomId);
+ FsPermission mapredSysPerms =
+ new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
+ FileSystem.mkdirs(jobDirectory.getFileSystem(conf), jobDirectory,
+ mapredSysPerms);
conf.set(JOB_DIR_LABEL, jobDirectory.toString());
//get a tmp directory for input splits
FileSystem jobfs = jobDirectory.getFileSystem(conf);
- jobfs.mkdirs(jobDirectory);
Path srcFiles = new Path(jobDirectory, "_har_src_files");
conf.set(SRC_LIST_LABEL, srcFiles.toString());
SequenceFile.Writer srcWriter = SequenceFile.createWriter(jobfs, conf,
diff --git a/src/tools/org/apache/hadoop/tools/rumen/DeskewedJobTraceReader.java b/src/tools/org/apache/hadoop/tools/rumen/DeskewedJobTraceReader.java
new file mode 100644
index 0000000..c2512e3
--- /dev/null
+++ b/src/tools/org/apache/hadoop/tools/rumen/DeskewedJobTraceReader.java
@@ -0,0 +1,199 @@
+/**
+ * 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.tools.rumen;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.PriorityQueue;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class DeskewedJobTraceReader implements Closeable {
+ // underlying engine
+ private final JobTraceReader reader;
+
+ // configuration variables
+ private final int skewBufferLength;
+
+ private final boolean abortOnUnfixableSkew;
+
+ // state variables
+ private long skewMeasurementLatestSubmitTime = Long.MIN_VALUE;
+
+ private long returnedLatestSubmitTime = Long.MIN_VALUE;
+
+ private int maxSkewBufferNeeded = 0;
+
+ // a submit time will NOT be in countedRepeatedSubmitTimesSoFar if
+ // it only occurs once. This situation is represented by having the
+ // time in submitTimesSoFar only. A submit time that occurs twice or more
+ // appears in countedRepeatedSubmitTimesSoFar [with the appropriate range
+ // value] AND submitTimesSoFar
+ private TreeMap<Long, Integer> countedRepeatedSubmitTimesSoFar =
+ new TreeMap<Long, Integer>();
+ private TreeSet<Long> submitTimesSoFar = new TreeSet<Long>();
+
+ private final PriorityQueue<LoggedJob> skewBuffer;
+
+ static final private Log LOG =
+ LogFactory.getLog(DeskewedJobTraceReader.class);
+
+ static private class JobComparator implements Comparator<LoggedJob> {
+ @Override
+ public int compare(LoggedJob j1, LoggedJob j2) {
+ return (j1.getSubmitTime() < j2.getSubmitTime()) ? -1 : (j1
+ .getSubmitTime() == j2.getSubmitTime()) ? 0 : 1;
+ }
+ }
+
+ /**
+ * Constructor.
+ *
+ * @param reader
+ * the {@link JobTraceReader} that's being protected
+ * @param skewBufferSize
+ * [the number of late jobs that can preced a later out-of-order
+ * earlier job
+ * @throws IOException
+ */
+ public DeskewedJobTraceReader(JobTraceReader reader, int skewBufferLength,
+ boolean abortOnUnfixableSkew) throws IOException {
+ this.reader = reader;
+
+ this.skewBufferLength = skewBufferLength;
+
+ this.abortOnUnfixableSkew = abortOnUnfixableSkew;
+
+ skewBuffer =
+ new PriorityQueue<LoggedJob>(skewBufferLength + 1, new JobComparator());
+
+ fillSkewBuffer();
+ }
+
+ public DeskewedJobTraceReader(JobTraceReader reader) throws IOException {
+ this(reader, 0, true);
+ }
+
+ private LoggedJob rawNextJob() throws IOException {
+ LoggedJob result = reader.getNext();
+
+ if ((!abortOnUnfixableSkew || skewBufferLength > 0) && result != null) {
+ long thisTime = result.getSubmitTime();
+
+ if (submitTimesSoFar.contains(thisTime)) {
+ Integer myCount = countedRepeatedSubmitTimesSoFar.get(thisTime);
+
+ countedRepeatedSubmitTimesSoFar.put(thisTime, myCount == null ? 2
+ : myCount + 1);
+ } else {
+ submitTimesSoFar.add(thisTime);
+ }
+
+ if (thisTime < skewMeasurementLatestSubmitTime) {
+ Iterator<Long> endCursor = submitTimesSoFar.descendingIterator();
+
+ int thisJobNeedsSkew = 0;
+
+ Long keyNeedingSkew;
+
+ while (endCursor.hasNext()
+ && (keyNeedingSkew = endCursor.next()) > thisTime) {
+ Integer keyNeedsSkewAmount =
+ countedRepeatedSubmitTimesSoFar.get(keyNeedingSkew);
+
+ thisJobNeedsSkew +=
+ keyNeedsSkewAmount == null ? 1 : keyNeedsSkewAmount;
+ }
+
+ maxSkewBufferNeeded = Math.max(maxSkewBufferNeeded, thisJobNeedsSkew);
+ }
+
+ skewMeasurementLatestSubmitTime =
+ Math.max(thisTime, skewMeasurementLatestSubmitTime);
+ }
+
+ return result;
+ }
+
+ static class OutOfOrderException extends RuntimeException {
+ static final long serialVersionUID = 1L;
+
+ public OutOfOrderException(String text) {
+ super(text);
+ }
+ }
+
+ LoggedJob nextJob() throws IOException, OutOfOrderException {
+ LoggedJob newJob = rawNextJob();
+
+ if (newJob != null) {
+ skewBuffer.add(newJob);
+ }
+
+ LoggedJob result = skewBuffer.poll();
+
+ while (result != null && result.getSubmitTime() < returnedLatestSubmitTime) {
+ LOG.error("The current job was submitted earlier than the previous one");
+ LOG.error("Its jobID is " + result.getJobID());
+ LOG.error("Its submit time is " + result.getSubmitTime()
+ + ",but the previous one was " + returnedLatestSubmitTime);
+
+ if (abortOnUnfixableSkew) {
+ throw new OutOfOrderException("Job submit time is "
+ + result.getSubmitTime() + ",but the previous one was "
+ + returnedLatestSubmitTime);
+ }
+
+ result = rawNextJob();
+ }
+
+ if (result != null) {
+ returnedLatestSubmitTime = result.getSubmitTime();
+ }
+
+ return result;
+ }
+
+ private void fillSkewBuffer() throws IOException {
+ for (int i = 0; i < skewBufferLength; ++i) {
+ LoggedJob newJob = rawNextJob();
+
+ if (newJob == null) {
+ return;
+ }
+
+ skewBuffer.add(newJob);
+ }
+ }
+
+ int neededSkewBufferSize() {
+ return maxSkewBufferNeeded;
+ }
+
+ @Override
+ public void close() throws IOException {
+ reader.close();
+ }
+
+}
diff --git a/src/tools/org/apache/hadoop/tools/rumen/Folder.java b/src/tools/org/apache/hadoop/tools/rumen/Folder.java
new file mode 100644
index 0000000..fff63b9
--- /dev/null
+++ b/src/tools/org/apache/hadoop/tools/rumen/Folder.java
@@ -0,0 +1,561 @@
+/**
+ * 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.tools.rumen;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.codehaus.jackson.JsonEncoding;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonGenerator;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+
+public class Folder extends Configured implements Tool {
+ private long outputDuration = -1;
+ private long inputCycle = -1;
+ private double concentration = 1.0;
+ private long randomSeed = 0; // irrelevant if seeded == false
+ private boolean seeded = false;
+ private boolean debug = false;
+ private boolean allowMissorting = false;
+ private int skewBufferLength = 0;
+
+ static final private Log LOG = LogFactory.getLog(Folder.class);
+
+ private DeskewedJobTraceReader reader = null;
+ private JsonGenerator outGen = null;
+
+ private List<Path> tempPaths = new LinkedList<Path>();
+
+ private Path tempDir = null;
+
+ private long firstJobSubmitTime;
+
+ private double timeDilation;
+
+ private double transcriptionRateFraction;
+
+ private int transcriptionRateInteger;
+
+ private Random random;
+
+ static private final long TICKS_PER_SECOND = 1000L;
+
+ // error return codes
+ static private final int NON_EXISTENT_FILES = 1;
+ static private final int NO_INPUT_CYCLE_LENGTH = 2;
+ static private final int EMPTY_JOB_TRACE = 3;
+ static private final int OUT_OF_ORDER_JOBS = 4;
+ static private final int ALL_JOBS_SIMULTANEOUS = 5;
+ static private final int IO_ERROR = 6;
+ static private final int OTHER_ERROR = 7;
+
+ private Set<Closeable> closees = new HashSet<Closeable>();
+ private Set<Path> deletees = new HashSet<Path>();
+
+ static long parseDuration(String durationString) {
+ String numeral = durationString.substring(0, durationString.length() - 1);
+ char durationCode = durationString.charAt(durationString.length() - 1);
+
+ long result = Integer.parseInt(numeral);
+
+ if (result <= 0) {
+ throw new IllegalArgumentException("Negative durations are not allowed");
+ }
+
+ switch (durationCode) {
+ case 'D':
+ case 'd':
+ return 24L * 60L * 60L * TICKS_PER_SECOND * result;
+ case 'H':
+ case 'h':
+ return 60L * 60L * TICKS_PER_SECOND * result;
+ case 'M':
+ case 'm':
+ return 60L * TICKS_PER_SECOND * result;
+ case 'S':
+ case 's':
+ return TICKS_PER_SECOND * result;
+ default:
+ throw new IllegalArgumentException("Missing or invalid duration code");
+ }
+ }
+
+ private int initialize(String[] args) throws IllegalArgumentException {
+ String tempDirName = null;
+ String inputPathName = null;
+ String outputPathName = null;
+
+ for (int i = 0; i < args.length; ++i) {
+ String thisArg = args[i];
+
+ if (thisArg.equalsIgnoreCase("-output-duration")) {
+ outputDuration = parseDuration(args[++i]);
+ } else if (thisArg.equalsIgnoreCase("-input-cycle")) {
+ inputCycle = parseDuration(args[++i]);
+ } else if (thisArg.equalsIgnoreCase("-concentration")) {
+ concentration = Double.parseDouble(args[++i]);
+ } else if (thisArg.equalsIgnoreCase("-debug")) {
+ debug = true;
+ } else if (thisArg.equalsIgnoreCase("-allow-missorting")) {
+ allowMissorting = true;
+ } else if (thisArg.equalsIgnoreCase("-seed")) {
+ seeded = true;
+ randomSeed = Long.parseLong(args[++i]);
+ } else if (thisArg.equalsIgnoreCase("-skew-buffer-length")) {
+ skewBufferLength = Integer.parseInt(args[++i]);
+ } else if (thisArg.equalsIgnoreCase("-temp-directory")) {
+ tempDirName = args[++i];
+ } else if (thisArg.equals("") || thisArg.startsWith("-")) {
+ throw new IllegalArgumentException("Illegal switch argument, "
+ + thisArg + " at position " + i);
+ } else {
+ inputPathName = thisArg;
+ outputPathName = args[++i];
+
+ if (i != args.length - 1) {
+ throw new IllegalArgumentException("Too many non-switch arguments");
+ }
+ }
+ }
+
+ try {
+ Configuration conf = getConf();
+ Path inPath = new Path(inputPathName);
+ reader =
+ new DeskewedJobTraceReader(new JobTraceReader(inPath, conf),
+ skewBufferLength, !allowMissorting);
+ Path outPath = new Path(outputPathName);
+
+ ObjectMapper outMapper = new ObjectMapper();
+ outMapper.configure(
+ SerializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
+ JsonFactory outFactory = outMapper.getJsonFactory();
+ FileSystem outFS = outPath.getFileSystem(conf);
+
+ CompressionCodec codec =
+ new CompressionCodecFactory(conf).getCodec(outPath);
+ OutputStream output;
+ Compressor compressor = null;
+ if (codec != null) {
+ compressor = CodecPool.getCompressor(codec);
+ output = codec.createOutputStream(outFS.create(outPath), compressor);
+ } else {
+ output = outFS.create(outPath);
+ }
+
+ outGen = outFactory.createJsonGenerator(output, JsonEncoding.UTF8);
+ outGen.useDefaultPrettyPrinter();
+
+ tempDir =
+ tempDirName == null ? outPath.getParent() : new Path(tempDirName);
+
+ FileSystem fs = tempDir.getFileSystem(getConf());
+ if (!fs.getFileStatus(tempDir).isDir()) {
+ throw new IOException("Your temp directory is not a directory");
+ }
+
+ if (inputCycle <= 0) {
+ LOG.error("You must have an input cycle length.");
+ return NO_INPUT_CYCLE_LENGTH;
+ }
+
+ if (outputDuration <= 0) {
+ outputDuration = 60L * 60L * TICKS_PER_SECOND;
+ }
+
+ if (inputCycle <= 0) {
+ inputCycle = outputDuration;
+ }
+
+ timeDilation = (double) outputDuration / (double) inputCycle;
+
+ random = seeded ? new Random(randomSeed) : new Random();
+
+ if (debug) {
+ randomSeed = random.nextLong();
+
+ LOG.warn("This run effectively has a -seed of " + randomSeed);
+
+ random = new Random(randomSeed);
+
+ seeded = true;
+ }
+ } catch (IOException e) {
+ e.printStackTrace(System.err);
+
+ return NON_EXISTENT_FILES;
+ }
+
+ return 0;
+ }
+
+ @Override
+ public int run(String[] args) throws IOException {
+ int result = initialize(args);
+
+ if (result != 0) {
+ return result;
+ }
+
+ return run();
+ }
+
+ public int run() throws IOException {
+ class JobEntryComparator implements
+ Comparator<Pair<LoggedJob, JobTraceReader>> {
+ public int compare(Pair<LoggedJob, JobTraceReader> p1,
+ Pair<LoggedJob, JobTraceReader> p2) {
+ LoggedJob j1 = p1.first();
+ LoggedJob j2 = p2.first();
+
+ return (j1.getSubmitTime() < j2.getSubmitTime()) ? -1 : (j1
+ .getSubmitTime() == j2.getSubmitTime()) ? 0 : 1;
+ }
+ }
+
+ ObjectMapper outMapper = new ObjectMapper();
+ outMapper.configure(
+ SerializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
+ JsonFactory outFactory = outMapper.getJsonFactory();
+
+ // we initialize an empty heap so if we take an error before establishing
+ // a real one the finally code goes through
+ Queue<Pair<LoggedJob, JobTraceReader>> heap =
+ new PriorityQueue<Pair<LoggedJob, JobTraceReader>>();
+
+ try {
+ LoggedJob job = reader.nextJob();
+
+ if (job == null) {
+ LOG.error("The job trace is empty");
+
+ return EMPTY_JOB_TRACE;
+ }
+
+ firstJobSubmitTime = job.getSubmitTime();
+ long lastJobSubmitTime = firstJobSubmitTime;
+
+ int numberJobs = 0;
+
+ long currentIntervalEnd = Long.MIN_VALUE;
+
+ Path nextSegment = null;
+ OutputStream tempUncompOut = null;
+ JsonGenerator tempGen = null;
+
+ if (debug) {
+ LOG.debug("The first job has a submit time of " + firstJobSubmitTime);
+ }
+
+ final Configuration conf = getConf();
+
+ try {
+ // At the top of this loop, skewBuffer has at most
+ // skewBufferLength entries.
+ while (job != null) {
+ final Random tempNameGenerator = new Random();
+
+ lastJobSubmitTime = job.getSubmitTime();
+
+ ++numberJobs;
+
+ if (job.getSubmitTime() >= currentIntervalEnd) {
+ if (tempGen != null) {
+ tempGen.close();
+ }
+ for (int i = 0; i < 3 && tempUncompOut == null; ++i) {
+ try {
+ nextSegment =
+ new Path(tempDir, "segment-" + tempNameGenerator.nextLong()
+ + ".json.gz");
+
+ if (debug) {
+ LOG.debug("The next segment name is " + nextSegment);
+ }
+
+ FileSystem fs = nextSegment.getFileSystem(conf);
+
+ try {
+ if (!fs.exists(nextSegment)) {
+ tempUncompOut = fs.create(nextSegment, false);
+ }
+
+ continue;
+ } catch (IOException e) {
+ // no code -- file did not already exist
+ }
+ } catch (IOException e) {
+ // no code -- file exists now, or directory bad. We try three
+ // times.
+ }
+ }
+
+ if (debug) {
+ LOG.debug("Creating " + nextSegment
+ + " for a job with a submit time of " + job.getSubmitTime());
+ }
+
+ deletees.add(nextSegment);
+
+ tempPaths.add(nextSegment);
+
+ CompressionCodec codec =
+ new CompressionCodecFactory(conf).getCodec(nextSegment);
+ OutputStream output;
+ Compressor compressor = null;
+ if (codec != null) {
+ compressor = CodecPool.getCompressor(codec);
+ output = codec.createOutputStream(tempUncompOut, compressor);
+ } else {
+ output = tempUncompOut;
+ }
+
+ tempUncompOut = null;
+
+ tempGen = outFactory.createJsonGenerator(output, JsonEncoding.UTF8);
+ if (debug) {
+ tempGen.useDefaultPrettyPrinter();
+ }
+
+ long currentIntervalNumber =
+ (job.getSubmitTime() - firstJobSubmitTime) / inputCycle;
+
+ currentIntervalEnd =
+ firstJobSubmitTime + ((currentIntervalNumber + 1) * inputCycle);
+ }
+
+ // the temp files contain UDadjusted times, but each temp file's
+ // content is in the same input cycle interval.
+ tempGen.writeObject(job);
+
+ job = reader.nextJob();
+ }
+ } catch (DeskewedJobTraceReader.OutOfOrderException e) {
+ return OUT_OF_ORDER_JOBS;
+ } finally {
+ if (tempGen != null) {
+ tempGen.close();
+ }
+ }
+
+ if (lastJobSubmitTime <= firstJobSubmitTime) {
+ LOG.error("All of your job[s] have the same submit time."
+ + " Please just use your input file.");
+
+ return ALL_JOBS_SIMULTANEOUS;
+ }
+
+ double submitTimeSpan = lastJobSubmitTime - firstJobSubmitTime;
+
+ LOG.warn("Your input trace spans "
+ + (lastJobSubmitTime - firstJobSubmitTime) + " ticks.");
+
+ double foldingRatio =
+ submitTimeSpan * (numberJobs + 1) / numberJobs / inputCycle;
+
+ if (debug) {
+ LOG.warn("run: submitTimeSpan = " + submitTimeSpan + ", numberJobs = "
+ + numberJobs + ", inputCycle = " + inputCycle);
+ }
+
+ if (reader.neededSkewBufferSize() > 0) {
+ LOG.warn("You needed a -skew-buffer-length of "
+ + reader.neededSkewBufferSize() + " but no more, for this input.");
+ }
+
+ double tProbability = timeDilation * concentration / foldingRatio;
+
+ if (debug) {
+ LOG.warn("run: timeDilation = " + timeDilation + ", concentration = "
+ + concentration + ", foldingRatio = " + foldingRatio);
+ LOG.warn("The transcription probability is " + tProbability);
+ }
+
+ transcriptionRateInteger = (int) Math.floor(tProbability);
+ transcriptionRateFraction = tProbability - Math.floor(tProbability);
+
+ // Now read all the inputs in parallel
+ heap =
+ new PriorityQueue<Pair<LoggedJob, JobTraceReader>>(tempPaths.size(),
+ new JobEntryComparator());
+
+ for (Path tempPath : tempPaths) {
+ JobTraceReader thisReader = new JobTraceReader(tempPath, conf);
+
+ closees.add(thisReader);
+
+ LoggedJob streamFirstJob = thisReader.getNext();
+
+ long thisIndex =
+ (streamFirstJob.getSubmitTime() - firstJobSubmitTime) / inputCycle;
+
+ if (debug) {
+ LOG.debug("A job with submit time of "
+ + streamFirstJob.getSubmitTime() + " is in interval # "
+ + thisIndex);
+ }
+
+ adjustJobTimes(streamFirstJob);
+
+ if (debug) {
+ LOG.debug("That job's submit time is adjusted to "
+ + streamFirstJob.getSubmitTime());
+ }
+
+ heap
+ .add(new Pair<LoggedJob, JobTraceReader>(streamFirstJob, thisReader));
+ }
+
+ Pair<LoggedJob, JobTraceReader> next = heap.poll();
+
+ while (next != null) {
+ maybeOutput(next.first());
+
+ if (debug) {
+ LOG.debug("The most recent job has an adjusted submit time of "
+ + next.first().getSubmitTime());
+ LOG.debug(" Its replacement in the heap will come from input engine "
+ + next.second());
+ }
+
+ LoggedJob replacement = next.second().getNext();
+
+ if (replacement == null) {
+ next.second().close();
+
+ if (debug) {
+ LOG.debug("That input engine is depleted.");
+ }
+ } else {
+ adjustJobTimes(replacement);
+
+ if (debug) {
+ LOG.debug("The replacement has an adjusted submit time of "
+ + replacement.getSubmitTime());
+ }
+
+ heap.add(new Pair<LoggedJob, JobTraceReader>(replacement, next
+ .second()));
+ }
+
+ next = heap.poll();
+ }
+ } finally {
+ IOUtils.cleanup(null, reader);
+ if (outGen != null) {
+ outGen.close();
+ }
+ for (Pair<LoggedJob, JobTraceReader> heapEntry : heap) {
+ heapEntry.second().close();
+ }
+ for (Closeable closee : closees) {
+ closee.close();
+ }
+ if (!debug) {
+ Configuration conf = getConf();
+
+ for (Path deletee : deletees) {
+ FileSystem fs = deletee.getFileSystem(conf);
+
+ try {
+ fs.delete(deletee, false);
+ } catch (IOException e) {
+ // no code
+ }
+ }
+ }
+ }
+
+ return 0;
+ }
+
+ private void maybeOutput(LoggedJob job) throws IOException {
+ for (int i = 0; i < transcriptionRateInteger; ++i) {
+ outGen.writeObject(job);
+ }
+
+ if (random.nextDouble() < transcriptionRateFraction) {
+ outGen.writeObject(job);
+ }
+ }
+
+ private void adjustJobTimes(LoggedJob adjustee) {
+ long offsetInCycle =
+ (adjustee.getSubmitTime() - firstJobSubmitTime) % inputCycle;
+
+ long outputOffset = (long) ((double) offsetInCycle * timeDilation);
+
+ long adjustment =
+ firstJobSubmitTime + outputOffset - adjustee.getSubmitTime();
+
+ adjustee.adjustTimes(adjustment);
+ }
+
+ /**
+ * @param args
+ */
+ public static void main(String[] args) {
+ Folder instance = new Folder();
+
+ int result = 0;
+
+ try {
+ result = ToolRunner.run(instance, args);
+ } catch (IOException e) {
+ e.printStackTrace(System.err);
+ System.exit(IO_ERROR);
+ } catch (Exception e) {
+ e.printStackTrace(System.err);
+ System.exit(OTHER_ERROR);
+ }
+
+ if (result != 0) {
+ System.exit(result);
+ }
+
+ return;
+ }
+}
diff --git a/src/tools/org/apache/hadoop/tools/rumen/HadoopLogsAnalyzer.java b/src/tools/org/apache/hadoop/tools/rumen/HadoopLogsAnalyzer.java
index acc603c..2425a6f 100644
--- a/src/tools/org/apache/hadoop/tools/rumen/HadoopLogsAnalyzer.java
+++ b/src/tools/org/apache/hadoop/tools/rumen/HadoopLogsAnalyzer.java
@@ -1285,7 +1285,7 @@
attempt.setLocation(host.makeLoggedLocation());
}
- ArrayList<LoggedLocation> locs = task.getPreferredLocations();
+ List<LoggedLocation> locs = task.getPreferredLocations();
if (host != null && locs != null) {
for (LoggedLocation loc : locs) {
diff --git a/src/tools/org/apache/hadoop/tools/rumen/JsonObjectMapperParser.java b/src/tools/org/apache/hadoop/tools/rumen/JsonObjectMapperParser.java
index cffc65a..b90a3f3 100644
--- a/src/tools/org/apache/hadoop/tools/rumen/JsonObjectMapperParser.java
+++ b/src/tools/org/apache/hadoop/tools/rumen/JsonObjectMapperParser.java
@@ -36,7 +36,9 @@
/**
* A simple wrapper for parsing JSON-encoded data using ObjectMapper.
- * @param <T> The (base) type of the object(s) to be parsed by this parser.
+ *
+ * @param <T>
+ * The (base) type of the object(s) to be parsed by this parser.
*/
class JsonObjectMapperParser<T> implements Closeable {
private final ObjectMapper mapper;
@@ -47,7 +49,7 @@
/**
* Constructor.
*
- * @param path
+ * @param path
* Path to the JSON data file, possibly compressed.
* @param conf
* @throws IOException
diff --git a/src/tools/org/apache/hadoop/tools/rumen/LoggedJob.java b/src/tools/org/apache/hadoop/tools/rumen/LoggedJob.java
index 51da246..1e6ca17 100644
--- a/src/tools/org/apache/hadoop/tools/rumen/LoggedJob.java
+++ b/src/tools/org/apache/hadoop/tools/rumen/LoggedJob.java
@@ -102,6 +102,24 @@
setJobID(jobID);
}
+ void adjustTimes(long adjustment) {
+ submitTime += adjustment;
+ launchTime += adjustment;
+ finishTime += adjustment;
+
+ for (LoggedTask task : mapTasks) {
+ task.adjustTimes(adjustment);
+ }
+
+ for (LoggedTask task : reduceTasks) {
+ task.adjustTimes(adjustment);
+ }
+
+ for (LoggedTask task : otherTasks) {
+ task.adjustTimes(adjustment);
+ }
+ }
+
@SuppressWarnings("unused")
// for input parameter ignored.
@JsonAnySetter
diff --git a/src/tools/org/apache/hadoop/tools/rumen/LoggedLocation.java b/src/tools/org/apache/hadoop/tools/rumen/LoggedLocation.java
index 4fae258..1291f6f 100644
--- a/src/tools/org/apache/hadoop/tools/rumen/LoggedLocation.java
+++ b/src/tools/org/apache/hadoop/tools/rumen/LoggedLocation.java
@@ -18,7 +18,10 @@
package org.apache.hadoop.tools.rumen;
import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
@@ -41,12 +44,15 @@
*
*/
public class LoggedLocation implements DeepCompare {
+ static final Map<List<String>, List<String>> layersCache =
+ new HashMap<List<String>, List<String>>();
+
/**
* The full path from the root of the network to the host.
*
* NOTE that this assumes that the network topology is a tree.
*/
- List<String> layers = new ArrayList<String>();
+ List<String> layers = Collections.emptyList();
static private Set<String> alreadySeenAnySetterAttributes =
new TreeSet<String>();
@@ -56,7 +62,26 @@
}
void setLayers(List<String> layers) {
- this.layers = layers;
+ if (layers == null || layers.isEmpty()) {
+ this.layers = Collections.emptyList();
+ } else {
+ synchronized (layersCache) {
+ List<String> found = layersCache.get(layers);
+ if (found == null) {
+ // make a copy with interned string.
+ List<String> clone = new ArrayList<String>(layers.size());
+ for (String s : layers) {
+ clone.add(s.intern());
+ }
+ // making it read-only as we are sharing them.
+ List<String> readonlyLayers = Collections.unmodifiableList(clone);
+ layersCache.put(readonlyLayers, readonlyLayers);
+ this.layers = readonlyLayers;
+ } else {
+ this.layers = found;
+ }
+ }
+ }
}
@SuppressWarnings("unused")
diff --git a/src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java b/src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java
index aa8888e..105d7f6 100644
--- a/src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java
+++ b/src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.tools.rumen;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
@@ -44,9 +45,7 @@
Pre21JobHistoryConstants.Values taskType;
Pre21JobHistoryConstants.Values taskStatus;
List<LoggedTaskAttempt> attempts = new ArrayList<LoggedTaskAttempt>();
-
- ArrayList<LoggedLocation> preferredLocations =
- new ArrayList<LoggedLocation>();
+ List<LoggedLocation> preferredLocations = Collections.emptyList();
int numberMaps = -1;
int numberReduces = -1;
@@ -69,6 +68,15 @@
super();
}
+ void adjustTimes(long adjustment) {
+ startTime += adjustment;
+ finishTime += adjustment;
+
+ for (LoggedTaskAttempt attempt : attempts) {
+ attempt.adjustTimes(adjustment);
+ }
+ }
+
public long getInputBytes() {
return inputBytes;
}
@@ -130,15 +138,23 @@
}
void setAttempts(List<LoggedTaskAttempt> attempts) {
- this.attempts = attempts;
+ if (attempts == null) {
+ this.attempts = new ArrayList<LoggedTaskAttempt>();
+ } else {
+ this.attempts = attempts;
+ }
}
- public ArrayList<LoggedLocation> getPreferredLocations() {
+ public List<LoggedLocation> getPreferredLocations() {
return preferredLocations;
}
- void setPreferredLocations(ArrayList<LoggedLocation> preferredLocations) {
- this.preferredLocations = preferredLocations;
+ void setPreferredLocations(List<LoggedLocation> preferredLocations) {
+ if (preferredLocations == null || preferredLocations.isEmpty()) {
+ this.preferredLocations = Collections.emptyList();
+ } else {
+ this.preferredLocations = preferredLocations;
+ }
}
public int getNumberMaps() {
@@ -204,8 +220,8 @@
}
}
- private void compareLoggedLocations(ArrayList<LoggedLocation> c1,
- ArrayList<LoggedLocation> c2, TreePath loc, String eltname)
+ private void compareLoggedLocations(List<LoggedLocation> c1,
+ List<LoggedLocation> c2, TreePath loc, String eltname)
throws DeepInequalityException {
if (c1 == null && c2 == null) {
return;
diff --git a/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java b/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java
index fda0e96..cf61da0 100644
--- a/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java
+++ b/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java
@@ -82,6 +82,11 @@
}
}
+ void adjustTimes(long adjustment) {
+ startTime += adjustment;
+ finishTime += adjustment;
+ }
+
public long getShuffleFinished() {
return shuffleFinished;
}
@@ -135,7 +140,7 @@
}
void setHostName(String hostName) {
- this.hostName = hostName;
+ this.hostName = hostName.intern();
}
public long getHdfsBytesRead() {
diff --git a/src/tools/org/apache/hadoop/tools/rumen/TaskInfo.java b/src/tools/org/apache/hadoop/tools/rumen/TaskInfo.java
index 8da16f9..4cc6816 100644
--- a/src/tools/org/apache/hadoop/tools/rumen/TaskInfo.java
+++ b/src/tools/org/apache/hadoop/tools/rumen/TaskInfo.java
@@ -24,7 +24,8 @@
private final int recsOut;
private final long maxMemory;
- public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut, int maxMemory) {
+ public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
+ long maxMemory) {
this.bytesIn = bytesIn;
this.recsIn = recsIn;
this.bytesOut = bytesOut;
diff --git a/src/tools/org/apache/hadoop/tools/rumen/ZombieCluster.java b/src/tools/org/apache/hadoop/tools/rumen/ZombieCluster.java
index b476116..49deec3 100644
--- a/src/tools/org/apache/hadoop/tools/rumen/ZombieCluster.java
+++ b/src/tools/org/apache/hadoop/tools/rumen/ZombieCluster.java
@@ -45,7 +45,7 @@
* @param defaultNode
* The default node setting.
*/
- ZombieCluster(LoggedNetworkTopology topology, MachineNode defaultNode) {
+ public ZombieCluster(LoggedNetworkTopology topology, MachineNode defaultNode) {
buildCluster(topology, defaultNode);
}
diff --git a/src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java b/src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java
index 922849f..d2582c2 100644
--- a/src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java
+++ b/src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java
@@ -619,38 +619,42 @@
Values type = loggedTask.getTaskType();
if ((type != Values.MAP) && (type != Values.REDUCE)) {
throw new IllegalArgumentException(
- "getTaskInfo only supports MAP or REDUCE tasks: " + type.toString() +
- " for task = " + loggedTask.getTaskID());
+ "getTaskInfo only supports MAP or REDUCE tasks: " + type.toString()
+ + " for task = " + loggedTask.getTaskID());
}
for (LoggedTaskAttempt attempt : attempts) {
attempt = sanitizeLoggedTaskAttempt(attempt);
// ignore bad attempts or unsuccessful attempts.
- if ((attempt == null)
- || (attempt.getResult() != Values.SUCCESS)) {
+ if ((attempt == null) || (attempt.getResult() != Values.SUCCESS)) {
continue;
}
if (type == Values.MAP) {
inputBytes = attempt.getHdfsBytesRead();
inputRecords = attempt.getMapInputRecords();
- outputBytes = attempt.getMapOutputBytes();
+ outputBytes =
+ (job.getTotalReduces() > 0) ? attempt.getMapOutputBytes() : attempt
+ .getHdfsBytesWritten();
outputRecords = attempt.getMapOutputRecords();
- heapMegabytes = (job.getJobMapMB() > 0) ? job.getJobMapMB()
- : job.getHeapMegabytes();
+ heapMegabytes =
+ (job.getJobMapMB() > 0) ? job.getJobMapMB() : job
+ .getHeapMegabytes();
} else {
inputBytes = attempt.getReduceShuffleBytes();
inputRecords = attempt.getReduceInputRecords();
outputBytes = attempt.getHdfsBytesWritten();
outputRecords = attempt.getReduceOutputRecords();
- heapMegabytes = (job.getJobReduceMB() > 0) ? job.getJobReduceMB()
- : job.getHeapMegabytes();
+ heapMegabytes =
+ (job.getJobReduceMB() > 0) ? job.getJobReduceMB() : job
+ .getHeapMegabytes();
}
break;
}
- TaskInfo taskInfo = new TaskInfo(inputBytes, (int) inputRecords,
- outputBytes, (int) outputRecords, (int) heapMegabytes);
+ TaskInfo taskInfo =
+ new TaskInfo(inputBytes, (int) inputRecords, outputBytes,
+ (int) outputRecords, (int) heapMegabytes);
return taskInfo;
}
@@ -869,8 +873,9 @@
private LoggedTaskAttempt getLoggedTaskAttempt(TaskType taskType,
int taskNumber, int taskAttemptNumber) {
buildMaps();
- TaskAttemptID id = new TaskAttemptID(getMaskedTaskID(taskType, taskNumber),
- taskAttemptNumber);
+ TaskAttemptID id =
+ new TaskAttemptID(getMaskedTaskID(taskType, taskNumber),
+ taskAttemptNumber);
return loggedTaskAttemptMap.get(id);
}
diff --git a/src/webapps/job/jobdetails.jsp b/src/webapps/job/jobdetails.jsp
index a15ea02..116ab10 100644
--- a/src/webapps/job/jobdetails.jsp
+++ b/src/webapps/job/jobdetails.jsp
@@ -26,7 +26,10 @@
import="java.util.*"
import="java.text.DecimalFormat"
import="org.apache.hadoop.mapred.*"
+ import="org.apache.hadoop.mapreduce.TaskType"
import="org.apache.hadoop.util.*"
+ import="org.apache.hadoop.fs.Path"
+ import="org.apache.hadoop.mapreduce.jobhistory.JobHistory"
%>
<%! private static final long serialVersionUID = 1L;
@@ -219,7 +222,14 @@
<%
if (job == null) {
- out.print("<b>Job " + jobId + " not found.</b><br>\n");
+ String historyFile = tracker.getJobHistory().getHistoryFilePath(jobIdObj);
+ if (historyFile == null) {
+ out.println("<h2>Job " + jobId + " not known!</h2>");
+ return;
+ }
+ String historyUrl = "/jobdetailshistory.jsp?jobid=" +
+ jobId + "&logFile=" + historyFile;
+ response.sendRedirect(response.encodeRedirectURL(historyUrl));
return;
}
JobProfile profile = job.getProfile();
@@ -231,7 +241,8 @@
out.print("<b>Job File:</b> <a href=\"jobconf.jsp?jobid=" + jobId + "\">"
+ profile.getJobFile() + "</a><br>\n");
out.print("<b>Job Setup:</b>");
- printJobLevelTaskSummary(out, jobId, "setup", job.getSetupTasks());
+ printJobLevelTaskSummary(out, jobId, "setup",
+ job.getTasks(TaskType.JOB_SETUP));
out.print("<br>\n");
if (runState == JobStatus.RUNNING) {
out.print("<b>Status:</b> Running<br>\n");
@@ -263,7 +274,8 @@
}
}
out.print("<b>Job Cleanup:</b>");
- printJobLevelTaskSummary(out, jobId, "cleanup", job.getCleanupTasks());
+ printJobLevelTaskSummary(out, jobId, "cleanup",
+ job.getTasks(TaskType.JOB_CLEANUP));
out.print("<br>\n");
if (flakyTaskTrackers > 0) {
out.print("<b>Black-listed TaskTrackers:</b> " +
@@ -282,9 +294,9 @@
"<th><a href=\"jobfailures.jsp?jobid=" + jobId +
"\">Failed/Killed<br>Task Attempts</a></th></tr>\n");
printTaskSummary(out, jobId, "map", status.mapProgress(),
- job.getMapTasks());
+ job.getTasks(TaskType.MAP));
printTaskSummary(out, jobId, "reduce", status.reduceProgress(),
- job.getReduceTasks());
+ job.getTasks(TaskType.REDUCE));
out.print("</table>\n");
%>
@@ -359,7 +371,7 @@
style="width:100%" type="image/svg+xml" pluginspage="http://www.adobe.com/svg/viewer/install/" />
<%}%>
-<%if(job.getReduceTasks().length > 0) { %>
+<%if(job.getTasks(TaskType.REDUCE).length > 0) { %>
<hr>Reduce Completion Graph -
<%if("off".equals(session.getAttribute("reduce.graph"))) { %>
<a href="/jobdetails.jsp?jobid=<%=jobId%>&refresh=<%=refresh%>&reduce.graph=on" > open </a>
diff --git a/src/webapps/job/jobfailures.jsp b/src/webapps/job/jobfailures.jsp
index 2f58c4b..7d53091 100644
--- a/src/webapps/job/jobfailures.jsp
+++ b/src/webapps/job/jobfailures.jsp
@@ -24,6 +24,7 @@
import="java.io.*"
import="java.util.*"
import="org.apache.hadoop.mapred.*"
+ import="org.apache.hadoop.mapreduce.TaskType"
import="org.apache.hadoop.util.*"
%>
@@ -147,13 +148,13 @@
out.print("<tr><th>Attempt</th><th>Task</th><th>Machine</th><th>State</th>" +
"<th>Error</th><th>Logs</th></tr>\n");
if (includeMap) {
- TaskInProgress[] tips = job.getMapTasks();
+ TaskInProgress[] tips = job.getTasks(TaskType.MAP);
for(int i=0; i < tips.length; ++i) {
printFailedAttempts(out, tracker, jobId, tips[i], state);
}
}
if (includeReduce) {
- TaskInProgress[] tips = job.getReduceTasks();
+ TaskInProgress[] tips = job.getTasks(TaskType.REDUCE);
for(int i=0; i < tips.length; ++i) {
printFailedAttempts(out, tracker, jobId, tips[i], state);
}
diff --git a/src/webapps/static/jobconf.xsl b/src/webapps/static/jobconf.xsl
index f3c2e33..7536365 100644
--- a/src/webapps/static/jobconf.xsl
+++ b/src/webapps/static/jobconf.xsl
@@ -1,4 +1,21 @@
<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
<xsl:output method="html"/>
<xsl:template match="configuration">