diff --git a/.gitignore b/.gitignore index 98dfe5a27b..e6cdca77e6 100644 --- a/.gitignore +++ b/.gitignore @@ -1,7 +1,26 @@ *~ *.swp -build -work +*.iml +.idea/ +/build/ +work/ +out/ .DS_Store third_party/libmesos.so third_party/libmesos.dylib +conf/java-opts +conf/spark-env.sh +conf/log4j.properties +target/ +reports/ +.project +.classpath +.scala_dependencies +lib_managed/ +src_managed/ +project/boot/ +project/plugins/project/build.properties +project/build/target/ +project/plugins/target/ +project/plugins/lib_managed/ +project/plugins/src_managed/ diff --git a/LICENSE b/LICENSE new file mode 100644 index 0000000000..d17afa1fc6 --- /dev/null +++ b/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2010, Regents of the University of California. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the University of California, Berkeley nor the + names of its contributors may be used to endorse or promote + products derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED +TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/Makefile b/Makefile deleted file mode 100644 index 15ab516d1f..0000000000 --- a/Makefile +++ /dev/null @@ -1,79 +0,0 @@ -EMPTY = -SPACE = $(EMPTY) $(EMPTY) - -# Build up classpath by concatenating some strings -JARS = third_party/mesos.jar -JARS += third_party/asm-3.2/lib/all/asm-all-3.2.jar -JARS += third_party/colt.jar -JARS += third_party/guava-r07/guava-r07.jar -JARS += third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar -JARS += third_party/hadoop-0.20.0/lib/commons-logging-1.0.4.jar -JARS += third_party/scalatest-1.2/scalatest-1.2.jar -JARS += third_party/scalacheck_2.8.0-1.7.jar -JARS += third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar -JARS += third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar -JARS += third_party/apache-log4j-1.2.16/log4j-1.2.16.jar -JARS += third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar -JARS += third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar -CLASSPATH = $(subst $(SPACE),:,$(JARS)) - -SCALA_SOURCES = src/examples/*.scala src/scala/spark/*.scala src/scala/spark/repl/*.scala -SCALA_SOURCES += src/test/spark/*.scala src/test/spark/repl/*.scala - -JAVA_SOURCES = $(wildcard src/java/spark/compress/lzf/*.java) - -ifeq ($(USE_FSC),1) - COMPILER_NAME = fsc -else - COMPILER_NAME = scalac -endif - -ifeq ($(SCALA_HOME),) - COMPILER = $(COMPILER_NAME) -else - COMPILER = $(SCALA_HOME)/bin/$(COMPILER_NAME) -endif - -CONF_FILES = conf/spark-env.sh conf/log4j.properties conf/java-opts - -all: scala java conf-files - -build/classes: - mkdir -p build/classes - -scala: build/classes java - $(COMPILER) -d build/classes -classpath build/classes:$(CLASSPATH) $(SCALA_SOURCES) - -java: $(JAVA_SOURCES) build/classes - javac -d build/classes $(JAVA_SOURCES) - -native: java - $(MAKE) -C src/native - -jar: build/spark.jar build/spark-dep.jar - -dep-jar: build/spark-dep.jar - -build/spark.jar: scala java - jar cf build/spark.jar -C build/classes spark - -build/spark-dep.jar: - mkdir -p build/dep - cd build/dep && for i in $(JARS); do jar xf ../../$$i; done - jar cf build/spark-dep.jar -C build/dep . - -conf-files: $(CONF_FILES) - -$(CONF_FILES): %: | %.template - cp $@.template $@ - -test: all - ./alltests - -default: all - -clean: - $(MAKE) -C src/native clean - rm -rf build - -.phony: default all clean scala java native jar dep-jar conf-files diff --git a/README b/README index f084f22a1f..3fbf8c98de 100644 --- a/README +++ b/README @@ -1,24 +1,32 @@ +ONLINE DOCUMENTATION + +You can find the latest Spark documentation, including a programming guide, +on the project wiki at http://github.com/mesos/spark/wiki. This file only +contains basic setup instructions. + + + BUILDING -Spark requires Scala 2.8. This version has been tested with 2.8.0.final. +Spark requires Scala 2.8. This version has been tested with 2.8.1.final. -To build and run Spark, you will need to have Scala's bin in your $PATH, -or you will need to set the SCALA_HOME environment variable to point -to where you've installed Scala. Scala must be accessible through one -of these methods on Mesos slave nodes as well as on the master. +The project is built using Simple Build Tool (SBT), which is packaged with it. +To build Spark and its example programs, run sbt/sbt compile. -To build Spark and the example programs, run make. +To run Spark, you will need to have Scala's bin in your $PATH, or you +will need to set the SCALA_HOME environment variable to point to where +you've installed Scala. Scala must be accessible through one of these +methods on Mesos slave nodes as well as on the master. To run one of the examples, use ./run . For example, -./run SparkLR will run the Logistic Regression example. Each of the -example programs prints usage help if no params are given. +./run spark.examples.SparkLR will run the Logistic Regression example. +Each of the example programs prints usage help if no params are given. All of the Spark samples take a parameter that is the Mesos master to connect to. This can be a Mesos URL, or "local" to run locally with one thread, or "local[N]" to run locally with N threads. -Tip: If you are building Spark and examples repeatedly, export USE_FSC=1 -to have the Makefile use the fsc compiler daemon instead of scalac. + CONFIGURATION diff --git a/alltests b/alltests deleted file mode 100755 index cd11604855..0000000000 --- a/alltests +++ /dev/null @@ -1,11 +0,0 @@ -#!/bin/bash -FWDIR="`dirname $0`" -if [ "x$SPARK_MEM" == "x" ]; then - export SPARK_MEM=500m -fi -RESULTS_DIR="$FWDIR/build/test_results" -if [ -d $RESULTS_DIR ]; then - rm -r $RESULTS_DIR -fi -mkdir -p $RESULTS_DIR -$FWDIR/run org.scalatest.tools.Runner -p $FWDIR/build/classes -u $RESULTS_DIR -o $@ diff --git a/conf/java-opts b/conf/java-opts index 83c61c0688..1a598061f0 100644 --- a/conf/java-opts +++ b/conf/java-opts @@ -1,4 +1,4 @@ --Dspark.shuffle.class=spark.TrackedCustomBlockedInMemoryShuffle +-Dspark.shuffle.class=spark.CustomBlockedInMemoryShuffle -Dspark.shuffle.masterHostAddress=127.0.0.1 -Dspark.shuffle.masterTrackerPort=22222 -Dspark.shuffle.trackerStrategy=spark.BalanceRemainingShuffleTrackerStrategy diff --git a/conf/spark-env.sh b/conf/spark-env.sh index 77f9cb69b9..5f6c8269e8 100755 --- a/conf/spark-env.sh +++ b/conf/spark-env.sh @@ -10,4 +10,4 @@ # be in the same format as the JVM's -Xmx option, e.g. 300m or 1g). # - SPARK_LIBRARY_PATH, to add extra search paths for native libraries. -MESOS_HOME=/home/mosharaf/Work/mesos +MESOS_HOME=/Users/mosharaf/Work/mesos diff --git a/third_party/apache-log4j-1.2.16/log4j-1.2.16.jar b/core/lib/apache-log4j-1.2.16/log4j-1.2.16.jar similarity index 100% rename from third_party/apache-log4j-1.2.16/log4j-1.2.16.jar rename to core/lib/apache-log4j-1.2.16/log4j-1.2.16.jar diff --git a/third_party/asm-3.2/.DS_Store b/core/lib/asm-3.2/.DS_Store similarity index 100% rename from third_party/asm-3.2/.DS_Store rename to core/lib/asm-3.2/.DS_Store diff --git a/third_party/asm-3.2/lib/all/README.txt b/core/lib/asm-3.2/lib/all/README.txt similarity index 100% rename from third_party/asm-3.2/lib/all/README.txt rename to core/lib/asm-3.2/lib/all/README.txt diff --git a/third_party/asm-3.2/lib/all/asm-all-3.2.jar b/core/lib/asm-3.2/lib/all/asm-all-3.2.jar similarity index 100% rename from third_party/asm-3.2/lib/all/asm-all-3.2.jar rename to core/lib/asm-3.2/lib/all/asm-all-3.2.jar diff --git a/third_party/asm-3.2/lib/all/asm-all-3.2.pom b/core/lib/asm-3.2/lib/all/asm-all-3.2.pom similarity index 100% rename from third_party/asm-3.2/lib/all/asm-all-3.2.pom rename to core/lib/asm-3.2/lib/all/asm-all-3.2.pom diff --git a/third_party/asm-3.2/lib/all/asm-debug-all-3.2.jar b/core/lib/asm-3.2/lib/all/asm-debug-all-3.2.jar similarity index 100% rename from third_party/asm-3.2/lib/all/asm-debug-all-3.2.jar rename to core/lib/asm-3.2/lib/all/asm-debug-all-3.2.jar diff --git a/third_party/asm-3.2/lib/all/asm-debug-all-3.2.pom b/core/lib/asm-3.2/lib/all/asm-debug-all-3.2.pom similarity index 100% rename from third_party/asm-3.2/lib/all/asm-debug-all-3.2.pom rename to core/lib/asm-3.2/lib/all/asm-debug-all-3.2.pom diff --git a/third_party/asm-3.2/lib/asm-3.2.jar b/core/lib/asm-3.2/lib/asm-3.2.jar similarity index 100% rename from third_party/asm-3.2/lib/asm-3.2.jar rename to core/lib/asm-3.2/lib/asm-3.2.jar diff --git a/third_party/asm-3.2/lib/asm-3.2.pom b/core/lib/asm-3.2/lib/asm-3.2.pom similarity index 100% rename from third_party/asm-3.2/lib/asm-3.2.pom rename to core/lib/asm-3.2/lib/asm-3.2.pom diff --git a/third_party/asm-3.2/lib/asm-analysis-3.2.jar b/core/lib/asm-3.2/lib/asm-analysis-3.2.jar similarity index 100% rename from third_party/asm-3.2/lib/asm-analysis-3.2.jar rename to core/lib/asm-3.2/lib/asm-analysis-3.2.jar diff --git a/third_party/asm-3.2/lib/asm-analysis-3.2.pom b/core/lib/asm-3.2/lib/asm-analysis-3.2.pom similarity index 100% rename from third_party/asm-3.2/lib/asm-analysis-3.2.pom rename to core/lib/asm-3.2/lib/asm-analysis-3.2.pom diff --git a/third_party/asm-3.2/lib/asm-commons-3.2.jar b/core/lib/asm-3.2/lib/asm-commons-3.2.jar similarity index 100% rename from third_party/asm-3.2/lib/asm-commons-3.2.jar rename to core/lib/asm-3.2/lib/asm-commons-3.2.jar diff --git a/third_party/asm-3.2/lib/asm-commons-3.2.pom b/core/lib/asm-3.2/lib/asm-commons-3.2.pom similarity index 100% rename from third_party/asm-3.2/lib/asm-commons-3.2.pom rename to core/lib/asm-3.2/lib/asm-commons-3.2.pom diff --git a/third_party/asm-3.2/lib/asm-parent-3.2.pom b/core/lib/asm-3.2/lib/asm-parent-3.2.pom similarity index 100% rename from third_party/asm-3.2/lib/asm-parent-3.2.pom rename to core/lib/asm-3.2/lib/asm-parent-3.2.pom diff --git a/third_party/asm-3.2/lib/asm-tree-3.2.jar b/core/lib/asm-3.2/lib/asm-tree-3.2.jar similarity index 100% rename from third_party/asm-3.2/lib/asm-tree-3.2.jar rename to core/lib/asm-3.2/lib/asm-tree-3.2.jar diff --git a/third_party/asm-3.2/lib/asm-tree-3.2.pom b/core/lib/asm-3.2/lib/asm-tree-3.2.pom similarity index 100% rename from third_party/asm-3.2/lib/asm-tree-3.2.pom rename to core/lib/asm-3.2/lib/asm-tree-3.2.pom diff --git a/third_party/asm-3.2/lib/asm-util-3.2.jar b/core/lib/asm-3.2/lib/asm-util-3.2.jar similarity index 100% rename from third_party/asm-3.2/lib/asm-util-3.2.jar rename to core/lib/asm-3.2/lib/asm-util-3.2.jar diff --git a/third_party/asm-3.2/lib/asm-util-3.2.pom b/core/lib/asm-3.2/lib/asm-util-3.2.pom similarity index 100% rename from third_party/asm-3.2/lib/asm-util-3.2.pom rename to core/lib/asm-3.2/lib/asm-util-3.2.pom diff --git a/third_party/asm-3.2/lib/asm-xml-3.2.jar b/core/lib/asm-3.2/lib/asm-xml-3.2.jar similarity index 100% rename from third_party/asm-3.2/lib/asm-xml-3.2.jar rename to core/lib/asm-3.2/lib/asm-xml-3.2.jar diff --git a/third_party/asm-3.2/lib/asm-xml-3.2.pom b/core/lib/asm-3.2/lib/asm-xml-3.2.pom similarity index 100% rename from third_party/asm-3.2/lib/asm-xml-3.2.pom rename to core/lib/asm-3.2/lib/asm-xml-3.2.pom diff --git a/third_party/colt.jar b/core/lib/colt.jar similarity index 100% rename from third_party/colt.jar rename to core/lib/colt.jar diff --git a/core/lib/compress-lzf-0.6.0/LICENSE b/core/lib/compress-lzf-0.6.0/LICENSE new file mode 100644 index 0000000000..c5da4e1348 --- /dev/null +++ b/core/lib/compress-lzf-0.6.0/LICENSE @@ -0,0 +1,11 @@ +Copyright 2009-2010 Ning, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); you may not +use this file except in compliance with the License. You may obtain a copy of +the License at http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS,WITHOUT +WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +License for the specific language governing permissions and limitations under +the License. diff --git a/core/lib/compress-lzf-0.6.0/compress-lzf-0.6.0.jar b/core/lib/compress-lzf-0.6.0/compress-lzf-0.6.0.jar new file mode 100644 index 0000000000..6cb5c4c92b Binary files /dev/null and b/core/lib/compress-lzf-0.6.0/compress-lzf-0.6.0.jar differ diff --git a/third_party/guava-r07/COPYING b/core/lib/guava-r07/COPYING similarity index 100% rename from third_party/guava-r07/COPYING rename to core/lib/guava-r07/COPYING diff --git a/third_party/guava-r07/README b/core/lib/guava-r07/README similarity index 100% rename from third_party/guava-r07/README rename to core/lib/guava-r07/README diff --git a/third_party/guava-r07/guava-r07.jar b/core/lib/guava-r07/guava-r07.jar similarity index 100% rename from third_party/guava-r07/guava-r07.jar rename to core/lib/guava-r07/guava-r07.jar diff --git a/third_party/hadoop-0.20.0/CHANGES.txt b/core/lib/hadoop-0.20.2/CHANGES.txt similarity index 95% rename from third_party/hadoop-0.20.0/CHANGES.txt rename to core/lib/hadoop-0.20.2/CHANGES.txt index 95c8b5c08b..b77d5f1c86 100644 --- a/third_party/hadoop-0.20.0/CHANGES.txt +++ b/core/lib/hadoop-0.20.2/CHANGES.txt @@ -1,5 +1,387 @@ Hadoop Change Log +Release 0.20.3 - Unreleased + +Release 0.20.2 - 2010-2-19 + + NEW FEATURES + + HADOOP-6218. Adds a feature where TFile can be split by Record + Sequence number. (Hong Tang and Raghu Angadi via ddas) + + BUG FIXES + + MAPREDUCE-112. Add counters for reduce input, output records to the new API. + (Jothi Padmanabhan via cdouglas) + + HADOOP-6231. Allow caching of filesystem instances to be disabled on a + per-instance basis (Tom White and Ben Slusky via mahadev) + + MAPREDUCE-826. harchive doesn't use ToolRunner / harchive returns 0 even + if the job fails with exception (koji via mahadev) + + MAPREDUCE-979. Fixed JobConf APIs related to memory parameters to return + values of new configuration variables when deprecated variables are + disabled. (Sreekanth Ramakrishnan via yhemanth) + + HDFS-686. NullPointerException is thrown while merging edit log and image. + (hairong) + + HDFS-677. Rename failure when both source and destination quota exceeds + results in deletion of source. (suresh) + + HDFS-709. Fix TestDFSShell failure due to rename bug introduced by + HDFS-677. (suresh) + + HDFS-579. Fix DfsTask to follow the semantics of 0.19, regarding non-zero + return values as failures. (Christian Kunz via cdouglas) + + MAPREDUCE-1070. Prevent a deadlock in the fair scheduler servlet. + (Todd Lipcon via cdouglas) + + HADOOP-5759. Fix for IllegalArgumentException when CombineFileInputFormat + is used as job InputFormat. (Amareshwari Sriramadasu via zshao) + + HADOOP-6097. Fix Path conversion in makeQualified and reset LineReader byte + count at the start of each block in Hadoop archives. (Ben Slusky, Tom + White, and Mahadev Konar via cdouglas) + + HDFS-723. Fix deadlock in DFSClient#DFSOutputStream. (hairong) + + HDFS-732. DFSClient.DFSOutputStream.close() should throw an exception if + the stream cannot be closed successfully. (szetszwo) + + MAPREDUCE-1163. Remove unused, hard-coded paths from libhdfs. (Allen + Wittenauer via cdouglas) + + HDFS-761. Fix failure to process rename operation from edits log due to + quota verification. (suresh) + + MAPREDUCE-623. Resolve javac warnings in mapreduce. (Jothi Padmanabhan + via sharad) + + HADOOP-6575. Remove call to fault injection tests not present in 0.20. + (cdouglas) + + HADOOP-6576. Fix streaming test failures on 0.20. (Todd Lipcon via cdouglas) + + IMPROVEMENTS + + HADOOP-5611. Fix C++ libraries to build on Debian Lenny. (Todd Lipcon + via tomwhite) + + MAPREDUCE-1068. Fix streaming job to show proper message if file is + is not present. (Amareshwari Sriramadasu via sharad) + + HDFS-596. Fix memory leak in hdfsFreeFileInfo() for libhdfs. + (Zhang Bingjun via dhruba) + + MAPREDUCE-1147. Add map output counters to new API. (Amar Kamat via + cdouglas) + + HADOOP-6269. Fix threading issue with defaultResource in Configuration. + (Sreekanth Ramakrishnan via cdouglas) + + MAPREDUCE-1182. Fix overflow in reduce causing allocations to exceed the + configured threshold. (cdouglas) + + HADOOP-6386. NameNode's HttpServer can't instantiate InetSocketAddress: + IllegalArgumentException is thrown. (cos) + + HDFS-185. Disallow chown, chgrp, chmod, setQuota, and setSpaceQuota when + name-node is in safemode. (Ravi Phulari via shv) + + HADOOP-6428. HttpServer sleeps with negative values (cos) + + HADOOP-5623. Fixes a problem to do with status messages getting overwritten + in streaming jobs. (Rick Cox and Jothi Padmanabhan via tomwhite) + + HADOOP-6315. Avoid incorrect use of BuiltInflater/BuiltInDeflater in + GzipCodec. (Aaron Kimball via cdouglas) + + HDFS-187. Initialize secondary namenode http address in TestStartup. + (Todd Lipcon via szetszwo) + + MAPREDUCE-433. Use more reliable counters in TestReduceFetch. (cdouglas) + + HDFS-792. DFSClient 0.20.1 is incompatible with HDFS 0.20.2. + (Tod Lipcon via hairong) + + HADOOP-6498. IPC client bug may cause rpc call hang. (Ruyue Ma and + hairong via hairong) + + HADOOP-6596. Failing tests prevent the rest of test targets from + execution. (cos) + + HADOOP-6524. Contrib tests are failing Clover'ed build. (cos) + + HDFS-919. Create test to validate the BlocksVerified metric (Gary Murry + via cos) + + HDFS-907. Add tests for getBlockLocations and totalLoad metrics. + (Ravi Phulari via cos) + + MAPREDUCE-1251. c++ utils doesn't compile. (Eli Collins via tomwhite) + + HADOOP-5612. Some c++ scripts are not chmodded before ant execution. + (Todd Lipcon via tomwhite) + +Release 0.20.1 - 2009-09-01 + + INCOMPATIBLE CHANGES + + HADOOP-5726. Remove pre-emption from capacity scheduler code base. + (Rahul Kumar Singh via yhemanth) + + HADOOP-5881. Simplify memory monitoring and scheduling related + configuration. (Vinod Kumar Vavilapalli via yhemanth) + + NEW FEATURES + + HADOOP-6080. Introduce -skipTrash option to rm and rmr. + (Jakob Homan via shv) + + HADOOP-3315. Add a new, binary file foramt, TFile. (Hong Tang via cdouglas) + + IMPROVEMENTS + + HADOOP-5711. Change Namenode file close log to info. (szetszwo) + + HADOOP-5736. Update the capacity scheduler documentation for features + like memory based scheduling, job initialization and removal of pre-emption. + (Sreekanth Ramakrishnan via yhemanth) + + HADOOP-4674. Fix fs help messages for -test, -text, -tail, -stat + and -touchz options. (Ravi Phulari via szetszwo) + + HADOOP-4372. Improves the way history filenames are obtained and manipulated. + (Amar Kamat via ddas) + + HADOOP-5897. Add name-node metrics to capture java heap usage. + (Suresh Srinivas via shv) + + HDFS-438. Improve help message for space quota command. (Raghu Angadi) + + MAPREDUCE-767. Remove the dependence on the CLI 2.0 snapshot. + (Amar Kamat via ddas) + + OPTIMIZATIONS + + BUG FIXES + + HADOOP-5691. Makes org.apache.hadoop.mapreduce.Reducer concrete class + instead of abstract. (Amareshwari Sriramadasu via sharad) + + HADOOP-5646. Fixes a problem in TestQueueCapacities. + (Vinod Kumar Vavilapalli via ddas) + + HADOOP-5655. TestMRServerPorts fails on java.net.BindException. (Devaraj + Das via hairong) + + HADOOP-5654. TestReplicationPolicy. fails on java.net.BindException. + (hairong) + + HADOOP-5688. Fix HftpFileSystem checksum path construction. (Tsz Wo + (Nicholas) Sze via cdouglas) + + HADOOP-5213. Fix Null pointer exception caused when bzip2compression + was used and user closed a output stream without writing any data. + (Zheng Shao via dhruba) + + HADOOP-5718. Remove the check for the default queue in capacity scheduler. + (Sreekanth Ramakrishnan via yhemanth) + + HADOOP-5719. Remove jobs that failed initialization from the waiting queue + in the capacity scheduler. (Sreekanth Ramakrishnan via yhemanth) + + HADOOP-4744. Attaching another fix to the jetty port issue. The TaskTracker + kills itself if it ever discovers that the port to which jetty is actually + bound is invalid (-1). (ddas) + + HADOOP-5349. Fixes a problem in LocalDirAllocator to check for the return + path value that is returned for the case where the file we want to write + is of an unknown size. (Vinod Kumar Vavilapalli via ddas) + + HADOOP-5636. Prevents a job from going to RUNNING state after it has been + KILLED (this used to happen when the SetupTask would come back with a + success after the job has been killed). (Amar Kamat via ddas) + + HADOOP-5641. Fix a NullPointerException in capacity scheduler's memory + based scheduling code when jobs get retired. (yhemanth) + + HADOOP-5828. Use absolute path for mapred.local.dir of JobTracker in + MiniMRCluster. (yhemanth) + + HADOOP-4981. Fix capacity scheduler to schedule speculative tasks + correctly in the presence of High RAM jobs. + (Sreekanth Ramakrishnan via yhemanth) + + HADOOP-5210. Solves a problem in the progress report of the reduce task. + (Ravi Gummadi via ddas) + + HADOOP-5850. Fixes a problem to do with not being able to jobs with + 0 maps/reduces. (Vinod K V via ddas) + + HADOOP-5728. Fixed FSEditLog.printStatistics IndexOutOfBoundsException. + (Wang Xu via johan) + + HADOOP-4626. Correct the API links in hdfs forrest doc so that they + point to the same version of hadoop. (szetszwo) + + HADOOP-5883. Fixed tasktracker memory monitoring to account for + momentary spurts in memory usage due to java's fork() model. + (yhemanth) + + HADOOP-5539. Fixes a problem to do with not preserving intermediate + output compression for merged data. + (Jothi Padmanabhan and Billy Pearson via ddas) + + HADOOP-5932. Fixes a problem in capacity scheduler in computing + available memory on a tasktracker. + (Vinod Kumar Vavilapalli via yhemanth) + + HADOOP-5648. Fixes a build issue in not being able to generate gridmix.jar + in hadoop binary tarball. (Giridharan Kesavan via gkesavan) + + HADOOP-5908. Fixes a problem to do with ArithmeticException in the + JobTracker when there are jobs with 0 maps. (Amar Kamat via ddas) + + HADOOP-5924. Fixes a corner case problem to do with job recovery with + empty history files. Also, after a JT restart, sends KillTaskAction to + tasks that report back but the corresponding job hasn't been initialized + yet. (Amar Kamat via ddas) + + HADOOP-5882. Fixes a reducer progress update problem for new mapreduce + api. (Amareshwari Sriramadasu via sharad) + + HADOOP-5746. Fixes a corner case problem in Streaming, where if an + exception happens in MROutputThread after the last call to the map/reduce + method, the exception goes undetected. (Amar Kamat via ddas) + + HADOOP-5884. Fixes accounting in capacity scheduler so that high RAM jobs + take more slots. (Vinod Kumar Vavilapalli via yhemanth) + + HADOOP-5937. Correct a safemode message in FSNamesystem. (Ravi Phulari + via szetszwo) + + HADOOP-5869. Fix bug in assignment of setup / cleanup task that was + causing TestQueueCapacities to fail. + (Sreekanth Ramakrishnan via yhemanth) + + HADOOP-5921. Fixes a problem in the JobTracker where it sometimes never + used to come up due to a system file creation on JobTracker's system-dir + failing. This problem would sometimes show up only when the FS for the + system-dir (usually HDFS) is started at nearly the same time as the + JobTracker. (Amar Kamat via ddas) + + HADOOP-5920. Fixes a testcase failure for TestJobHistory. + (Amar Kamat via ddas) + + HDFS-26. Better error message to users when commands fail because of + lack of quota. Allow quota to be set even if the limit is lower than + current consumption. (Boris Shkolnik via rangadi) + + MAPREDUCE-2. Fixes a bug in KeyFieldBasedPartitioner in handling empty + keys. (Amar Kamat via sharad) + + MAPREDUCE-130. Delete the jobconf copy from the log directory of the + JobTracker when the job is retired. (Amar Kamat via sharad) + + MAPREDUCE-657. Fix hardcoded filesystem problem in CompletedJobStatusStore. + (Amar Kamat via sharad) + + MAPREDUCE-179. Update progress in new RecordReaders. (cdouglas) + + MAPREDUCE-124. Fix a bug in failure handling of abort task of + OutputCommiter. (Amareshwari Sriramadasu via sharad) + + HADOOP-6139. Fix the FsShell help messages for rm and rmr. (Jakob Homan + via szetszwo) + + HADOOP-6141. Fix a few bugs in 0.20 test-patch.sh. (Hong Tang via + szetszwo) + + HADOOP-6145. Fix FsShell rm/rmr error messages when there is a FNFE. + (Jakob Homan via szetszwo) + + MAPREDUCE-565. Fix partitioner to work with new API. (Owen O'Malley via + cdouglas) + + MAPREDUCE-465. Fix a bug in MultithreadedMapRunner. (Amareshwari + Sriramadasu via sharad) + + MAPREDUCE-18. Puts some checks to detect cases where jetty serves up + incorrect output during shuffle. (Ravi Gummadi via ddas) + + MAPREDUCE-735. Fixes a problem in the KeyFieldHelper to do with + the end index for some inputs (Amar Kamat via ddas) + + HADOOP-6150. Users should be able to instantiate comparator using TFile + API. (Hong Tang via rangadi) + + MAPREDUCE-383. Fix a bug in Pipes combiner due to bytes count not + getting reset after the spill. (Christian Kunz via sharad) + + MAPREDUCE-40. Keep memory management backwards compatible for job + configuration parameters and limits. (Rahul Kumar Singh via yhemanth) + + MAPREDUCE-796. Fixes a ClassCastException in an exception log in + MultiThreadedMapRunner. (Amar Kamat via ddas) + + MAPREDUCE-838. Fixes a problem in the way commit of task outputs + happens. The bug was that even if commit failed, the task would + be declared as successful. (Amareshwari Sriramadasu via ddas) + + MAPREDUCE-805. Fixes some deadlocks in the JobTracker due to the fact + the JobTracker lock hierarchy wasn't maintained in some JobInProgress + method calls. (Amar Kamat via ddas) + + HDFS-167. Fix a bug in DFSClient that caused infinite retries on write. + (Bill Zeller via szetszwo) + + HDFS-527. Remove unnecessary DFSClient constructors. (szetszwo) + + MAPREDUCE-832. Reduce number of warning messages printed when + deprecated memory variables are used. (Rahul Kumar Singh via yhemanth) + + MAPREDUCE-745. Fixes a testcase problem to do with generation of JobTracker + IDs. (Amar Kamat via ddas) + + MAPREDUCE-834. Enables memory management on tasktrackers when old + memory management parameters are used in configuration. + (Sreekanth Ramakrishnan via yhemanth) + + MAPREDUCE-818. Fixes Counters#getGroup API. (Amareshwari Sriramadasu + via sharad) + + MAPREDUCE-807. Handles the AccessControlException during the deletion of + mapred.system.dir in the JobTracker. The JobTracker will bail out if it + encounters such an exception. (Amar Kamat via ddas) + + HADOOP-6213. Remove commons dependency on commons-cli2. (Amar Kamat via + sharad) + + MAPREDUCE-430. Fix a bug related to task getting stuck in case of + OOM error. (Amar Kamat via ddas) + + HADOOP-6215. fix GenericOptionParser to deal with -D with '=' in the + value. (Amar Kamat via sharad) + + MAPREDUCE-421. Fix Pipes to use returned system exit code. + (Christian Kunz via omalley) + + HDFS-525. The SimpleDateFormat object in ListPathsServlet is not thread + safe. (Suresh Srinivas and cdouglas) + + MAPREDUCE-911. Fix a bug in TestTaskFail related to speculative + execution. (Amareshwari Sriramadasu via sharad) + + MAPREDUCE-687. Fix an assertion in TestMiniMRMapRedDebugScript. + (Amareshwari Sriramadasu via sharad) + + MAPREDUCE-924. Fixes the TestPipes testcase to use Tool. + (Amareshwari Sriramadasu via sharad) + Release 0.20.0 - 2009-04-15 INCOMPATIBLE CHANGES @@ -361,6 +743,9 @@ Release 0.20.0 - 2009-04-15 HADOOP-5521. Removes dependency of TestJobInProgress on RESTART_COUNT JobHistory tag. (Ravi Gummadi via ddas) + HADOOP-5714. Add a metric for NameNode getFileInfo operation. (Jakob Homan + via szetszwo) + OPTIMIZATIONS HADOOP-3293. Fixes FileInputFormat to do provide locations for splits @@ -945,6 +1330,18 @@ Release 0.19.2 - Unreleased HADOOP-5551. Prevent directory destruction on file create. (Brian Bockelman via shv) + HADOOP-5671. Fix FNF exceptions when copying from old versions of + HftpFileSystem. (Tsz Wo (Nicholas), SZE via cdouglas) + + HADOOP-5579. Set errno correctly in libhdfs for permission, quota, and FNF + conditions. (Brian Bockelman via cdouglas) + + HADOOP-5816. Fixes a problem in the KeyFieldBasedComparator to do with + ArrayIndexOutOfBounds exception. (He Yongqiang via ddas) + + HADOOP-5951. Add Apache license header to StorageInfo.java. (Suresh + Srinivas via szetszwo) + Release 0.19.1 - 2009-02-23 IMPROVEMENTS @@ -2035,6 +2432,12 @@ Release 0.18.4 - Unreleased HADOOP-5557. Fixes some minor problems in TestOverReplicatedBlocks. (szetszwo) + HADOOP-5644. Namenode is stuck in safe mode. (suresh Srinivas via hairong) + + HADOOP-6017. Lease Manager in NameNode does not handle certain characters + in filenames. This results in fatal errors in Secondary NameNode and while + restrating NameNode. (Tsz Wo (Nicholas), SZE via rangadi) + Release 0.18.3 - 2009-01-27 IMPROVEMENTS diff --git a/third_party/hadoop-0.20.0/LICENSE.txt b/core/lib/hadoop-0.20.2/LICENSE.txt similarity index 100% rename from third_party/hadoop-0.20.0/LICENSE.txt rename to core/lib/hadoop-0.20.2/LICENSE.txt diff --git a/third_party/hadoop-0.20.0/NOTICE.txt b/core/lib/hadoop-0.20.2/NOTICE.txt similarity index 100% rename from third_party/hadoop-0.20.0/NOTICE.txt rename to core/lib/hadoop-0.20.2/NOTICE.txt diff --git a/third_party/hadoop-0.20.0/README.txt b/core/lib/hadoop-0.20.2/README.txt similarity index 100% rename from third_party/hadoop-0.20.0/README.txt rename to core/lib/hadoop-0.20.2/README.txt diff --git a/third_party/hadoop-0.20.0/bin/hadoop b/core/lib/hadoop-0.20.2/bin/hadoop similarity index 100% rename from third_party/hadoop-0.20.0/bin/hadoop rename to core/lib/hadoop-0.20.2/bin/hadoop diff --git a/third_party/hadoop-0.20.0/bin/hadoop-config.sh b/core/lib/hadoop-0.20.2/bin/hadoop-config.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/hadoop-config.sh rename to core/lib/hadoop-0.20.2/bin/hadoop-config.sh diff --git a/third_party/hadoop-0.20.0/bin/hadoop-daemon.sh b/core/lib/hadoop-0.20.2/bin/hadoop-daemon.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/hadoop-daemon.sh rename to core/lib/hadoop-0.20.2/bin/hadoop-daemon.sh diff --git a/third_party/hadoop-0.20.0/bin/hadoop-daemons.sh b/core/lib/hadoop-0.20.2/bin/hadoop-daemons.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/hadoop-daemons.sh rename to core/lib/hadoop-0.20.2/bin/hadoop-daemons.sh diff --git a/third_party/hadoop-0.20.0/bin/rcc b/core/lib/hadoop-0.20.2/bin/rcc similarity index 100% rename from third_party/hadoop-0.20.0/bin/rcc rename to core/lib/hadoop-0.20.2/bin/rcc diff --git a/third_party/hadoop-0.20.0/bin/slaves.sh b/core/lib/hadoop-0.20.2/bin/slaves.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/slaves.sh rename to core/lib/hadoop-0.20.2/bin/slaves.sh diff --git a/third_party/hadoop-0.20.0/bin/start-all.sh b/core/lib/hadoop-0.20.2/bin/start-all.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/start-all.sh rename to core/lib/hadoop-0.20.2/bin/start-all.sh diff --git a/third_party/hadoop-0.20.0/bin/start-balancer.sh b/core/lib/hadoop-0.20.2/bin/start-balancer.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/start-balancer.sh rename to core/lib/hadoop-0.20.2/bin/start-balancer.sh diff --git a/third_party/hadoop-0.20.0/bin/start-dfs.sh b/core/lib/hadoop-0.20.2/bin/start-dfs.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/start-dfs.sh rename to core/lib/hadoop-0.20.2/bin/start-dfs.sh diff --git a/third_party/hadoop-0.20.0/bin/start-mapred.sh b/core/lib/hadoop-0.20.2/bin/start-mapred.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/start-mapred.sh rename to core/lib/hadoop-0.20.2/bin/start-mapred.sh diff --git a/third_party/hadoop-0.20.0/bin/stop-all.sh b/core/lib/hadoop-0.20.2/bin/stop-all.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/stop-all.sh rename to core/lib/hadoop-0.20.2/bin/stop-all.sh diff --git a/third_party/hadoop-0.20.0/bin/stop-balancer.sh b/core/lib/hadoop-0.20.2/bin/stop-balancer.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/stop-balancer.sh rename to core/lib/hadoop-0.20.2/bin/stop-balancer.sh diff --git a/third_party/hadoop-0.20.0/bin/stop-dfs.sh b/core/lib/hadoop-0.20.2/bin/stop-dfs.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/stop-dfs.sh rename to core/lib/hadoop-0.20.2/bin/stop-dfs.sh diff --git a/third_party/hadoop-0.20.0/bin/stop-mapred.sh b/core/lib/hadoop-0.20.2/bin/stop-mapred.sh similarity index 100% rename from third_party/hadoop-0.20.0/bin/stop-mapred.sh rename to core/lib/hadoop-0.20.2/bin/stop-mapred.sh diff --git a/third_party/hadoop-0.20.0/build.xml b/core/lib/hadoop-0.20.2/build.xml similarity index 98% rename from third_party/hadoop-0.20.0/build.xml rename to core/lib/hadoop-0.20.2/build.xml index 68932d7d9f..602fcfe1b1 100644 --- a/third_party/hadoop-0.20.0/build.xml +++ b/core/lib/hadoop-0.20.2/build.xml @@ -27,7 +27,7 @@ - + @@ -137,7 +137,7 @@ - + @@ -704,6 +704,7 @@ + @@ -728,6 +729,10 @@ + + + + @@ -743,18 +748,30 @@ - Tests failed! + + + + Tests failed! + + + - - + + + + + + + Tests failed! + @@ -1393,6 +1410,7 @@ + @@ -1410,6 +1428,7 @@ + @@ -1432,6 +1451,7 @@ depends="check-c++-makefiles" if="need.c++.examples.pipes.makefile"> + diff --git a/third_party/hadoop-0.20.0/conf/capacity-scheduler.xml b/core/lib/hadoop-0.20.2/conf/capacity-scheduler.xml similarity index 60% rename from third_party/hadoop-0.20.0/conf/capacity-scheduler.xml rename to core/lib/hadoop-0.20.2/conf/capacity-scheduler.xml index d22a3964b4..e35b335607 100644 --- a/third_party/hadoop-0.20.0/conf/capacity-scheduler.xml +++ b/core/lib/hadoop-0.20.2/conf/capacity-scheduler.xml @@ -8,21 +8,13 @@ - mapred.capacity-scheduler.queue.default.guaranteed-capacity + mapred.capacity-scheduler.queue.default.capacity 100 Percentage of the number of slots in the cluster that are - guaranteed to be available for jobs in this queue. + to be available for jobs in this queue. - - mapred.capacity-scheduler.queue.default.reclaim-time-limit - 300 - The amount of time, in seconds, before which - resources distributed to other queues will be reclaimed. - - - mapred.capacity-scheduler.queue.default.supports-priority false @@ -54,28 +46,9 @@ - - - mapred.capacity-scheduler.reclaimCapacity.interval - 5 - The time interval, in seconds, between which the scheduler - periodically determines whether capacity needs to be reclaimed for - any queue. - - - - - mapred.capacity-scheduler.default-reclaim-time-limit - 300 - The amount of time, in seconds, before which - resources distributed to other queues will be reclaimed by default - in a job queue. - - - mapred.capacity-scheduler.default-supports-priority false @@ -83,37 +56,6 @@ account in scheduling decisions by default in a job queue. - - - mapred.capacity-scheduler.task.default-pmem-percentage-in-vmem - -1 - If mapred.task.maxpmem is set to -1, this configuration will - be used to calculate job's physical memory requirements as a percentage of - the job's virtual memory requirements set via mapred.task.maxvmem. This - property thus provides default value of physical memory for job's that - don't explicitly specify physical memory requirements. - - If not explicitly set to a valid value, scheduler will not consider - physical memory for scheduling even if virtual memory based scheduling is - enabled(by setting valid values for both mapred.task.default.maxvmem and - mapred.task.limit.maxvmem). - - - - - mapred.capacity-scheduler.task.limit.maxpmem - -1 - Configuration that provides an upper limit on the maximum - physical memory that can be specified by a job. The job configuration - mapred.task.maxpmem should be less than this value. If not, the job will - be rejected by the scheduler. - - If it is set to -1, scheduler will not consider physical memory for - scheduling even if virtual memory based scheduling is enabled(by setting - valid values for both mapred.task.default.maxvmem and - mapred.task.limit.maxvmem). - - mapred.capacity-scheduler.default-minimum-user-limit-percent diff --git a/third_party/hadoop-0.20.0/conf/configuration.xsl b/core/lib/hadoop-0.20.2/conf/configuration.xsl similarity index 100% rename from third_party/hadoop-0.20.0/conf/configuration.xsl rename to core/lib/hadoop-0.20.2/conf/configuration.xsl diff --git a/third_party/hadoop-0.20.0/conf/core-site.xml b/core/lib/hadoop-0.20.2/conf/core-site.xml similarity index 100% rename from third_party/hadoop-0.20.0/conf/core-site.xml rename to core/lib/hadoop-0.20.2/conf/core-site.xml diff --git a/third_party/hadoop-0.20.0/conf/hadoop-env.sh b/core/lib/hadoop-0.20.2/conf/hadoop-env.sh similarity index 100% rename from third_party/hadoop-0.20.0/conf/hadoop-env.sh rename to core/lib/hadoop-0.20.2/conf/hadoop-env.sh diff --git a/third_party/hadoop-0.20.0/conf/hadoop-metrics.properties b/core/lib/hadoop-0.20.2/conf/hadoop-metrics.properties similarity index 100% rename from third_party/hadoop-0.20.0/conf/hadoop-metrics.properties rename to core/lib/hadoop-0.20.2/conf/hadoop-metrics.properties diff --git a/third_party/hadoop-0.20.0/conf/hadoop-policy.xml b/core/lib/hadoop-0.20.2/conf/hadoop-policy.xml similarity index 100% rename from third_party/hadoop-0.20.0/conf/hadoop-policy.xml rename to core/lib/hadoop-0.20.2/conf/hadoop-policy.xml diff --git a/third_party/hadoop-0.20.0/conf/hdfs-site.xml b/core/lib/hadoop-0.20.2/conf/hdfs-site.xml similarity index 100% rename from third_party/hadoop-0.20.0/conf/hdfs-site.xml rename to core/lib/hadoop-0.20.2/conf/hdfs-site.xml diff --git a/third_party/hadoop-0.20.0/conf/log4j.properties b/core/lib/hadoop-0.20.2/conf/log4j.properties similarity index 100% rename from third_party/hadoop-0.20.0/conf/log4j.properties rename to core/lib/hadoop-0.20.2/conf/log4j.properties diff --git a/third_party/hadoop-0.20.0/conf/mapred-site.xml b/core/lib/hadoop-0.20.2/conf/mapred-site.xml similarity index 100% rename from third_party/hadoop-0.20.0/conf/mapred-site.xml rename to core/lib/hadoop-0.20.2/conf/mapred-site.xml diff --git a/third_party/hadoop-0.20.0/conf/masters b/core/lib/hadoop-0.20.2/conf/masters similarity index 100% rename from third_party/hadoop-0.20.0/conf/masters rename to core/lib/hadoop-0.20.2/conf/masters diff --git a/third_party/hadoop-0.20.0/conf/slaves b/core/lib/hadoop-0.20.2/conf/slaves similarity index 100% rename from third_party/hadoop-0.20.0/conf/slaves rename to core/lib/hadoop-0.20.2/conf/slaves diff --git a/third_party/hadoop-0.20.0/conf/ssl-client.xml.example b/core/lib/hadoop-0.20.2/conf/ssl-client.xml.example similarity index 100% rename from third_party/hadoop-0.20.0/conf/ssl-client.xml.example rename to core/lib/hadoop-0.20.2/conf/ssl-client.xml.example diff --git a/third_party/hadoop-0.20.0/conf/ssl-server.xml.example b/core/lib/hadoop-0.20.2/conf/ssl-server.xml.example similarity index 100% rename from third_party/hadoop-0.20.0/conf/ssl-server.xml.example rename to core/lib/hadoop-0.20.2/conf/ssl-server.xml.example diff --git a/third_party/hadoop-0.20.0/hadoop-0.20.0-ant.jar b/core/lib/hadoop-0.20.2/hadoop-0.20.2-ant.jar similarity index 63% rename from third_party/hadoop-0.20.0/hadoop-0.20.0-ant.jar rename to core/lib/hadoop-0.20.2/hadoop-0.20.2-ant.jar index 0c2b74e74f..ad504d3bff 100644 Binary files a/third_party/hadoop-0.20.0/hadoop-0.20.0-ant.jar and b/core/lib/hadoop-0.20.2/hadoop-0.20.2-ant.jar differ diff --git a/third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar b/core/lib/hadoop-0.20.2/hadoop-0.20.2-core.jar similarity index 64% rename from third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar rename to core/lib/hadoop-0.20.2/hadoop-0.20.2-core.jar index c99ce6d44d..32ae0a1c9c 100644 Binary files a/third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar and b/core/lib/hadoop-0.20.2/hadoop-0.20.2-core.jar differ diff --git a/third_party/hadoop-0.20.0/hadoop-0.20.0-examples.jar b/core/lib/hadoop-0.20.2/hadoop-0.20.2-examples.jar similarity index 91% rename from third_party/hadoop-0.20.0/hadoop-0.20.0-examples.jar rename to core/lib/hadoop-0.20.2/hadoop-0.20.2-examples.jar index 23b88f885d..0b1f32805b 100644 Binary files a/third_party/hadoop-0.20.0/hadoop-0.20.0-examples.jar and b/core/lib/hadoop-0.20.2/hadoop-0.20.2-examples.jar differ diff --git a/third_party/hadoop-0.20.0/hadoop-0.20.0-test.jar b/core/lib/hadoop-0.20.2/hadoop-0.20.2-test.jar similarity index 74% rename from third_party/hadoop-0.20.0/hadoop-0.20.0-test.jar rename to core/lib/hadoop-0.20.2/hadoop-0.20.2-test.jar index 02b17d4160..8ee3099df2 100644 Binary files a/third_party/hadoop-0.20.0/hadoop-0.20.0-test.jar and b/core/lib/hadoop-0.20.2/hadoop-0.20.2-test.jar differ diff --git a/third_party/hadoop-0.20.0/hadoop-0.20.0-tools.jar b/core/lib/hadoop-0.20.2/hadoop-0.20.2-tools.jar similarity index 66% rename from third_party/hadoop-0.20.0/hadoop-0.20.0-tools.jar rename to core/lib/hadoop-0.20.2/hadoop-0.20.2-tools.jar index 60f5e600a6..deee713f30 100644 Binary files a/third_party/hadoop-0.20.0/hadoop-0.20.0-tools.jar and b/core/lib/hadoop-0.20.2/hadoop-0.20.2-tools.jar differ diff --git a/third_party/hadoop-0.20.0/ivy.xml b/core/lib/hadoop-0.20.2/ivy.xml similarity index 97% rename from third_party/hadoop-0.20.0/ivy.xml rename to core/lib/hadoop-0.20.2/ivy.xml index 051ac6efb0..4fad1800b0 100644 --- a/third_party/hadoop-0.20.0/ivy.xml +++ b/core/lib/hadoop-0.20.2/ivy.xml @@ -80,10 +80,10 @@ - + - + + diff --git a/third_party/hadoop-0.20.0/ivy/hadoop-core.pom b/core/lib/hadoop-0.20.2/ivy/hadoop-core.pom similarity index 100% rename from third_party/hadoop-0.20.0/ivy/hadoop-core.pom rename to core/lib/hadoop-0.20.2/ivy/hadoop-core.pom diff --git a/third_party/hadoop-0.20.0/ivy/ivy-2.0.0-rc2.jar b/core/lib/hadoop-0.20.2/ivy/ivy-2.0.0-rc2.jar similarity index 100% rename from third_party/hadoop-0.20.0/ivy/ivy-2.0.0-rc2.jar rename to core/lib/hadoop-0.20.2/ivy/ivy-2.0.0-rc2.jar diff --git a/third_party/hadoop-0.20.0/ivy/ivysettings.xml b/core/lib/hadoop-0.20.2/ivy/ivysettings.xml similarity index 100% rename from third_party/hadoop-0.20.0/ivy/ivysettings.xml rename to core/lib/hadoop-0.20.2/ivy/ivysettings.xml diff --git a/third_party/hadoop-0.20.0/ivy/libraries.properties b/core/lib/hadoop-0.20.2/ivy/libraries.properties similarity index 97% rename from third_party/hadoop-0.20.0/ivy/libraries.properties rename to core/lib/hadoop-0.20.2/ivy/libraries.properties index 17cf390d92..c264797ef2 100644 --- a/third_party/hadoop-0.20.0/ivy/libraries.properties +++ b/core/lib/hadoop-0.20.2/ivy/libraries.properties @@ -21,7 +21,7 @@ apacheant.version=1.7.0 checkstyle.version=4.2 -commons-cli.version=2.0-SNAPSHOT +commons-cli.version=1.2 commons-codec.version=1.3 commons-collections.version=3.1 commons-httpclient.version=3.0.1 @@ -57,6 +57,8 @@ kfs.version=0.1 log4j.version=1.2.15 lucene-core.version=2.3.1 +mockito-all.version=1.8.0 + oro.version=2.0.8 rats-lib.version=0.5.1 diff --git a/core/lib/hadoop-0.20.2/lib/commons-cli-1.2.jar b/core/lib/hadoop-0.20.2/lib/commons-cli-1.2.jar new file mode 100644 index 0000000000..ce4b9fffe4 Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/commons-cli-1.2.jar differ diff --git a/third_party/hadoop-0.20.0/lib/commons-codec-1.3.jar b/core/lib/hadoop-0.20.2/lib/commons-codec-1.3.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/commons-codec-1.3.jar rename to core/lib/hadoop-0.20.2/lib/commons-codec-1.3.jar diff --git a/third_party/hadoop-0.20.0/lib/commons-el-1.0.jar b/core/lib/hadoop-0.20.2/lib/commons-el-1.0.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/commons-el-1.0.jar rename to core/lib/hadoop-0.20.2/lib/commons-el-1.0.jar diff --git a/third_party/hadoop-0.20.0/lib/commons-httpclient-3.0.1.jar b/core/lib/hadoop-0.20.2/lib/commons-httpclient-3.0.1.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/commons-httpclient-3.0.1.jar rename to core/lib/hadoop-0.20.2/lib/commons-httpclient-3.0.1.jar diff --git a/third_party/hadoop-0.20.0/lib/commons-logging-1.0.4.jar b/core/lib/hadoop-0.20.2/lib/commons-logging-1.0.4.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/commons-logging-1.0.4.jar rename to core/lib/hadoop-0.20.2/lib/commons-logging-1.0.4.jar diff --git a/third_party/hadoop-0.20.0/lib/commons-logging-api-1.0.4.jar b/core/lib/hadoop-0.20.2/lib/commons-logging-api-1.0.4.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/commons-logging-api-1.0.4.jar rename to core/lib/hadoop-0.20.2/lib/commons-logging-api-1.0.4.jar diff --git a/third_party/hadoop-0.20.0/lib/commons-net-1.4.1.jar b/core/lib/hadoop-0.20.2/lib/commons-net-1.4.1.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/commons-net-1.4.1.jar rename to core/lib/hadoop-0.20.2/lib/commons-net-1.4.1.jar diff --git a/third_party/hadoop-0.20.0/lib/core-3.1.1.jar b/core/lib/hadoop-0.20.2/lib/core-3.1.1.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/core-3.1.1.jar rename to core/lib/hadoop-0.20.2/lib/core-3.1.1.jar diff --git a/third_party/hadoop-0.20.0/lib/hsqldb-1.8.0.10.LICENSE.txt b/core/lib/hadoop-0.20.2/lib/hsqldb-1.8.0.10.LICENSE.txt similarity index 100% rename from third_party/hadoop-0.20.0/lib/hsqldb-1.8.0.10.LICENSE.txt rename to core/lib/hadoop-0.20.2/lib/hsqldb-1.8.0.10.LICENSE.txt diff --git a/third_party/hadoop-0.20.0/lib/hsqldb-1.8.0.10.jar b/core/lib/hadoop-0.20.2/lib/hsqldb-1.8.0.10.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/hsqldb-1.8.0.10.jar rename to core/lib/hadoop-0.20.2/lib/hsqldb-1.8.0.10.jar diff --git a/third_party/hadoop-0.20.0/lib/jasper-compiler-5.5.12.jar b/core/lib/hadoop-0.20.2/lib/jasper-compiler-5.5.12.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/jasper-compiler-5.5.12.jar rename to core/lib/hadoop-0.20.2/lib/jasper-compiler-5.5.12.jar diff --git a/third_party/hadoop-0.20.0/lib/jasper-runtime-5.5.12.jar b/core/lib/hadoop-0.20.2/lib/jasper-runtime-5.5.12.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/jasper-runtime-5.5.12.jar rename to core/lib/hadoop-0.20.2/lib/jasper-runtime-5.5.12.jar diff --git a/third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.17.0.xml b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.17.0.xml similarity index 100% rename from third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.17.0.xml rename to core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.17.0.xml diff --git a/third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.18.1.xml b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.18.1.xml similarity index 100% rename from third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.18.1.xml rename to core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.18.1.xml diff --git a/third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.18.2.xml b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.18.2.xml similarity index 100% rename from third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.18.2.xml rename to core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.18.2.xml diff --git a/third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.18.3.xml b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.18.3.xml similarity index 100% rename from third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.18.3.xml rename to core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.18.3.xml diff --git a/third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.19.0.xml b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.19.0.xml similarity index 100% rename from third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.19.0.xml rename to core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.19.0.xml diff --git a/third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.19.1.xml b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.19.1.xml similarity index 100% rename from third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.19.1.xml rename to core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.19.1.xml diff --git a/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.19.2.xml b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.19.2.xml new file mode 100644 index 0000000000..bbce108837 --- /dev/null +++ b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.19.2.xml @@ -0,0 +1,44204 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + final. + + @param name resource to be added, the classpath is examined for a file + with that name.]]> + + + + + + final. + + @param url url of the resource to be added, the local filesystem is + examined directly to find the resource, without referring to + the classpath.]]> + + + + + + final. + + @param file file-path of resource to be added, the local filesystem is + examined directly to find the resource, without referring to + the classpath.]]> + + + + + + final. + + @param in InputStream to deserialize the object from.]]> + + + + + + + + + + + name property, null if + no such property exists. + + Values are processed for variable expansion + before being returned. + + @param name the property name. + @return the value of the name property, + or null if no such property exists.]]> + + + + + + name property, without doing + variable expansion. + + @param name the property name. + @return the value of the name property, + or null if no such property exists.]]> + + + + + + + value of the name property. + + @param name property name. + @param value property value.]]> + + + + + + + name property. If no such property + exists, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value, or defaultValue if the property + doesn't exist.]]> + + + + + + + name property as an int. + + If no such property exists, or if the specified value is not a valid + int, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as an int, + or defaultValue.]]> + + + + + + + name property to an int. + + @param name property name. + @param value int value of the property.]]> + + + + + + + name property as a long. + If no such property is specified, or if the specified value is not a valid + long, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as a long, + or defaultValue.]]> + + + + + + + name property to a long. + + @param name property name. + @param value long value of the property.]]> + + + + + + + name property as a float. + If no such property is specified, or if the specified value is not a valid + float, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as a float, + or defaultValue.]]> + + + + + + + name property as a boolean. + If no such property is specified, or if the specified value is not a valid + boolean, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as a boolean, + or defaultValue.]]> + + + + + + + name property to a boolean. + + @param name property name. + @param value boolean value of the property.]]> + + + + + + + + + + + + + name property as + a collection of Strings. + If no such property is specified then empty collection is returned. +

+ This is an optimized version of {@link #getStrings(String)} + + @param name property name. + @return property value as a collection of Strings.]]> + + + + + + name property as + an array of Strings. + If no such property is specified then null is returned. + + @param name property name. + @return property value as an array of Strings, + or null.]]> + + + + + + + name property as + an array of Strings. + If no such property is specified then default value is returned. + + @param name property name. + @param defaultValue The default value + @return property value as an array of Strings, + or default value.]]> + + + + + + + name property as + as comma delimited values. + + @param name property name. + @param values The values]]> + + + + + + + + + + + + + + name property + as an array of Class. + The value of the property specifies a list of comma separated class names. + If no such property is specified, then defaultValue is + returned. + + @param name the property name. + @param defaultValue default value. + @return property value as a Class[], + or defaultValue.]]> + + + + + + + name property as a Class. + If no such property is specified, then defaultValue is + returned. + + @param name the class name. + @param defaultValue default value. + @return property value as a Class, + or defaultValue.]]> + + + + + + + + name property as a Class + implementing the interface specified by xface. + + If no such property is specified, then defaultValue is + returned. + + An exception is thrown if the returned class does not implement the named + interface. + + @param name the class name. + @param defaultValue default value. + @param xface the interface implemented by the named class. + @return property value as a Class, + or defaultValue.]]> + + + + + + + + name property to the name of a + theClass implementing the given interface xface. + + An exception is thrown if theClass does not implement the + interface xface. + + @param name property name. + @param theClass property value. + @param xface the interface implemented by the named class.]]> + + + + + + + + dirsProp with + the given path. If dirsProp contains multiple directories, + then one is chosen based on path's hash code. If the selected + directory does not exist, an attempt is made to create it. + + @param dirsProp directory in which to locate the file. + @param path file-path. + @return local file under the directory with the given path.]]> + + + + + + + + dirsProp with + the given path. If dirsProp contains multiple directories, + then one is chosen based on path's hash code. If the selected + directory does not exist, an attempt is made to create it. + + @param dirsProp directory in which to locate the file. + @param path file-path. + @return local file under the directory with the given path.]]> + + + + + + + + + + + + name. + + @param name configuration resource name. + @return an input stream attached to the resource.]]> + + + + + + name. + + @param name configuration resource name. + @return a reader attached to the resource.]]> + + + + + + + + + + + + + + + String + key-value pairs in the configuration. + + @return an iterator over the entries.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + true to set quiet-mode on, false + to turn it off.]]> + + + + + + + + + + + + + + + + + + + Resources + +

Configurations are specified by resources. A resource contains a set of + name/value pairs as XML data. Each resource is named by either a + String or by a {@link Path}. If named by a String, + then the classpath is examined for a file with that name. If named by a + Path, then the local filesystem is examined directly, without + referring to the classpath. + +

Unless explicitly turned off, Hadoop by default specifies two + resources, loaded in-order from the classpath:

    +
  1. hadoop-default.xml + : Read-only defaults for hadoop.
  2. +
  3. hadoop-site.xml: Site-specific configuration for a given hadoop + installation.
  4. +
+ Applications may add additional resources, which are loaded + subsequent to these resources in the order they are added. + +

Final Parameters

+ +

Configuration parameters may be declared final. + Once a resource declares a value final, no subsequently-loaded + resource can alter that value. + For example, one might define a final parameter with: +

+  <property>
+    <name>dfs.client.buffer.dir</name>
+    <value>/tmp/hadoop/dfs/client</value>
+    <final>true</final>
+  </property>
+ + Administrators typically define parameters as final in + hadoop-site.xml for values that user applications may not alter. + +

Variable Expansion

+ +

Value strings are first processed for variable expansion. The + available properties are:

    +
  1. Other properties defined in this Configuration; and, if a name is + undefined here,
  2. +
  3. Properties in {@link System#getProperties()}.
  4. +
+ +

For example, if a configuration resource contains the following property + definitions: +

+  <property>
+    <name>basedir</name>
+    <value>/user/${user.name}</value>
+  </property>
+  
+  <property>
+    <name>tempdir</name>
+    <value>${basedir}/tmp</value>
+  </property>
+ + When conf.get("tempdir") is called, then ${basedir} + will be resolved to another property in this Configuration, while + ${user.name} would then ordinarily be resolved to the value + of the System property with that name.]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DistributedCache is a facility provided by the Map-Reduce + framework to cache files (text, archives, jars etc.) needed by applications. +

+ +

Applications specify the files, via urls (hdfs:// or http://) to be cached + via the {@link org.apache.hadoop.mapred.JobConf}. + The DistributedCache assumes that the + files specified via hdfs:// urls are already present on the + {@link FileSystem} at the path specified by the url.

+ +

The framework will copy the necessary files on to the slave node before + any tasks for the job are executed on that node. Its efficiency stems from + the fact that the files are only copied once per job and the ability to + cache archives which are un-archived on the slaves.

+ +

DistributedCache can be used to distribute simple, read-only + data/text files and/or more complex types such as archives, jars etc. + Archives (zip, tar and tgz/tar.gz files) are un-archived at the slave nodes. + Jars may be optionally added to the classpath of the tasks, a rudimentary + software distribution mechanism. Files have execution permissions. + Optionally users can also direct it to symlink the distributed cache file(s) + into the working directory of the task.

+ +

DistributedCache tracks modification timestamps of the cache + files. Clearly the cache files should not be modified by the application + or externally while the job is executing.

+ +

Here is an illustrative example on how to use the + DistributedCache:

+

+     // Setting up the cache for the application
+     
+     1. Copy the requisite files to the FileSystem:
+     
+     $ bin/hadoop fs -copyFromLocal lookup.dat /myapp/lookup.dat  
+     $ bin/hadoop fs -copyFromLocal map.zip /myapp/map.zip  
+     $ bin/hadoop fs -copyFromLocal mylib.jar /myapp/mylib.jar
+     $ bin/hadoop fs -copyFromLocal mytar.tar /myapp/mytar.tar
+     $ bin/hadoop fs -copyFromLocal mytgz.tgz /myapp/mytgz.tgz
+     $ bin/hadoop fs -copyFromLocal mytargz.tar.gz /myapp/mytargz.tar.gz
+     
+     2. Setup the application's JobConf:
+     
+     JobConf job = new JobConf();
+     DistributedCache.addCacheFile(new URI("/myapp/lookup.dat#lookup.dat"), 
+                                   job);
+     DistributedCache.addCacheArchive(new URI("/myapp/map.zip", job);
+     DistributedCache.addFileToClassPath(new Path("/myapp/mylib.jar"), job);
+     DistributedCache.addCacheArchive(new URI("/myapp/mytar.tar", job);
+     DistributedCache.addCacheArchive(new URI("/myapp/mytgz.tgz", job);
+     DistributedCache.addCacheArchive(new URI("/myapp/mytargz.tar.gz", job);
+     
+     3. Use the cached files in the {@link org.apache.hadoop.mapred.Mapper}
+     or {@link org.apache.hadoop.mapred.Reducer}:
+     
+     public static class MapClass extends MapReduceBase  
+     implements Mapper<K, V, K, V> {
+     
+       private Path[] localArchives;
+       private Path[] localFiles;
+       
+       public void configure(JobConf job) {
+         // Get the cached archives/files
+         localArchives = DistributedCache.getLocalCacheArchives(job);
+         localFiles = DistributedCache.getLocalCacheFiles(job);
+       }
+       
+       public void map(K key, V value, 
+                       OutputCollector<K, V> output, Reporter reporter) 
+       throws IOException {
+         // Use data from the cached archives/files here
+         // ...
+         // ...
+         output.collect(k, v);
+       }
+     }
+     
+ 

+ + @see org.apache.hadoop.mapred.JobConf + @see org.apache.hadoop.mapred.JobClient]]> +
+
+ +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + BufferedFSInputStream + with the specified buffer size, + and saves its argument, the input stream + in, for later use. An internal + buffer array of length size + is created and stored in buf. + + @param in the underlying input stream. + @param size the buffer size. + @exception IllegalArgumentException if size <= 0.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + setReplication of FileSystem + @param src file name + @param replication new replication + @throws IOException + @return true if successful; + false if file does not exist or is a directory]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ' + @deprecated Consider using {@link GenericOptionsParser} instead.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + fs.scheme.class whose value names the FileSystem class. + The entire URI is passed to the FileSystem instance's initialize method.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Return all the files that match filePattern and are not checksum + files. Results are sorted by their names. + +

+ A filename pattern is composed of regular characters and + special pattern matching characters, which are: + +

+
+
+

+

? +
Matches any single character. + +

+

* +
Matches zero or more characters. + +

+

[abc] +
Matches a single character from character set + {a,b,c}. + +

+

[a-b] +
Matches a single character from the character range + {a...b}. Note that character a must be + lexicographically less than or equal to character b. + +

+

[^a] +
Matches a single character that is not from character set or range + {a}. Note that the ^ character must occur + immediately to the right of the opening bracket. + +

+

\c +
Removes (escapes) any special meaning of character c. + +

+

{ab,cd} +
Matches a string from the string set {ab, cd} + +

+

{ab,c{de,fh}} +
Matches a string from the string set {ab, cde, cfh} + +
+
+
+ + @param pathPattern a regular expression specifying a pth pattern + + @return an array of paths that match the path pattern + @throws IOException]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + All user code that may potentially use the Hadoop Distributed + File System should be written to use a FileSystem object. The + Hadoop DFS is a multi-machine system that appears as a single + disk. It's useful because of its fault tolerance and potentially + very large capacity. + +

+ The local implementation is {@link LocalFileSystem} and distributed + implementation is DistributedFileSystem.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FilterFileSystem contains + some other file system, which it uses as + its basic file system, possibly transforming + the data along the way or providing additional + functionality. The class FilterFileSystem + itself simply overrides all methods of + FileSystem with versions that + pass all requests to the contained file + system. Subclasses of FilterFileSystem + may further override some of these methods + and may also provide additional methods + and fields.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + buf at offset + and checksum into checksum. + The method is used for implementing read, therefore, it should be optimized + for sequential reading + @param pos chunkPos + @param buf desitination buffer + @param offset offset in buf at which to store data + @param len maximun number of bytes to read + @return number of bytes read]]> + + + + + + + + + + + + + + + + + -1 if the end of the + stream is reached. + @exception IOException if an I/O error occurs.]]> + + + + + + + + + This method implements the general contract of the corresponding + {@link InputStream#read(byte[], int, int) read} method of + the {@link InputStream} class. As an additional + convenience, it attempts to read as many bytes as possible by repeatedly + invoking the read method of the underlying stream. This + iterated read continues until one of the following + conditions becomes true:

    + +
  • The specified number of bytes have been read, + +
  • The read method of the underlying stream returns + -1, indicating end-of-file. + +
If the first read on the underlying stream returns + -1 to indicate end-of-file then this method returns + -1. Otherwise this method returns the number of bytes + actually read. + + @param b destination buffer. + @param off offset at which to start storing bytes. + @param len maximum number of bytes to read. + @return the number of bytes read, or -1 if the end of + the stream has been reached. + @exception IOException if an I/O error occurs. + ChecksumException if any checksum error occurs]]> +
+ + + + + + + + + + + + + + + + + + n bytes of data from the + input stream. + +

This method may skip more bytes than are remaining in the backing + file. This produces no exception and the number of bytes skipped + may include some number of bytes that were beyond the EOF of the + backing file. Attempting to read from the stream after skipping past + the end will result in -1 indicating the end of the file. + +

If n is negative, no bytes are skipped. + + @param n the number of bytes to be skipped. + @return the actual number of bytes skipped. + @exception IOException if an I/O error occurs. + ChecksumException if the chunk to skip to is corrupted]]> + + + + + + + This method may seek past the end of the file. + This produces no exception and an attempt to read from + the stream will result in -1 indicating the end of the file. + + @param pos the postion to seek to. + @exception IOException if an I/O error occurs. + ChecksumException if the chunk to seek to is corrupted]]> + + + + + + + + + + len bytes from + stm + + @param stm an input stream + @param buf destiniation buffer + @param offset offset at which to store data + @param len number of bytes to read + @return actual number of bytes read + @throws IOException if there is any IO error]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + len bytes from the specified byte array + starting at offset off and generate a checksum for + each data chunk. + +

This method stores bytes from the given array into this + stream's buffer before it gets checksumed. The buffer gets checksumed + and flushed to the underlying output stream when all data + in a checksum chunk are in the buffer. If the buffer is empty and + requested length is at least as large as the size of next checksum chunk + size, this method will checksum and write the chunk directly + to the underlying output stream. Thus it avoids uneccessary data copy. + + @param b the data. + @param off the start offset in the data. + @param len the number of bytes to write. + @exception IOException if an I/O error occurs.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if and only if pathname + should be included]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + trash feature. Files are moved to a user's trash + directory, a subdirectory of their home directory named ".Trash". Files are + initially moved to a current sub-directory of the trash directory. + Within that sub-directory their original path is preserved. Periodically + one may checkpoint the current trash and remove older checkpoints. (This + design permits trash management without enumeration of the full trash + content, without date support in the filesystem, and without clock + synchronization.)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A {@link FileSystem} backed by an FTP client provided by Apache Commons Net. +

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class is a tool for migrating data from an older to a newer version + of an S3 filesystem. +

+

+ All files in the filesystem are migrated by re-writing the block metadata + - no datafiles are touched. +

]]> +
+
+ + + + + + + + + + + + + + + + + + + Extracts AWS credentials from the filesystem URI or configuration. +

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A block-based {@link FileSystem} backed by + Amazon S3. +

+ @see NativeS3FileSystem]]> +
+
+ + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + If f is a file, this method will make a single call to S3. + If f is a directory, this method will make a maximum of + (n / 1000) + 2 calls to S3, where n is the total number of + files and directories contained directly in f. +

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + A {@link FileSystem} for reading and writing files stored on + Amazon S3. + Unlike {@link org.apache.hadoop.fs.s3.S3FileSystem} this implementation + stores files on S3 in their + native form so they can be read by other S3 tools. +

+ @see org.apache.hadoop.fs.s3.S3FileSystem]]> +
+
+ +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + . + @param name The name of the server + @param port The port to use on the server + @param findPort whether the server should start at the given port and + increment by 1 until it finds a free port. + @param conf Configuration]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + points to the log directory + "/static/" -> points to common static files (src/webapps/static) + "/" -> the jsp server code from (src/webapps/)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + nth value.]]> + + + + + + + + + + + + + + + + + + + + + nth value in the file.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + public class IntArrayWritable extends ArrayWritable { + public IntArrayWritable() { + super(IntWritable.class); + } + } + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a ByteWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This saves memory over creating a new DataInputStream and + ByteArrayInputStream each time data is read. + +

Typical usage is something like the following:

+
+ DataInputBuffer buffer = new DataInputBuffer();
+ while (... loop condition ...) {
+   byte[] data = ... get data ...;
+   int dataLength = ... get data length ...;
+   buffer.reset(data, dataLength);
+   ... read buffer using DataInput methods ...
+ }
+ 
]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This saves memory over creating a new DataOutputStream and + ByteArrayOutputStream each time data is written. + +

Typical usage is something like the following:

+
+ DataOutputBuffer buffer = new DataOutputBuffer();
+ while (... loop condition ...) {
+   buffer.reset();
+   ... write buffer using DataOutput methods ...
+   byte[] data = buffer.getData();
+   int dataLength = buffer.getLength();
+   ... write data to its ultimate destination ...
+ }
+ 
]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + the class of the item + @param conf the configuration to store + @param item the object to be stored + @param keyName the name of the key to use + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param keyName the name of the key to use + @param itemClass the class of the item + @return restored object + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param items the objects to be stored + @param keyName the name of the key to use + @throws IndexOutOfBoundsException if the items array is empty + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param keyName the name of the key to use + @param itemClass the class of the item + @return restored object + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + DefaultStringifier offers convenience methods to store/load objects to/from + the configuration. + + @param the class of the objects to stringify]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a DoubleWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a FloatWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + When two sequence files, which have same Key type but different Value + types, are mapped out to reduce, multiple Value types is not allowed. + In this case, this class can help you wrap instances with different types. +

+ +

+ Compared with ObjectWritable, this class is much more effective, + because ObjectWritable will append the class declaration as a String + into the output file in every Key-Value pair. +

+ +

+ Generic Writable implements {@link Configurable} interface, so that it will be + configured by the framework. The configuration is passed to the wrapped objects + implementing {@link Configurable} interface before deserialization. +

+ + how to use it:
+ 1. Write your own class, such as GenericObject, which extends GenericWritable.
+ 2. Implements the abstract method getTypes(), defines + the classes which will be wrapped in GenericObject in application. + Attention: this classes defined in getTypes() method, must + implement Writable interface. +

+ + The code looks like this: +
+ public class GenericObject extends GenericWritable {
+ 
+   private static Class[] CLASSES = {
+               ClassType1.class, 
+               ClassType2.class,
+               ClassType3.class,
+               };
+
+   protected Class[] getTypes() {
+       return CLASSES;
+   }
+
+ }
+ 
+ + @since Nov 8, 2006]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This saves memory over creating a new InputStream and + ByteArrayInputStream each time data is read. + +

Typical usage is something like the following:

+
+ InputBuffer buffer = new InputBuffer();
+ while (... loop condition ...) {
+   byte[] data = ... get data ...;
+   int dataLength = ... get data length ...;
+   buffer.reset(data, dataLength);
+   ... read buffer using InputStream methods ...
+ }
+ 
+ @see DataInputBuffer + @see DataOutput]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a IntWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + closes the input and output streams + at the end. + @param in InputStrem to read from + @param out OutputStream to write to + @param conf the Configuration object]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ignore any {@link IOException} or + null pointers. Must only be used for cleanup in exception handlers. + @param log the log to record problems to at debug level. Can be null. + @param closeables the objects to close]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a LongWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A map is a directory containing two files, the data file, + containing all keys and values in the map, and a smaller index + file, containing a fraction of the keys. The fraction is determined by + {@link Writer#getIndexInterval()}. + +

The index file is read entirely into memory. Thus key implementations + should try to keep themselves small. + +

Map files are created by adding entries in-order. To maintain a large + database, perform updates by copying the previous version of a database and + merging in a sorted change list, to create a new version of the database in + a new file. Sorting large change lists can be done with {@link + SequenceFile.Sorter}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + key and + val. Returns true if such a pair exists and false when at + the end of the map]]> + + + + + + + + + + + + + + + + key or if it does not exist, at the first entry + after the named key. + +- * @param key - key that we're trying to find +- * @param val - data value if key is found +- * @return - the key that was the closest match or null if eof.]]> + + + + + + + + + key does not exist, return + the first entry that falls just before the key. Otherwise, + return the record that sorts just after. + @return - the key that was the closest match or null if eof.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is an MD5Hash whose digest contains the + same values.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This saves memory over creating a new OutputStream and + ByteArrayOutputStream each time data is written. + +

Typical usage is something like the following:

+
+ OutputBuffer buffer = new OutputBuffer();
+ while (... loop condition ...) {
+   buffer.reset();
+   ... write buffer using OutputStream methods ...
+   byte[] data = buffer.getData();
+   int dataLength = buffer.getLength();
+   ... write data to its ultimate destination ...
+ }
+ 
+ @see DataOutputBuffer + @see InputBuffer]]> +
+
+ + + + + + + + + + + + + + + A {@link Comparator} that operates directly on byte representations of + objects. +

+ @param + @see DeserializerComparator]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SequenceFiles are flat files consisting of binary key/value + pairs. + +

SequenceFile provides {@link Writer}, {@link Reader} and + {@link Sorter} classes for writing, reading and sorting respectively.

+ + There are three SequenceFile Writers based on the + {@link CompressionType} used to compress key/value pairs: +
    +
  1. + Writer : Uncompressed records. +
  2. +
  3. + RecordCompressWriter : Record-compressed files, only compress + values. +
  4. +
  5. + BlockCompressWriter : Block-compressed files, both keys & + values are collected in 'blocks' + separately and compressed. The size of + the 'block' is configurable. +
+ +

The actual compression algorithm used to compress key and/or values can be + specified by using the appropriate {@link CompressionCodec}.

+ +

The recommended way is to use the static createWriter methods + provided by the SequenceFile to chose the preferred format.

+ +

The {@link Reader} acts as the bridge and can read any of the above + SequenceFile formats.

+ +

SequenceFile Formats

+ +

Essentially there are 3 different formats for SequenceFiles + depending on the CompressionType specified. All of them share a + common header described below. + +

+
    +
  • + version - 3 bytes of magic header SEQ, followed by 1 byte of actual + version number (e.g. SEQ4 or SEQ6) +
  • +
  • + keyClassName -key class +
  • +
  • + valueClassName - value class +
  • +
  • + compression - A boolean which specifies if compression is turned on for + keys/values in this file. +
  • +
  • + blockCompression - A boolean which specifies if block-compression is + turned on for keys/values in this file. +
  • +
  • + compression codec - CompressionCodec class which is used for + compression of keys and/or values (if compression is + enabled). +
  • +
  • + metadata - {@link Metadata} for this file. +
  • +
  • + sync - A sync marker to denote end of the header. +
  • +
+ +
Uncompressed SequenceFile Format
+
    +
  • + Header +
  • +
  • + Record +
      +
    • Record length
    • +
    • Key length
    • +
    • Key
    • +
    • Value
    • +
    +
  • +
  • + A sync-marker every few 100 bytes or so. +
  • +
+ +
Record-Compressed SequenceFile Format
+
    +
  • + Header +
  • +
  • + Record +
      +
    • Record length
    • +
    • Key length
    • +
    • Key
    • +
    • Compressed Value
    • +
    +
  • +
  • + A sync-marker every few 100 bytes or so. +
  • +
+ +
Block-Compressed SequenceFile Format
+
    +
  • + Header +
  • +
  • + Record Block +
      +
    • Compressed key-lengths block-size
    • +
    • Compressed key-lengths block
    • +
    • Compressed keys block-size
    • +
    • Compressed keys block
    • +
    • Compressed value-lengths block-size
    • +
    • Compressed value-lengths block
    • +
    • Compressed values block-size
    • +
    • Compressed values block
    • +
    +
  • +
  • + A sync-marker every few 100 bytes or so. +
  • +
+ +

The compressed blocks of key lengths and value lengths consist of the + actual lengths of individual keys/values encoded in ZeroCompressedInteger + format.

+ + @see CompressionCodec]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + key, skipping its + value. True if another entry exists, and false at end of file.]]> + + + + + + + + key and + val. Returns true if such a pair exists and false when at + end of file]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The position passed must be a position returned by {@link + SequenceFile.Writer#getLength()} when writing this file. To seek to an arbitrary + position, use {@link SequenceFile.Reader#sync(long)}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SegmentDescriptor + @param segments the list of SegmentDescriptors + @param tmpDir the directory to write temporary files into + @return RawKeyValueIterator + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + For best performance, applications should make sure that the {@link + Writable#readFields(DataInput)} implementation of their keys is + very efficient. In particular, it should avoid allocating memory.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This always returns a synchronized position. In other words, + immediately after calling {@link SequenceFile.Reader#seek(long)} with a position + returned by this method, {@link SequenceFile.Reader#next(Writable)} may be called. However + the key may be earlier in the file than key last written when this + method was called (e.g., with block-compression, it may be the first key + in the block that was being written when this method was called).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + key. Returns + true if such a key exists and false when at the end of the set.]]> + + + + + + + key. + Returns key, or null if no match exists.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the class of the objects to stringify]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + position. Note that this + method avoids using the converter or doing String instatiation + @return the Unicode scalar value at position or -1 + if the position is invalid or points to a + trailing byte]]> + + + + + + + + + + what in the backing + buffer, starting as position start. The starting + position is measured in bytes and the return value is in + terms of byte position in the buffer. The backing buffer is + not converted to a string for this operation. + @return byte position of the first occurence of the search + string in the UTF-8 buffer or -1 if not found]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a Text with the same contents.]]> + + + + + + + + + + + + + + + + + + + + + + + + + replace is true, then + malformed input is replaced with the + substitution character, which is U+FFFD. Otherwise the + method throws a MalformedInputException.]]> + + + + + + + + + + + + + + + replace is true, then + malformed input is replaced with the + substitution character, which is U+FFFD. Otherwise the + method throws a MalformedInputException. + @return ByteBuffer: bytes stores at ByteBuffer.array() + and length is ByteBuffer.limit()]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + In + addition, it provides methods for string traversal without converting the + byte array to a string.

Also includes utilities for + serializing/deserialing a string, coding/decoding a string, checking if a + byte array contains valid UTF8 code, calculating the length of an encoded + string.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a UTF8 with the same contents.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + Also includes utilities for efficiently reading and writing UTF-8. + + @deprecated replaced by Text]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This is useful when a class may evolve, so that instances written by the + old version of the class may still be processed by the new version. To + handle this situation, {@link #readFields(DataInput)} + implementations should catch {@link VersionMismatchException}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a VIntWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a VLongWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + out. + + @param out DataOuput to serialize this object into. + @throws IOException]]> + + + + + + + in. + +

For efficiency, implementations should attempt to re-use storage in the + existing object where possible.

+ + @param in DataInput to deseriablize this object from. + @throws IOException]]> +
+ + + Any key or value type in the Hadoop Map-Reduce + framework implements this interface.

+ +

Implementations typically implement a static read(DataInput) + method which constructs a new instance, calls {@link #readFields(DataInput)} + and returns the instance.

+ +

Example:

+

+     public class MyWritable implements Writable {
+       // Some data     
+       private int counter;
+       private long timestamp;
+       
+       public void write(DataOutput out) throws IOException {
+         out.writeInt(counter);
+         out.writeLong(timestamp);
+       }
+       
+       public void readFields(DataInput in) throws IOException {
+         counter = in.readInt();
+         timestamp = in.readLong();
+       }
+       
+       public static MyWritable read(DataInput in) throws IOException {
+         MyWritable w = new MyWritable();
+         w.readFields(in);
+         return w;
+       }
+     }
+ 

]]> +
+ + + + + + + + WritableComparables can be compared to each other, typically + via Comparators. Any type which is to be used as a + key in the Hadoop Map-Reduce framework should implement this + interface.

+ +

Example:

+

+     public class MyWritableComparable implements WritableComparable {
+       // Some data
+       private int counter;
+       private long timestamp;
+       
+       public void write(DataOutput out) throws IOException {
+         out.writeInt(counter);
+         out.writeLong(timestamp);
+       }
+       
+       public void readFields(DataInput in) throws IOException {
+         counter = in.readInt();
+         timestamp = in.readLong();
+       }
+       
+       public int compareTo(MyWritableComparable w) {
+         int thisValue = this.value;
+         int thatValue = ((IntWritable)o).value;
+         return (thisValue < thatValue ? -1 : (thisValue==thatValue ? 0 : 1));
+       }
+     }
+ 

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The default implementation reads the data into two {@link + WritableComparable}s (using {@link + Writable#readFields(DataInput)}, then calls {@link + #compare(WritableComparable,WritableComparable)}.]]> + + + + + + + The default implementation uses the natural ordering, calling {@link + Comparable#compareTo(Object)}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This base implemenation uses the natural ordering. To define alternate + orderings, override {@link #compare(WritableComparable,WritableComparable)}. + +

One may optimize compare-intensive operations by overriding + {@link #compare(byte[],int,int,byte[],int,int)}. Static utility methods are + provided to assist in optimized implementations of this method.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Enum type + @param in DataInput to read from + @param enumType Class type of Enum + @return Enum represented by String read from DataInput + @throws IOException]]> + + + + + + + + + + + + + + + + len number of bytes in input streamin + @param in input stream + @param len number of bytes to skip + @throws IOException when skipped less number of bytes]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CompressionCodec for which to get the + Compressor + @return Compressor for the given + CompressionCodec from the pool or a new one]]> + + + + + + CompressionCodec for which to get the + Decompressor + @return Decompressor for the given + CompressionCodec the pool or a new one]]> + + + + + + Compressor to be returned to the pool]]> + + + + + + Decompressor to be returned to the + pool]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Implementations are assumed to be buffered. This permits clients to + reposition the underlying input stream then call {@link #resetState()}, + without having to also synchronize client buffers.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true indicating that more input data is required. + + @param b Input data + @param off Start offset + @param len Length]]> + + + + + true if the input data buffer is empty and + #setInput() should be called in order to provide more input.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the end of the compressed + data output stream has been reached.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true indicating that more input data is required. + + @param b Input data + @param off Start offset + @param len Length]]> + + + + + true if the input data buffer is empty and + #setInput() should be called in order to provide more input.]]> + + + + + + + + + + + + + true if a preset dictionary is needed for decompression. + @return true if a preset dictionary is needed for decompression]]> + + + + + true if the end of the compressed + data output stream has been reached.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if native-lzo library is loaded & initialized; + else false]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + lzo compression/decompression pair. + http://www.oberhumer.com/opensource/lzo/]]> + + + + + + + + + + + + + + + + + + + + + + + lzo compression/decompression pair compatible with lzop. + http://www.lzop.org/]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FIXME: This array should be in a private or package private location, + since it could be modified by malicious code. +

]]> +
+ + + + This interface is public for historical purposes. You should have no need to + use it. +

]]> +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Although BZip2 headers are marked with the magic "Bz" this + constructor expects the next byte in the stream to be the first one after + the magic. Thus callers have to skip the first two bytes. Otherwise this + constructor will throw an exception. +

+ + @throws IOException + if the stream content is malformed or an I/O error occurs. + @throws NullPointerException + if in == null]]> +
+
+ + + + + + + + + + + + + + + The decompression requires large amounts of memory. Thus you should call the + {@link #close() close()} method as soon as possible, to force + CBZip2InputStream to release the allocated memory. See + {@link CBZip2OutputStream CBZip2OutputStream} for information about memory + usage. +

+ +

+ CBZip2InputStream reads bytes from the compressed source stream via + the single byte {@link java.io.InputStream#read() read()} method exclusively. + Thus you should consider to use a buffered source stream. +

+ +

+ Instances of this class are not threadsafe. +

]]> +
+
+ + + + + + + + CBZip2OutputStream with a blocksize of 900k. + +

+ Attention: The caller is resonsible to write the two BZip2 magic + bytes "BZ" to the specified stream prior to calling this + constructor. +

+ + @param out * + the destination stream. + + @throws IOException + if an I/O error occurs in the specified stream. + @throws NullPointerException + if out == null.]]> +
+
+ + + + CBZip2OutputStream with specified blocksize. + +

+ Attention: The caller is resonsible to write the two BZip2 magic + bytes "BZ" to the specified stream prior to calling this + constructor. +

+ + + @param out + the destination stream. + @param blockSize + the blockSize as 100k units. + + @throws IOException + if an I/O error occurs in the specified stream. + @throws IllegalArgumentException + if (blockSize < 1) || (blockSize > 9). + @throws NullPointerException + if out == null. + + @see #MIN_BLOCKSIZE + @see #MAX_BLOCKSIZE]]> +
+
+ + + + + + + + + + + + + inputLength this method returns MAX_BLOCKSIZE + always. + + @param inputLength + The length of the data which will be compressed by + CBZip2OutputStream.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + == 1.]]> + + + + + == 9.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + If you are ever unlucky/improbable enough to get a stack overflow whilst + sorting, increase the following constant and try again. In practice I + have never seen the stack go above 27 elems, so the following limit seems + very generous. +

]]> +
+
+ + + The compression requires large amounts of memory. Thus you should call the + {@link #close() close()} method as soon as possible, to force + CBZip2OutputStream to release the allocated memory. +

+ +

+ You can shrink the amount of allocated memory and maybe raise the compression + speed by choosing a lower blocksize, which in turn may cause a lower + compression ratio. You can avoid unnecessary memory allocation by avoiding + using a blocksize which is bigger than the size of the input. +

+ +

+ You can compute the memory usage for compressing by the following formula: +

+ +
+ <code>400k + (9 * blocksize)</code>.
+ 
+ +

+ To get the memory required for decompression by {@link CBZip2InputStream + CBZip2InputStream} use +

+ +
+ <code>65k + (5 * blocksize)</code>.
+ 
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Memory usage by blocksize
Blocksize Compression
+ memory usage
Decompression
+ memory usage
100k1300k565k
200k2200k1065k
300k3100k1565k
400k4000k2065k
500k4900k2565k
600k5800k3065k
700k6700k3565k
800k7600k4065k
900k8500k4565k
+ +

+ For decompression CBZip2InputStream allocates less memory if the + bzipped input is smaller than one block. +

+ +

+ Instances of this class are not threadsafe. +

+ +

+ TODO: Update to BZip2 1.0.1 +

]]> +
+
+ +
+ + + + + + + + + + + + + + + + + true if lzo compressors are loaded & initialized, + else false]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if lzo decompressors are loaded & initialized, + else false]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @return the total (non-negative) number of uncompressed bytes input so far]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @return the total (non-negative) number of uncompressed bytes input so far]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if native-zlib is loaded & initialized + and can be loaded for this job, else false]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Keep trying a limited number of times, waiting a fixed time between attempts, + and then fail by re-throwing the exception. +

]]> +
+
+ + + + + + + Keep trying for a maximum time, waiting a fixed time between attempts, + and then fail by re-throwing the exception. +

]]> +
+
+ + + + + + + Keep trying a limited number of times, waiting a growing amount of time between attempts, + and then fail by re-throwing the exception. + The time between attempts is sleepTime mutliplied by the number of tries so far. +

]]> +
+
+ + + + + + + Keep trying a limited number of times, waiting a growing amount of time between attempts, + and then fail by re-throwing the exception. + The time between attempts is sleepTime mutliplied by a random + number in the range of [0, 2 to the number of retries) +

]]> +
+
+ + + + + + Set a default policy with some explicit handlers for specific exceptions. +

]]> +
+
+ + + + + + A retry policy for RemoteException + Set a default policy with some explicit handlers for specific exceptions. +

]]> +
+
+ + + + Try once, and fail by re-throwing the exception. + This corresponds to having no retry mechanism in place. +

]]> +
+
+ + + + Try once, and fail silently for void methods, or by + re-throwing the exception for non-void methods. +

]]> +
+
+ + + + Keep trying forever. +

]]> +
+
+ + + A collection of useful implementations of {@link RetryPolicy}. +

]]> +
+
+ + + + + + + + + + Determines whether the framework should retry a + method for the given exception, and the number + of retries that have been made for that operation + so far. +

+ @param e The exception that caused the method to fail. + @param retries The number of times the method has been retried. + @return true if the method should be retried, + false if the method should not be retried + but shouldn't fail with an exception (only for void methods). + @throws Exception The re-thrown exception e indicating + that the method failed and should not be retried further.]]> +
+
+ + + Specifies a policy for retrying method failures. + Implementations of this interface should be immutable. +

]]> +
+
+ + + + + + + + + + + + Create a proxy for an interface of an implementation class + using the same retry policy for each method in the interface. +

+ @param iface the interface that the retry will implement + @param implementation the instance whose methods should be retried + @param retryPolicy the policy for retirying method call failures + @return the retry proxy]]> +
+
+ + + + + + + Create a proxy for an interface of an implementation class + using the a set of retry policies specified by method name. + If no retry policy is defined for a method then a default of + {@link RetryPolicies#TRY_ONCE_THEN_FAIL} is used. +

+ @param iface the interface that the retry will implement + @param implementation the instance whose methods should be retried + @param methodNameToPolicyMap a map of method names to retry policies + @return the retry proxy]]> +
+
+ + + A factory for creating retry proxies. +

]]> +
+
+ +
+ + + + + + + + Prepare the deserializer for reading.

]]> +
+
+ + + + + + Deserialize the next object from the underlying input stream. + If the object t is non-null then this deserializer + may set its internal state to the next object read from the input + stream. Otherwise, if the object t is null a new + deserialized object will be created. +

+ @return the deserialized object]]> +
+
+ + + + Close the underlying input stream and clear up any resources.

]]> +
+
+ + + Provides a facility for deserializing objects of type from an + {@link InputStream}. +

+ +

+ Deserializers are stateful, but must not buffer the input since + other producers may read from the input between calls to + {@link #deserialize(Object)}. +

+ @param ]]> +
+
+ + + + + + + + + + + + + + + + + + A {@link RawComparator} that uses a {@link Deserializer} to deserialize + the objects to be compared so that the standard {@link Comparator} can + be used to compare them. +

+

+ One may optimize compare-intensive operations by using a custom + implementation of {@link RawComparator} that operates directly + on byte representations. +

+ @param ]]> +
+
+ + + + + + + + + + + + + + + + + + An experimental {@link Serialization} for Java {@link Serializable} classes. +

+ @see JavaSerializationComparator]]> +
+
+ + + + + + + + + + + + + A {@link RawComparator} that uses a {@link JavaSerialization} + {@link Deserializer} to deserialize objects that are then compared via + their {@link Comparable} interfaces. +

+ @param + @see JavaSerialization]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + Encapsulates a {@link Serializer}/{@link Deserializer} pair. +

+ @param ]]> +
+
+ + + + + + + Serializations are found by reading the io.serializations + property from conf, which is a comma-delimited list of + classnames. +

]]> +
+
+ + + + + + + + + + + + A factory for {@link Serialization}s. +

]]> +
+
+ + + + + + + + Prepare the serializer for writing.

]]> +
+
+ + + + + Serialize t to the underlying output stream.

]]> +
+
+ + + + Close the underlying output stream and clear up any resources.

]]> +
+
+ + + Provides a facility for serializing objects of type to an + {@link OutputStream}. +

+ +

+ Serializers are stateful, but must not buffer the output since + other producers may write to the output between calls to + {@link #serialize(Object)}. +

+ @param ]]> +
+
+ + + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + param, to the IPC server running at + address, returning the value. Throws exceptions if there are + network problems or if the remote code threw an exception.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Unwraps any IOException. + + @param lookupTypes the desired exception class. + @return IOException, which is either the lookupClass exception or this.]]> + + + + + This unwraps any Throwable that has a constructor taking + a String as a parameter. + Otherwise it returns this. + + @return Throwable]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + protocol is a Java interface. All parameters and return types must + be one of: + +
  • a primitive type, boolean, byte, + char, short, int, long, + float, double, or void; or
  • + +
  • a {@link String}; or
  • + +
  • a {@link Writable}; or
  • + +
  • an array of the above types
+ + All methods in the protocol should throw only IOException. No field data of + the protocol instance is transmitted.]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + handlerCount determines + the number of handler threads that will be used to process calls.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + This class has a number of metrics variables that are publicly accessible; + these variables (objects) have methods to update their values; + for example: +

{@link #rpcQueueTime}.inc(time)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + For the statistics that are sampled and averaged, one must specify + a metrics context that does periodic update calls. Most do. + The default Null metrics context however does NOT. So if you aren't + using any other metrics context then you can turn on the viewing and averaging + of sampled metrics by specifying the following two lines + in the hadoop-meterics.properties file: +

+        rpc.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+        rpc.period=10
+  
+

+ Note that the metrics are collected regardless of the context used. + The context with the update thread is used to average the data periodically]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobTracker, + as {@link JobTracker.State} + + @return the current state of the JobTracker.]]> + + + + + + + + + + + + ClusterStatus provides clients with information such as: +

    +
  1. + Size of the cluster. +
  2. +
  3. + Task capacity of the cluster. +
  4. +
  5. + The number of currently running map & reduce tasks. +
  6. +
  7. + State of the JobTracker. +
  8. +

+ +

Clients can query for the latest ClusterStatus, via + {@link JobClient#getClusterStatus()}.

+ + @see JobClient]]> + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Counters represent global counters, defined either by the + Map-Reduce framework or applications. Each Counter can be of + any {@link Enum} type.

+ +

Counters are bunched into {@link Group}s, each comprising of + counters from a particular Enum class.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Group of counters, comprising of counters from a particular + counter {@link Enum} class. + +

Grouphandles localization of the class name and the + counter names.

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FileInputFormat implementations can override this and return + false to ensure that individual input files are never split-up + so that {@link Mapper}s process entire files. + + @param fs the file system that the file is on + @param filename the file name to check + @return is this file splitable?]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FileInputFormat is the base class for all file-based + InputFormats. This provides a generic implementation of + {@link #getSplits(JobConf, int)}. + Subclasses of FileInputFormat can also override the + {@link #isSplitable(FileSystem, Path)} method to ensure input-files are + not split-up and are processed as a whole by {@link Mapper}s.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the job output should be compressed, + false otherwise]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tasks' Side-Effect Files + +

Note: The following is valid only if the {@link OutputCommitter} + is {@link FileOutputCommitter}. If OutputCommitter is not + a FileOutputCommitter, the task's temporary output + directory is same as {@link #getOutputPath(JobConf)} i.e. + ${mapred.output.dir}$

+ +

Some applications need to create/write-to side-files, which differ from + the actual job-outputs. + +

In such cases there could be issues with 2 instances of the same TIP + (running simultaneously e.g. speculative tasks) trying to open/write-to the + same file (path) on HDFS. Hence the application-writer will have to pick + unique names per task-attempt (e.g. using the attemptid, say + attempt_200709221812_0001_m_000000_0), not just per TIP.

+ +

To get around this the Map-Reduce framework helps the application-writer + out by maintaining a special + ${mapred.output.dir}/_temporary/_${taskid} + sub-directory for each task-attempt on HDFS where the output of the + task-attempt goes. On successful completion of the task-attempt the files + in the ${mapred.output.dir}/_temporary/_${taskid} (only) + are promoted to ${mapred.output.dir}. Of course, the + framework discards the sub-directory of unsuccessful task-attempts. This + is completely transparent to the application.

+ +

The application-writer can take advantage of this by creating any + side-files required in ${mapred.work.output.dir} during execution + of his reduce-task i.e. via {@link #getWorkOutputPath(JobConf)}, and the + framework will move them out similarly - thus she doesn't have to pick + unique paths per task-attempt.

+ +

Note: the value of ${mapred.work.output.dir} during + execution of a particular task-attempt is actually + ${mapred.output.dir}/_temporary/_{$taskid}, and this value is + set by the map-reduce framework. So, just create any side-files in the + path returned by {@link #getWorkOutputPath(JobConf)} from map/reduce + task to take advantage of this feature.

+ +

The entire discussion holds true for maps of jobs with + reducer=NONE (i.e. 0 reduces) since output of the map, in that case, + goes directly to HDFS.

+ + @return the {@link Path} to the task's temporary output directory + for the map-reduce job.]]> +
+
+ + + + + + + + + + + + + The generated name can be used to create custom files from within the + different tasks for the job, the names for different tasks will not collide + with each other.

+ +

The given name is postfixed with the task type, 'm' for maps, 'r' for + reduces and the task partition number. For example, give a name 'test' + running on the first map o the job the generated name will be + 'test-m-00000'.

+ + @param conf the configuration for the job. + @param name the name to make unique. + @return a unique name accross all tasks of the job.]]> +
+
+ + + + + The path can be used to create custom files from within the map and + reduce tasks. The path name will be unique for each task. The path parent + will be the job output directory.

ls + +

This method uses the {@link #getUniqueName} method to make the file name + unique for the task.

+ + @param conf the configuration for the job. + @param name the name for the file. + @return a unique path accross all tasks of the job.]]> +
+
+ + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Each {@link InputSplit} is then assigned to an individual {@link Mapper} + for processing.

+ +

Note: The split is a logical split of the inputs and the + input files are not physically split into chunks. For e.g. a split could + be <input-file-path, start, offset> tuple. + + @param job job configuration. + @param numSplits the desired number of splits, a hint. + @return an array of {@link InputSplit}s for the job.]]> + + + + + + + + + It is the responsibility of the RecordReader to respect + record boundaries while processing the logical split to present a + record-oriented view to the individual task.

+ + @param split the {@link InputSplit} + @param job the job that this split belongs to + @return a {@link RecordReader}]]> +
+
+ + InputFormat describes the input-specification for a + Map-Reduce job. + +

The Map-Reduce framework relies on the InputFormat of the + job to:

+

    +
  1. + Validate the input-specification of the job. +
  2. + Split-up the input file(s) into logical {@link InputSplit}s, each of + which is then assigned to an individual {@link Mapper}. +
  3. +
  4. + Provide the {@link RecordReader} implementation to be used to glean + input records from the logical InputSplit for processing by + the {@link Mapper}. +
  5. +
+ +

The default behavior of file-based {@link InputFormat}s, typically + sub-classes of {@link FileInputFormat}, is to split the + input into logical {@link InputSplit}s based on the total size, in + bytes, of the input files. However, the {@link FileSystem} blocksize of + the input files is treated as an upper bound for input splits. A lower bound + on the split size can be set via + + mapred.min.split.size.

+ +

Clearly, logical splits based on input-size is insufficient for many + applications since record boundaries are to respected. In such cases, the + application has to also implement a {@link RecordReader} on whom lies the + responsibilty to respect record-boundaries and present a record-oriented + view of the logical InputSplit to the individual task. + + @see InputSplit + @see RecordReader + @see JobClient + @see FileInputFormat]]> + + + + + + + + + + InputSplit. + + @return the number of bytes in the input split. + @throws IOException]]> + + + + + + InputSplit is + located as an array of Strings. + @throws IOException]]> + + + + InputSplit represents the data to be processed by an + individual {@link Mapper}. + +

Typically, it presents a byte-oriented view on the input and is the + responsibility of {@link RecordReader} of the job to process this and present + a record-oriented view. + + @see InputFormat + @see RecordReader]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobClient.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + jobid doesn't correspond to any known job. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobClient is the primary interface for the user-job to interact + with the {@link JobTracker}. + + JobClient provides facilities to submit jobs, track their + progress, access component-tasks' reports/logs, get the Map-Reduce cluster + status information etc. + +

The job submission process involves: +

    +
  1. + Checking the input and output specifications of the job. +
  2. +
  3. + Computing the {@link InputSplit}s for the job. +
  4. +
  5. + Setup the requisite accounting information for the {@link DistributedCache} + of the job, if necessary. +
  6. +
  7. + Copying the job's jar and configuration to the map-reduce system directory + on the distributed file-system. +
  8. +
  9. + Submitting the job to the JobTracker and optionally monitoring + it's status. +
  10. +

+ + Normally the user creates the application, describes various facets of the + job via {@link JobConf} and then uses the JobClient to submit + the job and monitor its progress. + +

Here is an example on how to use JobClient:

+

+     // Create a new JobConf
+     JobConf job = new JobConf(new Configuration(), MyJob.class);
+     
+     // Specify various job-specific parameters     
+     job.setJobName("myjob");
+     
+     job.setInputPath(new Path("in"));
+     job.setOutputPath(new Path("out"));
+     
+     job.setMapperClass(MyJob.MyMapper.class);
+     job.setReducerClass(MyJob.MyReducer.class);
+
+     // Submit the job, then poll for progress until the job is complete
+     JobClient.runJob(job);
+ 

+ +

Job Control

+ +

At times clients would chain map-reduce jobs to accomplish complex tasks + which cannot be done via a single map-reduce job. This is fairly easy since + the output of the job, typically, goes to distributed file-system and that + can be used as the input for the next job.

+ +

However, this also means that the onus on ensuring jobs are complete + (success/failure) lies squarely on the clients. In such situations the + various job-control options are: +

    +
  1. + {@link #runJob(JobConf)} : submits the job and returns only after + the job has completed. +
  2. +
  3. + {@link #submitJob(JobConf)} : only submits the job, then poll the + returned handle to the {@link RunningJob} to query status and make + scheduling decisions. +
  4. +
  5. + {@link JobConf#setJobEndNotificationURI(String)} : setup a notification + on job-completion, thus avoiding polling. +
  6. +

+ + @see JobConf + @see ClusterStatus + @see Tool + @see DistributedCache]]> +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + If the parameter {@code loadDefaults} is false, the new instance + will not load resources from the default files. + + @param loadDefaults specifies whether to load from the default files]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if framework should keep the intermediate files + for failed tasks, false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the outputs of the maps are to be compressed, + false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This comparator should be provided if the equivalence rules for keys + for sorting the intermediates are different from those for grouping keys + before each call to + {@link Reducer#reduce(Object, java.util.Iterator, OutputCollector, Reporter)}.

+ +

For key-value pairs (K1,V1) and (K2,V2), the values (V1, V2) are passed + in a single call to the reduce function if K1 and K2 compare as equal.

+ +

Since {@link #setOutputKeyComparatorClass(Class)} can be used to control + how keys are sorted, this can be used in conjunction to simulate + secondary sort on values.

+ +

Note: This is not a guarantee of the reduce sort being + stable in any sense. (In any case, with the order of available + map-outputs to the reduce being non-deterministic, it wouldn't make + that much sense.)

+ + @param theClass the comparator class to be used for grouping keys. + It should implement RawComparator. + @see #setOutputKeyComparatorClass(Class)]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + combiner class used to combine map-outputs + before being sent to the reducers. Typically the combiner is same as the + the {@link Reducer} for the job i.e. {@link #getReducerClass()}. + + @return the user-defined combiner class used to combine map-outputs.]]> + + + + + + combiner class used to combine map-outputs + before being sent to the reducers. + +

The combiner is a task-level aggregation operation which, in some cases, + helps to cut down the amount of data transferred from the {@link Mapper} to + the {@link Reducer}, leading to better performance.

+ +

Typically the combiner is same as the the Reducer for the + job i.e. {@link #setReducerClass(Class)}.

+ + @param theClass the user-defined combiner class used to combine + map-outputs.]]> +
+
+ + + true. + + @return true if speculative execution be used for this job, + false otherwise.]]> + + + + + + true if speculative execution + should be turned on, else false.]]> + + + + + true. + + @return true if speculative execution be + used for this job for map tasks, + false otherwise.]]> + + + + + + true if speculative execution + should be turned on for map tasks, + else false.]]> + + + + + true. + + @return true if speculative execution be used + for reduce tasks for this job, + false otherwise.]]> + + + + + + true if speculative execution + should be turned on for reduce tasks, + else false.]]> + + + + + 1. + + @return the number of reduce tasks for this job.]]> + + + + + + Note: This is only a hint to the framework. The actual + number of spawned map tasks depends on the number of {@link InputSplit}s + generated by the job's {@link InputFormat#getSplits(JobConf, int)}. + + A custom {@link InputFormat} is typically used to accurately control + the number of map tasks for the job.

+ +

How many maps?

+ +

The number of maps is usually driven by the total size of the inputs + i.e. total number of blocks of the input files.

+ +

The right level of parallelism for maps seems to be around 10-100 maps + per-node, although it has been set up to 300 or so for very cpu-light map + tasks. Task setup takes awhile, so it is best if the maps take at least a + minute to execute.

+ +

The default behavior of file-based {@link InputFormat}s is to split the + input into logical {@link InputSplit}s based on the total size, in + bytes, of input files. However, the {@link FileSystem} blocksize of the + input files is treated as an upper bound for input splits. A lower bound + on the split size can be set via + + mapred.min.split.size.

+ +

Thus, if you expect 10TB of input data and have a blocksize of 128MB, + you'll end up with 82,000 maps, unless {@link #setNumMapTasks(int)} is + used to set it even higher.

+ + @param n the number of map tasks for this job. + @see InputFormat#getSplits(JobConf, int) + @see FileInputFormat + @see FileSystem#getDefaultBlockSize() + @see FileStatus#getBlockSize()]]> +
+
+ + + 1. + + @return the number of reduce tasks for this job.]]> + + + + + + How many reduces? + +

The right number of reduces seems to be 0.95 or + 1.75 multiplied by (<no. of nodes> * + + mapred.tasktracker.reduce.tasks.maximum). +

+ +

With 0.95 all of the reduces can launch immediately and + start transfering map outputs as the maps finish. With 1.75 + the faster nodes will finish their first round of reduces and launch a + second wave of reduces doing a much better job of load balancing.

+ +

Increasing the number of reduces increases the framework overhead, but + increases load balancing and lowers the cost of failures.

+ +

The scaling factors above are slightly less than whole numbers to + reserve a few reduce slots in the framework for speculative-tasks, failures + etc.

+ +

Reducer NONE

+ +

It is legal to set the number of reduce-tasks to zero.

+ +

In this case the output of the map-tasks directly go to distributed + file-system, to the path set by + {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Also, the + framework doesn't sort the map-outputs before writing it out to HDFS.

+ + @param n the number of reduce tasks for this job.]]> +
+
+ + + mapred.map.max.attempts + property. If this property is not already set, the default is 4 attempts. + + @return the max number of attempts per map task.]]> + + + + + + + + + + + mapred.reduce.max.attempts + property. If this property is not already set, the default is 4 attempts. + + @return the max number of attempts per reduce task.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + noFailures, the + tasktracker is blacklisted for this job. + + @param noFailures maximum no. of failures of a given job per tasktracker.]]> + + + + + blacklisted for this job. + + @return the maximum no. of failures of a given job per tasktracker.]]> + + + + + failed. + + Defaults to zero, i.e. any failed map-task results in + the job being declared as {@link JobStatus#FAILED}. + + @return the maximum percentage of map tasks that can fail without + the job being aborted.]]> + + + + + + failed. + + @param percent the maximum percentage of map tasks that can fail without + the job being aborted.]]> + + + + + failed. + + Defaults to zero, i.e. any failed reduce-task results + in the job being declared as {@link JobStatus#FAILED}. + + @return the maximum percentage of reduce tasks that can fail without + the job being aborted.]]> + + + + + + failed. + + @param percent the maximum percentage of reduce tasks that can fail without + the job being aborted.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The debug script can aid debugging of failed map tasks. The script is + given task's stdout, stderr, syslog, jobconf files as arguments.

+ +

The debug command, run on the node where the map failed, is:

+

+ $script $stdout $stderr $syslog $jobconf. +

+ +

The script file is distributed through {@link DistributedCache} + APIs. The script needs to be symlinked.

+ +

Here is an example on how to submit a script +

+ job.setMapDebugScript("./myscript");
+ DistributedCache.createSymlink(job);
+ DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
+ 

+ + @param mDbgScript the script name]]> +
+
+ + + + + + + + + The debug script can aid debugging of failed reduce tasks. The script + is given task's stdout, stderr, syslog, jobconf files as arguments.

+ +

The debug command, run on the node where the map failed, is:

+

+ $script $stdout $stderr $syslog $jobconf. +

+ +

The script file is distributed through {@link DistributedCache} + APIs. The script file needs to be symlinked

+ +

Here is an example on how to submit a script +

+ job.setReduceDebugScript("./myscript");
+ DistributedCache.createSymlink(job);
+ DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
+ 

+ + @param rDbgScript the script name]]> +
+
+ + + + + + + + null if it hasn't + been set. + @see #setJobEndNotificationURI(String)]]> + + + + + + The uri can contain 2 special parameters: $jobId and + $jobStatus. Those, if present, are replaced by the job's + identifier and completion-status respectively.

+ +

This is typically used by application-writers to implement chaining of + Map-Reduce jobs in an asynchronous manner.

+ + @param uri the job end notification uri + @see JobStatus + @see Job Completion and Chaining]]> +
+
+ + + + When a job starts, a shared directory is created at location + + ${mapred.local.dir}/taskTracker/jobcache/$jobid/work/ . + This directory is exposed to the users through + job.local.dir . + So, the tasks can use this space + as scratch space and share files among them.

+ This value is available as System property also. + + @return The localized job specific shared directory]]> +
+
+ + + + + + + + + + + + + + + + + + JobConf is the primary interface for a user to describe a + map-reduce job to the Hadoop framework for execution. The framework tries to + faithfully execute the job as-is described by JobConf, however: +
    +
  1. + Some configuration parameters might have been marked as + + final by administrators and hence cannot be altered. +
  2. +
  3. + While some job parameters are straight-forward to set + (e.g. {@link #setNumReduceTasks(int)}), some parameters interact subtly + rest of the framework and/or job-configuration and is relatively more + complex for the user to control finely (e.g. {@link #setNumMapTasks(int)}). +
  4. +

+ +

JobConf typically specifies the {@link Mapper}, combiner + (if any), {@link Partitioner}, {@link Reducer}, {@link InputFormat} and + {@link OutputFormat} implementations to be used etc. + +

Optionally JobConf is used to specify other advanced facets + of the job such as Comparators to be used, files to be put in + the {@link DistributedCache}, whether or not intermediate and/or job outputs + are to be compressed (and how), debugability via user-provided scripts + ( {@link #setMapDebugScript(String)}/{@link #setReduceDebugScript(String)}), + for doing post-processing on task logs, task's stdout, stderr, syslog. + and etc.

+ +

Here is an example on how to configure a job via JobConf:

+

+     // Create a new JobConf
+     JobConf job = new JobConf(new Configuration(), MyJob.class);
+     
+     // Specify various job-specific parameters     
+     job.setJobName("myjob");
+     
+     FileInputFormat.setInputPaths(job, new Path("in"));
+     FileOutputFormat.setOutputPath(job, new Path("out"));
+     
+     job.setMapperClass(MyJob.MyMapper.class);
+     job.setCombinerClass(MyJob.MyReducer.class);
+     job.setReducerClass(MyJob.MyReducer.class);
+     
+     job.setInputFormat(SequenceFileInputFormat.class);
+     job.setOutputFormat(SequenceFileOutputFormat.class);
+ 

+ + @see JobClient + @see ClusterStatus + @see Tool + @see DistributedCache]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + .]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + any job + run on the jobtracker started at 200707121733, we would use : +
 
+ JobID.getTaskIDsPattern("200707121733", null);
+ 
+ which will return : +
 "job_200707121733_[0-9]*" 
+ @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @return a regex pattern matching JobIDs]]> +
+
+ + + An example JobID is : + job_200707121733_0003 , which represents the third job + running at the jobtracker started at 200707121733. +

+ Applications should never construct or parse JobID strings, but rather + use appropriate constructors or {@link #forName(String)} method. + + @see TaskID + @see TaskAttemptID + @see JobTracker#getNewJobId() + @see JobTracker#getStartTime()]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + "N/A" + + @return Scheduling information associated to particular Job Queue]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + -archives + -files inputjar args]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + zero. + + @param conf configuration for the JobTracker. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Output pairs need not be of the same types as input pairs. A given + input pair may map to zero or many output pairs. Output pairs are + collected with calls to + {@link OutputCollector#collect(Object,Object)}.

+ +

Applications can use the {@link Reporter} provided to report progress + or just indicate that they are alive. In scenarios where the application + takes an insignificant amount of time to process individual key/value + pairs, this is crucial since the framework might assume that the task has + timed-out and kill that task. The other way of avoiding this is to set + + mapred.task.timeout to a high-enough value (or even zero for no + time-outs).

+ + @param key the input key. + @param value the input value. + @param output collects mapped keys and values. + @param reporter facility to report progress.]]> +
+ + + Maps are the individual tasks which transform input records into a + intermediate records. The transformed intermediate records need not be of + the same type as the input records. A given input pair may map to zero or + many output pairs.

+ +

The Hadoop Map-Reduce framework spawns one map task for each + {@link InputSplit} generated by the {@link InputFormat} for the job. + Mapper implementations can access the {@link JobConf} for the + job via the {@link JobConfigurable#configure(JobConf)} and initialize + themselves. Similarly they can use the {@link Closeable#close()} method for + de-initialization.

+ +

The framework then calls + {@link #map(Object, Object, OutputCollector, Reporter)} + for each key/value pair in the InputSplit for that task.

+ +

All intermediate values associated with a given output key are + subsequently grouped by the framework, and passed to a {@link Reducer} to + determine the final output. Users can control the grouping by specifying + a Comparator via + {@link JobConf#setOutputKeyComparatorClass(Class)}.

+ +

The grouped Mapper outputs are partitioned per + Reducer. Users can control which keys (and hence records) go to + which Reducer by implementing a custom {@link Partitioner}. + +

Users can optionally specify a combiner, via + {@link JobConf#setCombinerClass(Class)}, to perform local aggregation of the + intermediate outputs, which helps to cut down the amount of data transferred + from the Mapper to the Reducer. + +

The intermediate, grouped outputs are always stored in + {@link SequenceFile}s. Applications can specify if and how the intermediate + outputs are to be compressed and which {@link CompressionCodec}s are to be + used via the JobConf.

+ +

If the job has + zero + reduces then the output of the Mapper is directly written + to the {@link FileSystem} without grouping by keys.

+ +

Example:

+

+     public class MyMapper<K extends WritableComparable, V extends Writable> 
+     extends MapReduceBase implements Mapper<K, V, K, V> {
+     
+       static enum MyCounters { NUM_RECORDS }
+       
+       private String mapTaskId;
+       private String inputFile;
+       private int noRecords = 0;
+       
+       public void configure(JobConf job) {
+         mapTaskId = job.get("mapred.task.id");
+         inputFile = job.get("mapred.input.file");
+       }
+       
+       public void map(K key, V val,
+                       OutputCollector<K, V> output, Reporter reporter)
+       throws IOException {
+         // Process the <key, value> pair (assume this takes a while)
+         // ...
+         // ...
+         
+         // Let the framework know that we are alive, and kicking!
+         // reporter.progress();
+         
+         // Process some more
+         // ...
+         // ...
+         
+         // Increment the no. of <key, value> pairs processed
+         ++noRecords;
+
+         // Increment counters
+         reporter.incrCounter(NUM_RECORDS, 1);
+        
+         // Every 100 records update application-level status
+         if ((noRecords%100) == 0) {
+           reporter.setStatus(mapTaskId + " processed " + noRecords + 
+                              " from input-file: " + inputFile); 
+         }
+         
+         // Output the result
+         output.collect(key, val);
+       }
+     }
+ 

+ +

Applications may write a custom {@link MapRunnable} to exert greater + control on map processing e.g. multi-threaded Mappers etc.

+ + @see JobConf + @see InputFormat + @see Partitioner + @see Reducer + @see MapReduceBase + @see MapRunnable + @see SequenceFile]]> +
+
+ + + + + + + + + + + + + + + + + + + + + Provides default no-op implementations for a few methods, most non-trivial + applications need to override some of them.

]]> +
+
+ + + + + + + + + + + <key, value> pairs. + +

Mapping of input records to output records is complete when this method + returns.

+ + @param input the {@link RecordReader} to read the input records. + @param output the {@link OutputCollector} to collect the outputrecords. + @param reporter {@link Reporter} to report progress, status-updates etc. + @throws IOException]]> +
+
+ + Custom implementations of MapRunnable can exert greater + control on map processing e.g. multi-threaded, asynchronous mappers etc.

+ + @see Mapper]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + nearly + equal content length.
+ Subclasses implement {@link #getRecordReader(InputSplit, JobConf, Reporter)} + to construct RecordReader's for MultiFileSplit's. + @see MultiFileSplit]]> +
+
+ + + + + + + + + + + + + + + + + th Path]]> + + + + + + + + + + + th Path]]> + + + + + + + + + + + + + + + + + + + + + + + MultiFileSplit can be used to implement {@link RecordReader}'s, with + reading one record per file. + @see FileSplit + @see MultiFileInputFormat]]> + + + + + + + + + + + + + + + <key, value> pairs output by {@link Mapper}s + and {@link Reducer}s. + +

OutputCollector is the generalization of the facility + provided by the Map-Reduce framework to collect data output by either the + Mapper or the Reducer i.e. intermediate outputs + or the output of the job.

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + OutputCommitter describes the commit of task output for a + Map-Reduce job. + +

The Map-Reduce framework relies on the OutputCommitter of + the job to:

+

    +
  1. + Setup the job during initialization. For example, create the temporary + output directory for the job during the initialization of the job. +
  2. +
  3. + Cleanup the job after the job completion. For example, remove the + temporary output directory after the job completion. +
  4. +
  5. + Setup the task temporary output. +
  6. +
  7. + Check whether a task needs a commit. This is to avoid the commit + procedure if a task does not need commit. +
  8. +
  9. + Commit of the task output. +
  10. +
  11. + Discard the task commit. +
  12. +
+ + @see FileOutputCommitter + @see JobContext + @see TaskAttemptContext]]> +
+
+ + + + + + + + + + + + + + + + + + + This is to validate the output specification for the job when it is + a job is submitted. Typically checks that it does not already exist, + throwing an exception when it already exists, so that output is not + overwritten.

+ + @param ignored + @param job job configuration. + @throws IOException when output should not be attempted]]> +
+
+ + OutputFormat describes the output-specification for a + Map-Reduce job. + +

The Map-Reduce framework relies on the OutputFormat of the + job to:

+

    +
  1. + Validate the output-specification of the job. For e.g. check that the + output directory doesn't already exist. +
  2. + Provide the {@link RecordWriter} implementation to be used to write out + the output files of the job. Output files are stored in a + {@link FileSystem}. +
  3. +
+ + @see RecordWriter + @see JobConf]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + Typically a hash function on a all or a subset of the key.

+ + @param key the key to be paritioned. + @param value the entry value. + @param numPartitions the total number of partitions. + @return the partition number for the key.]]> +
+
+ + Partitioner controls the partitioning of the keys of the + intermediate map-outputs. The key (or a subset of the key) is used to derive + the partition, typically by a hash function. The total number of partitions + is the same as the number of reduce tasks for the job. Hence this controls + which of the m reduce tasks the intermediate key (and hence the + record) is sent for reduction.

+ + @see Reducer]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 0.0 to 1.0. + @throws IOException]]> + + + + RecordReader reads <key, value> pairs from an + {@link InputSplit}. + +

RecordReader, typically, converts the byte-oriented view of + the input, provided by the InputSplit, and presents a + record-oriented view for the {@link Mapper} & {@link Reducer} tasks for + processing. It thus assumes the responsibility of processing record + boundaries and presenting the tasks with keys and values.

+ + @see InputSplit + @see InputFormat]]> +
+
+ + + + + + + + + + + + + + + + RecordWriter to future operations. + + @param reporter facility to report progress. + @throws IOException]]> + + + + RecordWriter writes the output <key, value> pairs + to an output file. + +

RecordWriter implementations write the job outputs to the + {@link FileSystem}. + + @see OutputFormat]]> + + + + + + + + + + + + + + + Reduces values for a given key. + +

The framework calls this method for each + <key, (list of values)> pair in the grouped inputs. + Output values must be of the same type as input values. Input keys must + not be altered. The framework will reuse the key and value objects + that are passed into the reduce, therefore the application should clone + the objects they want to keep a copy of. In many cases, all values are + combined into zero or one value. +

+ +

Output pairs are collected with calls to + {@link OutputCollector#collect(Object,Object)}.

+ +

Applications can use the {@link Reporter} provided to report progress + or just indicate that they are alive. In scenarios where the application + takes an insignificant amount of time to process individual key/value + pairs, this is crucial since the framework might assume that the task has + timed-out and kill that task. The other way of avoiding this is to set + + mapred.task.timeout to a high-enough value (or even zero for no + time-outs).

+ + @param key the key. + @param values the list of values to reduce. + @param output to collect keys and combined values. + @param reporter facility to report progress.]]> +
+ + + The number of Reducers for the job is set by the user via + {@link JobConf#setNumReduceTasks(int)}. Reducer implementations + can access the {@link JobConf} for the job via the + {@link JobConfigurable#configure(JobConf)} method and initialize themselves. + Similarly they can use the {@link Closeable#close()} method for + de-initialization.

+ +

Reducer has 3 primary phases:

+
    +
  1. + +

    Shuffle

    + +

    Reducer is input the grouped output of a {@link Mapper}. + In the phase the framework, for each Reducer, fetches the + relevant partition of the output of all the Mappers, via HTTP. +

    +
  2. + +
  3. +

    Sort

    + +

    The framework groups Reducer inputs by keys + (since different Mappers may have output the same key) in this + stage.

    + +

    The shuffle and sort phases occur simultaneously i.e. while outputs are + being fetched they are merged.

    + +
    SecondarySort
    + +

    If equivalence rules for keys while grouping the intermediates are + different from those for grouping keys before reduction, then one may + specify a Comparator via + {@link JobConf#setOutputValueGroupingComparator(Class)}.Since + {@link JobConf#setOutputKeyComparatorClass(Class)} can be used to + control how intermediate keys are grouped, these can be used in conjunction + to simulate secondary sort on values.

    + + + For example, say that you want to find duplicate web pages and tag them + all with the url of the "best" known example. You would set up the job + like: +
      +
    • Map Input Key: url
    • +
    • Map Input Value: document
    • +
    • Map Output Key: document checksum, url pagerank
    • +
    • Map Output Value: url
    • +
    • Partitioner: by checksum
    • +
    • OutputKeyComparator: by checksum and then decreasing pagerank
    • +
    • OutputValueGroupingComparator: by checksum
    • +
    +
  4. + +
  5. +

    Reduce

    + +

    In this phase the + {@link #reduce(Object, Iterator, OutputCollector, Reporter)} + method is called for each <key, (list of values)> pair in + the grouped inputs.

    +

    The output of the reduce task is typically written to the + {@link FileSystem} via + {@link OutputCollector#collect(Object, Object)}.

    +
  6. +
+ +

The output of the Reducer is not re-sorted.

+ +

Example:

+

+     public class MyReducer<K extends WritableComparable, V extends Writable> 
+     extends MapReduceBase implements Reducer<K, V, K, V> {
+     
+       static enum MyCounters { NUM_RECORDS }
+        
+       private String reduceTaskId;
+       private int noKeys = 0;
+       
+       public void configure(JobConf job) {
+         reduceTaskId = job.get("mapred.task.id");
+       }
+       
+       public void reduce(K key, Iterator<V> values,
+                          OutputCollector<K, V> output, 
+                          Reporter reporter)
+       throws IOException {
+       
+         // Process
+         int noValues = 0;
+         while (values.hasNext()) {
+           V value = values.next();
+           
+           // Increment the no. of values for this key
+           ++noValues;
+           
+           // Process the <key, value> pair (assume this takes a while)
+           // ...
+           // ...
+           
+           // Let the framework know that we are alive, and kicking!
+           if ((noValues%10) == 0) {
+             reporter.progress();
+           }
+         
+           // Process some more
+           // ...
+           // ...
+           
+           // Output the <key, value> 
+           output.collect(key, value);
+         }
+         
+         // Increment the no. of <key, list of values> pairs processed
+         ++noKeys;
+         
+         // Increment counters
+         reporter.incrCounter(NUM_RECORDS, 1);
+         
+         // Every 100 keys update application-level status
+         if ((noKeys%100) == 0) {
+           reporter.setStatus(reduceTaskId + " processed " + noKeys);
+         }
+       }
+     }
+ 

+ + @see Mapper + @see Partitioner + @see Reporter + @see MapReduceBase]]> +
+
+ + + + + + + + + + + + + + + Counter of the given group/name.]]> + + + + + + + Enum. + @param amount A non-negative amount by which the counter is to + be incremented.]]> + + + + + + + + + + + + + + InputSplit that the map is reading from. + @throws UnsupportedOperationException if called outside a mapper]]> + + + + + + + + + {@link Mapper} and {@link Reducer} can use the Reporter + provided to report progress or just indicate that they are alive. In + scenarios where the application takes an insignificant amount of time to + process individual key/value pairs, this is crucial since the framework + might assume that the task has timed-out and kill that task. + +

Applications can also update {@link Counters} via the provided + Reporter .

+ + @see Progressable + @see Counters]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + progress of the job's map-tasks, as a float between 0.0 + and 1.0. When all map tasks have completed, the function returns 1.0. + + @return the progress of the job's map-tasks. + @throws IOException]]> + + + + + + progress of the job's reduce-tasks, as a float between 0.0 + and 1.0. When all reduce tasks have completed, the function returns 1.0. + + @return the progress of the job's reduce-tasks. + @throws IOException]]> + + + + + + progress of the job's cleanup-tasks, as a float between 0.0 + and 1.0. When all cleanup tasks have completed, the function returns 1.0. + + @return the progress of the job's cleanup-tasks. + @throws IOException]]> + + + + + + progress of the job's setup-tasks, as a float between 0.0 + and 1.0. When all setup tasks have completed, the function returns 1.0. + + @return the progress of the job's setup-tasks. + @throws IOException]]> + + + + + + true if the job is complete, else false. + @throws IOException]]> + + + + + + true if the job succeeded, else false. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + RunningJob is the user-interface to query for details on a + running Map-Reduce job. + +

Clients can get hold of RunningJob via the {@link JobClient} + and then query the running-job for details such as name, configuration, + progress etc.

+ + @see JobClient]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This allows the user to specify the key class to be different + from the actual class ({@link BytesWritable}) used for writing

+ + @param conf the {@link JobConf} to modify + @param theClass the SequenceFile output key class.]]> +
+
+ + + + + This allows the user to specify the value class to be different + from the actual class ({@link BytesWritable}) used for writing

+ + @param conf the {@link JobConf} to modify + @param theClass the SequenceFile output key class.]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + f. The filtering criteria is + MD5(key) % f == 0.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + f using + the criteria record# % f == 0. + For example, if the frequency is 10, one out of 10 records is returned.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if auto increment + {@link SkipBadRecords#COUNTER_MAP_PROCESSED_RECORDS}. + false otherwise.]]> + + + + + + + + + + + + + true if auto increment + {@link SkipBadRecords#COUNTER_REDUCE_PROCESSED_GROUPS}. + false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hadoop provides an optional mode of execution in which the bad records + are detected and skipped in further attempts. + +

This feature can be used when map/reduce tasks crashes deterministically on + certain input. This happens due to bugs in the map/reduce function. The usual + course would be to fix these bugs. But sometimes this is not possible; + perhaps the bug is in third party libraries for which the source code is + not available. Due to this, the task never reaches to completion even with + multiple attempts and complete data for that task is lost.

+ +

With this feature, only a small portion of data is lost surrounding + the bad record, which may be acceptable for some user applications. + see {@link SkipBadRecords#setMapperMaxSkipRecords(Configuration, long)}

+ +

The skipping mode gets kicked off after certain no of failures + see {@link SkipBadRecords#setAttemptsToStartSkipping(Configuration, int)}

+ +

In the skipping mode, the map/reduce task maintains the record range which + is getting processed at all times. Before giving the input to the + map/reduce function, it sends this record range to the Task tracker. + If task crashes, the Task tracker knows which one was the last reported + range. On further attempts that range get skipped.

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + all task attempt IDs + of any jobtracker, in any job, of the first + map task, we would use : +
 
+ TaskAttemptID.getTaskAttemptIDsPattern(null, null, true, 1, null);
+ 
+ which will return : +
 "attempt_[^_]*_[0-9]*_m_000001_[0-9]*" 
+ @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @param isMap whether the tip is a map, or null + @param taskId taskId number, or null + @param attemptId the task attempt number, or null + @return a regex pattern matching TaskAttemptIDs]]> +
+
+ + + An example TaskAttemptID is : + attempt_200707121733_0003_m_000005_0 , which represents the + zeroth task attempt for the fifth map task in the third job + running at the jobtracker started at 200707121733. +

+ Applications should never construct or parse TaskAttemptID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the first map task + of any jobtracker, of any job, we would use : +

 
+ TaskID.getTaskIDsPattern(null, null, true, 1);
+ 
+ which will return : +
 "task_[^_]*_[0-9]*_m_000001*" 
+ @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @param isMap whether the tip is a map, or null + @param taskId taskId number, or null + @return a regex pattern matching TaskIDs]]> +
+ + + + An example TaskID is : + task_200707121733_0003_m_000005 , which represents the + fifth map task in the third job running at the jobtracker + started at 200707121733. +

+ Applications should never construct or parse TaskID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskAttemptID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hadoop.log.dir.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the Job was added.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ([,]*) + func ::= tbl(,"") + class ::= @see java.lang.Class#forName(java.lang.String) + path ::= @see org.apache.hadoop.fs.Path#Path(java.lang.String) + } + Reads expression from the mapred.join.expr property and + user-supplied join types from mapred.join.define.<ident> + types. Paths supplied to tbl are given as input paths to the + InputFormat class listed. + @see #compose(java.lang.String, java.lang.Class, java.lang.String...)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ,

) }]]> + + + + + + + + (tbl(,),tbl(,),...,tbl(,)) }]]> + + + + + + + + (tbl(,),tbl(,),...,tbl(,)) }]]> + + + + mapred.join.define.<ident> to a classname. In the expression + mapred.join.expr, the identifier will be assumed to be a + ComposableRecordReader. + mapred.join.keycomparator can be a classname used to compare keys + in the join. + @see JoinRecordReader + @see MultiFilterRecordReader]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ...... + }]]> + + + + + + + + + + + + + + + + + + + + + capacity children to position + id in the parent reader. + The id of a root CompositeRecordReader is -1 by convention, but relying + on this is not recommended.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + override(S1,S2,S3) will prefer values + from S3 over S2, and values from S2 over S1 for all keys + emitted from all sources.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + [,,...,]]]> + + + + + + + out. + TupleWritable format: + {@code + ...... + }]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It has to be specified how key and values are passed from one element of + the chain to the next, by value or by reference. If a Mapper leverages the + assumed semantics that the key and values are not modified by the collector + 'by value' must be used. If the Mapper does not expect this semantics, as + an optimization to avoid serialization and deserialization 'by reference' + can be used. +

+ For the added Mapper the configuration given for it, + mapperConf, have precedence over the job's JobConf. This + precedence is in effect when the task is running. +

+ 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 +

+ + @param job job's JobConf to add the Mapper class. + @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 byValue indicates if key/values should be passed by value + to the next Mapper in the chain, if any. + @param mapperConf a JobConf with the configuration for the Mapper + class. It is recommended to use a JobConf without default values using the + JobConf(boolean loadDefaults) constructor with FALSE.]]> + + + + + + + If this method is overriden super.configure(...) should be + invoked at the beginning of the overwriter method.]]> + + + + + + + + + + map(...) methods of the Mappers in the chain.]]> + + + + + + + If this method is overriden super.close() should be + invoked at the end of the overwriter method.]]> + + + + + 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. +

+ 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. +

+ 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 maching output and input key and + value classes as no conversion is done by the chaining code. +

+ Using the ChainMapper and the ChainReducer classes is possible to compose + Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And + immediate benefit of this pattern is a dramatic reduction in disk IO. +

+ 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. +

+ ChainMapper usage pattern: +

+

+ ...
+ conf.setJobName("chain");
+ conf.setInputFormat(TextInputFormat.class);
+ conf.setOutputFormat(TextOutputFormat.class);
+ 

+ JobConf mapAConf = new JobConf(false); + ... + ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class, + Text.class, Text.class, true, mapAConf); +

+ JobConf mapBConf = new JobConf(false); + ... + ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class, + LongWritable.class, Text.class, false, mapBConf); +

+ JobConf reduceConf = new JobConf(false); + ... + ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class, + Text.class, Text.class, true, reduceConf); +

+ ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class, + LongWritable.class, Text.class, false, null); +

+ ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class, + LongWritable.class, LongWritable.class, true, null); +

+ FileInputFormat.setInputPaths(conf, inDir); + FileOutputFormat.setOutputPath(conf, outDir); + ... +

+ JobClient jc = new JobClient(conf); + RunningJob job = jc.submitJob(conf); + ... +

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + It has to be specified how key and values are passed from one element of + the chain to the next, by value or by reference. If a Reducer leverages the + assumed semantics that the key and values are not modified by the collector + 'by value' must be used. If the Reducer does not expect this semantics, as + an optimization to avoid serialization and deserialization 'by reference' + can be used. +

+ For the added Reducer the configuration given for it, + reducerConf, have precedence over the job's JobConf. This + precedence is in effect when the task is running. +

+ 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. + + @param job job's JobConf to add the Reducer class. + @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 byValue indicates if key/values should be passed by value + to the next Mapper in the chain, if any. + @param reducerConf a JobConf with the configuration for the Reducer + class. It is recommended to use a JobConf without default values using the + JobConf(boolean loadDefaults) constructor with FALSE.]]> + + + + + + + + + + + + + + It has to be specified how key and values are passed from one element of + the chain to the next, by value or by reference. If a Mapper leverages the + assumed semantics that the key and values are not modified by the collector + 'by value' must be used. If the Mapper does not expect this semantics, as + an optimization to avoid serialization and deserialization 'by reference' + can be used. +

+ For the added Mapper the configuration given for it, + mapperConf, have precedence over the job's JobConf. This + precedence is in effect when the task is running. +

+ 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 + . + + @param job chain job's JobConf to add the Mapper class. + @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 byValue indicates if key/values should be passed by value + to the next Mapper in the chain, if any. + @param mapperConf a JobConf with the configuration for the Mapper + class. It is recommended to use a JobConf without default values using the + JobConf(boolean loadDefaults) constructor with FALSE.]]> + + + + + + + If this method is overriden super.configure(...) should be + invoked at the beginning of the overwriter method.]]> + + + + + + + + + + reduce(...) method of the Reducer with the + map(...) methods of the Mappers in the chain.]]> + + + + + + + If this method is overriden super.close() should be + invoked at the end of the overwriter method.]]> + + + + + For each record output by the Reducer, 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. +

+ 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. +

+ 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 maching output and input key and + value classes as no conversion is done by the chaining code. +

+ Using the ChainMapper and the ChainReducer classes is possible to compose + Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And + immediate benefit of this pattern is a dramatic reduction in disk IO. +

+ 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. +

+ ChainReducer usage pattern: +

+

+ ...
+ conf.setJobName("chain");
+ conf.setInputFormat(TextInputFormat.class);
+ conf.setOutputFormat(TextOutputFormat.class);
+ 

+ JobConf mapAConf = new JobConf(false); + ... + ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class, + Text.class, Text.class, true, mapAConf); +

+ JobConf mapBConf = new JobConf(false); + ... + ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class, + LongWritable.class, Text.class, false, mapBConf); +

+ JobConf reduceConf = new JobConf(false); + ... + ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class, + Text.class, Text.class, true, reduceConf); +

+ ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class, + LongWritable.class, Text.class, false, null); +

+ ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class, + LongWritable.class, LongWritable.class, true, null); +

+ FileInputFormat.setInputPaths(conf, inDir); + FileOutputFormat.setOutputPath(conf, outDir); + ... +

+ JobClient jc = new JobClient(conf); + RunningJob job = jc.submitJob(conf); + ... +

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + all splits. + @param freq The frequency with which records will be emitted.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + all splits. + This will read every split at the client, which is very expensive. + @param freq Probability with which a key will be chosen. + @param numSamples Total number of samples to obtain from all selected + splits.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + all splits. + Takes the first numSamples / numSplits records from each split. + @param numSamples Total number of samples to obtain from all selected + splits.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the name output is multi, false + if it is single. If the name output is not defined it returns + false]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @param conf job conf to add the named output + @param namedOutput named output name, it has to be a word, letters + and numbers only, cannot be the word 'part' as + that is reserved for the + default output. + @param outputFormatClass OutputFormat class. + @param keyClass key class + @param valueClass value class]]> + + + + + + + + + + + + @param conf job conf to add the named output + @param namedOutput named output name, it has to be a word, letters + and numbers only, cannot be the word 'part' as + that is reserved for the + default output. + @param outputFormatClass OutputFormat class. + @param keyClass key class + @param valueClass value class]]> + + + + + + + + By default these counters are disabled. +

+ MultipleOutputs supports counters, by default the are disabled. + The counters group is the {@link MultipleOutputs} class name. +

+ The names of the counters are the same as the named outputs. For multi + named outputs the name of the counter is the concatenation of the named + output, and underscore '_' and the multiname. + + @param conf job conf to enableadd the named output. + @param enabled indicates if the counters will be enabled or not.]]> +
+
+ + + + + By default these counters are disabled. +

+ MultipleOutputs supports counters, by default the are disabled. + The counters group is the {@link MultipleOutputs} class name. +

+ The names of the counters are the same as the named outputs. For multi + named outputs the name of the counter is the concatenation of the named + output, and underscore '_' and the multiname. + + + @param conf job conf to enableadd the named output. + @return TRUE if the counters are enabled, FALSE if they are disabled.]]> +
+
+ + + + + + + + + + + + + @param namedOutput the named output name + @param reporter the reporter + @return the output collector for the given named output + @throws IOException thrown if output collector could not be created]]> + + + + + + + + + + + @param namedOutput the named output name + @param multiName the multi name part + @param reporter the reporter + @return the output collector for the given named output + @throws IOException thrown if output collector could not be created]]> + + + + + + + If overriden subclasses must invoke super.close() at the + end of their close() + + @throws java.io.IOException thrown if any of the MultipleOutput files + could not be closed properly.]]> + + + + OutputCollector passed to + the map() and reduce() methods of the + Mapper and Reducer implementations. +

+ Each additional output, or named output, may be configured with its own + OutputFormat, with its own key class and with its own value + class. +

+ A named output can be a single file or a multi file. The later is refered as + a multi named output. +

+ A multi named output is an unbound set of files all sharing the same + OutputFormat, key class and value class configuration. +

+ When named outputs are used within a Mapper implementation, + key/values written to a name output are not part of the reduce phase, only + key/values written to the job OutputCollector are part of the + reduce phase. +

+ MultipleOutputs supports counters, by default the are disabled. The counters + group is the {@link MultipleOutputs} class name. +

+ The names of the counters are the same as the named outputs. For multi + named outputs the name of the counter is the concatenation of the named + output, and underscore '_' and the multiname. +

+ Job configuration usage pattern is: +

+
+ JobConf conf = new JobConf();
+
+ conf.setInputPath(inDir);
+ FileOutputFormat.setOutputPath(conf, outDir);
+
+ conf.setMapperClass(MOMap.class);
+ conf.setReducerClass(MOReduce.class);
+ ...
+
+ // Defines additional single text based output 'text' for the job
+ MultipleOutputs.addNamedOutput(conf, "text", TextOutputFormat.class,
+ LongWritable.class, Text.class);
+
+ // Defines additional multi sequencefile based output 'sequence' for the
+ // job
+ MultipleOutputs.addMultiNamedOutput(conf, "seq",
+   SequenceFileOutputFormat.class,
+   LongWritable.class, Text.class);
+ ...
+
+ JobClient jc = new JobClient();
+ RunningJob job = jc.submitJob(conf);
+
+ ...
+ 
+

+ Job configuration usage pattern is: +

+
+ public class MOReduce implements
+   Reducer<WritableComparable, Writable> {
+ private MultipleOutputs mos;
+
+ public void configure(JobConf conf) {
+ ...
+ mos = new MultipleOutputs(conf);
+ }
+
+ public void reduce(WritableComparable key, Iterator<Writable> values,
+ OutputCollector output, Reporter reporter)
+ throws IOException {
+ ...
+ mos.getCollector("text", reporter).collect(key, new Text("Hello"));
+ mos.getCollector("seq", "A", reporter).collect(key, new Text("Bye"));
+ mos.getCollector("seq", "B", reporter).collect(key, new Text("Chau"));
+ ...
+ }
+
+ public void close() throws IOException {
+ mos.close();
+ ...
+ }
+
+ }
+ 
]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It can be used instead of the default implementation, + @link org.apache.hadoop.mapred.MapRunner, when the Map operation is not CPU + bound in order to improve throughput. +

+ Map implementations using this MapRunnable must be thread-safe. +

+ The Map-Reduce job has to be configured to use this MapRunnable class (using + the JobConf.setMapRunnerClass method) and + the number of thread the thread-pool can use with the + mapred.map.multithreadedrunner.threads property, its default + value is 10 threads. +

]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + pairs. Uses + {@link StringTokenizer} to break text into tokens.]]> + + + + + + + + + + + + total.order.partitioner.natural.order is not false, a trie + of the first total.order.partitioner.max.trie.depth(2) + 1 bytes + will be built. Otherwise, keys will be located using a binary search of + the partition keyset using the {@link org.apache.hadoop.io.RawComparator} + defined for this job. The input file must be sorted with the same + comparator and contain {@link + org.apache.hadoop.mapred.JobConf#getNumReduceTasks} - 1 keys.]]> + + + + + + + + + + + + R reduces, there are R-1 + keys in the SequenceFile.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + generateKeyValPairs(Object key, Object value); public void + configure(JobConfjob); } + + The package also provides a base class, ValueAggregatorBaseDescriptor, + implementing the above interface. The user can extend the base class and + implement generateKeyValPairs accordingly. + + The primary work of generateKeyValPairs is to emit one or more key/value + pairs based on the input key/value pair. The key in an output key/value pair + encode two pieces of information: aggregation type and aggregation id. The + value will be aggregated onto the aggregation id according the aggregation + type. + + This class offers a function to generate a map/reduce job using Aggregate + framework. The function takes the following parameters: input directory spec + input format (text or sequence file) output directory a file specifying the + user plugin class]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The job can be configured using the static methods in this class, + {@link DBInputFormat}, and {@link DBOutputFormat}. +

+ Alternatively, the properties can be set in the configuration with proper + values. + + @see DBConfiguration#configureDB(JobConf, String, String, String, String) + @see DBInputFormat#setInput(JobConf, Class, String, String) + @see DBInputFormat#setInput(JobConf, Class, String, String, String, String...) + @see DBOutputFormat#setOutput(JobConf, String, String...)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 20070101 AND length > 0)' + @param orderBy the fieldNames in the orderBy clause. + @param fieldNames The field names in the table + @see #setInput(JobConf, Class, String, String)]]> + + + + + + + + + + + + + + DBInputFormat emits LongWritables containing the record number as + key and DBWritables as value. + + The SQL query, and input class can be using one of the two + setInput methods.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + {@link DBOutputFormat} accepts <key,value> pairs, where + key has a type extending DBWritable. Returned {@link RecordWriter} + writes only the key to the database with a batch SQL query.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DBWritable. DBWritable, is similar to {@link Writable} + except that the {@link #write(PreparedStatement)} method takes a + {@link PreparedStatement}, and {@link #readFields(ResultSet)} + takes a {@link ResultSet}. +

+ Implementations are responsible for writing the fields of the object + to PreparedStatement, and reading the fields of the object from the + ResultSet. + +

Example:

+ If we have the following table in the database : +
+ CREATE TABLE MyTable (
+   counter        INTEGER NOT NULL,
+   timestamp      BIGINT  NOT NULL,
+ );
+ 
+ then we can read/write the tuples from/to the table with : +

+ public class MyWritable implements Writable, DBWritable {
+   // Some data     
+   private int counter;
+   private long timestamp;
+       
+   //Writable#write() implementation
+   public void write(DataOutput out) throws IOException {
+     out.writeInt(counter);
+     out.writeLong(timestamp);
+   }
+       
+   //Writable#readFields() implementation
+   public void readFields(DataInput in) throws IOException {
+     counter = in.readInt();
+     timestamp = in.readLong();
+   }
+       
+   public void write(PreparedStatement statement) throws SQLException {
+     statement.setInt(1, counter);
+     statement.setLong(2, timestamp);
+   }
+       
+   public void readFields(ResultSet resultSet) throws SQLException {
+     counter = resultSet.getInt(1);
+     timestamp = resultSet.getLong(2);
+   } 
+ }
+ 

]]> +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + When constructing the instance, if the factory property + contextName.class exists, + its value is taken to be the name of the class to instantiate. Otherwise, + the default is to create an instance of + org.apache.hadoop.metrics.spi.NullContext, which is a + dummy "no-op" context which will cause all metric data to be discarded. + + @param contextName the name of the context + @return the named MetricsContext]]> + + + + + + + + + + + + + + When the instance is constructed, this method checks if the file + hadoop-metrics.properties exists on the class path. If it + exists, it must be in the format defined by java.util.Properties, and all + the properties in the file are set as attributes on the newly created + ContextFactory instance. + + @return the singleton ContextFactory instance]]> + + + + getFactory() method.]]> + + + + + + + + + + + + + + + + + + + startMonitoring() again after calling + this. + @see #close()]]> + + + + + + + + + + + + + + + + recordName. + Throws an exception if the metrics implementation is configured with a fixed + set of record names and recordName is not in that set. + + @param recordName the name of the record + @throws MetricsException if recordName conflicts with configuration data]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A record name identifies the kind of data to be reported. For example, a + program reporting statistics relating to the disks on a computer might use + a record name "diskStats".

+ + A record has zero or more tags. A tag has a name and a value. To + continue the example, the "diskStats" record might use a tag named + "diskName" to identify a particular disk. Sometimes it is useful to have + more than one tag, so there might also be a "diskType" with value "ide" or + "scsi" or whatever.

+ + A record also has zero or more metrics. These are the named + values that are to be reported to the metrics system. In the "diskStats" + example, possible metric names would be "diskPercentFull", "diskPercentBusy", + "kbReadPerSecond", etc.

+ + The general procedure for using a MetricsRecord is to fill in its tag and + metric values, and then call update() to pass the record to the + client library. + Metric data is not immediately sent to the metrics system + each time that update() is called. + An internal table is maintained, identified by the record name. This + table has columns + corresponding to the tag and the metric names, and rows + corresponding to each unique set of tag values. An update + either modifies an existing row in the table, or adds a new row with a set of + tag values that are different from all the other rows. Note that if there + are no tags, then there can be at most one row in the table.

+ + Once a row is added to the table, its data will be sent to the metrics system + on every timer period, whether or not it has been updated since the previous + timer period. If this is inappropriate, for example if metrics were being + reported by some transient object in an application, the remove() + method can be used to remove the row and thus stop the data from being + sent.

+ + Note that the update() method is atomic. This means that it is + safe for different threads to be updating the same metric. More precisely, + it is OK for different threads to call update() on MetricsRecord instances + with the same set of tag names and tag values. Different threads should + not use the same MetricsRecord instance at the same time.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + MetricsContext.registerUpdater().]]> + + + + + + + + + + + + + + + + + + + + + + + + + fileName attribute, + if specified. Otherwise the data will be written to standard + output.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class is configured by setting ContextFactory attributes which in turn + are usually configured through a properties file. All the attributes are + prefixed by the contextName. For example, the properties file might contain: +

+ myContextName.fileName=/tmp/metrics.log
+ myContextName.period=5
+ 
]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + contextName.tableName. The returned map consists of + those attributes with the contextName and tableName stripped off.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + recordName. + Throws an exception if the metrics implementation is configured with a fixed + set of record names and recordName is not in that set. + + @param recordName the name of the record + @throws MetricsException if recordName conflicts with configuration data]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class implements the internal table of metric data, and the timer + on which data is to be sent to the metrics system. Subclasses must + override the abstract emitRecord method in order to transmit + the data.

]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + update + and remove().]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hostname or hostname:port. If + the specs string is null, defaults to localhost:defaultPort. + + @return a list of InetSocketAddress objects.]]> + + + + + + + + + + + + + + + + + + + ,name=" + Where the and are the supplied parameters + + @param serviceName + @param nameName + @param theMbean - the MBean to register + @return the named used to register the MBean]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hadoop.rpc.socket.factory.class.<ClassName>. When no + such parameter exists then fall back on the default socket factory as + configured by hadoop.rpc.socket.factory.class.default. If + this default socket factory is not configured, then fall back on the JVM + default socket factory. + + @param conf the configuration + @param clazz the class (usually a {@link VersionedProtocol}) + @return a socket factory]]> + + + + + + hadoop.rpc.socket.factory.default + + @param conf the configuration + @return the default socket factory as specified in the configuration or + the JVM default socket factory if the configuration does not + contain a default socket factory property.]]> + + + + + + + + + + + + + : + ://:/]]> + + + + + + + + : + ://:/]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + From documentation for {@link #getInputStream(Socket, long)}:
+ Returns InputStream for the socket. If the socket has an associated + SocketChannel then it returns a + {@link SocketInputStream} with the given timeout. If the socket does not + have a channel, {@link Socket#getInputStream()} is returned. In the later + case, the timeout argument is ignored and the timeout set with + {@link Socket#setSoTimeout(int)} applies for reads.

+ + Any socket created using socket factories returned by {@link #NetUtils}, + must use this interface instead of {@link Socket#getInputStream()}. + + @see #getInputStream(Socket, long) + + @param socket + @return InputStream for reading from the socket. + @throws IOException]]> +
+
+ + + + + +
+ + Any socket created using socket factories returned by {@link #NetUtils}, + must use this interface instead of {@link Socket#getInputStream()}. + + @see Socket#getChannel() + + @param socket + @param timeout timeout in milliseconds. This may not always apply. zero + for waiting as long as necessary. + @return InputStream for reading from the socket. + @throws IOException]]> +
+
+ + + + +
+ + From documentation for {@link #getOutputStream(Socket, long)} :
+ Returns OutputStream for the socket. If the socket has an associated + SocketChannel then it returns a + {@link SocketOutputStream} with the given timeout. If the socket does not + have a channel, {@link Socket#getOutputStream()} is returned. In the later + case, the timeout argument is ignored and the write will wait until + data is available.

+ + Any socket created using socket factories returned by {@link #NetUtils}, + must use this interface instead of {@link Socket#getOutputStream()}. + + @see #getOutputStream(Socket, long) + + @param socket + @return OutputStream for writing to the socket. + @throws IOException]]> +
+
+ + + + + +
+ + Any socket created using socket factories returned by {@link #NetUtils}, + must use this interface instead of {@link Socket#getOutputStream()}. + + @see Socket#getChannel() + + @param socket + @param timeout timeout in milliseconds. This may not always apply. zero + for waiting as long as necessary. + @return OutputStream for writing to the socket. + @throws IOException]]> +
+
+ + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + node + + @param node + a node + @return true if node is already in the tree; false otherwise]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + scope + if scope starts with ~, choose one from the all nodes except for the + ones in scope; otherwise, choose one from scope + @param scope range of nodes from which a node will be choosen + @return the choosen node]]> + + + + + + + scope but not in excludedNodes + if scope starts with ~, return the number of nodes that are not + in scope and excludedNodes; + @param scope a path string that may start with ~ + @param excludedNodes a list of nodes + @return number of available nodes]]> + + + + + + + + + + + + reader + It linearly scans the array, if a local node is found, swap it with + the first element of the array. + If a local rack node is found, swap it with the first element following + the local node. + If neither local node or local rack node is found, put a random replica + location at postion 0. + It leaves the rest nodes untouched.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + Create a new input stream with the given timeout. If the timeout + is zero, it will be treated as infinite timeout. The socket's + channel will be configured to be non-blocking. + + @see SocketInputStream#SocketInputStream(ReadableByteChannel, long) + + @param socket should have a channel associated with it. + @param timeout timeout timeout in milliseconds. must not be negative. + @throws IOException]]> +
+
+ + + +
+ + Create a new input stream with the given timeout. If the timeout + is zero, it will be treated as infinite timeout. The socket's + channel will be configured to be non-blocking. + @see SocketInputStream#SocketInputStream(ReadableByteChannel, long) + + @param socket should have a channel associated with it. + @throws IOException]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + + + + + +
+ + Create a new ouput stream with the given timeout. If the timeout + is zero, it will be treated as infinite timeout. The socket's + channel will be configured to be non-blocking. + + @see SocketOutputStream#SocketOutputStream(WritableByteChannel, long) + + @param socket should have a channel associated with it. + @param timeout timeout timeout in milliseconds. must not be negative. + @throws IOException]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + = getCount(). + @param newCapacity The new capacity in bytes.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Index idx = startVector(...); + while (!idx.done()) { + .... // read element of a vector + idx.incr(); + } + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This task takes the given record definition files and compiles them into + java or c++ + files. It is then up to the user to compile the generated files. + +

The task requires the file or the nested fileset element to be + specified. Optional attributes are language (set the output + language, default is "java"), + destdir (name of the destination directory for generated java/c++ + code, default is ".") and failonerror (specifies error handling + behavior. default is true). +

Usage

+
+ <recordcc
+       destdir="${basedir}/gensrc"
+       language="java">
+   <fileset include="**\/*.jr" />
+ </recordcc>
+ 
]]> +
+
+ +
+ + + + + + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ugi as a comma separated string in + conf as a property attr + + The String starts with the user name followed by the default group names, + and other group names. + + @param conf configuration + @param attr property name + @param ugi a UnixUserGroupInformation]]> + + + + + + + + conf + + The object is expected to store with the property name attr + as a comma separated string that starts + with the user name followed by group names. + If the property name is not defined, return null. + It's assumed that there is only one UGI per user. If this user already + has a UGI in the ugi map, return the ugi in the map. + Otherwise, construct a UGI from the configuration, store it in the + ugi map and return it. + + @param conf configuration + @param attr property name + @return a UnixUGI + @throws LoginException if the stored string is ill-formatted.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This tool supports archiving and anaylzing (sort/grep) of log-files. + It takes as input + a) Input uri which will serve uris of the logs to be archived. + b) Output directory (not mandatory). + b) Directory on dfs to archive the logs. + c) The sort/grep patterns for analyzing the files and separator for boundaries. + Usage: + Logalyzer -archive -archiveDir -analysis -logs -grep -sort -separator +

]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + in]]> + + + + + + + out.]]> + + + + + + + + + + reset is true, then resets the checksum. + @return number of bytes written. Will be equal to getChecksumSize();]]> + + + + + + + + + reset is true, then resets the checksum. + @return number of bytes written. Will be equal to getChecksumSize();]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + GenericOptionsParser to parse only the generic Hadoop + arguments. + + The array of string arguments other than the generic arguments can be + obtained by {@link #getRemainingArgs()}. + + @param conf the Configuration to modify. + @param args command-line arguments.]]> + + + + + GenericOptionsParser to parse given options as well + as generic Hadoop options. + + The resulting CommandLine object can be obtained by + {@link #getCommandLine()}. + + @param conf the configuration to modify + @param options options built by the caller + @param args User-specified arguments]]> + + + + + Strings containing the un-parsed arguments + or empty array if commandLine was not defined.]]> + + + + + CommandLine object + to process the parsed arguments. + + Note: If the object is created with + {@link #GenericOptionsParser(Configuration, String[])}, then returned + object will only contain parsed generic options. + + @return CommandLine representing list of arguments + parsed against Options descriptor.]]> + + + + + + + + + + + + + + + + + GenericOptionsParser is a utility to parse command line + arguments generic to the Hadoop framework. + + GenericOptionsParser recognizes several standarad command + line arguments, enabling applications to easily specify a namenode, a + jobtracker, additional configuration resources etc. + +

Generic Options

+ +

The supported generic options are:

+

+     -conf <configuration file>     specify a configuration file
+     -D <property=value>            use value for given property
+     -fs <local|namenode:port>      specify a namenode
+     -jt <local|jobtracker:port>    specify a job tracker
+     -files <comma separated list of files>    specify comma separated
+                            files to be copied to the map reduce cluster
+     -libjars <comma separated list of jars>   specify comma separated
+                            jar files to include in the classpath.
+     -archives <comma separated list of archives>    specify comma
+             separated archives to be unarchived on the compute machines.
+
+ 

+ +

The general command line syntax is:

+

+ bin/hadoop command [genericOptions] [commandOptions]
+ 

+ +

Generic command line arguments might modify + Configuration objects, given to constructors.

+ +

The functionality is implemented using Commons CLI.

+ +

Examples:

+

+ $ bin/hadoop dfs -fs darwin:8020 -ls /data
+ list /data directory in dfs with namenode darwin:8020
+ 
+ $ bin/hadoop dfs -D fs.default.name=darwin:8020 -ls /data
+ list /data directory in dfs with namenode darwin:8020
+     
+ $ bin/hadoop dfs -conf hadoop-site.xml -ls /data
+ list /data directory in dfs with conf specified in hadoop-site.xml
+     
+ $ bin/hadoop job -D mapred.job.tracker=darwin:50020 -submit job.xml
+ submit a job to job tracker darwin:50020
+     
+ $ bin/hadoop job -jt darwin:50020 -submit job.xml
+ submit a job to job tracker darwin:50020
+     
+ $ bin/hadoop job -jt local -submit job.xml
+ submit a job to local runner
+ 
+ $ bin/hadoop jar -libjars testlib.jar 
+ -archives test.tgz -files file.txt inputjar args
+ job submission with libjars, files and archives
+ 

+ + @see Tool + @see ToolRunner]]> +
+
+ + + + + + + + + Class<T>) of the + argument of type T. + @param The type of the argument + @param t the object to get it class + @return Class<T>]]> + + + + + + + List<T> to a an array of + T[]. + @param c the Class object of the items in the list + @param list the list to convert]]> + + + + + + List<T> to a an array of + T[]. + @param list the list to convert + @throws ArrayIndexOutOfBoundsException if the list is empty. + Use {@link #toArray(Class, List)} if the list may be empty.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + io.file.buffer.size specified in the given + Configuration. + @param in input stream + @param conf configuration + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if native-hadoop is loaded, + else false]]> + + + + + + true if native hadoop libraries, if present, can be + used for this job; false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + { pq.top().change(); pq.adjustTop(); } + instead of
+  { o = pq.pop(); o.change(); pq.push(o); }
+ 
]]> +
+
+ + + + + + + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Clients and/or applications can use the provided Progressable + to explicitly report progress to the Hadoop framework. This is especially + important for operations which take an insignificant amount of time since, + in-lieu of the reported progress, the framework has to assume that an error + has occured and time-out the operation.

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Class is to be obtained + @return the correctly typed Class of the given object.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hadoop Pipes + or Hadoop Streaming. + + It also checks to ensure that we are running on a *nix platform else + (e.g. in Cygwin/Windows) it returns null. + @param conf configuration + @return a String[] with the ulimit command arguments or + null if we are running on a non *nix platform or + if the limit is unspecified.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Shell interface. + @param cmd shell command to execute. + @return the output of the executed command.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + Shell can be used to run unix commands like du or + df. It also offers facilities to gate commands by + time-intervals.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ShellCommandExecutorshould be used in cases where the output + of the command needs no explicit parsing and where the command, working + directory and the environment remains unchanged. The output of the command + is stored as-is and is expected to be small.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ArrayList of string values]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + charToEscape in the string + with the escape char escapeChar + + @param str string + @param escapeChar escape char + @param charToEscape the char to be escaped + @return an escaped string]]> + + + + + + + + + + + + + + + + + + + + + + charToEscape in the string + with the escape char escapeChar + + @param str string + @param escapeChar escape char + @param charToEscape the escaped char + @return an unescaped string]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tool, is the standard for any Map-Reduce tool/application. + The tool/application should delegate the handling of + + standard command-line options to {@link ToolRunner#run(Tool, String[])} + and only handle its custom arguments.

+ +

Here is how a typical Tool is implemented:

+

+     public class MyApp extends Configured implements Tool {
+     
+       public int run(String[] args) throws Exception {
+         // Configuration processed by ToolRunner
+         Configuration conf = getConf();
+         
+         // Create a JobConf using the processed conf
+         JobConf job = new JobConf(conf, MyApp.class);
+         
+         // Process custom command-line options
+         Path in = new Path(args[1]);
+         Path out = new Path(args[2]);
+         
+         // Specify various job-specific parameters     
+         job.setJobName("my-app");
+         job.setInputPath(in);
+         job.setOutputPath(out);
+         job.setMapperClass(MyApp.MyMapper.class);
+         job.setReducerClass(MyApp.MyReducer.class);
+
+         // Submit the job, then poll for progress until the job is complete
+         JobClient.runJob(job);
+       }
+       
+       public static void main(String[] args) throws Exception {
+         // Let ToolRunner handle generic command-line options 
+         int res = ToolRunner.run(new Configuration(), new Sort(), args);
+         
+         System.exit(res);
+       }
+     }
+ 

+ + @see GenericOptionsParser + @see ToolRunner]]> +
+
+ + + + + + + + + + + + Tool by {@link Tool#run(String[])}, after + parsing with the given generic arguments. Uses the given + Configuration, or builds one if null. + + Sets the Tool's configuration with the possibly modified + version of the conf. + + @param conf Configuration for the Tool. + @param tool Tool to run. + @param args command-line arguments to the tool. + @return exit code of the {@link Tool#run(String[])} method.]]> + + + + + + + + Tool with its Configuration. + + Equivalent to run(tool.getConf(), tool, args). + + @param tool Tool to run. + @param args command-line arguments to the tool. + @return exit code of the {@link Tool#run(String[])} method.]]> + + + + + + + + + + ToolRunner can be used to run classes implementing + Tool interface. It works in conjunction with + {@link GenericOptionsParser} to parse the + + generic hadoop command line arguments and modifies the + Configuration of the Tool. The + application-specific options are passed along without being modified. +

+ + @see Tool + @see GenericOptionsParser]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ + diff --git a/third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.20.0.xml b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.20.1.xml similarity index 96% rename from third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.20.0.xml rename to core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.20.1.xml index 9067cf1158..fc056397c8 100644 --- a/third_party/hadoop-0.20.0/lib/jdiff/hadoop_0.20.0.xml +++ b/core/lib/hadoop-0.20.2/lib/jdiff/hadoop_0.20.1.xml @@ -1,15 +1,15 @@ - + - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • "none" - No compression. +
  • "lzo" - LZO compression. +
  • "gz" - GZIP compression. + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • Block Compression. +
  • Named meta data blocks. +
  • Sorted or unsorted keys. +
  • Seek by key or by file offset. + + The memory footprint of a TFile includes the following: +
      +
    • Some constant overhead of reading or writing a compressed block. +
        +
      • Each compressed block requires one compression/decompression codec for + I/O. +
      • Temporary space to buffer the key. +
      • Temporary space to buffer the value (for TFile.Writer only). Values are + chunk encoded, so that we buffer at most one chunk of user data. By default, + the chunk buffer is 1MB. Reading chunked value does not require additional + memory. +
      +
    • TFile index, which is proportional to the total number of Data Blocks. + The total amount of memory needed to hold the index can be estimated as + (56+AvgKeySize)*NumBlocks. +
    • MetaBlock index, which is proportional to the total number of Meta + Blocks.The total amount of memory needed to hold the index for Meta Blocks + can be estimated as (40+AvgMetaBlockName)*NumMetaBlock. +
    +

    + The behavior of TFile can be customized by the following variables through + Configuration: +

      +
    • tfile.io.chunk.size: Value chunk size. Integer (in bytes). Default + to 1MB. Values of the length less than the chunk size is guaranteed to have + known value length in read time (See + {@link TFile.Reader.Scanner.Entry#isValueLengthKnown()}). +
    • tfile.fs.output.buffer.size: Buffer size used for + FSDataOutputStream. Integer (in bytes). Default to 256KB. +
    • tfile.fs.input.buffer.size: Buffer size used for + FSDataInputStream. Integer (in bytes). Default to 256KB. +
    +

    + Suggestions on performance optimization. +

      +
    • Minimum block size. We recommend a setting of minimum block size between + 256KB to 1MB for general usage. Larger block size is preferred if files are + primarily for sequential access. However, it would lead to inefficient random + access (because there are more data to decompress). Smaller blocks are good + for random access, but require more memory to hold the block index, and may + be slower to create (because we must flush the compressor stream at the + conclusion of each data block, which leads to an FS I/O flush). Further, due + to the internal caching in Compression codec, the smallest possible block + size would be around 20KB-30KB. +
    • The current implementation does not offer true multi-threading for + reading. The implementation uses FSDataInputStream seek()+read(), which is + shown to be much faster than positioned-read call in single thread mode. + However, it also means that if multiple threads attempt to access the same + TFile (using multiple scanners) simultaneously, the actual I/O is carried out + sequentially even if they access different DFS blocks. +
    • Compression codec. Use "none" if the data is not very compressable (by + compressable, I mean a compression ratio at least 2:1). Generally, use "lzo" + as the starting point for experimenting. "gz" overs slightly better + compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to + decompress, comparing to "lzo". +
    • File system buffering, if the underlying FSDataInputStream and + FSDataOutputStream is already adequately buffered; or if applications + reads/writes keys and values in large buffers, we can reduce the sizes of + input/output buffering in TFile layer by setting the configuration parameters + "tfile.fs.input.buffer.size" and "tfile.fs.output.buffer.size". +
    + + Some design rationale behind TFile can be found at Hadoop-3315.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + entry of the TFile. + @param endKey + End key of the scan. If null, scan up to the last entry + of the TFile. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Use {@link Scanner#atEnd()} to test whether the cursor is at the end + location of the scanner. +

    + Use {@link Scanner#advance()} to move the cursor to the next key-value + pair (or end if none exists). Use seekTo methods ( + {@link Scanner#seekTo(byte[])} or + {@link Scanner#seekTo(byte[], int, int)}) to seek to any arbitrary + location in the covered range (including backward seeking). Use + {@link Scanner#rewind()} to seek back to the beginning of the scanner. + Use {@link Scanner#seekToEnd()} to seek to the end of the scanner. +

    + Actual keys and values may be obtained through {@link Scanner.Entry} + object, which is obtained through {@link Scanner#entry()}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

  • Algorithmic comparator: binary comparators that is language + independent. Currently, only "memcmp" is supported. +
  • Language-specific comparator: binary comparators that can + only be constructed in specific language. For Java, the syntax + is "jclass:", followed by the class name of the RawComparator. + Currently, we only support RawComparators that can be + constructed through the default constructor (with no + parameters). Parameterized RawComparators such as + {@link WritableComparator} or + {@link JavaSerializationComparator} may not be directly used. + One should write a wrapper class that inherits from such classes + and use its default constructor to perform proper + initialization. + + @param conf + The configuration object. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + If an exception is thrown, the TFile will be in an inconsistent + state. The only legitimate call after that would be close]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Utils#writeVLong(out, n). + + @param out + output stream + @param n + The integer to be encoded + @throws IOException + @see Utils#writeVLong(DataOutput, long)]]> + + + + + + + + +
  • if n in [-32, 127): encode in one byte with the actual value. + Otherwise, +
  • if n in [-20*2^8, 20*2^8): encode in two bytes: byte[0] = n/256 - 52; + byte[1]=n&0xff. Otherwise, +
  • if n IN [-16*2^16, 16*2^16): encode in three bytes: byte[0]=n/2^16 - + 88; byte[1]=(n>>8)&0xff; byte[2]=n&0xff. Otherwise, +
  • if n in [-8*2^24, 8*2^24): encode in four bytes: byte[0]=n/2^24 - 112; + byte[1] = (n>>16)&0xff; byte[2] = (n>>8)&0xff; byte[3]=n&0xff. Otherwise: +
  • if n in [-2^31, 2^31): encode in five bytes: byte[0]=-125; byte[1] = + (n>>24)&0xff; byte[2]=(n>>16)&0xff; byte[3]=(n>>8)&0xff; byte[4]=n&0xff; +
  • if n in [-2^39, 2^39): encode in six bytes: byte[0]=-124; byte[1] = + (n>>32)&0xff; byte[2]=(n>>24)&0xff; byte[3]=(n>>16)&0xff; + byte[4]=(n>>8)&0xff; byte[5]=n&0xff +
  • if n in [-2^47, 2^47): encode in seven bytes: byte[0]=-123; byte[1] = + (n>>40)&0xff; byte[2]=(n>>32)&0xff; byte[3]=(n>>24)&0xff; + byte[4]=(n>>16)&0xff; byte[5]=(n>>8)&0xff; byte[6]=n&0xff; +
  • if n in [-2^55, 2^55): encode in eight bytes: byte[0]=-122; byte[1] = + (n>>48)&0xff; byte[2] = (n>>40)&0xff; byte[3]=(n>>32)&0xff; + byte[4]=(n>>24)&0xff; byte[5]=(n>>16)&0xff; byte[6]=(n>>8)&0xff; + byte[7]=n&0xff; +
  • if n in [-2^63, 2^63): encode in nine bytes: byte[0]=-121; byte[1] = + (n>>54)&0xff; byte[2] = (n>>48)&0xff; byte[3] = (n>>40)&0xff; + byte[4]=(n>>32)&0xff; byte[5]=(n>>24)&0xff; byte[6]=(n>>16)&0xff; + byte[7]=(n>>8)&0xff; byte[8]=n&0xff; + + + @param out + output stream + @param n + the integer number + @throws IOException]]> + + + + + + + (int)Utils#readVLong(in). + + @param in + input stream + @return the decoded integer + @throws IOException + + @see Utils#readVLong(DataInput)]]> + + + + + + + +
  • if (FB >= -32), return (long)FB; +
  • if (FB in [-72, -33]), return (FB+52)<<8 + NB[0]&0xff; +
  • if (FB in [-104, -73]), return (FB+88)<<16 + (NB[0]&0xff)<<8 + + NB[1]&0xff; +
  • if (FB in [-120, -105]), return (FB+112)<<24 + (NB[0]&0xff)<<16 + + (NB[1]&0xff)<<8 + NB[2]&0xff; +
  • if (FB in [-128, -121]), return interpret NB[FB+129] as a signed + big-endian integer. + + @param in + input stream + @return the decoded long integer. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @param cmp + Comparator for the key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @param cmp + Comparator for the key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Job Completion and Chaining]]> + @see Job Completion and Chaining]]> - - - - - - - - - + - - - - - - - - - + - + + + + + + + + + mapred.task.maxvmem is split into + mapred.job.map.memory.mb + and mapred.job.map.memory.mb,mapred + each of the new key are set + as mapred.task.maxvmem / 1024 + as new values are in MB + + @return The maximum amount of memory any task of this job will use, in + bytes. + @see #setMaxVirtualMemoryForTask(long) + @deprecated Use {@link #getMemoryForMapTask()} and + {@link #getMemoryForReduceTask()}]]> + + + + + + + mapred.task.maxvmem is split into + mapred.job.map.memory.mb + and mapred.job.map.memory.mb,mapred + each of the new key are set + as mapred.task.maxvmem / 1024 + as new values are in MB + + @param vmem Maximum amount of virtual memory in bytes any task of this job + can use. + @see #getMaxVirtualMemoryForTask() + @deprecated + Use {@link #setMemoryForMapTask(long mem)} and + Use {@link #setMemoryForReduceTask(long mem)}]]> + + + + + + + + + + + + deprecated="deprecated, no comment"> - + - + deprecated="deprecated, no comment"> - + + deprecated="deprecated, no comment"> - - - If a job doesn't specify its virtual memory requirement by setting - {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to {@link #DISABLED_MEMORY_LIMIT}, - tasks are assured a memory limit set to this property. This property is - disabled by default, and if not explicitly set to a valid value by the - administrators and if a job doesn't specify its virtual memory - requirements, the job's tasks will not be assured anything and may be - killed by a TT that intends to control the total memory usage of the tasks - via memory management functionality. - -

    - - This value should in general be less than the cluster-wide configuration - {@link #UPPER_LIMIT_ON_TASK_VMEM_PROPERTY} . If not or if it not set, - TaskTracker's memory management may be disabled and a scheduler's memory - based scheduling decisions will be affected. Please refer to the - documentation of the configured scheduler to see how this property is used.]]> + - + deprecated="deprecated, no comment"> - - - This value will be used by TaskTrackers for monitoring the memory usage of - tasks of this jobs. If a TaskTracker's memory management functionality is - enabled, each task of this job will be allowed to use a maximum virtual - memory specified by this property. If the task's memory usage goes over - this value, the task will be failed by the TT. If not set, the cluster-wide - configuration {@link #MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY} is used as the - default value for memory requirements. If this property cascaded with - {@link #MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY} becomes equal to -1, job's - tasks will not be assured anything and may be killed by a TT that intends - to control the total memory usage of the tasks via memory management - functionality. If the memory management functionality is disabled on a TT, - this value is ignored. - -

    - - This value should also be not more than the cluster-wide configuration - {@link #UPPER_LIMIT_ON_TASK_VMEM_PROPERTY} which has to be set by the site - administrators. - -

    - - This value may be used by schedulers that support scheduling based on job's - memory requirements. In general, a task of this job will be scheduled on a - TaskTracker only if the amount of virtual memory still unoccupied on the - TaskTracker is greater than or equal to this value. But different - schedulers can take different decisions. Please refer to the documentation - of the scheduler being configured to see if it does memory based scheduling - and if it does, how this property is used by that scheduler. - - @see #setMaxVirtualMemoryForTask(long) - @see #getMaxVirtualMemoryForTask()]]> + - - - - This value may be used by schedulers that support scheduling based on job's - memory requirements. In general, a task of this job will be scheduled on a - TaskTracker, only if the amount of physical memory still unoccupied on the - TaskTracker is greater than or equal to this value. But different - schedulers can take different decisions. Please refer to the documentation - of the scheduler being configured to see how it does memory based - scheduling and how this variable is used by that scheduler. - - @see #setMaxPhysicalMemoryForTask(long) - @see #getMaxPhysicalMemoryForTask()]]> + - - - - If it is not set on a TaskTracker, TaskTracker's memory management will be - disabled.]]> + @@ -35211,7 +36846,8 @@ + deprecated="Use + {@link #logSubmitted(JobID, JobConf, String, long, boolean)} instead."> @@ -35225,9 +36861,22 @@ @param jobConf job conf of the job @param jobConfPath path to job conf xml file in HDFS. @param submitTime time when job tracker received the job - @throws IOException]]> + @throws IOException + @deprecated Use + {@link #logSubmitted(JobID, JobConf, String, long, boolean)} instead.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + final. + + @param name resource to be added, the classpath is examined for a file + with that name.]]> + + + + + + final. + + @param url url of the resource to be added, the local filesystem is + examined directly to find the resource, without referring to + the classpath.]]> + + + + + + final. + + @param file file-path of resource to be added, the local filesystem is + examined directly to find the resource, without referring to + the classpath.]]> + + + + + + final. + + @param in InputStream to deserialize the object from.]]> + + + + + + + + + + + name property, null if + no such property exists. + + Values are processed for variable expansion + before being returned. + + @param name the property name. + @return the value of the name property, + or null if no such property exists.]]> + + + + + + name property, without doing + variable expansion. + + @param name the property name. + @return the value of the name property, + or null if no such property exists.]]> + + + + + + + value of the name property. + + @param name property name. + @param value property value.]]> + + + + + + + + + + + + + + name property. If no such property + exists, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value, or defaultValue if the property + doesn't exist.]]> + + + + + + + name property as an int. + + If no such property exists, or if the specified value is not a valid + int, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as an int, + or defaultValue.]]> + + + + + + + name property to an int. + + @param name property name. + @param value int value of the property.]]> + + + + + + + name property as a long. + If no such property is specified, or if the specified value is not a valid + long, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as a long, + or defaultValue.]]> + + + + + + + name property to a long. + + @param name property name. + @param value long value of the property.]]> + + + + + + + name property as a float. + If no such property is specified, or if the specified value is not a valid + float, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as a float, + or defaultValue.]]> + + + + + + + name property to a float. + + @param name property name. + @param value property value.]]> + + + + + + + name property as a boolean. + If no such property is specified, or if the specified value is not a valid + boolean, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as a boolean, + or defaultValue.]]> + + + + + + + name property to a boolean. + + @param name property name. + @param value boolean value of the property.]]> + + + + + + + + + + + + + + + + + + + + name property as + a collection of Strings. + If no such property is specified then empty collection is returned. +

    + This is an optimized version of {@link #getStrings(String)} + + @param name property name. + @return property value as a collection of Strings.]]> + + + + + + name property as + an array of Strings. + If no such property is specified then null is returned. + + @param name property name. + @return property value as an array of Strings, + or null.]]> + + + + + + + name property as + an array of Strings. + If no such property is specified then default value is returned. + + @param name property name. + @param defaultValue The default value + @return property value as an array of Strings, + or default value.]]> + + + + + + + name property as + as comma delimited values. + + @param name property name. + @param values The values]]> + + + + + + + + + + + + + + name property + as an array of Class. + The value of the property specifies a list of comma separated class names. + If no such property is specified, then defaultValue is + returned. + + @param name the property name. + @param defaultValue default value. + @return property value as a Class[], + or defaultValue.]]> + + + + + + + name property as a Class. + If no such property is specified, then defaultValue is + returned. + + @param name the class name. + @param defaultValue default value. + @return property value as a Class, + or defaultValue.]]> + + + + + + + + name property as a Class + implementing the interface specified by xface. + + If no such property is specified, then defaultValue is + returned. + + An exception is thrown if the returned class does not implement the named + interface. + + @param name the class name. + @param defaultValue default value. + @param xface the interface implemented by the named class. + @return property value as a Class, + or defaultValue.]]> + + + + + + + + name property to the name of a + theClass implementing the given interface xface. + + An exception is thrown if theClass does not implement the + interface xface. + + @param name property name. + @param theClass property value. + @param xface the interface implemented by the named class.]]> + + + + + + + + dirsProp with + the given path. If dirsProp contains multiple directories, + then one is chosen based on path's hash code. If the selected + directory does not exist, an attempt is made to create it. + + @param dirsProp directory in which to locate the file. + @param path file-path. + @return local file under the directory with the given path.]]> + + + + + + + + dirsProp with + the given path. If dirsProp contains multiple directories, + then one is chosen based on path's hash code. If the selected + directory does not exist, an attempt is made to create it. + + @param dirsProp directory in which to locate the file. + @param path file-path. + @return local file under the directory with the given path.]]> + + + + + + + + + + + + name. + + @param name configuration resource name. + @return an input stream attached to the resource.]]> + + + + + + name. + + @param name configuration resource name. + @return a reader attached to the resource.]]> + + + + + + + + + + + + + + + String + key-value pairs in the configuration. + + @return an iterator over the entries.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + true to set quiet-mode on, false + to turn it off.]]> + + + + + + + + + + + + + + + + + + + Resources + +

    Configurations are specified by resources. A resource contains a set of + name/value pairs as XML data. Each resource is named by either a + String or by a {@link Path}. If named by a String, + then the classpath is examined for a file with that name. If named by a + Path, then the local filesystem is examined directly, without + referring to the classpath. + +

    Unless explicitly turned off, Hadoop by default specifies two + resources, loaded in-order from the classpath:

      +
    1. core-default.xml + : Read-only defaults for hadoop.
    2. +
    3. core-site.xml: Site-specific configuration for a given hadoop + installation.
    4. +
    + Applications may add additional resources, which are loaded + subsequent to these resources in the order they are added. + +

    Final Parameters

    + +

    Configuration parameters may be declared final. + Once a resource declares a value final, no subsequently-loaded + resource can alter that value. + For example, one might define a final parameter with: +

    +  <property>
    +    <name>dfs.client.buffer.dir</name>
    +    <value>/tmp/hadoop/dfs/client</value>
    +    <final>true</final>
    +  </property>
    + + Administrators typically define parameters as final in + core-site.xml for values that user applications may not alter. + +

    Variable Expansion

    + +

    Value strings are first processed for variable expansion. The + available properties are:

      +
    1. Other properties defined in this Configuration; and, if a name is + undefined here,
    2. +
    3. Properties in {@link System#getProperties()}.
    4. +
    + +

    For example, if a configuration resource contains the following property + definitions: +

    +  <property>
    +    <name>basedir</name>
    +    <value>/user/${user.name}</value>
    +  </property>
    +  
    +  <property>
    +    <name>tempdir</name>
    +    <value>${basedir}/tmp</value>
    +  </property>
    + + When conf.get("tempdir") is called, then ${basedir} + will be resolved to another property in this Configuration, while + ${user.name} would then ordinarily be resolved to the value + of the System property with that name.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DistributedCache is a facility provided by the Map-Reduce + framework to cache files (text, archives, jars etc.) needed by applications. +

    + +

    Applications specify the files, via urls (hdfs:// or http://) to be cached + via the {@link org.apache.hadoop.mapred.JobConf}. + The DistributedCache assumes that the + files specified via hdfs:// urls are already present on the + {@link FileSystem} at the path specified by the url.

    + +

    The framework will copy the necessary files on to the slave node before + any tasks for the job are executed on that node. Its efficiency stems from + the fact that the files are only copied once per job and the ability to + cache archives which are un-archived on the slaves.

    + +

    DistributedCache can be used to distribute simple, read-only + data/text files and/or more complex types such as archives, jars etc. + Archives (zip, tar and tgz/tar.gz files) are un-archived at the slave nodes. + Jars may be optionally added to the classpath of the tasks, a rudimentary + software distribution mechanism. Files have execution permissions. + Optionally users can also direct it to symlink the distributed cache file(s) + into the working directory of the task.

    + +

    DistributedCache tracks modification timestamps of the cache + files. Clearly the cache files should not be modified by the application + or externally while the job is executing.

    + +

    Here is an illustrative example on how to use the + DistributedCache:

    +

    +     // Setting up the cache for the application
    +     
    +     1. Copy the requisite files to the FileSystem:
    +     
    +     $ bin/hadoop fs -copyFromLocal lookup.dat /myapp/lookup.dat  
    +     $ bin/hadoop fs -copyFromLocal map.zip /myapp/map.zip  
    +     $ bin/hadoop fs -copyFromLocal mylib.jar /myapp/mylib.jar
    +     $ bin/hadoop fs -copyFromLocal mytar.tar /myapp/mytar.tar
    +     $ bin/hadoop fs -copyFromLocal mytgz.tgz /myapp/mytgz.tgz
    +     $ bin/hadoop fs -copyFromLocal mytargz.tar.gz /myapp/mytargz.tar.gz
    +     
    +     2. Setup the application's JobConf:
    +     
    +     JobConf job = new JobConf();
    +     DistributedCache.addCacheFile(new URI("/myapp/lookup.dat#lookup.dat"), 
    +                                   job);
    +     DistributedCache.addCacheArchive(new URI("/myapp/map.zip", job);
    +     DistributedCache.addFileToClassPath(new Path("/myapp/mylib.jar"), job);
    +     DistributedCache.addCacheArchive(new URI("/myapp/mytar.tar", job);
    +     DistributedCache.addCacheArchive(new URI("/myapp/mytgz.tgz", job);
    +     DistributedCache.addCacheArchive(new URI("/myapp/mytargz.tar.gz", job);
    +     
    +     3. Use the cached files in the {@link org.apache.hadoop.mapred.Mapper}
    +     or {@link org.apache.hadoop.mapred.Reducer}:
    +     
    +     public static class MapClass extends MapReduceBase  
    +     implements Mapper<K, V, K, V> {
    +     
    +       private Path[] localArchives;
    +       private Path[] localFiles;
    +       
    +       public void configure(JobConf job) {
    +         // Get the cached archives/files
    +         localArchives = DistributedCache.getLocalCacheArchives(job);
    +         localFiles = DistributedCache.getLocalCacheFiles(job);
    +       }
    +       
    +       public void map(K key, V value, 
    +                       OutputCollector<K, V> output, Reporter reporter) 
    +       throws IOException {
    +         // Use data from the cached archives/files here
    +         // ...
    +         // ...
    +         output.collect(k, v);
    +       }
    +     }
    +     
    + 

    + + @see org.apache.hadoop.mapred.JobConf + @see org.apache.hadoop.mapred.JobClient]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + BufferedFSInputStream + with the specified buffer size, + and saves its argument, the input stream + in, for later use. An internal + buffer array of length size + is created and stored in buf. + + @param in the underlying input stream. + @param size the buffer size. + @exception IllegalArgumentException if size <= 0.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + setReplication of FileSystem + @param src file name + @param replication new replication + @throws IOException + @return true if successful; + false if file does not exist or is a directory]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + fs.scheme.class whose value names the FileSystem class. + The entire URI is passed to the FileSystem instance's initialize method.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Return all the files that match filePattern and are not checksum + files. Results are sorted by their names. + +

    + A filename pattern is composed of regular characters and + special pattern matching characters, which are: + +

    +
    +
    +

    +

    ? +
    Matches any single character. + +

    +

    * +
    Matches zero or more characters. + +

    +

    [abc] +
    Matches a single character from character set + {a,b,c}. + +

    +

    [a-b] +
    Matches a single character from the character range + {a...b}. Note that character a must be + lexicographically less than or equal to character b. + +

    +

    [^a] +
    Matches a single character that is not from character set or range + {a}. Note that the ^ character must occur + immediately to the right of the opening bracket. + +

    +

    \c +
    Removes (escapes) any special meaning of character c. + +

    +

    {ab,cd} +
    Matches a string from the string set {ab, cd} + +

    +

    {ab,c{de,fh}} +
    Matches a string from the string set {ab, cde, cfh} + +
    +
    +
    + + @param pathPattern a regular expression specifying a pth pattern + + @return an array of paths that match the path pattern + @throws IOException]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + All user code that may potentially use the Hadoop Distributed + File System should be written to use a FileSystem object. The + Hadoop DFS is a multi-machine system that appears as a single + disk. It's useful because of its fault tolerance and potentially + very large capacity. + +

    + The local implementation is {@link LocalFileSystem} and distributed + implementation is DistributedFileSystem.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FilterFileSystem contains + some other file system, which it uses as + its basic file system, possibly transforming + the data along the way or providing additional + functionality. The class FilterFileSystem + itself simply overrides all methods of + FileSystem with versions that + pass all requests to the contained file + system. Subclasses of FilterFileSystem + may further override some of these methods + and may also provide additional methods + and fields.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + buf at offset + and checksum into checksum. + The method is used for implementing read, therefore, it should be optimized + for sequential reading + @param pos chunkPos + @param buf desitination buffer + @param offset offset in buf at which to store data + @param len maximun number of bytes to read + @return number of bytes read]]> + + + + + + + + + + + + + + + + + -1 if the end of the + stream is reached. + @exception IOException if an I/O error occurs.]]> + + + + + + + + + This method implements the general contract of the corresponding + {@link InputStream#read(byte[], int, int) read} method of + the {@link InputStream} class. As an additional + convenience, it attempts to read as many bytes as possible by repeatedly + invoking the read method of the underlying stream. This + iterated read continues until one of the following + conditions becomes true:

      + +
    • The specified number of bytes have been read, + +
    • The read method of the underlying stream returns + -1, indicating end-of-file. + +
    If the first read on the underlying stream returns + -1 to indicate end-of-file then this method returns + -1. Otherwise this method returns the number of bytes + actually read. + + @param b destination buffer. + @param off offset at which to start storing bytes. + @param len maximum number of bytes to read. + @return the number of bytes read, or -1 if the end of + the stream has been reached. + @exception IOException if an I/O error occurs. + ChecksumException if any checksum error occurs]]> +
    + + + + + + + + + + + + + + + + + + n bytes of data from the + input stream. + +

    This method may skip more bytes than are remaining in the backing + file. This produces no exception and the number of bytes skipped + may include some number of bytes that were beyond the EOF of the + backing file. Attempting to read from the stream after skipping past + the end will result in -1 indicating the end of the file. + +

    If n is negative, no bytes are skipped. + + @param n the number of bytes to be skipped. + @return the actual number of bytes skipped. + @exception IOException if an I/O error occurs. + ChecksumException if the chunk to skip to is corrupted]]> + + + + + + + This method may seek past the end of the file. + This produces no exception and an attempt to read from + the stream will result in -1 indicating the end of the file. + + @param pos the postion to seek to. + @exception IOException if an I/O error occurs. + ChecksumException if the chunk to seek to is corrupted]]> + + + + + + + + + + len bytes from + stm + + @param stm an input stream + @param buf destiniation buffer + @param offset offset at which to store data + @param len number of bytes to read + @return actual number of bytes read + @throws IOException if there is any IO error]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + len bytes from the specified byte array + starting at offset off and generate a checksum for + each data chunk. + +

    This method stores bytes from the given array into this + stream's buffer before it gets checksumed. The buffer gets checksumed + and flushed to the underlying output stream when all data + in a checksum chunk are in the buffer. If the buffer is empty and + requested length is at least as large as the size of next checksum chunk + size, this method will checksum and write the chunk directly + to the underlying output stream. Thus it avoids uneccessary data copy. + + @param b the data. + @param off the start offset in the data. + @param len the number of bytes to write. + @exception IOException if an I/O error occurs.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if and only if pathname + should be included]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + trash feature. Files are moved to a user's trash + directory, a subdirectory of their home directory named ".Trash". Files are + initially moved to a current sub-directory of the trash directory. + Within that sub-directory their original path is preserved. Periodically + one may checkpoint the current trash and remove older checkpoints. (This + design permits trash management without enumeration of the full trash + content, without date support in the filesystem, and without clock + synchronization.)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A {@link FileSystem} backed by an FTP client provided by Apache Commons Net. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (cause==null ? null : cause.toString()) (which + typically contains the class and detail message of cause). + @param cause the cause (which is saved for later retrieval by the + {@link #getCause()} method). (A null value is + permitted, and indicates that the cause is nonexistent or + unknown.)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class is a tool for migrating data from an older to a newer version + of an S3 filesystem. +

    +

    + All files in the filesystem are migrated by re-writing the block metadata + - no datafiles are touched. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + Extracts AWS credentials from the filesystem URI or configuration. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A block-based {@link FileSystem} backed by + Amazon S3. +

    + @see NativeS3FileSystem]]> +
    +
    + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + If f is a file, this method will make a single call to S3. + If f is a directory, this method will make a maximum of + (n / 1000) + 2 calls to S3, where n is the total number of + files and directories contained directly in f. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + A {@link FileSystem} for reading and writing files stored on + Amazon S3. + Unlike {@link org.apache.hadoop.fs.s3.S3FileSystem} this implementation + stores files on S3 in their + native form so they can be read by other S3 tools. +

    + @see org.apache.hadoop.fs.s3.S3FileSystem]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + . + @param name The name of the server + @param port The port to use on the server + @param findPort whether the server should start at the given port and + increment by 1 until it finds a free port. + @param conf Configuration]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + points to the log directory + "/static/" -> points to common static files (src/webapps/static) + "/" -> the jsp server code from (src/webapps/)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + nth value.]]> + + + + + + + + + + + + + + + + + + + + + nth value in the file.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + public class IntArrayWritable extends ArrayWritable { + public IntArrayWritable() { + super(IntWritable.class); + } + } + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a ByteWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This saves memory over creating a new DataInputStream and + ByteArrayInputStream each time data is read. + +

    Typical usage is something like the following:

    +
    + DataInputBuffer buffer = new DataInputBuffer();
    + while (... loop condition ...) {
    +   byte[] data = ... get data ...;
    +   int dataLength = ... get data length ...;
    +   buffer.reset(data, dataLength);
    +   ... read buffer using DataInput methods ...
    + }
    + 
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This saves memory over creating a new DataOutputStream and + ByteArrayOutputStream each time data is written. + +

    Typical usage is something like the following:

    +
    + DataOutputBuffer buffer = new DataOutputBuffer();
    + while (... loop condition ...) {
    +   buffer.reset();
    +   ... write buffer using DataOutput methods ...
    +   byte[] data = buffer.getData();
    +   int dataLength = buffer.getLength();
    +   ... write data to its ultimate destination ...
    + }
    + 
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + the class of the item + @param conf the configuration to store + @param item the object to be stored + @param keyName the name of the key to use + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param keyName the name of the key to use + @param itemClass the class of the item + @return restored object + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param items the objects to be stored + @param keyName the name of the key to use + @throws IndexOutOfBoundsException if the items array is empty + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param keyName the name of the key to use + @param itemClass the class of the item + @return restored object + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + DefaultStringifier offers convenience methods to store/load objects to/from + the configuration. + + @param the class of the objects to stringify]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a DoubleWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a FloatWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + When two sequence files, which have same Key type but different Value + types, are mapped out to reduce, multiple Value types is not allowed. + In this case, this class can help you wrap instances with different types. +

    + +

    + Compared with ObjectWritable, this class is much more effective, + because ObjectWritable will append the class declaration as a String + into the output file in every Key-Value pair. +

    + +

    + Generic Writable implements {@link Configurable} interface, so that it will be + configured by the framework. The configuration is passed to the wrapped objects + implementing {@link Configurable} interface before deserialization. +

    + + how to use it:
    + 1. Write your own class, such as GenericObject, which extends GenericWritable.
    + 2. Implements the abstract method getTypes(), defines + the classes which will be wrapped in GenericObject in application. + Attention: this classes defined in getTypes() method, must + implement Writable interface. +

    + + The code looks like this: +
    + public class GenericObject extends GenericWritable {
    + 
    +   private static Class[] CLASSES = {
    +               ClassType1.class, 
    +               ClassType2.class,
    +               ClassType3.class,
    +               };
    +
    +   protected Class[] getTypes() {
    +       return CLASSES;
    +   }
    +
    + }
    + 
    + + @since Nov 8, 2006]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This saves memory over creating a new InputStream and + ByteArrayInputStream each time data is read. + +

    Typical usage is something like the following:

    +
    + InputBuffer buffer = new InputBuffer();
    + while (... loop condition ...) {
    +   byte[] data = ... get data ...;
    +   int dataLength = ... get data length ...;
    +   buffer.reset(data, dataLength);
    +   ... read buffer using InputStream methods ...
    + }
    + 
    + @see DataInputBuffer + @see DataOutput]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a IntWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + closes the input and output streams + at the end. + @param in InputStrem to read from + @param out OutputStream to write to + @param conf the Configuration object]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ignore any {@link IOException} or + null pointers. Must only be used for cleanup in exception handlers. + @param log the log to record problems to at debug level. Can be null. + @param closeables the objects to close]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a LongWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A map is a directory containing two files, the data file, + containing all keys and values in the map, and a smaller index + file, containing a fraction of the keys. The fraction is determined by + {@link Writer#getIndexInterval()}. + +

    The index file is read entirely into memory. Thus key implementations + should try to keep themselves small. + +

    Map files are created by adding entries in-order. To maintain a large + database, perform updates by copying the previous version of a database and + merging in a sorted change list, to create a new version of the database in + a new file. Sorting large change lists can be done with {@link + SequenceFile.Sorter}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + key and + val. Returns true if such a pair exists and false when at + the end of the map]]> + + + + + + + + + + + + + + + + key or if it does not exist, at the first entry + after the named key. + +- * @param key - key that we're trying to find +- * @param val - data value if key is found +- * @return - the key that was the closest match or null if eof.]]> + + + + + + + + + key does not exist, return + the first entry that falls just before the key. Otherwise, + return the record that sorts just after. + @return - the key that was the closest match or null if eof.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is an MD5Hash whose digest contains the + same values.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This saves memory over creating a new OutputStream and + ByteArrayOutputStream each time data is written. + +

    Typical usage is something like the following:

    +
    + OutputBuffer buffer = new OutputBuffer();
    + while (... loop condition ...) {
    +   buffer.reset();
    +   ... write buffer using OutputStream methods ...
    +   byte[] data = buffer.getData();
    +   int dataLength = buffer.getLength();
    +   ... write data to its ultimate destination ...
    + }
    + 
    + @see DataOutputBuffer + @see InputBuffer]]> +
    +
    + + + + + + + + + + + + + + + A {@link Comparator} that operates directly on byte representations of + objects. +

    + @param + @see DeserializerComparator]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SequenceFiles are flat files consisting of binary key/value + pairs. + +

    SequenceFile provides {@link Writer}, {@link Reader} and + {@link Sorter} classes for writing, reading and sorting respectively.

    + + There are three SequenceFile Writers based on the + {@link CompressionType} used to compress key/value pairs: +
      +
    1. + Writer : Uncompressed records. +
    2. +
    3. + RecordCompressWriter : Record-compressed files, only compress + values. +
    4. +
    5. + BlockCompressWriter : Block-compressed files, both keys & + values are collected in 'blocks' + separately and compressed. The size of + the 'block' is configurable. +
    + +

    The actual compression algorithm used to compress key and/or values can be + specified by using the appropriate {@link CompressionCodec}.

    + +

    The recommended way is to use the static createWriter methods + provided by the SequenceFile to chose the preferred format.

    + +

    The {@link Reader} acts as the bridge and can read any of the above + SequenceFile formats.

    + +

    SequenceFile Formats

    + +

    Essentially there are 3 different formats for SequenceFiles + depending on the CompressionType specified. All of them share a + common header described below. + +

    +
      +
    • + version - 3 bytes of magic header SEQ, followed by 1 byte of actual + version number (e.g. SEQ4 or SEQ6) +
    • +
    • + keyClassName -key class +
    • +
    • + valueClassName - value class +
    • +
    • + compression - A boolean which specifies if compression is turned on for + keys/values in this file. +
    • +
    • + blockCompression - A boolean which specifies if block-compression is + turned on for keys/values in this file. +
    • +
    • + compression codec - CompressionCodec class which is used for + compression of keys and/or values (if compression is + enabled). +
    • +
    • + metadata - {@link Metadata} for this file. +
    • +
    • + sync - A sync marker to denote end of the header. +
    • +
    + +
    Uncompressed SequenceFile Format
    +
      +
    • + Header +
    • +
    • + Record +
        +
      • Record length
      • +
      • Key length
      • +
      • Key
      • +
      • Value
      • +
      +
    • +
    • + A sync-marker every few 100 bytes or so. +
    • +
    + +
    Record-Compressed SequenceFile Format
    +
      +
    • + Header +
    • +
    • + Record +
        +
      • Record length
      • +
      • Key length
      • +
      • Key
      • +
      • Compressed Value
      • +
      +
    • +
    • + A sync-marker every few 100 bytes or so. +
    • +
    + +
    Block-Compressed SequenceFile Format
    +
      +
    • + Header +
    • +
    • + Record Block +
        +
      • Compressed key-lengths block-size
      • +
      • Compressed key-lengths block
      • +
      • Compressed keys block-size
      • +
      • Compressed keys block
      • +
      • Compressed value-lengths block-size
      • +
      • Compressed value-lengths block
      • +
      • Compressed values block-size
      • +
      • Compressed values block
      • +
      +
    • +
    • + A sync-marker every few 100 bytes or so. +
    • +
    + +

    The compressed blocks of key lengths and value lengths consist of the + actual lengths of individual keys/values encoded in ZeroCompressedInteger + format.

    + + @see CompressionCodec]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + key, skipping its + value. True if another entry exists, and false at end of file.]]> + + + + + + + + key and + val. Returns true if such a pair exists and false when at + end of file]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The position passed must be a position returned by {@link + SequenceFile.Writer#getLength()} when writing this file. To seek to an arbitrary + position, use {@link SequenceFile.Reader#sync(long)}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SegmentDescriptor + @param segments the list of SegmentDescriptors + @param tmpDir the directory to write temporary files into + @return RawKeyValueIterator + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + For best performance, applications should make sure that the {@link + Writable#readFields(DataInput)} implementation of their keys is + very efficient. In particular, it should avoid allocating memory.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This always returns a synchronized position. In other words, + immediately after calling {@link SequenceFile.Reader#seek(long)} with a position + returned by this method, {@link SequenceFile.Reader#next(Writable)} may be called. However + the key may be earlier in the file than key last written when this + method was called (e.g., with block-compression, it may be the first key + in the block that was being written when this method was called).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + key. Returns + true if such a key exists and false when at the end of the set.]]> + + + + + + + key. + Returns key, or null if no match exists.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the class of the objects to stringify]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + position. Note that this + method avoids using the converter or doing String instatiation + @return the Unicode scalar value at position or -1 + if the position is invalid or points to a + trailing byte]]> + + + + + + + + + + what in the backing + buffer, starting as position start. The starting + position is measured in bytes and the return value is in + terms of byte position in the buffer. The backing buffer is + not converted to a string for this operation. + @return byte position of the first occurence of the search + string in the UTF-8 buffer or -1 if not found]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a Text with the same contents.]]> + + + + + + + + + + + + + + + + + + + + + + + + + replace is true, then + malformed input is replaced with the + substitution character, which is U+FFFD. Otherwise the + method throws a MalformedInputException.]]> + + + + + + + + + + + + + + + replace is true, then + malformed input is replaced with the + substitution character, which is U+FFFD. Otherwise the + method throws a MalformedInputException. + @return ByteBuffer: bytes stores at ByteBuffer.array() + and length is ByteBuffer.limit()]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + In + addition, it provides methods for string traversal without converting the + byte array to a string.

    Also includes utilities for + serializing/deserialing a string, coding/decoding a string, checking if a + byte array contains valid UTF8 code, calculating the length of an encoded + string.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a UTF8 with the same contents.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + Also includes utilities for efficiently reading and writing UTF-8. + + @deprecated replaced by Text]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This is useful when a class may evolve, so that instances written by the + old version of the class may still be processed by the new version. To + handle this situation, {@link #readFields(DataInput)} + implementations should catch {@link VersionMismatchException}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a VIntWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a VLongWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + out. + + @param out DataOuput to serialize this object into. + @throws IOException]]> + + + + + + + in. + +

    For efficiency, implementations should attempt to re-use storage in the + existing object where possible.

    + + @param in DataInput to deseriablize this object from. + @throws IOException]]> +
    + + + Any key or value type in the Hadoop Map-Reduce + framework implements this interface.

    + +

    Implementations typically implement a static read(DataInput) + method which constructs a new instance, calls {@link #readFields(DataInput)} + and returns the instance.

    + +

    Example:

    +

    +     public class MyWritable implements Writable {
    +       // Some data     
    +       private int counter;
    +       private long timestamp;
    +       
    +       public void write(DataOutput out) throws IOException {
    +         out.writeInt(counter);
    +         out.writeLong(timestamp);
    +       }
    +       
    +       public void readFields(DataInput in) throws IOException {
    +         counter = in.readInt();
    +         timestamp = in.readLong();
    +       }
    +       
    +       public static MyWritable read(DataInput in) throws IOException {
    +         MyWritable w = new MyWritable();
    +         w.readFields(in);
    +         return w;
    +       }
    +     }
    + 

    ]]> +
    + + + + + + + + WritableComparables can be compared to each other, typically + via Comparators. Any type which is to be used as a + key in the Hadoop Map-Reduce framework should implement this + interface.

    + +

    Example:

    +

    +     public class MyWritableComparable implements WritableComparable {
    +       // Some data
    +       private int counter;
    +       private long timestamp;
    +       
    +       public void write(DataOutput out) throws IOException {
    +         out.writeInt(counter);
    +         out.writeLong(timestamp);
    +       }
    +       
    +       public void readFields(DataInput in) throws IOException {
    +         counter = in.readInt();
    +         timestamp = in.readLong();
    +       }
    +       
    +       public int compareTo(MyWritableComparable w) {
    +         int thisValue = this.value;
    +         int thatValue = ((IntWritable)o).value;
    +         return (thisValue < thatValue ? -1 : (thisValue==thatValue ? 0 : 1));
    +       }
    +     }
    + 

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The default implementation reads the data into two {@link + WritableComparable}s (using {@link + Writable#readFields(DataInput)}, then calls {@link + #compare(WritableComparable,WritableComparable)}.]]> + + + + + + + The default implementation uses the natural ordering, calling {@link + Comparable#compareTo(Object)}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This base implemenation uses the natural ordering. To define alternate + orderings, override {@link #compare(WritableComparable,WritableComparable)}. + +

    One may optimize compare-intensive operations by overriding + {@link #compare(byte[],int,int,byte[],int,int)}. Static utility methods are + provided to assist in optimized implementations of this method.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Enum type + @param in DataInput to read from + @param enumType Class type of Enum + @return Enum represented by String read from DataInput + @throws IOException]]> + + + + + + + + + + + + + + + + len number of bytes in input streamin + @param in input stream + @param len number of bytes to skip + @throws IOException when skipped less number of bytes]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CompressionCodec for which to get the + Compressor + @return Compressor for the given + CompressionCodec from the pool or a new one]]> + + + + + + CompressionCodec for which to get the + Decompressor + @return Decompressor for the given + CompressionCodec the pool or a new one]]> + + + + + + Compressor to be returned to the pool]]> + + + + + + Decompressor to be returned to the + pool]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Implementations are assumed to be buffered. This permits clients to + reposition the underlying input stream then call {@link #resetState()}, + without having to also synchronize client buffers.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true indicating that more input data is required. + + @param b Input data + @param off Start offset + @param len Length]]> + + + + + true if the input data buffer is empty and + #setInput() should be called in order to provide more input.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the end of the compressed + data output stream has been reached.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true indicating that more input data is required. + + @param b Input data + @param off Start offset + @param len Length]]> + + + + + true if the input data buffer is empty and + #setInput() should be called in order to provide more input.]]> + + + + + + + + + + + + + true if a preset dictionary is needed for decompression. + @return true if a preset dictionary is needed for decompression]]> + + + + + true if the end of the compressed + data output stream has been reached.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FIXME: This array should be in a private or package private location, + since it could be modified by malicious code. +

    ]]> +
    + + + + This interface is public for historical purposes. You should have no need to + use it. +

    ]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Although BZip2 headers are marked with the magic "Bz" this + constructor expects the next byte in the stream to be the first one after + the magic. Thus callers have to skip the first two bytes. Otherwise this + constructor will throw an exception. +

    + + @throws IOException + if the stream content is malformed or an I/O error occurs. + @throws NullPointerException + if in == null]]> +
    +
    + + + + + + + + + + + + + + + The decompression requires large amounts of memory. Thus you should call the + {@link #close() close()} method as soon as possible, to force + CBZip2InputStream to release the allocated memory. See + {@link CBZip2OutputStream CBZip2OutputStream} for information about memory + usage. +

    + +

    + CBZip2InputStream reads bytes from the compressed source stream via + the single byte {@link java.io.InputStream#read() read()} method exclusively. + Thus you should consider to use a buffered source stream. +

    + +

    + Instances of this class are not threadsafe. +

    ]]> +
    +
    + + + + + + + + CBZip2OutputStream with a blocksize of 900k. + +

    + Attention: The caller is resonsible to write the two BZip2 magic + bytes "BZ" to the specified stream prior to calling this + constructor. +

    + + @param out * + the destination stream. + + @throws IOException + if an I/O error occurs in the specified stream. + @throws NullPointerException + if out == null.]]> +
    +
    + + + + CBZip2OutputStream with specified blocksize. + +

    + Attention: The caller is resonsible to write the two BZip2 magic + bytes "BZ" to the specified stream prior to calling this + constructor. +

    + + + @param out + the destination stream. + @param blockSize + the blockSize as 100k units. + + @throws IOException + if an I/O error occurs in the specified stream. + @throws IllegalArgumentException + if (blockSize < 1) || (blockSize > 9). + @throws NullPointerException + if out == null. + + @see #MIN_BLOCKSIZE + @see #MAX_BLOCKSIZE]]> +
    +
    + + + + + + + + + + + + + inputLength this method returns MAX_BLOCKSIZE + always. + + @param inputLength + The length of the data which will be compressed by + CBZip2OutputStream.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + == 1.]]> + + + + + == 9.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + If you are ever unlucky/improbable enough to get a stack overflow whilst + sorting, increase the following constant and try again. In practice I + have never seen the stack go above 27 elems, so the following limit seems + very generous. +

    ]]> +
    +
    + + + The compression requires large amounts of memory. Thus you should call the + {@link #close() close()} method as soon as possible, to force + CBZip2OutputStream to release the allocated memory. +

    + +

    + You can shrink the amount of allocated memory and maybe raise the compression + speed by choosing a lower blocksize, which in turn may cause a lower + compression ratio. You can avoid unnecessary memory allocation by avoiding + using a blocksize which is bigger than the size of the input. +

    + +

    + You can compute the memory usage for compressing by the following formula: +

    + +
    + <code>400k + (9 * blocksize)</code>.
    + 
    + +

    + To get the memory required for decompression by {@link CBZip2InputStream + CBZip2InputStream} use +

    + +
    + <code>65k + (5 * blocksize)</code>.
    + 
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Memory usage by blocksize
    Blocksize Compression
    + memory usage
    Decompression
    + memory usage
    100k1300k565k
    200k2200k1065k
    300k3100k1565k
    400k4000k2065k
    500k4900k2565k
    600k5800k3065k
    700k6700k3565k
    800k7600k4065k
    900k8500k4565k
    + +

    + For decompression CBZip2InputStream allocates less memory if the + bzipped input is smaller than one block. +

    + +

    + Instances of this class are not threadsafe. +

    + +

    + TODO: Update to BZip2 1.0.1 +

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @return the total (non-negative) number of uncompressed bytes input so far]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @return the total (non-negative) number of uncompressed bytes input so far]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if native-zlib is loaded & initialized + and can be loaded for this job, else false]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • "none" - No compression. +
  • "lzo" - LZO compression. +
  • "gz" - GZIP compression. + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • Block Compression. +
  • Named meta data blocks. +
  • Sorted or unsorted keys. +
  • Seek by key or by file offset. + + The memory footprint of a TFile includes the following: +
      +
    • Some constant overhead of reading or writing a compressed block. +
        +
      • Each compressed block requires one compression/decompression codec for + I/O. +
      • Temporary space to buffer the key. +
      • Temporary space to buffer the value (for TFile.Writer only). Values are + chunk encoded, so that we buffer at most one chunk of user data. By default, + the chunk buffer is 1MB. Reading chunked value does not require additional + memory. +
      +
    • TFile index, which is proportional to the total number of Data Blocks. + The total amount of memory needed to hold the index can be estimated as + (56+AvgKeySize)*NumBlocks. +
    • MetaBlock index, which is proportional to the total number of Meta + Blocks.The total amount of memory needed to hold the index for Meta Blocks + can be estimated as (40+AvgMetaBlockName)*NumMetaBlock. +
    +

    + The behavior of TFile can be customized by the following variables through + Configuration: +

      +
    • tfile.io.chunk.size: Value chunk size. Integer (in bytes). Default + to 1MB. Values of the length less than the chunk size is guaranteed to have + known value length in read time (See + {@link TFile.Reader.Scanner.Entry#isValueLengthKnown()}). +
    • tfile.fs.output.buffer.size: Buffer size used for + FSDataOutputStream. Integer (in bytes). Default to 256KB. +
    • tfile.fs.input.buffer.size: Buffer size used for + FSDataInputStream. Integer (in bytes). Default to 256KB. +
    +

    + Suggestions on performance optimization. +

      +
    • Minimum block size. We recommend a setting of minimum block size between + 256KB to 1MB for general usage. Larger block size is preferred if files are + primarily for sequential access. However, it would lead to inefficient random + access (because there are more data to decompress). Smaller blocks are good + for random access, but require more memory to hold the block index, and may + be slower to create (because we must flush the compressor stream at the + conclusion of each data block, which leads to an FS I/O flush). Further, due + to the internal caching in Compression codec, the smallest possible block + size would be around 20KB-30KB. +
    • The current implementation does not offer true multi-threading for + reading. The implementation uses FSDataInputStream seek()+read(), which is + shown to be much faster than positioned-read call in single thread mode. + However, it also means that if multiple threads attempt to access the same + TFile (using multiple scanners) simultaneously, the actual I/O is carried out + sequentially even if they access different DFS blocks. +
    • Compression codec. Use "none" if the data is not very compressable (by + compressable, I mean a compression ratio at least 2:1). Generally, use "lzo" + as the starting point for experimenting. "gz" overs slightly better + compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to + decompress, comparing to "lzo". +
    • File system buffering, if the underlying FSDataInputStream and + FSDataOutputStream is already adequately buffered; or if applications + reads/writes keys and values in large buffers, we can reduce the sizes of + input/output buffering in TFile layer by setting the configuration parameters + "tfile.fs.input.buffer.size" and "tfile.fs.output.buffer.size". +
    + + Some design rationale behind TFile can be found at Hadoop-3315.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + entry of the TFile. + @param endKey + End key of the scan. If null, scan up to the last entry + of the TFile. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Use {@link Scanner#atEnd()} to test whether the cursor is at the end + location of the scanner. +

    + Use {@link Scanner#advance()} to move the cursor to the next key-value + pair (or end if none exists). Use seekTo methods ( + {@link Scanner#seekTo(byte[])} or + {@link Scanner#seekTo(byte[], int, int)}) to seek to any arbitrary + location in the covered range (including backward seeking). Use + {@link Scanner#rewind()} to seek back to the beginning of the scanner. + Use {@link Scanner#seekToEnd()} to seek to the end of the scanner. +

    + Actual keys and values may be obtained through {@link Scanner.Entry} + object, which is obtained through {@link Scanner#entry()}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

  • Algorithmic comparator: binary comparators that is language + independent. Currently, only "memcmp" is supported. +
  • Language-specific comparator: binary comparators that can + only be constructed in specific language. For Java, the syntax + is "jclass:", followed by the class name of the RawComparator. + Currently, we only support RawComparators that can be + constructed through the default constructor (with no + parameters). Parameterized RawComparators such as + {@link WritableComparator} or + {@link JavaSerializationComparator} may not be directly used. + One should write a wrapper class that inherits from such classes + and use its default constructor to perform proper + initialization. + + @param conf + The configuration object. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + If an exception is thrown, the TFile will be in an inconsistent + state. The only legitimate call after that would be close]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Utils#writeVLong(out, n). + + @param out + output stream + @param n + The integer to be encoded + @throws IOException + @see Utils#writeVLong(DataOutput, long)]]> + + + + + + + + +
  • if n in [-32, 127): encode in one byte with the actual value. + Otherwise, +
  • if n in [-20*2^8, 20*2^8): encode in two bytes: byte[0] = n/256 - 52; + byte[1]=n&0xff. Otherwise, +
  • if n IN [-16*2^16, 16*2^16): encode in three bytes: byte[0]=n/2^16 - + 88; byte[1]=(n>>8)&0xff; byte[2]=n&0xff. Otherwise, +
  • if n in [-8*2^24, 8*2^24): encode in four bytes: byte[0]=n/2^24 - 112; + byte[1] = (n>>16)&0xff; byte[2] = (n>>8)&0xff; byte[3]=n&0xff. Otherwise: +
  • if n in [-2^31, 2^31): encode in five bytes: byte[0]=-125; byte[1] = + (n>>24)&0xff; byte[2]=(n>>16)&0xff; byte[3]=(n>>8)&0xff; byte[4]=n&0xff; +
  • if n in [-2^39, 2^39): encode in six bytes: byte[0]=-124; byte[1] = + (n>>32)&0xff; byte[2]=(n>>24)&0xff; byte[3]=(n>>16)&0xff; + byte[4]=(n>>8)&0xff; byte[5]=n&0xff +
  • if n in [-2^47, 2^47): encode in seven bytes: byte[0]=-123; byte[1] = + (n>>40)&0xff; byte[2]=(n>>32)&0xff; byte[3]=(n>>24)&0xff; + byte[4]=(n>>16)&0xff; byte[5]=(n>>8)&0xff; byte[6]=n&0xff; +
  • if n in [-2^55, 2^55): encode in eight bytes: byte[0]=-122; byte[1] = + (n>>48)&0xff; byte[2] = (n>>40)&0xff; byte[3]=(n>>32)&0xff; + byte[4]=(n>>24)&0xff; byte[5]=(n>>16)&0xff; byte[6]=(n>>8)&0xff; + byte[7]=n&0xff; +
  • if n in [-2^63, 2^63): encode in nine bytes: byte[0]=-121; byte[1] = + (n>>54)&0xff; byte[2] = (n>>48)&0xff; byte[3] = (n>>40)&0xff; + byte[4]=(n>>32)&0xff; byte[5]=(n>>24)&0xff; byte[6]=(n>>16)&0xff; + byte[7]=(n>>8)&0xff; byte[8]=n&0xff; + + + @param out + output stream + @param n + the integer number + @throws IOException]]> + + + + + + + (int)Utils#readVLong(in). + + @param in + input stream + @return the decoded integer + @throws IOException + + @see Utils#readVLong(DataInput)]]> + + + + + + + +
  • if (FB >= -32), return (long)FB; +
  • if (FB in [-72, -33]), return (FB+52)<<8 + NB[0]&0xff; +
  • if (FB in [-104, -73]), return (FB+88)<<16 + (NB[0]&0xff)<<8 + + NB[1]&0xff; +
  • if (FB in [-120, -105]), return (FB+112)<<24 + (NB[0]&0xff)<<16 + + (NB[1]&0xff)<<8 + NB[2]&0xff; +
  • if (FB in [-128, -121]), return interpret NB[FB+129] as a signed + big-endian integer. + + @param in + input stream + @return the decoded long integer. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @param cmp + Comparator for the key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @param cmp + Comparator for the key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Keep trying a limited number of times, waiting a fixed time between attempts, + and then fail by re-throwing the exception. +

    ]]> +
    +
    + + + + + + + Keep trying for a maximum time, waiting a fixed time between attempts, + and then fail by re-throwing the exception. +

    ]]> +
    +
    + + + + + + + Keep trying a limited number of times, waiting a growing amount of time between attempts, + and then fail by re-throwing the exception. + The time between attempts is sleepTime mutliplied by the number of tries so far. +

    ]]> +
    +
    + + + + + + + Keep trying a limited number of times, waiting a growing amount of time between attempts, + and then fail by re-throwing the exception. + The time between attempts is sleepTime mutliplied by a random + number in the range of [0, 2 to the number of retries) +

    ]]> +
    +
    + + + + + + Set a default policy with some explicit handlers for specific exceptions. +

    ]]> +
    +
    + + + + + + A retry policy for RemoteException + Set a default policy with some explicit handlers for specific exceptions. +

    ]]> +
    +
    + + + + Try once, and fail by re-throwing the exception. + This corresponds to having no retry mechanism in place. +

    ]]> +
    +
    + + + + Try once, and fail silently for void methods, or by + re-throwing the exception for non-void methods. +

    ]]> +
    +
    + + + + Keep trying forever. +

    ]]> +
    +
    + + + A collection of useful implementations of {@link RetryPolicy}. +

    ]]> +
    +
    + + + + + + + + + + Determines whether the framework should retry a + method for the given exception, and the number + of retries that have been made for that operation + so far. +

    + @param e The exception that caused the method to fail. + @param retries The number of times the method has been retried. + @return true if the method should be retried, + false if the method should not be retried + but shouldn't fail with an exception (only for void methods). + @throws Exception The re-thrown exception e indicating + that the method failed and should not be retried further.]]> +
    +
    + + + Specifies a policy for retrying method failures. + Implementations of this interface should be immutable. +

    ]]> +
    +
    + + + + + + + + + + + + Create a proxy for an interface of an implementation class + using the same retry policy for each method in the interface. +

    + @param iface the interface that the retry will implement + @param implementation the instance whose methods should be retried + @param retryPolicy the policy for retirying method call failures + @return the retry proxy]]> +
    +
    + + + + + + + Create a proxy for an interface of an implementation class + using the a set of retry policies specified by method name. + If no retry policy is defined for a method then a default of + {@link RetryPolicies#TRY_ONCE_THEN_FAIL} is used. +

    + @param iface the interface that the retry will implement + @param implementation the instance whose methods should be retried + @param methodNameToPolicyMap a map of method names to retry policies + @return the retry proxy]]> +
    +
    + + + A factory for creating retry proxies. +

    ]]> +
    +
    + +
    + + + + + + + + Prepare the deserializer for reading.

    ]]> +
    +
    + + + + + + Deserialize the next object from the underlying input stream. + If the object t is non-null then this deserializer + may set its internal state to the next object read from the input + stream. Otherwise, if the object t is null a new + deserialized object will be created. +

    + @return the deserialized object]]> +
    +
    + + + + Close the underlying input stream and clear up any resources.

    ]]> +
    +
    + + + Provides a facility for deserializing objects of type from an + {@link InputStream}. +

    + +

    + Deserializers are stateful, but must not buffer the input since + other producers may read from the input between calls to + {@link #deserialize(Object)}. +

    + @param ]]> +
    +
    + + + + + + + + + + + + + + + + + + A {@link RawComparator} that uses a {@link Deserializer} to deserialize + the objects to be compared so that the standard {@link Comparator} can + be used to compare them. +

    +

    + One may optimize compare-intensive operations by using a custom + implementation of {@link RawComparator} that operates directly + on byte representations. +

    + @param ]]> +
    +
    + + + + + + + + + + + + + + + + + + An experimental {@link Serialization} for Java {@link Serializable} classes. +

    + @see JavaSerializationComparator]]> +
    +
    + + + + + + + + + + + + + A {@link RawComparator} that uses a {@link JavaSerialization} + {@link Deserializer} to deserialize objects that are then compared via + their {@link Comparable} interfaces. +

    + @param + @see JavaSerialization]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + Encapsulates a {@link Serializer}/{@link Deserializer} pair. +

    + @param ]]> +
    +
    + + + + + + + Serializations are found by reading the io.serializations + property from conf, which is a comma-delimited list of + classnames. +

    ]]> +
    +
    + + + + + + + + + + + + A factory for {@link Serialization}s. +

    ]]> +
    +
    + + + + + + + + Prepare the serializer for writing.

    ]]> +
    +
    + + + + + Serialize t to the underlying output stream.

    ]]> +
    +
    + + + + Close the underlying output stream and clear up any resources.

    ]]> +
    +
    + + + Provides a facility for serializing objects of type to an + {@link OutputStream}. +

    + +

    + Serializers are stateful, but must not buffer the output since + other producers may write to the output between calls to + {@link #serialize(Object)}. +

    + @param ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + param, to the IPC server running at + address, returning the value. Throws exceptions if there are + network problems or if the remote code threw an exception. + @deprecated Use {@link #call(Writable, InetSocketAddress, Class, UserGroupInformation)} instead]]> + + + + + + + + + + param, to the IPC server running at + address with the ticket credentials, returning + the value. + Throws exceptions if there are network problems or if the remote code + threw an exception. + @deprecated Use {@link #call(Writable, InetSocketAddress, Class, UserGroupInformation)} instead]]> + + + + + + + + + + + param, to the IPC server running at + address which is servicing the protocol protocol, + with the ticket credentials, returning the value. + Throws exceptions if there are network problems or if the remote code + threw an exception.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Unwraps any IOException. + + @param lookupTypes the desired exception class. + @return IOException, which is either the lookupClass exception or this.]]> + + + + + This unwraps any Throwable that has a constructor taking + a String as a parameter. + Otherwise it returns this. + + @return Throwable]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + protocol is a Java interface. All parameters and return types must + be one of: + +
    • a primitive type, boolean, byte, + char, short, int, long, + float, double, or void; or
    • + +
    • a {@link String}; or
    • + +
    • a {@link Writable}; or
    • + +
    • an array of the above types
    + + All methods in the protocol should throw only IOException. No field data of + the protocol instance is transmitted.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + handlerCount determines + the number of handler threads that will be used to process calls.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + ,name=RpcActivityForPort" + + Many of the activity metrics are sampled and averaged on an interval + which can be specified in the metrics config file. +

    + For the metrics that are sampled and averaged, one must specify + a metrics context that does periodic update calls. Most metrics contexts do. + The default Null metrics context however does NOT. So if you aren't + using any other metrics context then you can turn on the viewing and averaging + of sampled metrics by specifying the following two lines + in the hadoop-meterics.properties file: +

    +        rpc.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
    +        rpc.period=10
    +  
    +

    + Note that the metrics are collected regardless of the context used. + The context with the update thread is used to average the data periodically + + + + Impl details: We use a dynamic mbean that gets the list of the metrics + from the metrics registry passed as an argument to the constructor]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class has a number of metrics variables that are publicly accessible; + these variables (objects) have methods to update their values; + for example: +

    {@link #rpcQueueTime}.inc(time)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + For the statistics that are sampled and averaged, one must specify + a metrics context that does periodic update calls. Most do. + The default Null metrics context however does NOT. So if you aren't + using any other metrics context then you can turn on the viewing and averaging + of sampled metrics by specifying the following two lines + in the hadoop-meterics.properties file: +

    +        rpc.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
    +        rpc.period=10
    +  
    +

    + Note that the metrics are collected regardless of the context used. + The context with the update thread is used to average the data periodically]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + When constructing the instance, if the factory property + contextName.class exists, + its value is taken to be the name of the class to instantiate. Otherwise, + the default is to create an instance of + org.apache.hadoop.metrics.spi.NullContext, which is a + dummy "no-op" context which will cause all metric data to be discarded. + + @param contextName the name of the context + @return the named MetricsContext]]> + + + + + + + + + + + + + + + + + + + + + When the instance is constructed, this method checks if the file + hadoop-metrics.properties exists on the class path. If it + exists, it must be in the format defined by java.util.Properties, and all + the properties in the file are set as attributes on the newly created + ContextFactory instance. + + @return the singleton ContextFactory instance]]> + + + + getFactory() method.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + startMonitoring() again after calling + this. + @see #close()]]> + + + + + + + + + + + + + + + + recordName. + Throws an exception if the metrics implementation is configured with a fixed + set of record names and recordName is not in that set. + + @param recordName the name of the record + @throws MetricsException if recordName conflicts with configuration data]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A record name identifies the kind of data to be reported. For example, a + program reporting statistics relating to the disks on a computer might use + a record name "diskStats".

    + + A record has zero or more tags. A tag has a name and a value. To + continue the example, the "diskStats" record might use a tag named + "diskName" to identify a particular disk. Sometimes it is useful to have + more than one tag, so there might also be a "diskType" with value "ide" or + "scsi" or whatever.

    + + A record also has zero or more metrics. These are the named + values that are to be reported to the metrics system. In the "diskStats" + example, possible metric names would be "diskPercentFull", "diskPercentBusy", + "kbReadPerSecond", etc.

    + + The general procedure for using a MetricsRecord is to fill in its tag and + metric values, and then call update() to pass the record to the + client library. + Metric data is not immediately sent to the metrics system + each time that update() is called. + An internal table is maintained, identified by the record name. This + table has columns + corresponding to the tag and the metric names, and rows + corresponding to each unique set of tag values. An update + either modifies an existing row in the table, or adds a new row with a set of + tag values that are different from all the other rows. Note that if there + are no tags, then there can be at most one row in the table.

    + + Once a row is added to the table, its data will be sent to the metrics system + on every timer period, whether or not it has been updated since the previous + timer period. If this is inappropriate, for example if metrics were being + reported by some transient object in an application, the remove() + method can be used to remove the row and thus stop the data from being + sent.

    + + Note that the update() method is atomic. This means that it is + safe for different threads to be updating the same metric. More precisely, + it is OK for different threads to call update() on MetricsRecord instances + with the same set of tag names and tag values. Different threads should + not use the same MetricsRecord instance at the same time.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + MetricsContext.registerUpdater().]]> + + + + + + + + + + + + + + + + + + + + + + + + + fileName attribute, + if specified. Otherwise the data will be written to standard + output.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class is configured by setting ContextFactory attributes which in turn + are usually configured through a properties file. All the attributes are + prefixed by the contextName. For example, the properties file might contain: +

    + myContextName.fileName=/tmp/metrics.log
    + myContextName.period=5
    + 
    ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + contextName.tableName. The returned map consists of + those attributes with the contextName and tableName stripped off.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + recordName. + Throws an exception if the metrics implementation is configured with a fixed + set of record names and recordName is not in that set. + + @param recordName the name of the record + @throws MetricsException if recordName conflicts with configuration data]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class implements the internal table of metric data, and the timer + on which data is to be sent to the metrics system. Subclasses must + override the abstract emitRecord method in order to transmit + the data.

    ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + update + and remove().]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hostname or hostname:port. If + the specs string is null, defaults to localhost:defaultPort. + + @return a list of InetSocketAddress objects.]]> + + + + + + + + + + + + + + + + + + + ,name=" + Where the and are the supplied parameters + + @param serviceName + @param nameName + @param theMbean - the MBean to register + @return the named used to register the MBean]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hadoop.rpc.socket.factory.class.<ClassName>. When no + such parameter exists then fall back on the default socket factory as + configured by hadoop.rpc.socket.factory.class.default. If + this default socket factory is not configured, then fall back on the JVM + default socket factory. + + @param conf the configuration + @param clazz the class (usually a {@link VersionedProtocol}) + @return a socket factory]]> + + + + + + hadoop.rpc.socket.factory.default + + @param conf the configuration + @return the default socket factory as specified in the configuration or + the JVM default socket factory if the configuration does not + contain a default socket factory property.]]> + + + + + + + + + + + + + : + ://:/]]> + + + + + + + + : + ://:/]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + From documentation for {@link #getInputStream(Socket, long)}:
    + Returns InputStream for the socket. If the socket has an associated + SocketChannel then it returns a + {@link SocketInputStream} with the given timeout. If the socket does not + have a channel, {@link Socket#getInputStream()} is returned. In the later + case, the timeout argument is ignored and the timeout set with + {@link Socket#setSoTimeout(int)} applies for reads.

    + + Any socket created using socket factories returned by {@link #NetUtils}, + must use this interface instead of {@link Socket#getInputStream()}. + + @see #getInputStream(Socket, long) + + @param socket + @return InputStream for reading from the socket. + @throws IOException]]> +
    +
    + + + + + +
    + + Any socket created using socket factories returned by {@link #NetUtils}, + must use this interface instead of {@link Socket#getInputStream()}. + + @see Socket#getChannel() + + @param socket + @param timeout timeout in milliseconds. This may not always apply. zero + for waiting as long as necessary. + @return InputStream for reading from the socket. + @throws IOException]]> +
    +
    + + + + +
    + + From documentation for {@link #getOutputStream(Socket, long)} :
    + Returns OutputStream for the socket. If the socket has an associated + SocketChannel then it returns a + {@link SocketOutputStream} with the given timeout. If the socket does not + have a channel, {@link Socket#getOutputStream()} is returned. In the later + case, the timeout argument is ignored and the write will wait until + data is available.

    + + Any socket created using socket factories returned by {@link #NetUtils}, + must use this interface instead of {@link Socket#getOutputStream()}. + + @see #getOutputStream(Socket, long) + + @param socket + @return OutputStream for writing to the socket. + @throws IOException]]> +
    +
    + + + + + +
    + + Any socket created using socket factories returned by {@link #NetUtils}, + must use this interface instead of {@link Socket#getOutputStream()}. + + @see Socket#getChannel() + + @param socket + @param timeout timeout in milliseconds. This may not always apply. zero + for waiting as long as necessary. + @return OutputStream for writing to the socket. + @throws IOException]]> +
    +
    + + + + + + + socket.connect(endpoint, timeout). If + socket.getChannel() returns a non-null channel, + connect is implemented using Hadoop's selectors. This is done mainly + to avoid Sun's connect implementation from creating thread-local + selectors, since Hadoop does not have control on when these are closed + and could end up taking all the available file descriptors. + + @see java.net.Socket#connect(java.net.SocketAddress, int) + + @param socket + @param endpoint + @param timeout - timeout in milliseconds]]> + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + node + + @param node + a node + @return true if node is already in the tree; false otherwise]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + scope + if scope starts with ~, choose one from the all nodes except for the + ones in scope; otherwise, choose one from scope + @param scope range of nodes from which a node will be choosen + @return the choosen node]]> + + + + + + + scope but not in excludedNodes + if scope starts with ~, return the number of nodes that are not + in scope and excludedNodes; + @param scope a path string that may start with ~ + @param excludedNodes a list of nodes + @return number of available nodes]]> + + + + + + + + + + + + reader + It linearly scans the array, if a local node is found, swap it with + the first element of the array. + If a local rack node is found, swap it with the first element following + the local node. + If neither local node or local rack node is found, put a random replica + location at postion 0. + It leaves the rest nodes untouched.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + Create a new input stream with the given timeout. If the timeout + is zero, it will be treated as infinite timeout. The socket's + channel will be configured to be non-blocking. + + @see SocketInputStream#SocketInputStream(ReadableByteChannel, long) + + @param socket should have a channel associated with it. + @param timeout timeout timeout in milliseconds. must not be negative. + @throws IOException]]> +
    +
    + + + +
    + + Create a new input stream with the given timeout. If the timeout + is zero, it will be treated as infinite timeout. The socket's + channel will be configured to be non-blocking. + @see SocketInputStream#SocketInputStream(ReadableByteChannel, long) + + @param socket should have a channel associated with it. + @throws IOException]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + +
    + + Create a new ouput stream with the given timeout. If the timeout + is zero, it will be treated as infinite timeout. The socket's + channel will be configured to be non-blocking. + + @see SocketOutputStream#SocketOutputStream(WritableByteChannel, long) + + @param socket should have a channel associated with it. + @param timeout timeout timeout in milliseconds. must not be negative. + @throws IOException]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + = getCount(). + @param newCapacity The new capacity in bytes.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Index idx = startVector(...); + while (!idx.done()) { + .... // read element of a vector + idx.incr(); + } + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This task takes the given record definition files and compiles them into + java or c++ + files. It is then up to the user to compile the generated files. + +

    The task requires the file or the nested fileset element to be + specified. Optional attributes are language (set the output + language, default is "java"), + destdir (name of the destination directory for generated java/c++ + code, default is ".") and failonerror (specifies error handling + behavior. default is true). +

    Usage

    +
    + <recordcc
    +       destdir="${basedir}/gensrc"
    +       language="java">
    +   <fileset include="**\/*.jr" />
    + </recordcc>
    + 
    ]]> +
    +
    + +
    + + + + + + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (cause==null ? null : cause.toString()) (which + typically contains the class and detail message of cause). + @param cause the cause (which is saved for later retrieval by the + {@link #getCause()} method). (A null value is + permitted, and indicates that the cause is nonexistent or + unknown.)]]> + + + + + + + + + + + + + Group with the given groupname. + @param group group name]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ugi. + @param ugi user + @return the {@link Subject} for the user identified by ugi]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ugi as a comma separated string in + conf as a property attr + + The String starts with the user name followed by the default group names, + and other group names. + + @param conf configuration + @param attr property name + @param ugi a UnixUserGroupInformation]]> + + + + + + + + conf + + The object is expected to store with the property name attr + as a comma separated string that starts + with the user name followed by group names. + If the property name is not defined, return null. + It's assumed that there is only one UGI per user. If this user already + has a UGI in the ugi map, return the ugi in the map. + Otherwise, construct a UGI from the configuration, store it in the + ugi map and return it. + + @param conf configuration + @param attr property name + @return a UnixUGI + @throws LoginException if the stored string is ill-formatted.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + User with the given username. + @param user user name]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (cause==null ? null : cause.toString()) (which + typically contains the class and detail message of cause). + @param cause the cause (which is saved for later retrieval by the + {@link #getCause()} method). (A null value is + permitted, and indicates that the cause is nonexistent or + unknown.)]]> + + + + + + + + + + + + + + does not provide the stack trace for security purposes.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + service as related to + Service Level Authorization for Hadoop. + + Each service defines it's configuration key and also the necessary + {@link Permission} required to access the service.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + in]]> + + + + + + + out.]]> + + + + + + + + + + reset is true, then resets the checksum. + @return number of bytes written. Will be equal to getChecksumSize();]]> + + + + + + + + + reset is true, then resets the checksum. + @return number of bytes written. Will be equal to getChecksumSize();]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + GenericOptionsParser to parse only the generic Hadoop + arguments. + + The array of string arguments other than the generic arguments can be + obtained by {@link #getRemainingArgs()}. + + @param conf the Configuration to modify. + @param args command-line arguments.]]> + + + + + GenericOptionsParser to parse given options as well + as generic Hadoop options. + + The resulting CommandLine object can be obtained by + {@link #getCommandLine()}. + + @param conf the configuration to modify + @param options options built by the caller + @param args User-specified arguments]]> + + + + + Strings containing the un-parsed arguments + or empty array if commandLine was not defined.]]> + + + + + + + + + + CommandLine object + to process the parsed arguments. + + Note: If the object is created with + {@link #GenericOptionsParser(Configuration, String[])}, then returned + object will only contain parsed generic options. + + @return CommandLine representing list of arguments + parsed against Options descriptor.]]> + + + + + + + + + + + + + + + + + GenericOptionsParser is a utility to parse command line + arguments generic to the Hadoop framework. + + GenericOptionsParser recognizes several standarad command + line arguments, enabling applications to easily specify a namenode, a + jobtracker, additional configuration resources etc. + +

    Generic Options

    + +

    The supported generic options are:

    +

    +     -conf <configuration file>     specify a configuration file
    +     -D <property=value>            use value for given property
    +     -fs <local|namenode:port>      specify a namenode
    +     -jt <local|jobtracker:port>    specify a job tracker
    +     -files <comma separated list of files>    specify comma separated
    +                            files to be copied to the map reduce cluster
    +     -libjars <comma separated list of jars>   specify comma separated
    +                            jar files to include in the classpath.
    +     -archives <comma separated list of archives>    specify comma
    +             separated archives to be unarchived on the compute machines.
    +
    + 

    + +

    The general command line syntax is:

    +

    + bin/hadoop command [genericOptions] [commandOptions]
    + 

    + +

    Generic command line arguments might modify + Configuration objects, given to constructors.

    + +

    The functionality is implemented using Commons CLI.

    + +

    Examples:

    +

    + $ bin/hadoop dfs -fs darwin:8020 -ls /data
    + list /data directory in dfs with namenode darwin:8020
    + 
    + $ bin/hadoop dfs -D fs.default.name=darwin:8020 -ls /data
    + list /data directory in dfs with namenode darwin:8020
    +     
    + $ bin/hadoop dfs -conf hadoop-site.xml -ls /data
    + list /data directory in dfs with conf specified in hadoop-site.xml
    +     
    + $ bin/hadoop job -D mapred.job.tracker=darwin:50020 -submit job.xml
    + submit a job to job tracker darwin:50020
    +     
    + $ bin/hadoop job -jt darwin:50020 -submit job.xml
    + submit a job to job tracker darwin:50020
    +     
    + $ bin/hadoop job -jt local -submit job.xml
    + submit a job to local runner
    + 
    + $ bin/hadoop jar -libjars testlib.jar 
    + -archives test.tgz -files file.txt inputjar args
    + job submission with libjars, files and archives
    + 

    + + @see Tool + @see ToolRunner]]> +
    +
    + + + + + + + + + Class<T>) of the + argument of type T. + @param The type of the argument + @param t the object to get it class + @return Class<T>]]> + + + + + + + List<T> to a an array of + T[]. + @param c the Class object of the items in the list + @param list the list to convert]]> + + + + + + List<T> to a an array of + T[]. + @param list the list to convert + @throws ArrayIndexOutOfBoundsException if the list is empty. + Use {@link #toArray(Class, List)} if the list may be empty.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + io.file.buffer.size specified in the given + Configuration. + @param in input stream + @param conf configuration + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if native-hadoop is loaded, + else false]]> + + + + + + true if native hadoop libraries, if present, can be + used for this job; false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + { pq.top().change(); pq.adjustTop(); } + instead of
    +  { o = pq.pop(); o.change(); pq.push(o); }
    + 
    ]]> +
    +
    + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Clients and/or applications can use the provided Progressable + to explicitly report progress to the Hadoop framework. This is especially + important for operations which take an insignificant amount of time since, + in-lieu of the reported progress, the framework has to assume that an error + has occured and time-out the operation.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Class is to be obtained + @return the correctly typed Class of the given object.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hadoop Pipes + or Hadoop Streaming. + + It also checks to ensure that we are running on a *nix platform else + (e.g. in Cygwin/Windows) it returns null. + @param conf configuration + @return a String[] with the ulimit command arguments or + null if we are running on a non *nix platform or + if the limit is unspecified.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Shell interface. + @param cmd shell command to execute. + @return the output of the executed command.]]> + + + + + + + + Shell interface. + @param env the map of environment key=value + @param cmd shell command to execute. + @return the output of the executed command.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + Shell can be used to run unix commands like du or + df. It also offers facilities to gate commands by + time-intervals.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ShellCommandExecutorshould be used in cases where the output + of the command needs no explicit parsing and where the command, working + directory and the environment remains unchanged. The output of the command + is stored as-is and is expected to be small.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ArrayList of string values]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + charToEscape in the string + with the escape char escapeChar + + @param str string + @param escapeChar escape char + @param charToEscape the char to be escaped + @return an escaped string]]> + + + + + + + + + + + + + + + + + + + + + + charToEscape in the string + with the escape char escapeChar + + @param str string + @param escapeChar escape char + @param charToEscape the escaped char + @return an unescaped string]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tool, is the standard for any Map-Reduce tool/application. + The tool/application should delegate the handling of + + standard command-line options to {@link ToolRunner#run(Tool, String[])} + and only handle its custom arguments.

    + +

    Here is how a typical Tool is implemented:

    +

    +     public class MyApp extends Configured implements Tool {
    +     
    +       public int run(String[] args) throws Exception {
    +         // Configuration processed by ToolRunner
    +         Configuration conf = getConf();
    +         
    +         // Create a JobConf using the processed conf
    +         JobConf job = new JobConf(conf, MyApp.class);
    +         
    +         // Process custom command-line options
    +         Path in = new Path(args[1]);
    +         Path out = new Path(args[2]);
    +         
    +         // Specify various job-specific parameters     
    +         job.setJobName("my-app");
    +         job.setInputPath(in);
    +         job.setOutputPath(out);
    +         job.setMapperClass(MyApp.MyMapper.class);
    +         job.setReducerClass(MyApp.MyReducer.class);
    +
    +         // Submit the job, then poll for progress until the job is complete
    +         JobClient.runJob(job);
    +       }
    +       
    +       public static void main(String[] args) throws Exception {
    +         // Let ToolRunner handle generic command-line options 
    +         int res = ToolRunner.run(new Configuration(), new Sort(), args);
    +         
    +         System.exit(res);
    +       }
    +     }
    + 

    + + @see GenericOptionsParser + @see ToolRunner]]> +
    +
    + + + + + + + + + + + + Tool by {@link Tool#run(String[])}, after + parsing with the given generic arguments. Uses the given + Configuration, or builds one if null. + + Sets the Tool's configuration with the possibly modified + version of the conf. + + @param conf Configuration for the Tool. + @param tool Tool to run. + @param args command-line arguments to the tool. + @return exit code of the {@link Tool#run(String[])} method.]]> + + + + + + + + Tool with its Configuration. + + Equivalent to run(tool.getConf(), tool, args). + + @param tool Tool to run. + @param args command-line arguments to the tool. + @return exit code of the {@link Tool#run(String[])} method.]]> + + + + + + + + + + ToolRunner can be used to run classes implementing + Tool interface. It works in conjunction with + {@link GenericOptionsParser} to parse the + + generic hadoop command line arguments and modifies the + Configuration of the Tool. The + application-specific options are passed along without being modified. +

    + + @see Tool + @see GenericOptionsParser]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + this filter. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Bloom filter, as defined by Bloom in 1970. +

    + The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by + the networking research community in the past decade thanks to the bandwidth efficiencies that it + offers for the transmission of set membership information between networked hosts. A sender encodes + the information into a bit vector, the Bloom filter, that is more compact than a conventional + representation. Computation and space costs for construction are linear in the number of elements. + The receiver uses the filter to test whether various elements are members of the set. Though the + filter will occasionally return a false positive, it will never return a false negative. When creating + the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size. + +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + + @see Space/Time Trade-Offs in Hash Coding with Allowable Errors]]> + + + + + + + + + + + + + this filter. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}).]]> + + + + + + + + + this counting Bloom filter. +

    + Invariant: nothing happens if the specified key does not belong to this counter Bloom filter. + @param key The key to remove.]]> + + + + + + + + + + + + key -> count map. +

    NOTE: due to the bucket size of this filter, inserting the same + key more than 15 times will cause an overflow at all filter positions + associated with this key, and it will significantly increase the error + rate for this and other keys. For this reason the filter can only be + used to store small count values 0 <= N << 15. + @param key key to be tested + @return 0 if the key is not present. Otherwise, a positive value v will + be returned such that v == count with probability equal to the + error rate of this filter, and v > count otherwise. + Additionally, if the filter experienced an underflow as a result of + {@link #delete(Key)} operation, the return value may be lower than the + count with the probability of the false negative rate of such + filter.]]> + + + + + + + + + + + + + + + + + + + + + + counting Bloom filter, as defined by Fan et al. in a ToN + 2000 paper. +

    + A counting Bloom filter is an improvement to standard a Bloom filter as it + allows dynamic additions and deletions of set membership information. This + is achieved through the use of a counting vector instead of a bit vector. +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + + @see Summary cache: a scalable wide-area web cache sharing protocol]]> + + + + + + + + + + + + + + Builds an empty Dynamic Bloom filter. + @param vectorSize The number of bits in the vector. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}). + @param nr The threshold for the maximum number of keys to record in a + dynamic Bloom filter row.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + dynamic Bloom filter, as defined in the INFOCOM 2006 paper. +

    + A dynamic Bloom filter (DBF) makes use of a s * m bit matrix but + each of the s rows is a standard Bloom filter. The creation + process of a DBF is iterative. At the start, the DBF is a 1 * m + bit matrix, i.e., it is composed of a single standard Bloom filter. + It assumes that nr elements are recorded in the + initial bit vector, where nr <= n (n is + the cardinality of the set A to record in the filter). +

    + As the size of A grows during the execution of the application, + several keys must be inserted in the DBF. When inserting a key into the DBF, + one must first get an active Bloom filter in the matrix. A Bloom filter is + active when the number of recorded keys, nr, is + strictly less than the current cardinality of A, n. + If an active Bloom filter is found, the key is inserted and + nr is incremented by one. On the other hand, if there + is no active Bloom filter, a new one is created (i.e., a new row is added to + the matrix) according to the current size of A and the element + is added in this new Bloom filter and the nr value of + this new Bloom filter is set to one. A given key is said to belong to the + DBF if the k positions are set to one in one of the matrix rows. +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + @see BloomFilter A Bloom filter + + @see Theory and Network Applications of Dynamic Bloom Filters]]> + + + + + + + + + + + this filter. + @param nbHash The number of hash functions to consider. + @param hashType type of the hashing function (see {@link Hash}).]]> + + + + + + this filter. + @param key The key to add.]]> + + + + + + this filter. + @param key The key to test. + @return boolean True if the specified key belongs to this filter. + False otherwise.]]> + + + + + + this filter and a specified filter. +

    + Invariant: The result is assigned to this filter. + @param filter The filter to AND with.]]> + + + + + + this filter and a specified filter. +

    + Invariant: The result is assigned to this filter. + @param filter The filter to OR with.]]> + + + + + + this filter and a specified filter. +

    + Invariant: The result is assigned to this filter. + @param filter The filter to XOR with.]]> + + + + + this filter. +

    + The result is assigned to this filter.]]> + + + + + + this filter. + @param keys The list of keys.]]> + + + + + + this filter. + @param keys The collection of keys.]]> + + + + + + this filter. + @param keys The array of keys.]]> + + + + + + + + + + + + + this filter.]]> + + + + + + + + + + + + + + + + + + + + A filter is a data structure which aims at offering a lossy summary of a set A. The + key idea is to map entries of A (also called keys) into several positions + in a vector through the use of several hash functions. +

    + Typically, a filter will be implemented as a Bloom filter (or a Bloom filter extension). +

    + It must be extended in order to define the real behavior. + + @see Key The general behavior of a key + @see HashFunction A hash function]]> + + + + + + + + + Builds a hash function that must obey to a given maximum number of returned values and a highest value. + @param maxValue The maximum highest returned value. + @param nbHash The number of resulting hashed values. + @param hashType type of the hashing function (see {@link Hash}).]]> + + + + + this hash function. A NOOP]]> + + + + + + + + + + + + + + + + + + + + + + + + + Builds a key with a default weight. + @param value The byte value of this key.]]> + + + + + + Builds a key with a specified weight. + @param value The value of this key. + @param weight The weight associated to this key.]]> + + + + + + + + + + + + this key.]]> + + + + + this key.]]> + + + + + + this key with a specified value. + @param weight The increment.]]> + + + + + this key by one.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The idea is to randomly select a bit to reset.]]> + + + + + + The idea is to select the bit to reset that will generate the minimum + number of false negative.]]> + + + + + + The idea is to select the bit to reset that will remove the maximum number + of false positive.]]> + + + + + + The idea is to select the bit to reset that will, at the same time, remove + the maximum number of false positve while minimizing the amount of false + negative generated.]]> + + + + + Originally created by + European Commission One-Lab Project 034819.]]> + + + + + + + + + + + + + + this filter. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}).]]> + + + + + + + + + this retouched Bloom filter. +

    + Invariant: if the false positive is null, nothing happens. + @param key The false positive key to add.]]> + + + + + + this retouched Bloom filter. + @param coll The collection of false positive.]]> + + + + + + this retouched Bloom filter. + @param keys The list of false positive.]]> + + + + + + this retouched Bloom filter. + @param keys The array of false positive.]]> + + + + + + + this retouched Bloom filter. + @param scheme The selective clearing scheme to apply.]]> + + + + + + + + + + + + retouched Bloom filter, as defined in the CoNEXT 2006 paper. +

    + It allows the removal of selected false positives at the cost of introducing + random false negatives, and with the benefit of eliminating some random false + positives at the same time. + +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + @see BloomFilter A Bloom filter + @see RemoveScheme The different selective clearing algorithms + + @see Retouched Bloom Filters: Allowing Networked Applications to Trade Off Selected False Positives Against False Negatives]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + length, and + the provided seed value + @param bytes input bytes + @param length length of the valid bytes to consider + @param initval seed value + @return hash value]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The best hash table sizes are powers of 2. There is no need to do mod + a prime (mod is sooo slow!). If you need less than 32 bits, use a bitmask. + For example, if you need only 10 bits, do + h = (h & hashmask(10)); + In which case, the hash table should have hashsize(10) elements. + +

    If you are hashing n strings byte[][] k, do it like this: + for (int i = 0, h = 0; i < n; ++i) h = hash( k[i], h); + +

    By Bob Jenkins, 2006. bob_jenkins@burtleburtle.net. You may use this + code any way you wish, private, educational, or commercial. It's free. + +

    Use for hash table lookup, or anything where one collision in 2^^32 is + acceptable. Do NOT use for cryptographic purposes.]]> + + + + + + + + + + + lookup3.c, by Bob Jenkins, May 2006, Public Domain. + + You can use this free for any purpose. It's in the public domain. + It has no warranty. + + + @see lookup3.c + @see Hash Functions (and how this + function compares to others such as CRC, MD?, etc + @see Has update on the + Dr. Dobbs Article]]> + + + + + + + + + + + + + + + + The C version of MurmurHash 2.0 found at that site was ported + to Java by Andrzej Bialecki (ab at getopt org).

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobTracker, + as {@link JobTracker.State} + + @return the current state of the JobTracker.]]> + + + + + JobTracker + + @return the size of heap memory used by the JobTracker]]> + + + + + JobTracker + + @return the configured size of max heap memory that can be used by the JobTracker]]> + + + + + + + + + + + + ClusterStatus provides clients with information such as: +
      +
    1. + Size of the cluster. +
    2. +
    3. + Name of the trackers. +
    4. +
    5. + Task capacity of the cluster. +
    6. +
    7. + The number of currently running map & reduce tasks. +
    8. +
    9. + State of the JobTracker. +
    10. +

    + +

    Clients can query for the latest ClusterStatus, via + {@link JobClient#getClusterStatus()}.

    + + @see JobClient]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Counters represent global counters, defined either by the + Map-Reduce framework or applications. Each Counter can be of + any {@link Enum} type.

    + +

    Counters are bunched into {@link Group}s, each comprising of + counters from a particular Enum class. + @deprecated Use {@link org.apache.hadoop.mapreduce.Counters} instead.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Group of counters, comprising of counters from a particular + counter {@link Enum} class. + +

    Grouphandles localization of the class name and the + counter names.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FileInputFormat implementations can override this and return + false to ensure that individual input files are never split-up + so that {@link Mapper}s process entire files. + + @param fs the file system that the file is on + @param filename the file name to check + @return is this file splitable?]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FileInputFormat is the base class for all file-based + InputFormats. This provides a generic implementation of + {@link #getSplits(JobConf, int)}. + Subclasses of FileInputFormat can also override the + {@link #isSplitable(FileSystem, Path)} method to ensure input-files are + not split-up and are processed as a whole by {@link Mapper}s. + @deprecated Use {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat} + instead.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the job output should be compressed, + false otherwise]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tasks' Side-Effect Files + +

    Note: The following is valid only if the {@link OutputCommitter} + is {@link FileOutputCommitter}. If OutputCommitter is not + a FileOutputCommitter, the task's temporary output + directory is same as {@link #getOutputPath(JobConf)} i.e. + ${mapred.output.dir}$

    + +

    Some applications need to create/write-to side-files, which differ from + the actual job-outputs. + +

    In such cases there could be issues with 2 instances of the same TIP + (running simultaneously e.g. speculative tasks) trying to open/write-to the + same file (path) on HDFS. Hence the application-writer will have to pick + unique names per task-attempt (e.g. using the attemptid, say + attempt_200709221812_0001_m_000000_0), not just per TIP.

    + +

    To get around this the Map-Reduce framework helps the application-writer + out by maintaining a special + ${mapred.output.dir}/_temporary/_${taskid} + sub-directory for each task-attempt on HDFS where the output of the + task-attempt goes. On successful completion of the task-attempt the files + in the ${mapred.output.dir}/_temporary/_${taskid} (only) + are promoted to ${mapred.output.dir}. Of course, the + framework discards the sub-directory of unsuccessful task-attempts. This + is completely transparent to the application.

    + +

    The application-writer can take advantage of this by creating any + side-files required in ${mapred.work.output.dir} during execution + of his reduce-task i.e. via {@link #getWorkOutputPath(JobConf)}, and the + framework will move them out similarly - thus she doesn't have to pick + unique paths per task-attempt.

    + +

    Note: the value of ${mapred.work.output.dir} during + execution of a particular task-attempt is actually + ${mapred.output.dir}/_temporary/_{$taskid}, and this value is + set by the map-reduce framework. So, just create any side-files in the + path returned by {@link #getWorkOutputPath(JobConf)} from map/reduce + task to take advantage of this feature.

    + +

    The entire discussion holds true for maps of jobs with + reducer=NONE (i.e. 0 reduces) since output of the map, in that case, + goes directly to HDFS.

    + + @return the {@link Path} to the task's temporary output directory + for the map-reduce job.]]> +
    +
    + + + + + + + + + + + + + The generated name can be used to create custom files from within the + different tasks for the job, the names for different tasks will not collide + with each other.

    + +

    The given name is postfixed with the task type, 'm' for maps, 'r' for + reduces and the task partition number. For example, give a name 'test' + running on the first map o the job the generated name will be + 'test-m-00000'.

    + + @param conf the configuration for the job. + @param name the name to make unique. + @return a unique name accross all tasks of the job.]]> +
    +
    + + + + + The path can be used to create custom files from within the map and + reduce tasks. The path name will be unique for each task. The path parent + will be the job output directory.

    ls + +

    This method uses the {@link #getUniqueName} method to make the file name + unique for the task.

    + + @param conf the configuration for the job. + @param name the name for the file. + @return a unique path accross all tasks of the job.]]> +
    +
    + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Each {@link InputSplit} is then assigned to an individual {@link Mapper} + for processing.

    + +

    Note: The split is a logical split of the inputs and the + input files are not physically split into chunks. For e.g. a split could + be <input-file-path, start, offset> tuple. + + @param job job configuration. + @param numSplits the desired number of splits, a hint. + @return an array of {@link InputSplit}s for the job.]]> + + + + + + + + + It is the responsibility of the RecordReader to respect + record boundaries while processing the logical split to present a + record-oriented view to the individual task.

    + + @param split the {@link InputSplit} + @param job the job that this split belongs to + @return a {@link RecordReader}]]> +
    +
    + + InputFormat describes the input-specification for a + Map-Reduce job. + +

    The Map-Reduce framework relies on the InputFormat of the + job to:

    +

      +
    1. + Validate the input-specification of the job. +
    2. + Split-up the input file(s) into logical {@link InputSplit}s, each of + which is then assigned to an individual {@link Mapper}. +
    3. +
    4. + Provide the {@link RecordReader} implementation to be used to glean + input records from the logical InputSplit for processing by + the {@link Mapper}. +
    5. +
    + +

    The default behavior of file-based {@link InputFormat}s, typically + sub-classes of {@link FileInputFormat}, is to split the + input into logical {@link InputSplit}s based on the total size, in + bytes, of the input files. However, the {@link FileSystem} blocksize of + the input files is treated as an upper bound for input splits. A lower bound + on the split size can be set via + + mapred.min.split.size.

    + +

    Clearly, logical splits based on input-size is insufficient for many + applications since record boundaries are to respected. In such cases, the + application has to also implement a {@link RecordReader} on whom lies the + responsibilty to respect record-boundaries and present a record-oriented + view of the logical InputSplit to the individual task. + + @see InputSplit + @see RecordReader + @see JobClient + @see FileInputFormat + @deprecated Use {@link org.apache.hadoop.mapreduce.InputFormat} instead.]]> + + + + + + + + + + InputSplit. + + @return the number of bytes in the input split. + @throws IOException]]> + + + + + + InputSplit is + located as an array of Strings. + @throws IOException]]> + + + + InputSplit represents the data to be processed by an + individual {@link Mapper}. + +

    Typically, it presents a byte-oriented view on the input and is the + responsibility of {@link RecordReader} of the job to process this and present + a record-oriented view. + + @see InputFormat + @see RecordReader + @deprecated Use {@link org.apache.hadoop.mapreduce.InputSplit} instead.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobClient.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + jobid doesn't correspond to any known job. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobClient is the primary interface for the user-job to interact + with the {@link JobTracker}. + + JobClient provides facilities to submit jobs, track their + progress, access component-tasks' reports/logs, get the Map-Reduce cluster + status information etc. + +

    The job submission process involves: +

      +
    1. + Checking the input and output specifications of the job. +
    2. +
    3. + Computing the {@link InputSplit}s for the job. +
    4. +
    5. + Setup the requisite accounting information for the {@link DistributedCache} + of the job, if necessary. +
    6. +
    7. + Copying the job's jar and configuration to the map-reduce system directory + on the distributed file-system. +
    8. +
    9. + Submitting the job to the JobTracker and optionally monitoring + it's status. +
    10. +

    + + Normally the user creates the application, describes various facets of the + job via {@link JobConf} and then uses the JobClient to submit + the job and monitor its progress. + +

    Here is an example on how to use JobClient:

    +

    +     // Create a new JobConf
    +     JobConf job = new JobConf(new Configuration(), MyJob.class);
    +     
    +     // Specify various job-specific parameters     
    +     job.setJobName("myjob");
    +     
    +     job.setInputPath(new Path("in"));
    +     job.setOutputPath(new Path("out"));
    +     
    +     job.setMapperClass(MyJob.MyMapper.class);
    +     job.setReducerClass(MyJob.MyReducer.class);
    +
    +     // Submit the job, then poll for progress until the job is complete
    +     JobClient.runJob(job);
    + 

    + +

    Job Control

    + +

    At times clients would chain map-reduce jobs to accomplish complex tasks + which cannot be done via a single map-reduce job. This is fairly easy since + the output of the job, typically, goes to distributed file-system and that + can be used as the input for the next job.

    + +

    However, this also means that the onus on ensuring jobs are complete + (success/failure) lies squarely on the clients. In such situations the + various job-control options are: +

      +
    1. + {@link #runJob(JobConf)} : submits the job and returns only after + the job has completed. +
    2. +
    3. + {@link #submitJob(JobConf)} : only submits the job, then poll the + returned handle to the {@link RunningJob} to query status and make + scheduling decisions. +
    4. +
    5. + {@link JobConf#setJobEndNotificationURI(String)} : setup a notification + on job-completion, thus avoiding polling. +
    6. +

    + + @see JobConf + @see ClusterStatus + @see Tool + @see DistributedCache]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + If the parameter {@code loadDefaults} is false, the new instance + will not load resources from the default files. + + @param loadDefaults specifies whether to load from the default files]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if framework should keep the intermediate files + for failed tasks, false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the outputs of the maps are to be compressed, + false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This comparator should be provided if the equivalence rules for keys + for sorting the intermediates are different from those for grouping keys + before each call to + {@link Reducer#reduce(Object, java.util.Iterator, OutputCollector, Reporter)}.

    + +

    For key-value pairs (K1,V1) and (K2,V2), the values (V1, V2) are passed + in a single call to the reduce function if K1 and K2 compare as equal.

    + +

    Since {@link #setOutputKeyComparatorClass(Class)} can be used to control + how keys are sorted, this can be used in conjunction to simulate + secondary sort on values.

    + +

    Note: This is not a guarantee of the reduce sort being + stable in any sense. (In any case, with the order of available + map-outputs to the reduce being non-deterministic, it wouldn't make + that much sense.)

    + + @param theClass the comparator class to be used for grouping keys. + It should implement RawComparator. + @see #setOutputKeyComparatorClass(Class)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + combiner class used to combine map-outputs + before being sent to the reducers. Typically the combiner is same as the + the {@link Reducer} for the job i.e. {@link #getReducerClass()}. + + @return the user-defined combiner class used to combine map-outputs.]]> + + + + + + combiner class used to combine map-outputs + before being sent to the reducers. + +

    The combiner is an application-specified aggregation operation, which + can help cut down the amount of data transferred between the + {@link Mapper} and the {@link Reducer}, leading to better performance.

    + +

    The framework may invoke the combiner 0, 1, or multiple times, in both + the mapper and reducer tasks. In general, the combiner is called as the + sort/merge result is written to disk. The combiner must: +

      +
    • be side-effect free
    • +
    • have the same input and output key types and the same input and + output value types
    • +

    + +

    Typically the combiner is same as the Reducer for the + job i.e. {@link #setReducerClass(Class)}.

    + + @param theClass the user-defined combiner class used to combine + map-outputs.]]> +
    +
    + + + true. + + @return true if speculative execution be used for this job, + false otherwise.]]> + + + + + + true if speculative execution + should be turned on, else false.]]> + + + + + true. + + @return true if speculative execution be + used for this job for map tasks, + false otherwise.]]> + + + + + + true if speculative execution + should be turned on for map tasks, + else false.]]> + + + + + true. + + @return true if speculative execution be used + for reduce tasks for this job, + false otherwise.]]> + + + + + + true if speculative execution + should be turned on for reduce tasks, + else false.]]> + + + + + 1. + + @return the number of reduce tasks for this job.]]> + + + + + + Note: This is only a hint to the framework. The actual + number of spawned map tasks depends on the number of {@link InputSplit}s + generated by the job's {@link InputFormat#getSplits(JobConf, int)}. + + A custom {@link InputFormat} is typically used to accurately control + the number of map tasks for the job.

    + +

    How many maps?

    + +

    The number of maps is usually driven by the total size of the inputs + i.e. total number of blocks of the input files.

    + +

    The right level of parallelism for maps seems to be around 10-100 maps + per-node, although it has been set up to 300 or so for very cpu-light map + tasks. Task setup takes awhile, so it is best if the maps take at least a + minute to execute.

    + +

    The default behavior of file-based {@link InputFormat}s is to split the + input into logical {@link InputSplit}s based on the total size, in + bytes, of input files. However, the {@link FileSystem} blocksize of the + input files is treated as an upper bound for input splits. A lower bound + on the split size can be set via + + mapred.min.split.size.

    + +

    Thus, if you expect 10TB of input data and have a blocksize of 128MB, + you'll end up with 82,000 maps, unless {@link #setNumMapTasks(int)} is + used to set it even higher.

    + + @param n the number of map tasks for this job. + @see InputFormat#getSplits(JobConf, int) + @see FileInputFormat + @see FileSystem#getDefaultBlockSize() + @see FileStatus#getBlockSize()]]> +
    +
    + + + 1. + + @return the number of reduce tasks for this job.]]> + + + + + + How many reduces? + +

    The right number of reduces seems to be 0.95 or + 1.75 multiplied by (<no. of nodes> * + + mapred.tasktracker.reduce.tasks.maximum). +

    + +

    With 0.95 all of the reduces can launch immediately and + start transfering map outputs as the maps finish. With 1.75 + the faster nodes will finish their first round of reduces and launch a + second wave of reduces doing a much better job of load balancing.

    + +

    Increasing the number of reduces increases the framework overhead, but + increases load balancing and lowers the cost of failures.

    + +

    The scaling factors above are slightly less than whole numbers to + reserve a few reduce slots in the framework for speculative-tasks, failures + etc.

    + +

    Reducer NONE

    + +

    It is legal to set the number of reduce-tasks to zero.

    + +

    In this case the output of the map-tasks directly go to distributed + file-system, to the path set by + {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Also, the + framework doesn't sort the map-outputs before writing it out to HDFS.

    + + @param n the number of reduce tasks for this job.]]> +
    +
    + + + mapred.map.max.attempts + property. If this property is not already set, the default is 4 attempts. + + @return the max number of attempts per map task.]]> + + + + + + + + + + + mapred.reduce.max.attempts + property. If this property is not already set, the default is 4 attempts. + + @return the max number of attempts per reduce task.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + noFailures, the + tasktracker is blacklisted for this job. + + @param noFailures maximum no. of failures of a given job per tasktracker.]]> + + + + + blacklisted for this job. + + @return the maximum no. of failures of a given job per tasktracker.]]> + + + + + failed. + + Defaults to zero, i.e. any failed map-task results in + the job being declared as {@link JobStatus#FAILED}. + + @return the maximum percentage of map tasks that can fail without + the job being aborted.]]> + + + + + + failed. + + @param percent the maximum percentage of map tasks that can fail without + the job being aborted.]]> + + + + + failed. + + Defaults to zero, i.e. any failed reduce-task results + in the job being declared as {@link JobStatus#FAILED}. + + @return the maximum percentage of reduce tasks that can fail without + the job being aborted.]]> + + + + + + failed. + + @param percent the maximum percentage of reduce tasks that can fail without + the job being aborted.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The debug script can aid debugging of failed map tasks. The script is + given task's stdout, stderr, syslog, jobconf files as arguments.

    + +

    The debug command, run on the node where the map failed, is:

    +

    + $script $stdout $stderr $syslog $jobconf. +

    + +

    The script file is distributed through {@link DistributedCache} + APIs. The script needs to be symlinked.

    + +

    Here is an example on how to submit a script +

    + job.setMapDebugScript("./myscript");
    + DistributedCache.createSymlink(job);
    + DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
    + 

    + + @param mDbgScript the script name]]> +
    +
    + + + + + + + + + The debug script can aid debugging of failed reduce tasks. The script + is given task's stdout, stderr, syslog, jobconf files as arguments.

    + +

    The debug command, run on the node where the map failed, is:

    +

    + $script $stdout $stderr $syslog $jobconf. +

    + +

    The script file is distributed through {@link DistributedCache} + APIs. The script file needs to be symlinked

    + +

    Here is an example on how to submit a script +

    + job.setReduceDebugScript("./myscript");
    + DistributedCache.createSymlink(job);
    + DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
    + 

    + + @param rDbgScript the script name]]> +
    +
    + + + + + + + + null if it hasn't + been set. + @see #setJobEndNotificationURI(String)]]> + + + + + + The uri can contain 2 special parameters: $jobId and + $jobStatus. Those, if present, are replaced by the job's + identifier and completion-status respectively.

    + +

    This is typically used by application-writers to implement chaining of + Map-Reduce jobs in an asynchronous manner.

    + + @param uri the job end notification uri + @see JobStatus + @see Job Completion and Chaining]]> +
    +
    + + + + When a job starts, a shared directory is created at location + + ${mapred.local.dir}/taskTracker/jobcache/$jobid/work/ . + This directory is exposed to the users through + job.local.dir . + So, the tasks can use this space + as scratch space and share files among them.

    + This value is available as System property also. + + @return The localized job specific shared directory]]> +
    +
    + + + + For backward compatibility, if the job configuration sets the + key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different + from {@link #DISABLED_MEMORY_LIMIT}, that value will be used + after converting it from bytes to MB. + @return memory required to run a map task of the job, in MB, + or {@link #DISABLED_MEMORY_LIMIT} if unset.]]> + + + + + + + + + For backward compatibility, if the job configuration sets the + key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different + from {@link #DISABLED_MEMORY_LIMIT}, that value will be used + after converting it from bytes to MB. + @return memory required to run a reduce task of the job, in MB, + or {@link #DISABLED_MEMORY_LIMIT} if unset.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + This method is deprecated. Now, different memory limits can be + set for map and reduce tasks of a job, in MB. +

    + For backward compatibility, if the job configuration sets the + key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different + from {@link #DISABLED_MEMORY_LIMIT}, that value is returned. + Otherwise, this method will return the larger of the values returned by + {@link #getMemoryForMapTask()} and {@link #getMemoryForReduceTask()} + after converting them into bytes. + + @return Memory required to run a task of this job, in bytes, + or {@link #DISABLED_MEMORY_LIMIT}, if unset. + @see #setMaxVirtualMemoryForTask(long) + @deprecated Use {@link #getMemoryForMapTask()} and + {@link #getMemoryForReduceTask()}]]> + + + + + + + mapred.task.maxvmem is split into + mapred.job.map.memory.mb + and mapred.job.map.memory.mb,mapred + each of the new key are set + as mapred.task.maxvmem / 1024 + as new values are in MB + + @param vmem Maximum amount of virtual memory in bytes any task of this job + can use. + @see #getMaxVirtualMemoryForTask() + @deprecated + Use {@link #setMemoryForMapTask(long mem)} and + Use {@link #setMemoryForReduceTask(long mem)}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobConf is the primary interface for a user to describe a + map-reduce job to the Hadoop framework for execution. The framework tries to + faithfully execute the job as-is described by JobConf, however: +

      +
    1. + Some configuration parameters might have been marked as + + final by administrators and hence cannot be altered. +
    2. +
    3. + While some job parameters are straight-forward to set + (e.g. {@link #setNumReduceTasks(int)}), some parameters interact subtly + rest of the framework and/or job-configuration and is relatively more + complex for the user to control finely (e.g. {@link #setNumMapTasks(int)}). +
    4. +

    + +

    JobConf typically specifies the {@link Mapper}, combiner + (if any), {@link Partitioner}, {@link Reducer}, {@link InputFormat} and + {@link OutputFormat} implementations to be used etc. + +

    Optionally JobConf is used to specify other advanced facets + of the job such as Comparators to be used, files to be put in + the {@link DistributedCache}, whether or not intermediate and/or job outputs + are to be compressed (and how), debugability via user-provided scripts + ( {@link #setMapDebugScript(String)}/{@link #setReduceDebugScript(String)}), + for doing post-processing on task logs, task's stdout, stderr, syslog. + and etc.

    + +

    Here is an example on how to configure a job via JobConf:

    +

    +     // Create a new JobConf
    +     JobConf job = new JobConf(new Configuration(), MyJob.class);
    +     
    +     // Specify various job-specific parameters     
    +     job.setJobName("myjob");
    +     
    +     FileInputFormat.setInputPaths(job, new Path("in"));
    +     FileOutputFormat.setOutputPath(job, new Path("out"));
    +     
    +     job.setMapperClass(MyJob.MyMapper.class);
    +     job.setCombinerClass(MyJob.MyReducer.class);
    +     job.setReducerClass(MyJob.MyReducer.class);
    +     
    +     job.setInputFormat(SequenceFileInputFormat.class);
    +     job.setOutputFormat(SequenceFileOutputFormat.class);
    + 

    + + @see JobClient + @see ClusterStatus + @see Tool + @see DistributedCache + @deprecated Use {@link Configuration} instead]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + .]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + any job + run on the jobtracker started at 200707121733, we would use : +
     
    + JobID.getTaskIDsPattern("200707121733", null);
    + 
    + which will return : +
     "job_200707121733_[0-9]*" 
    + @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @return a regex pattern matching JobIDs]]> +
    +
    + + + An example JobID is : + job_200707121733_0003 , which represents the third job + running at the jobtracker started at 200707121733. +

    + Applications should never construct or parse JobID strings, but rather + use appropriate constructors or {@link #forName(String)} method. + + @see TaskID + @see TaskAttemptID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + "N/A" + + @return Scheduling information associated to particular Job Queue]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + zero. + + @param conf configuration for the JobTracker. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Output pairs need not be of the same types as input pairs. A given + input pair may map to zero or many output pairs. Output pairs are + collected with calls to + {@link OutputCollector#collect(Object,Object)}.

    + +

    Applications can use the {@link Reporter} provided to report progress + or just indicate that they are alive. In scenarios where the application + takes an insignificant amount of time to process individual key/value + pairs, this is crucial since the framework might assume that the task has + timed-out and kill that task. The other way of avoiding this is to set + + mapred.task.timeout to a high-enough value (or even zero for no + time-outs).

    + + @param key the input key. + @param value the input value. + @param output collects mapped keys and values. + @param reporter facility to report progress.]]> +
    + + + Maps are the individual tasks which transform input records into a + intermediate records. The transformed intermediate records need not be of + the same type as the input records. A given input pair may map to zero or + many output pairs.

    + +

    The Hadoop Map-Reduce framework spawns one map task for each + {@link InputSplit} generated by the {@link InputFormat} for the job. + Mapper implementations can access the {@link JobConf} for the + job via the {@link JobConfigurable#configure(JobConf)} and initialize + themselves. Similarly they can use the {@link Closeable#close()} method for + de-initialization.

    + +

    The framework then calls + {@link #map(Object, Object, OutputCollector, Reporter)} + for each key/value pair in the InputSplit for that task.

    + +

    All intermediate values associated with a given output key are + subsequently grouped by the framework, and passed to a {@link Reducer} to + determine the final output. Users can control the grouping by specifying + a Comparator via + {@link JobConf#setOutputKeyComparatorClass(Class)}.

    + +

    The grouped Mapper outputs are partitioned per + Reducer. Users can control which keys (and hence records) go to + which Reducer by implementing a custom {@link Partitioner}. + +

    Users can optionally specify a combiner, via + {@link JobConf#setCombinerClass(Class)}, to perform local aggregation of the + intermediate outputs, which helps to cut down the amount of data transferred + from the Mapper to the Reducer. + +

    The intermediate, grouped outputs are always stored in + {@link SequenceFile}s. Applications can specify if and how the intermediate + outputs are to be compressed and which {@link CompressionCodec}s are to be + used via the JobConf.

    + +

    If the job has + zero + reduces then the output of the Mapper is directly written + to the {@link FileSystem} without grouping by keys.

    + +

    Example:

    +

    +     public class MyMapper<K extends WritableComparable, V extends Writable> 
    +     extends MapReduceBase implements Mapper<K, V, K, V> {
    +     
    +       static enum MyCounters { NUM_RECORDS }
    +       
    +       private String mapTaskId;
    +       private String inputFile;
    +       private int noRecords = 0;
    +       
    +       public void configure(JobConf job) {
    +         mapTaskId = job.get("mapred.task.id");
    +         inputFile = job.get("map.input.file");
    +       }
    +       
    +       public void map(K key, V val,
    +                       OutputCollector<K, V> output, Reporter reporter)
    +       throws IOException {
    +         // Process the <key, value> pair (assume this takes a while)
    +         // ...
    +         // ...
    +         
    +         // Let the framework know that we are alive, and kicking!
    +         // reporter.progress();
    +         
    +         // Process some more
    +         // ...
    +         // ...
    +         
    +         // Increment the no. of <key, value> pairs processed
    +         ++noRecords;
    +
    +         // Increment counters
    +         reporter.incrCounter(NUM_RECORDS, 1);
    +        
    +         // Every 100 records update application-level status
    +         if ((noRecords%100) == 0) {
    +           reporter.setStatus(mapTaskId + " processed " + noRecords + 
    +                              " from input-file: " + inputFile); 
    +         }
    +         
    +         // Output the result
    +         output.collect(key, val);
    +       }
    +     }
    + 

    + +

    Applications may write a custom {@link MapRunnable} to exert greater + control on map processing e.g. multi-threaded Mappers etc.

    + + @see JobConf + @see InputFormat + @see Partitioner + @see Reducer + @see MapReduceBase + @see MapRunnable + @see SequenceFile + @deprecated Use {@link org.apache.hadoop.mapreduce.Mapper} instead.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + Provides default no-op implementations for a few methods, most non-trivial + applications need to override some of them.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + <key, value> pairs. + +

    Mapping of input records to output records is complete when this method + returns.

    + + @param input the {@link RecordReader} to read the input records. + @param output the {@link OutputCollector} to collect the outputrecords. + @param reporter {@link Reporter} to report progress, status-updates etc. + @throws IOException]]> +
    +
    + + Custom implementations of MapRunnable can exert greater + control on map processing e.g. multi-threaded, asynchronous mappers etc.

    + + @see Mapper + @deprecated Use {@link org.apache.hadoop.mapreduce.Mapper} instead.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + nearly + equal content length.
    + Subclasses implement {@link #getRecordReader(InputSplit, JobConf, Reporter)} + to construct RecordReader's for MultiFileSplit's. + @see MultiFileSplit + @deprecated Use {@link org.apache.hadoop.mapred.lib.CombineFileInputFormat} instead]]> +
    +
    + + + + + + + + + + + + + MultiFileSplit can be used to implement {@link RecordReader}'s, with + reading one record per file. + @see FileSplit + @see MultiFileInputFormat + @deprecated Use {@link org.apache.hadoop.mapred.lib.CombineFileSplit} instead]]> + + + + + + + + + + + + + + + <key, value> pairs output by {@link Mapper}s + and {@link Reducer}s. + +

    OutputCollector is the generalization of the facility + provided by the Map-Reduce framework to collect data output by either the + Mapper or the Reducer i.e. intermediate outputs + or the output of the job.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + OutputCommitter describes the commit of task output for a + Map-Reduce job. + +

    The Map-Reduce framework relies on the OutputCommitter of + the job to:

    +

      +
    1. + Setup the job during initialization. For example, create the temporary + output directory for the job during the initialization of the job. +
    2. +
    3. + Cleanup the job after the job completion. For example, remove the + temporary output directory after the job completion. +
    4. +
    5. + Setup the task temporary output. +
    6. +
    7. + Check whether a task needs a commit. This is to avoid the commit + procedure if a task does not need commit. +
    8. +
    9. + Commit of the task output. +
    10. +
    11. + Discard the task commit. +
    12. +
    + + @see FileOutputCommitter + @see JobContext + @see TaskAttemptContext + @deprecated Use {@link org.apache.hadoop.mapreduce.OutputCommitter} instead.]]> +
    +
    + + + + + + + + + + + + + + + + + + + This is to validate the output specification for the job when it is + a job is submitted. Typically checks that it does not already exist, + throwing an exception when it already exists, so that output is not + overwritten.

    + + @param ignored + @param job job configuration. + @throws IOException when output should not be attempted]]> +
    +
    + + OutputFormat describes the output-specification for a + Map-Reduce job. + +

    The Map-Reduce framework relies on the OutputFormat of the + job to:

    +

      +
    1. + Validate the output-specification of the job. For e.g. check that the + output directory doesn't already exist. +
    2. + Provide the {@link RecordWriter} implementation to be used to write out + the output files of the job. Output files are stored in a + {@link FileSystem}. +
    3. +
    + + @see RecordWriter + @see JobConf + @deprecated Use {@link org.apache.hadoop.mapreduce.OutputFormat} instead.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + Typically a hash function on a all or a subset of the key.

    + + @param key the key to be paritioned. + @param value the entry value. + @param numPartitions the total number of partitions. + @return the partition number for the key.]]> +
    +
    + + Partitioner controls the partitioning of the keys of the + intermediate map-outputs. The key (or a subset of the key) is used to derive + the partition, typically by a hash function. The total number of partitions + is the same as the number of reduce tasks for the job. Hence this controls + which of the m reduce tasks the intermediate key (and hence the + record) is sent for reduction.

    + + @see Reducer + @deprecated Use {@link org.apache.hadoop.mapreduce.Partitioner} instead.]]> +
    +
    + + + + + + + + + + + + + + + + + + + true if there exists a key/value, + false otherwise. + @throws IOException]]> + + + + + + + + + + + + + + + RawKeyValueIterator is an iterator used to iterate over + the raw keys and values during sort/merge of intermediate data.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 0.0 to 1.0. + @throws IOException]]> + + + + RecordReader reads <key, value> pairs from an + {@link InputSplit}. + +

    RecordReader, typically, converts the byte-oriented view of + the input, provided by the InputSplit, and presents a + record-oriented view for the {@link Mapper} & {@link Reducer} tasks for + processing. It thus assumes the responsibility of processing record + boundaries and presenting the tasks with keys and values.

    + + @see InputSplit + @see InputFormat]]> +
    +
    + + + + + + + + + + + + + + + + RecordWriter to future operations. + + @param reporter facility to report progress. + @throws IOException]]> + + + + RecordWriter writes the output <key, value> pairs + to an output file. + +

    RecordWriter implementations write the job outputs to the + {@link FileSystem}. + + @see OutputFormat]]> + + + + + + + + + + + + + + + Reduces values for a given key. + +

    The framework calls this method for each + <key, (list of values)> pair in the grouped inputs. + Output values must be of the same type as input values. Input keys must + not be altered. The framework will reuse the key and value objects + that are passed into the reduce, therefore the application should clone + the objects they want to keep a copy of. In many cases, all values are + combined into zero or one value. +

    + +

    Output pairs are collected with calls to + {@link OutputCollector#collect(Object,Object)}.

    + +

    Applications can use the {@link Reporter} provided to report progress + or just indicate that they are alive. In scenarios where the application + takes an insignificant amount of time to process individual key/value + pairs, this is crucial since the framework might assume that the task has + timed-out and kill that task. The other way of avoiding this is to set + + mapred.task.timeout to a high-enough value (or even zero for no + time-outs).

    + + @param key the key. + @param values the list of values to reduce. + @param output to collect keys and combined values. + @param reporter facility to report progress.]]> +
    + + + The number of Reducers for the job is set by the user via + {@link JobConf#setNumReduceTasks(int)}. Reducer implementations + can access the {@link JobConf} for the job via the + {@link JobConfigurable#configure(JobConf)} method and initialize themselves. + Similarly they can use the {@link Closeable#close()} method for + de-initialization.

    + +

    Reducer has 3 primary phases:

    +
      +
    1. + +

      Shuffle

      + +

      Reducer is input the grouped output of a {@link Mapper}. + In the phase the framework, for each Reducer, fetches the + relevant partition of the output of all the Mappers, via HTTP. +

      +
    2. + +
    3. +

      Sort

      + +

      The framework groups Reducer inputs by keys + (since different Mappers may have output the same key) in this + stage.

      + +

      The shuffle and sort phases occur simultaneously i.e. while outputs are + being fetched they are merged.

      + +
      SecondarySort
      + +

      If equivalence rules for keys while grouping the intermediates are + different from those for grouping keys before reduction, then one may + specify a Comparator via + {@link JobConf#setOutputValueGroupingComparator(Class)}.Since + {@link JobConf#setOutputKeyComparatorClass(Class)} can be used to + control how intermediate keys are grouped, these can be used in conjunction + to simulate secondary sort on values.

      + + + For example, say that you want to find duplicate web pages and tag them + all with the url of the "best" known example. You would set up the job + like: +
        +
      • Map Input Key: url
      • +
      • Map Input Value: document
      • +
      • Map Output Key: document checksum, url pagerank
      • +
      • Map Output Value: url
      • +
      • Partitioner: by checksum
      • +
      • OutputKeyComparator: by checksum and then decreasing pagerank
      • +
      • OutputValueGroupingComparator: by checksum
      • +
      +
    4. + +
    5. +

      Reduce

      + +

      In this phase the + {@link #reduce(Object, Iterator, OutputCollector, Reporter)} + method is called for each <key, (list of values)> pair in + the grouped inputs.

      +

      The output of the reduce task is typically written to the + {@link FileSystem} via + {@link OutputCollector#collect(Object, Object)}.

      +
    6. +
    + +

    The output of the Reducer is not re-sorted.

    + +

    Example:

    +

    +     public class MyReducer<K extends WritableComparable, V extends Writable> 
    +     extends MapReduceBase implements Reducer<K, V, K, V> {
    +     
    +       static enum MyCounters { NUM_RECORDS }
    +        
    +       private String reduceTaskId;
    +       private int noKeys = 0;
    +       
    +       public void configure(JobConf job) {
    +         reduceTaskId = job.get("mapred.task.id");
    +       }
    +       
    +       public void reduce(K key, Iterator<V> values,
    +                          OutputCollector<K, V> output, 
    +                          Reporter reporter)
    +       throws IOException {
    +       
    +         // Process
    +         int noValues = 0;
    +         while (values.hasNext()) {
    +           V value = values.next();
    +           
    +           // Increment the no. of values for this key
    +           ++noValues;
    +           
    +           // Process the <key, value> pair (assume this takes a while)
    +           // ...
    +           // ...
    +           
    +           // Let the framework know that we are alive, and kicking!
    +           if ((noValues%10) == 0) {
    +             reporter.progress();
    +           }
    +         
    +           // Process some more
    +           // ...
    +           // ...
    +           
    +           // Output the <key, value> 
    +           output.collect(key, value);
    +         }
    +         
    +         // Increment the no. of <key, list of values> pairs processed
    +         ++noKeys;
    +         
    +         // Increment counters
    +         reporter.incrCounter(NUM_RECORDS, 1);
    +         
    +         // Every 100 keys update application-level status
    +         if ((noKeys%100) == 0) {
    +           reporter.setStatus(reduceTaskId + " processed " + noKeys);
    +         }
    +       }
    +     }
    + 

    + + @see Mapper + @see Partitioner + @see Reporter + @see MapReduceBase + @deprecated Use {@link org.apache.hadoop.mapreduce.Reducer} instead.]]> +
    +
    + + + + + + + + + + + + + + Counter of the given group/name.]]> + + + + + + + Counter of the given group/name.]]> + + + + + + + Enum. + @param amount A non-negative amount by which the counter is to + be incremented.]]> + + + + + + + + + + + + + + InputSplit that the map is reading from. + @throws UnsupportedOperationException if called outside a mapper]]> + + + + + + + + + {@link Mapper} and {@link Reducer} can use the Reporter + provided to report progress or just indicate that they are alive. In + scenarios where the application takes an insignificant amount of time to + process individual key/value pairs, this is crucial since the framework + might assume that the task has timed-out and kill that task. + +

    Applications can also update {@link Counters} via the provided + Reporter .

    + + @see Progressable + @see Counters]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + progress of the job's map-tasks, as a float between 0.0 + and 1.0. When all map tasks have completed, the function returns 1.0. + + @return the progress of the job's map-tasks. + @throws IOException]]> + + + + + + progress of the job's reduce-tasks, as a float between 0.0 + and 1.0. When all reduce tasks have completed, the function returns 1.0. + + @return the progress of the job's reduce-tasks. + @throws IOException]]> + + + + + + progress of the job's cleanup-tasks, as a float between 0.0 + and 1.0. When all cleanup tasks have completed, the function returns 1.0. + + @return the progress of the job's cleanup-tasks. + @throws IOException]]> + + + + + + progress of the job's setup-tasks, as a float between 0.0 + and 1.0. When all setup tasks have completed, the function returns 1.0. + + @return the progress of the job's setup-tasks. + @throws IOException]]> + + + + + + true if the job is complete, else false. + @throws IOException]]> + + + + + + true if the job succeeded, else false. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + RunningJob is the user-interface to query for details on a + running Map-Reduce job. + +

    Clients can get hold of RunningJob via the {@link JobClient} + and then query the running-job for details such as name, configuration, + progress etc.

    + + @see JobClient]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This allows the user to specify the key class to be different + from the actual class ({@link BytesWritable}) used for writing

    + + @param conf the {@link JobConf} to modify + @param theClass the SequenceFile output key class.]]> +
    +
    + + + + + This allows the user to specify the value class to be different + from the actual class ({@link BytesWritable}) used for writing

    + + @param conf the {@link JobConf} to modify + @param theClass the SequenceFile output key class.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + f. The filtering criteria is + MD5(key) % f == 0.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + f using + the criteria record# % f == 0. + For example, if the frequency is 10, one out of 10 records is returned.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if auto increment + {@link SkipBadRecords#COUNTER_MAP_PROCESSED_RECORDS}. + false otherwise.]]> + + + + + + + + + + + + + true if auto increment + {@link SkipBadRecords#COUNTER_REDUCE_PROCESSED_GROUPS}. + false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hadoop provides an optional mode of execution in which the bad records + are detected and skipped in further attempts. + +

    This feature can be used when map/reduce tasks crashes deterministically on + certain input. This happens due to bugs in the map/reduce function. The usual + course would be to fix these bugs. But sometimes this is not possible; + perhaps the bug is in third party libraries for which the source code is + not available. Due to this, the task never reaches to completion even with + multiple attempts and complete data for that task is lost.

    + +

    With this feature, only a small portion of data is lost surrounding + the bad record, which may be acceptable for some user applications. + see {@link SkipBadRecords#setMapperMaxSkipRecords(Configuration, long)}

    + +

    The skipping mode gets kicked off after certain no of failures + see {@link SkipBadRecords#setAttemptsToStartSkipping(Configuration, int)}

    + +

    In the skipping mode, the map/reduce task maintains the record range which + is getting processed at all times. Before giving the input to the + map/reduce function, it sends this record range to the Task tracker. + If task crashes, the Task tracker knows which one was the last reported + range. On further attempts that range get skipped.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + all task attempt IDs + of any jobtracker, in any job, of the first + map task, we would use : +
     
    + TaskAttemptID.getTaskAttemptIDsPattern(null, null, true, 1, null);
    + 
    + which will return : +
     "attempt_[^_]*_[0-9]*_m_000001_[0-9]*" 
    + @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @param isMap whether the tip is a map, or null + @param taskId taskId number, or null + @param attemptId the task attempt number, or null + @return a regex pattern matching TaskAttemptIDs]]> +
    +
    + + + An example TaskAttemptID is : + attempt_200707121733_0003_m_000005_0 , which represents the + zeroth task attempt for the fifth map task in the third job + running at the jobtracker started at 200707121733. +

    + Applications should never construct or parse TaskAttemptID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the first map task + of any jobtracker, of any job, we would use : +

     
    + TaskID.getTaskIDsPattern(null, null, true, 1);
    + 
    + which will return : +
     "task_[^_]*_[0-9]*_m_000001*" 
    + @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @param isMap whether the tip is a map, or null + @param taskId taskId number, or null + @return a regex pattern matching TaskIDs]]> +
    + + + + + + + + An example TaskID is : + task_200707121733_0003_m_000005 , which represents the + fifth map task in the third job running at the jobtracker + started at 200707121733. +

    + Applications should never construct or parse TaskID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskAttemptID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hadoop.log.dir.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the Job was added.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ([,]*) + func ::= tbl(,"") + class ::= @see java.lang.Class#forName(java.lang.String) + path ::= @see org.apache.hadoop.fs.Path#Path(java.lang.String) + } + Reads expression from the mapred.join.expr property and + user-supplied join types from mapred.join.define.<ident> + types. Paths supplied to tbl are given as input paths to the + InputFormat class listed. + @see #compose(java.lang.String, java.lang.Class, java.lang.String...)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ,

    ) }]]> + + + + + + + + (tbl(,),tbl(,),...,tbl(,)) }]]> + + + + + + + + (tbl(,),tbl(,),...,tbl(,)) }]]> + + + + mapred.join.define.<ident> to a classname. In the expression + mapred.join.expr, the identifier will be assumed to be a + ComposableRecordReader. + mapred.join.keycomparator can be a classname used to compare keys + in the join. + @see JoinRecordReader + @see MultiFilterRecordReader]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ...... + }]]> + + + + + + + + + + + + + + + + + + + + + capacity children to position + id in the parent reader. + The id of a root CompositeRecordReader is -1 by convention, but relying + on this is not recommended.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + override(S1,S2,S3) will prefer values + from S3 over S2, and values from S2 over S1 for all keys + emitted from all sources.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + [,,...,]]]> + + + + + + + out. + TupleWritable format: + {@code + ...... + }]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It has to be specified how key and values are passed from one element of + the chain to the next, by value or by reference. If a Mapper leverages the + assumed semantics that the key and values are not modified by the collector + 'by value' must be used. If the Mapper does not expect this semantics, as + an optimization to avoid serialization and deserialization 'by reference' + can be used. +

    + For the added Mapper the configuration given for it, + mapperConf, have precedence over the job's JobConf. This + precedence is in effect when the task is running. +

    + 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 +

    + + @param job job's JobConf to add the Mapper class. + @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 byValue indicates if key/values should be passed by value + to the next Mapper in the chain, if any. + @param mapperConf a JobConf with the configuration for the Mapper + class. It is recommended to use a JobConf without default values using the + JobConf(boolean loadDefaults) constructor with FALSE.]]> + + + + + + + If this method is overriden super.configure(...) should be + invoked at the beginning of the overwriter method.]]> + + + + + + + + + + map(...) methods of the Mappers in the chain.]]> + + + + + + + If this method is overriden super.close() should be + invoked at the end of the overwriter method.]]> + + + + + 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. +

    + 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. +

    + 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 maching output and input key and + value classes as no conversion is done by the chaining code. +

    + Using the ChainMapper and the ChainReducer classes is possible to compose + Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And + immediate benefit of this pattern is a dramatic reduction in disk IO. +

    + 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. +

    + ChainMapper usage pattern: +

    +

    + ...
    + conf.setJobName("chain");
    + conf.setInputFormat(TextInputFormat.class);
    + conf.setOutputFormat(TextOutputFormat.class);
    + 

    + JobConf mapAConf = new JobConf(false); + ... + ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class, + Text.class, Text.class, true, mapAConf); +

    + JobConf mapBConf = new JobConf(false); + ... + ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class, + LongWritable.class, Text.class, false, mapBConf); +

    + JobConf reduceConf = new JobConf(false); + ... + ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class, + Text.class, Text.class, true, reduceConf); +

    + ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class, + LongWritable.class, Text.class, false, null); +

    + ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class, + LongWritable.class, LongWritable.class, true, null); +

    + FileInputFormat.setInputPaths(conf, inDir); + FileOutputFormat.setOutputPath(conf, outDir); + ... +

    + JobClient jc = new JobClient(conf); + RunningJob job = jc.submitJob(conf); + ... +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + It has to be specified how key and values are passed from one element of + the chain to the next, by value or by reference. If a Reducer leverages the + assumed semantics that the key and values are not modified by the collector + 'by value' must be used. If the Reducer does not expect this semantics, as + an optimization to avoid serialization and deserialization 'by reference' + can be used. +

    + For the added Reducer the configuration given for it, + reducerConf, have precedence over the job's JobConf. This + precedence is in effect when the task is running. +

    + 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. + + @param job job's JobConf to add the Reducer class. + @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 byValue indicates if key/values should be passed by value + to the next Mapper in the chain, if any. + @param reducerConf a JobConf with the configuration for the Reducer + class. It is recommended to use a JobConf without default values using the + JobConf(boolean loadDefaults) constructor with FALSE.]]> + + + + + + + + + + + + + + It has to be specified how key and values are passed from one element of + the chain to the next, by value or by reference. If a Mapper leverages the + assumed semantics that the key and values are not modified by the collector + 'by value' must be used. If the Mapper does not expect this semantics, as + an optimization to avoid serialization and deserialization 'by reference' + can be used. +

    + For the added Mapper the configuration given for it, + mapperConf, have precedence over the job's JobConf. This + precedence is in effect when the task is running. +

    + 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 + . + + @param job chain job's JobConf to add the Mapper class. + @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 byValue indicates if key/values should be passed by value + to the next Mapper in the chain, if any. + @param mapperConf a JobConf with the configuration for the Mapper + class. It is recommended to use a JobConf without default values using the + JobConf(boolean loadDefaults) constructor with FALSE.]]> + + + + + + + If this method is overriden super.configure(...) should be + invoked at the beginning of the overwriter method.]]> + + + + + + + + + + reduce(...) method of the Reducer with the + map(...) methods of the Mappers in the chain.]]> + + + + + + + If this method is overriden super.close() should be + invoked at the end of the overwriter method.]]> + + + + + For each record output by the Reducer, 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. +

    + 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. +

    + 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 maching output and input key and + value classes as no conversion is done by the chaining code. +

    + Using the ChainMapper and the ChainReducer classes is possible to compose + Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And + immediate benefit of this pattern is a dramatic reduction in disk IO. +

    + 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. +

    + ChainReducer usage pattern: +

    +

    + ...
    + conf.setJobName("chain");
    + conf.setInputFormat(TextInputFormat.class);
    + conf.setOutputFormat(TextOutputFormat.class);
    + 

    + JobConf mapAConf = new JobConf(false); + ... + ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class, + Text.class, Text.class, true, mapAConf); +

    + JobConf mapBConf = new JobConf(false); + ... + ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class, + LongWritable.class, Text.class, false, mapBConf); +

    + JobConf reduceConf = new JobConf(false); + ... + ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class, + Text.class, Text.class, true, reduceConf); +

    + ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class, + LongWritable.class, Text.class, false, null); +

    + ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class, + LongWritable.class, LongWritable.class, true, null); +

    + FileInputFormat.setInputPaths(conf, inDir); + FileOutputFormat.setOutputPath(conf, outDir); + ... +

    + JobClient jc = new JobClient(conf); + RunningJob job = jc.submitJob(conf); + ... +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + RecordReader's for CombineFileSplit's. + @see CombineFileSplit]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + th Path]]> + + + + + + th Path]]> + + + + + + + + + + + th Path]]> + + + + + + + + + + + + + + + + + + + + + + + + + + CombineFileSplit can be used to implement {@link org.apache.hadoop.mapred.RecordReader}'s, + with reading one record per file. + @see org.apache.hadoop.mapred.FileSplit + @see CombineFileInputFormat]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + all splits. + @param freq The frequency with which records will be emitted.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + all splits. + This will read every split at the client, which is very expensive. + @param freq Probability with which a key will be chosen. + @param numSamples Total number of samples to obtain from all selected + splits.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + all splits. + Takes the first numSamples / numSplits records from each split. + @param numSamples Total number of samples to obtain from all selected + splits.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the name output is multi, false + if it is single. If the name output is not defined it returns + false]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @param conf job conf to add the named output + @param namedOutput named output name, it has to be a word, letters + and numbers only, cannot be the word 'part' as + that is reserved for the + default output. + @param outputFormatClass OutputFormat class. + @param keyClass key class + @param valueClass value class]]> + + + + + + + + + + + + @param conf job conf to add the named output + @param namedOutput named output name, it has to be a word, letters + and numbers only, cannot be the word 'part' as + that is reserved for the + default output. + @param outputFormatClass OutputFormat class. + @param keyClass key class + @param valueClass value class]]> + + + + + + + + By default these counters are disabled. +

    + MultipleOutputs supports counters, by default the are disabled. + The counters group is the {@link MultipleOutputs} class name. +

    + The names of the counters are the same as the named outputs. For multi + named outputs the name of the counter is the concatenation of the named + output, and underscore '_' and the multiname. + + @param conf job conf to enableadd the named output. + @param enabled indicates if the counters will be enabled or not.]]> +
    +
    + + + + + By default these counters are disabled. +

    + MultipleOutputs supports counters, by default the are disabled. + The counters group is the {@link MultipleOutputs} class name. +

    + The names of the counters are the same as the named outputs. For multi + named outputs the name of the counter is the concatenation of the named + output, and underscore '_' and the multiname. + + + @param conf job conf to enableadd the named output. + @return TRUE if the counters are enabled, FALSE if they are disabled.]]> +
    +
    + + + + + + + + + + + + + @param namedOutput the named output name + @param reporter the reporter + @return the output collector for the given named output + @throws IOException thrown if output collector could not be created]]> + + + + + + + + + + + @param namedOutput the named output name + @param multiName the multi name part + @param reporter the reporter + @return the output collector for the given named output + @throws IOException thrown if output collector could not be created]]> + + + + + + + If overriden subclasses must invoke super.close() at the + end of their close() + + @throws java.io.IOException thrown if any of the MultipleOutput files + could not be closed properly.]]> + + + + OutputCollector passed to + the map() and reduce() methods of the + Mapper and Reducer implementations. +

    + Each additional output, or named output, may be configured with its own + OutputFormat, with its own key class and with its own value + class. +

    + A named output can be a single file or a multi file. The later is refered as + a multi named output. +

    + A multi named output is an unbound set of files all sharing the same + OutputFormat, key class and value class configuration. +

    + When named outputs are used within a Mapper implementation, + key/values written to a name output are not part of the reduce phase, only + key/values written to the job OutputCollector are part of the + reduce phase. +

    + MultipleOutputs supports counters, by default the are disabled. The counters + group is the {@link MultipleOutputs} class name. +

    + The names of the counters are the same as the named outputs. For multi + named outputs the name of the counter is the concatenation of the named + output, and underscore '_' and the multiname. +

    + Job configuration usage pattern is: +

    +
    + JobConf conf = new JobConf();
    +
    + conf.setInputPath(inDir);
    + FileOutputFormat.setOutputPath(conf, outDir);
    +
    + conf.setMapperClass(MOMap.class);
    + conf.setReducerClass(MOReduce.class);
    + ...
    +
    + // Defines additional single text based output 'text' for the job
    + MultipleOutputs.addNamedOutput(conf, "text", TextOutputFormat.class,
    + LongWritable.class, Text.class);
    +
    + // Defines additional multi sequencefile based output 'sequence' for the
    + // job
    + MultipleOutputs.addMultiNamedOutput(conf, "seq",
    +   SequenceFileOutputFormat.class,
    +   LongWritable.class, Text.class);
    + ...
    +
    + JobClient jc = new JobClient();
    + RunningJob job = jc.submitJob(conf);
    +
    + ...
    + 
    +

    + Job configuration usage pattern is: +

    +
    + public class MOReduce implements
    +   Reducer<WritableComparable, Writable> {
    + private MultipleOutputs mos;
    +
    + public void configure(JobConf conf) {
    + ...
    + mos = new MultipleOutputs(conf);
    + }
    +
    + public void reduce(WritableComparable key, Iterator<Writable> values,
    + OutputCollector output, Reporter reporter)
    + throws IOException {
    + ...
    + mos.getCollector("text", reporter).collect(key, new Text("Hello"));
    + mos.getCollector("seq", "A", reporter).collect(key, new Text("Bye"));
    + mos.getCollector("seq", "B", reporter).collect(key, new Text("Chau"));
    + ...
    + }
    +
    + public void close() throws IOException {
    + mos.close();
    + ...
    + }
    +
    + }
    + 
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It can be used instead of the default implementation, + @link org.apache.hadoop.mapred.MapRunner, when the Map operation is not CPU + bound in order to improve throughput. +

    + Map implementations using this MapRunnable must be thread-safe. +

    + The Map-Reduce job has to be configured to use this MapRunnable class (using + the JobConf.setMapRunnerClass method) and + the number of thread the thread-pool can use with the + mapred.map.multithreadedrunner.threads property, its default + value is 10 threads. +

    ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + pairs. Uses + {@link StringTokenizer} to break text into tokens. + @deprecated Use + {@link org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper} instead.]]> + + + + + + + + + + + + total.order.partitioner.natural.order is not false, a trie + of the first total.order.partitioner.max.trie.depth(2) + 1 bytes + will be built. Otherwise, keys will be located using a binary search of + the partition keyset using the {@link org.apache.hadoop.io.RawComparator} + defined for this job. The input file must be sorted with the same + comparator and contain {@link + org.apache.hadoop.mapred.JobConf#getNumReduceTasks} - 1 keys.]]> + + + + + + + + + + + + R reduces, there are R-1 + keys in the SequenceFile.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + generateKeyValPairs(Object key, Object value); public void + configure(JobConfjob); } + + The package also provides a base class, ValueAggregatorBaseDescriptor, + implementing the above interface. The user can extend the base class and + implement generateKeyValPairs accordingly. + + The primary work of generateKeyValPairs is to emit one or more key/value + pairs based on the input key/value pair. The key in an output key/value pair + encode two pieces of information: aggregation type and aggregation id. The + value will be aggregated onto the aggregation id according the aggregation + type. + + This class offers a function to generate a map/reduce job using Aggregate + framework. The function takes the following parameters: input directory spec + input format (text or sequence file) output directory a file specifying the + user plugin class]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The job can be configured using the static methods in this class, + {@link DBInputFormat}, and {@link DBOutputFormat}. +

    + Alternatively, the properties can be set in the configuration with proper + values. + + @see DBConfiguration#configureDB(JobConf, String, String, String, String) + @see DBInputFormat#setInput(JobConf, Class, String, String) + @see DBInputFormat#setInput(JobConf, Class, String, String, String, String...) + @see DBOutputFormat#setOutput(JobConf, String, String...)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 20070101 AND length > 0)' + @param orderBy the fieldNames in the orderBy clause. + @param fieldNames The field names in the table + @see #setInput(JobConf, Class, String, String)]]> + + + + + + + + + + + + + + DBInputFormat emits LongWritables containing the record number as + key and DBWritables as value. + + The SQL query, and input class can be using one of the two + setInput methods.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + {@link DBOutputFormat} accepts <key,value> pairs, where + key has a type extending DBWritable. Returned {@link RecordWriter} + writes only the key to the database with a batch SQL query.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DBWritable. DBWritable, is similar to {@link Writable} + except that the {@link #write(PreparedStatement)} method takes a + {@link PreparedStatement}, and {@link #readFields(ResultSet)} + takes a {@link ResultSet}. +

    + Implementations are responsible for writing the fields of the object + to PreparedStatement, and reading the fields of the object from the + ResultSet. + +

    Example:

    + If we have the following table in the database : +
    + CREATE TABLE MyTable (
    +   counter        INTEGER NOT NULL,
    +   timestamp      BIGINT  NOT NULL,
    + );
    + 
    + then we can read/write the tuples from/to the table with : +

    + public class MyWritable implements Writable, DBWritable {
    +   // Some data     
    +   private int counter;
    +   private long timestamp;
    +       
    +   //Writable#write() implementation
    +   public void write(DataOutput out) throws IOException {
    +     out.writeInt(counter);
    +     out.writeLong(timestamp);
    +   }
    +       
    +   //Writable#readFields() implementation
    +   public void readFields(DataInput in) throws IOException {
    +     counter = in.readInt();
    +     timestamp = in.readLong();
    +   }
    +       
    +   public void write(PreparedStatement statement) throws SQLException {
    +     statement.setInt(1, counter);
    +     statement.setLong(2, timestamp);
    +   }
    +       
    +   public void readFields(ResultSet resultSet) throws SQLException {
    +     counter = resultSet.getInt(1);
    +     timestamp = resultSet.getLong(2);
    +   } 
    + }
    + 

    ]]> +
    + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Counters represent global counters, defined either by the + Map-Reduce framework or applications. Each Counter is named by + an {@link Enum} and has a long for the value.

    + +

    Counters are bunched into Groups, each comprising of + counters from a particular Enum class.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Each {@link InputSplit} is then assigned to an individual {@link Mapper} + for processing.

    + +

    Note: The split is a logical split of the inputs and the + input files are not physically split into chunks. For e.g. a split could + be <input-file-path, start, offset> tuple. The InputFormat + also creates the {@link RecordReader} to read the {@link InputSplit}. + + @param context job configuration. + @return an array of {@link InputSplit}s for the job.]]> + + + + + + + + + + + + + InputFormat describes the input-specification for a + Map-Reduce job. + +

    The Map-Reduce framework relies on the InputFormat of the + job to:

    +

      +
    1. + Validate the input-specification of the job. +
    2. + Split-up the input file(s) into logical {@link InputSplit}s, each of + which is then assigned to an individual {@link Mapper}. +
    3. +
    4. + Provide the {@link RecordReader} implementation to be used to glean + input records from the logical InputSplit for processing by + the {@link Mapper}. +
    5. +
    + +

    The default behavior of file-based {@link InputFormat}s, typically + sub-classes of {@link FileInputFormat}, is to split the + input into logical {@link InputSplit}s based on the total size, in + bytes, of the input files. However, the {@link FileSystem} blocksize of + the input files is treated as an upper bound for input splits. A lower bound + on the split size can be set via + + mapred.min.split.size.

    + +

    Clearly, logical splits based on input-size is insufficient for many + applications since record boundaries are to respected. In such cases, the + application has to also implement a {@link RecordReader} on whom lies the + responsibility to respect record-boundaries and present a record-oriented + view of the logical InputSplit to the individual task. + + @see InputSplit + @see RecordReader + @see FileInputFormat]]> + + + + + + + + + + + + + + + + + + + + + + + InputSplit represents the data to be processed by an + individual {@link Mapper}. + +

    Typically, it presents a byte-oriented view on the input and is the + responsibility of {@link RecordReader} of the job to process this and present + a record-oriented view. + + @see InputFormat + @see RecordReader]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + InputFormat to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + OutputFormat to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + Mapper to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + + + + + + + + + + + + + + + + + + + Reducer to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + Partitioner to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + progress of the job's map-tasks, as a float between 0.0 + and 1.0. When all map tasks have completed, the function returns 1.0. + + @return the progress of the job's map-tasks. + @throws IOException]]> + + + + + + progress of the job's reduce-tasks, as a float between 0.0 + and 1.0. When all reduce tasks have completed, the function returns 1.0. + + @return the progress of the job's reduce-tasks. + @throws IOException]]> + + + + + + true if the job is complete, else false. + @throws IOException]]> + + + + + + true if the job succeeded, else false. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobTracker is lost]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1. + @return the number of reduce tasks for this job.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + An example JobID is : + job_200707121733_0003 , which represents the third job + running at the jobtracker started at 200707121733. +

    + Applications should never construct or parse JobID strings, but rather + use appropriate constructors or {@link #forName(String)} method. + + @see TaskID + @see TaskAttemptID + @see org.apache.hadoop.mapred.JobTracker#getNewJobId() + @see org.apache.hadoop.mapred.JobTracker#getStartTime()]]> + + + + + + + + + + + + + + + + + + + + + + + + + + the key input type to the Mapper + @param the value input type to the Mapper + @param the key output type from the Mapper + @param the value output type from the Mapper]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Maps are the individual tasks which transform input records into a + intermediate records. The transformed intermediate records need not be of + the same type as the input records. A given input pair may map to zero or + many output pairs.

    + +

    The Hadoop Map-Reduce framework spawns one map task for each + {@link InputSplit} generated by the {@link InputFormat} for the job. + Mapper implementations can access the {@link Configuration} for + the job via the {@link JobContext#getConfiguration()}. + +

    The framework first calls + {@link #setup(org.apache.hadoop.mapreduce.Mapper.Context)}, followed by + {@link #map(Object, Object, Context)} + for each key/value pair in the InputSplit. Finally + {@link #cleanup(Context)} is called.

    + +

    All intermediate values associated with a given output key are + subsequently grouped by the framework, and passed to a {@link Reducer} to + determine the final output. Users can control the sorting and grouping by + specifying two key {@link RawComparator} classes.

    + +

    The Mapper outputs are partitioned per + Reducer. Users can control which keys (and hence records) go to + which Reducer by implementing a custom {@link Partitioner}. + +

    Users can optionally specify a combiner, via + {@link Job#setCombinerClass(Class)}, to perform local aggregation of the + intermediate outputs, which helps to cut down the amount of data transferred + from the Mapper to the Reducer. + +

    Applications can specify if and how the intermediate + outputs are to be compressed and which {@link CompressionCodec}s are to be + used via the Configuration.

    + +

    If the job has zero + reduces then the output of the Mapper is directly written + to the {@link OutputFormat} without sorting by keys.

    + +

    Example:

    +

    + public class TokenCounterMapper 
    +     extends Mapper{
    +    
    +   private final static IntWritable one = new IntWritable(1);
    +   private Text word = new Text();
    +   
    +   public void map(Object key, Text value, Context context) throws IOException {
    +     StringTokenizer itr = new StringTokenizer(value.toString());
    +     while (itr.hasMoreTokens()) {
    +       word.set(itr.nextToken());
    +       context.collect(word, one);
    +     }
    +   }
    + }
    + 

    + +

    Applications may override the {@link #run(Context)} method to exert + greater control on map processing e.g. multi-threaded Mappers + etc.

    + + @see InputFormat + @see JobContext + @see Partitioner + @see Reducer]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + OutputCommitter describes the commit of task output for a + Map-Reduce job. + +

    The Map-Reduce framework relies on the OutputCommitter of + the job to:

    +

      +
    1. + Setup the job during initialization. For example, create the temporary + output directory for the job during the initialization of the job. +
    2. +
    3. + Cleanup the job after the job completion. For example, remove the + temporary output directory after the job completion. +
    4. +
    5. + Setup the task temporary output. +
    6. +
    7. + Check whether a task needs a commit. This is to avoid the commit + procedure if a task does not need commit. +
    8. +
    9. + Commit of the task output. +
    10. +
    11. + Discard the task commit. +
    12. +
    + + @see org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + @see JobContext + @see TaskAttemptContext]]> +
    +
    + + + + + + + + + + + + + + + + + + + This is to validate the output specification for the job when it is + a job is submitted. Typically checks that it does not already exist, + throwing an exception when it already exists, so that output is not + overwritten.

    + + @param context information about the job + @throws IOException when output should not be attempted]]> +
    +
    + + + + + + + + + + OutputFormat describes the output-specification for a + Map-Reduce job. + +

    The Map-Reduce framework relies on the OutputFormat of the + job to:

    +

      +
    1. + Validate the output-specification of the job. For e.g. check that the + output directory doesn't already exist. +
    2. + Provide the {@link RecordWriter} implementation to be used to write out + the output files of the job. Output files are stored in a + {@link FileSystem}. +
    3. +
    + + @see RecordWriter]]> +
    +
    + + + + + + + + + + + Typically a hash function on a all or a subset of the key.

    + + @param key the key to be partioned. + @param value the entry value. + @param numPartitions the total number of partitions. + @return the partition number for the key.]]> +
    +
    + + Partitioner controls the partitioning of the keys of the + intermediate map-outputs. The key (or a subset of the key) is used to derive + the partition, typically by a hash function. The total number of partitions + is the same as the number of reduce tasks for the job. Hence this controls + which of the m reduce tasks the intermediate key (and hence the + record) is sent for reduction.

    + + @see Reducer]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @param ]]> + + + + + + + + + + + + + + + + + + + + + + RecordWriter to future operations. + + @param context the context of the task + @throws IOException]]> + + + + RecordWriter writes the output <key, value> pairs + to an output file. + +

    RecordWriter implementations write the job outputs to the + {@link FileSystem}. + + @see OutputFormat]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the class of the input keys + @param the class of the input values + @param the class of the output keys + @param the class of the output values]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Reducer implementations + can access the {@link Configuration} for the job via the + {@link JobContext#getConfiguration()} method.

    + +

    Reducer has 3 primary phases:

    +
      +
    1. + +

      Shuffle

      + +

      The Reducer copies the sorted output from each + {@link Mapper} using HTTP across the network.

      +
    2. + +
    3. +

      Sort

      + +

      The framework merge sorts Reducer inputs by + keys + (since different Mappers may have output the same key).

      + +

      The shuffle and sort phases occur simultaneously i.e. while outputs are + being fetched they are merged.

      + +
      SecondarySort
      + +

      To achieve a secondary sort on the values returned by the value + iterator, the application should extend the key with the secondary + key and define a grouping comparator. The keys will be sorted using the + entire key, but will be grouped using the grouping comparator to decide + which keys and values are sent in the same call to reduce.The grouping + comparator is specified via + {@link Job#setGroupingComparatorClass(Class)}. The sort order is + controlled by + {@link Job#setSortComparatorClass(Class)}.

      + + + For example, say that you want to find duplicate web pages and tag them + all with the url of the "best" known example. You would set up the job + like: +
        +
      • Map Input Key: url
      • +
      • Map Input Value: document
      • +
      • Map Output Key: document checksum, url pagerank
      • +
      • Map Output Value: url
      • +
      • Partitioner: by checksum
      • +
      • OutputKeyComparator: by checksum and then decreasing pagerank
      • +
      • OutputValueGroupingComparator: by checksum
      • +
      +
    4. + +
    5. +

      Reduce

      + +

      In this phase the + {@link #reduce(Object, Iterable, Context)} + method is called for each <key, (collection of values)> in + the sorted inputs.

      +

      The output of the reduce task is typically written to a + {@link RecordWriter} via + {@link Context#write(Object, Object)}.

      +
    6. +
    + +

    The output of the Reducer is not re-sorted.

    + +

    Example:

    +

    + public class IntSumReducer extends Reducer {
    +   private IntWritable result = new IntWritable();
    + 
    +   public void reduce(Key key, Iterable values, 
    +                      Context context) throws IOException {
    +     int sum = 0;
    +     for (IntWritable val : values) {
    +       sum += val.get();
    +     }
    +     result.set(sum);
    +     context.collect(key, result);
    +   }
    + }
    + 

    + + @see Mapper + @see Partitioner]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + An example TaskAttemptID is : + attempt_200707121733_0003_m_000005_0 , which represents the + zeroth task attempt for the fifth map task in the third job + running at the jobtracker started at 200707121733. +

    + Applications should never construct or parse TaskAttemptID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + An example TaskID is : + task_200707121733_0003_m_000005 , which represents the + fifth map task in the third job running at the jobtracker + started at 200707121733. +

    + Applications should never construct or parse TaskID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskAttemptID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the input key type for the task + @param the input value type for the task + @param the output key type for the task + @param the output value type for the task]]> + + + + + + + + + + + + + + + + + + + FileInputFormat implementations can override this and return + false to ensure that individual input files are never split-up + so that {@link Mapper}s process entire files. + + @param context the job context + @param filename the file name to check + @return is this file splitable?]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FileInputFormat is the base class for all file-based + InputFormats. This provides a generic implementation of + {@link #getSplits(JobContext)}. + Subclasses of FileInputFormat can also override the + {@link #isSplitable(JobContext, Path)} method to ensure input-files are + not split-up and are processed as a whole by {@link Mapper}s.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the map's input key type + @param the map's input value type + @param the map's output key type + @param the map's output value type + @param job the job + @return the mapper class to run]]> + + + + + + + the map input key type + @param the map input value type + @param the map output key type + @param the map output value type + @param job the job to modify + @param cls the class to use as the mapper]]> + + + + + + + + + + + + + It can be used instead of the default implementation, + @link org.apache.hadoop.mapred.MapRunner, when the Map operation is not CPU + bound in order to improve throughput. +

    + Mapper implementations using this MapRunnable must be thread-safe. +

    + The Map-Reduce job has to be configured with the mapper to use via + {@link #setMapperClass(Configuration, Class)} and + the number of thread the thread-pool can use with the + {@link #getNumberOfThreads(Configuration) method. The default + value is 10 threads. +

    ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the job output should be compressed, + false otherwise]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tasks' Side-Effect Files + +

    Some applications need to create/write-to side-files, which differ from + the actual job-outputs. + +

    In such cases there could be issues with 2 instances of the same TIP + (running simultaneously e.g. speculative tasks) trying to open/write-to the + same file (path) on HDFS. Hence the application-writer will have to pick + unique names per task-attempt (e.g. using the attemptid, say + attempt_200709221812_0001_m_000000_0), not just per TIP.

    + +

    To get around this the Map-Reduce framework helps the application-writer + out by maintaining a special + ${mapred.output.dir}/_temporary/_${taskid} + sub-directory for each task-attempt on HDFS where the output of the + task-attempt goes. On successful completion of the task-attempt the files + in the ${mapred.output.dir}/_temporary/_${taskid} (only) + are promoted to ${mapred.output.dir}. Of course, the + framework discards the sub-directory of unsuccessful task-attempts. This + is completely transparent to the application.

    + +

    The application-writer can take advantage of this by creating any + side-files required in a work directory during execution + of his task i.e. via + {@link #getWorkOutputPath(TaskInputOutputContext)}, and + the framework will move them out similarly - thus she doesn't have to pick + unique paths per task-attempt.

    + +

    The entire discussion holds true for maps of jobs with + reducer=NONE (i.e. 0 reduces) since output of the map, in that case, + goes directly to HDFS.

    + + @return the {@link Path} to the task's temporary output directory + for the map-reduce job.]]> +
    + + + + + + + + + The path can be used to create custom files from within the map and + reduce tasks. The path name will be unique for each task. The path parent + will be the job output directory.

    ls + +

    This method uses the {@link #getUniqueFile} method to make the file name + unique for the task.

    + + @param context the context for the task. + @param name the name for the file. + @param extension the extension for the file + @return a unique path accross all tasks of the job.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This tool supports archiving and anaylzing (sort/grep) of log-files. + It takes as input + a) Input uri which will serve uris of the logs to be archived. + b) Output directory (not mandatory). + b) Directory on dfs to archive the logs. + c) The sort/grep patterns for analyzing the files and separator for boundaries. + Usage: + Logalyzer -archive -archiveDir -analysis -logs -grep -sort -separator +

    ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/third_party/hadoop-0.20.0/lib/jets3t-0.6.1.jar b/core/lib/hadoop-0.20.2/lib/jets3t-0.6.1.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/jets3t-0.6.1.jar rename to core/lib/hadoop-0.20.2/lib/jets3t-0.6.1.jar diff --git a/third_party/hadoop-0.20.0/lib/jetty-6.1.14.jar b/core/lib/hadoop-0.20.2/lib/jetty-6.1.14.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/jetty-6.1.14.jar rename to core/lib/hadoop-0.20.2/lib/jetty-6.1.14.jar diff --git a/third_party/hadoop-0.20.0/lib/jetty-util-6.1.14.jar b/core/lib/hadoop-0.20.2/lib/jetty-util-6.1.14.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/jetty-util-6.1.14.jar rename to core/lib/hadoop-0.20.2/lib/jetty-util-6.1.14.jar diff --git a/third_party/hadoop-0.20.0/lib/jsp-2.1/jsp-2.1.jar b/core/lib/hadoop-0.20.2/lib/jsp-2.1/jsp-2.1.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/jsp-2.1/jsp-2.1.jar rename to core/lib/hadoop-0.20.2/lib/jsp-2.1/jsp-2.1.jar diff --git a/third_party/hadoop-0.20.0/lib/jsp-2.1/jsp-api-2.1.jar b/core/lib/hadoop-0.20.2/lib/jsp-2.1/jsp-api-2.1.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/jsp-2.1/jsp-api-2.1.jar rename to core/lib/hadoop-0.20.2/lib/jsp-2.1/jsp-api-2.1.jar diff --git a/third_party/hadoop-0.20.0/lib/junit-3.8.1.jar b/core/lib/hadoop-0.20.2/lib/junit-3.8.1.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/junit-3.8.1.jar rename to core/lib/hadoop-0.20.2/lib/junit-3.8.1.jar diff --git a/third_party/hadoop-0.20.0/lib/kfs-0.2.2.jar b/core/lib/hadoop-0.20.2/lib/kfs-0.2.2.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/kfs-0.2.2.jar rename to core/lib/hadoop-0.20.2/lib/kfs-0.2.2.jar diff --git a/third_party/hadoop-0.20.0/lib/kfs-0.2.LICENSE.txt b/core/lib/hadoop-0.20.2/lib/kfs-0.2.LICENSE.txt similarity index 100% rename from third_party/hadoop-0.20.0/lib/kfs-0.2.LICENSE.txt rename to core/lib/hadoop-0.20.2/lib/kfs-0.2.LICENSE.txt diff --git a/third_party/hadoop-0.20.0/lib/log4j-1.2.15.jar b/core/lib/hadoop-0.20.2/lib/log4j-1.2.15.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/log4j-1.2.15.jar rename to core/lib/hadoop-0.20.2/lib/log4j-1.2.15.jar diff --git a/core/lib/hadoop-0.20.2/lib/mockito-all-1.8.0.jar b/core/lib/hadoop-0.20.2/lib/mockito-all-1.8.0.jar new file mode 100644 index 0000000000..3b336bf76c Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/mockito-all-1.8.0.jar differ diff --git a/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.a b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.a new file mode 100644 index 0000000000..b58ebdd435 Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.a differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.la b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.la similarity index 87% rename from third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.la rename to core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.la index 14941670b4..8d3da00ce6 100644 --- a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.la +++ b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.la @@ -14,7 +14,7 @@ library_names='libhadoop.so.1.0.0 libhadoop.so.1 libhadoop.so' old_library='libhadoop.a' # Libraries that this one depends upon. -dependency_libs=' -L/home/hadoopqa/tools/java/latest1.6-32/jre/lib/i386/server -ljvm -ldl' +dependency_libs=' -L/home/chrisdo/localwork/buildtools/buildtools/java/latest1.6-64/jre/lib/amd64/server -ljvm -ldl' # Version information for libhadoop. current=1 diff --git a/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.so b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.so new file mode 100644 index 0000000000..759d0bacec Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.so differ diff --git a/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.so.1 b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.so.1 new file mode 100644 index 0000000000..759d0bacec Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.so.1 differ diff --git a/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.so.1.0.0 b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.so.1.0.0 new file mode 100644 index 0000000000..759d0bacec Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/native/Linux-amd64-64/libhadoop.so.1.0.0 differ diff --git a/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.a b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.a new file mode 100644 index 0000000000..3ba7f1702d Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.a differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.la b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.la similarity index 87% rename from third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.la rename to core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.la index 2e772fb4ae..1905624b4b 100644 --- a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.la +++ b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.la @@ -14,7 +14,7 @@ library_names='libhadoop.so.1.0.0 libhadoop.so.1 libhadoop.so' old_library='libhadoop.a' # Libraries that this one depends upon. -dependency_libs=' -L/home/hadoopqa/tools/java/latest1.6-64/jre/lib/amd64/server -ljvm -ldl' +dependency_libs=' -L/home/chrisdo/localwork/buildtools/buildtools/java/latest1.6-32/jre/lib/i386/server -ljvm -ldl' # Version information for libhadoop. current=1 diff --git a/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.so b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.so new file mode 100644 index 0000000000..0716c1b7b8 Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.so differ diff --git a/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.so.1 b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.so.1 new file mode 100644 index 0000000000..0716c1b7b8 Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.so.1 differ diff --git a/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.so.1.0.0 b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.so.1.0.0 new file mode 100644 index 0000000000..0716c1b7b8 Binary files /dev/null and b/core/lib/hadoop-0.20.2/lib/native/Linux-i386-32/libhadoop.so.1.0.0 differ diff --git a/third_party/hadoop-0.20.0/lib/oro-2.0.8.jar b/core/lib/hadoop-0.20.2/lib/oro-2.0.8.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/oro-2.0.8.jar rename to core/lib/hadoop-0.20.2/lib/oro-2.0.8.jar diff --git a/third_party/hadoop-0.20.0/lib/servlet-api-2.5-6.1.14.jar b/core/lib/hadoop-0.20.2/lib/servlet-api-2.5-6.1.14.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/servlet-api-2.5-6.1.14.jar rename to core/lib/hadoop-0.20.2/lib/servlet-api-2.5-6.1.14.jar diff --git a/third_party/hadoop-0.20.0/lib/xmlenc-0.52.jar b/core/lib/hadoop-0.20.2/lib/xmlenc-0.52.jar similarity index 100% rename from third_party/hadoop-0.20.0/lib/xmlenc-0.52.jar rename to core/lib/hadoop-0.20.2/lib/xmlenc-0.52.jar diff --git a/third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar b/core/lib/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar similarity index 100% rename from third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar rename to core/lib/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar diff --git a/third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar b/core/lib/jetty-7.1.6.v20100715/servlet-api-2.5.jar similarity index 100% rename from third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar rename to core/lib/jetty-7.1.6.v20100715/servlet-api-2.5.jar diff --git a/core/lib/jline.jar b/core/lib/jline.jar new file mode 100644 index 0000000000..6ed67faab6 Binary files /dev/null and b/core/lib/jline.jar differ diff --git a/third_party/liblzf-3.5/Changes b/core/lib/liblzf-3.5/Changes similarity index 100% rename from third_party/liblzf-3.5/Changes rename to core/lib/liblzf-3.5/Changes diff --git a/third_party/liblzf-3.5/LICENSE b/core/lib/liblzf-3.5/LICENSE similarity index 100% rename from third_party/liblzf-3.5/LICENSE rename to core/lib/liblzf-3.5/LICENSE diff --git a/third_party/liblzf-3.5/Makefile b/core/lib/liblzf-3.5/Makefile similarity index 100% rename from third_party/liblzf-3.5/Makefile rename to core/lib/liblzf-3.5/Makefile diff --git a/third_party/liblzf-3.5/Makefile.in b/core/lib/liblzf-3.5/Makefile.in similarity index 100% rename from third_party/liblzf-3.5/Makefile.in rename to core/lib/liblzf-3.5/Makefile.in diff --git a/third_party/liblzf-3.5/README b/core/lib/liblzf-3.5/README similarity index 100% rename from third_party/liblzf-3.5/README rename to core/lib/liblzf-3.5/README diff --git a/third_party/liblzf-3.5/config.h b/core/lib/liblzf-3.5/config.h similarity index 100% rename from third_party/liblzf-3.5/config.h rename to core/lib/liblzf-3.5/config.h diff --git a/third_party/liblzf-3.5/config.h.in b/core/lib/liblzf-3.5/config.h.in similarity index 100% rename from third_party/liblzf-3.5/config.h.in rename to core/lib/liblzf-3.5/config.h.in diff --git a/third_party/liblzf-3.5/config.log b/core/lib/liblzf-3.5/config.log similarity index 100% rename from third_party/liblzf-3.5/config.log rename to core/lib/liblzf-3.5/config.log diff --git a/third_party/liblzf-3.5/config.status b/core/lib/liblzf-3.5/config.status similarity index 100% rename from third_party/liblzf-3.5/config.status rename to core/lib/liblzf-3.5/config.status diff --git a/third_party/liblzf-3.5/configure b/core/lib/liblzf-3.5/configure similarity index 100% rename from third_party/liblzf-3.5/configure rename to core/lib/liblzf-3.5/configure diff --git a/third_party/liblzf-3.5/configure.ac b/core/lib/liblzf-3.5/configure.ac similarity index 100% rename from third_party/liblzf-3.5/configure.ac rename to core/lib/liblzf-3.5/configure.ac diff --git a/third_party/liblzf-3.5/crc32.h b/core/lib/liblzf-3.5/crc32.h similarity index 100% rename from third_party/liblzf-3.5/crc32.h rename to core/lib/liblzf-3.5/crc32.h diff --git a/third_party/liblzf-3.5/cs/CLZF.cs b/core/lib/liblzf-3.5/cs/CLZF.cs similarity index 100% rename from third_party/liblzf-3.5/cs/CLZF.cs rename to core/lib/liblzf-3.5/cs/CLZF.cs diff --git a/third_party/liblzf-3.5/cs/README b/core/lib/liblzf-3.5/cs/README similarity index 100% rename from third_party/liblzf-3.5/cs/README rename to core/lib/liblzf-3.5/cs/README diff --git a/third_party/liblzf-3.5/install-sh b/core/lib/liblzf-3.5/install-sh similarity index 100% rename from third_party/liblzf-3.5/install-sh rename to core/lib/liblzf-3.5/install-sh diff --git a/third_party/liblzf-3.5/lzf.c b/core/lib/liblzf-3.5/lzf.c similarity index 100% rename from third_party/liblzf-3.5/lzf.c rename to core/lib/liblzf-3.5/lzf.c diff --git a/third_party/liblzf-3.5/lzf.h b/core/lib/liblzf-3.5/lzf.h similarity index 100% rename from third_party/liblzf-3.5/lzf.h rename to core/lib/liblzf-3.5/lzf.h diff --git a/third_party/liblzf-3.5/lzfP.h b/core/lib/liblzf-3.5/lzfP.h similarity index 100% rename from third_party/liblzf-3.5/lzfP.h rename to core/lib/liblzf-3.5/lzfP.h diff --git a/third_party/liblzf-3.5/lzf_c.c b/core/lib/liblzf-3.5/lzf_c.c similarity index 100% rename from third_party/liblzf-3.5/lzf_c.c rename to core/lib/liblzf-3.5/lzf_c.c diff --git a/third_party/liblzf-3.5/lzf_d.c b/core/lib/liblzf-3.5/lzf_d.c similarity index 100% rename from third_party/liblzf-3.5/lzf_d.c rename to core/lib/liblzf-3.5/lzf_d.c diff --git a/third_party/mesos.jar b/core/lib/mesos.jar similarity index 100% rename from third_party/mesos.jar rename to core/lib/mesos.jar diff --git a/third_party/scalacheck_2.8.0-1.7.jar b/core/lib/scalacheck_2.8.0-1.7.jar similarity index 100% rename from third_party/scalacheck_2.8.0-1.7.jar rename to core/lib/scalacheck_2.8.0-1.7.jar diff --git a/third_party/scalatest-1.2/LICENSE b/core/lib/scalatest-1.2/LICENSE similarity index 100% rename from third_party/scalatest-1.2/LICENSE rename to core/lib/scalatest-1.2/LICENSE diff --git a/third_party/scalatest-1.2/NOTICE b/core/lib/scalatest-1.2/NOTICE similarity index 100% rename from third_party/scalatest-1.2/NOTICE rename to core/lib/scalatest-1.2/NOTICE diff --git a/third_party/scalatest-1.2/README.txt b/core/lib/scalatest-1.2/README.txt similarity index 100% rename from third_party/scalatest-1.2/README.txt rename to core/lib/scalatest-1.2/README.txt diff --git a/third_party/scalatest-1.2/scalatest-1.2.jar b/core/lib/scalatest-1.2/scalatest-1.2.jar similarity index 100% rename from third_party/scalatest-1.2/scalatest-1.2.jar rename to core/lib/scalatest-1.2/scalatest-1.2.jar diff --git a/third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar b/core/lib/slf4j-1.6.1/slf4j-api-1.6.1.jar similarity index 100% rename from third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar rename to core/lib/slf4j-1.6.1/slf4j-api-1.6.1.jar diff --git a/third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar b/core/lib/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar similarity index 100% rename from third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar rename to core/lib/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar diff --git a/src/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala similarity index 100% rename from src/scala/spark/Accumulators.scala rename to core/src/main/scala/spark/Accumulators.scala diff --git a/src/scala/spark/BasicLocalFileShuffle.scala b/core/src/main/scala/spark/BasicLocalFileShuffle.scala similarity index 100% rename from src/scala/spark/BasicLocalFileShuffle.scala rename to core/src/main/scala/spark/BasicLocalFileShuffle.scala diff --git a/core/src/main/scala/spark/BitTorrentBroadcast.scala b/core/src/main/scala/spark/BitTorrentBroadcast.scala new file mode 100644 index 0000000000..e9339290cb --- /dev/null +++ b/core/src/main/scala/spark/BitTorrentBroadcast.scala @@ -0,0 +1,1236 @@ +package spark + +import java.io._ +import java.net._ +import java.util.{BitSet, Comparator, Random, Timer, TimerTask, UUID} +import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} + +import scala.collection.mutable.{ListBuffer, Map, Set} + +@serializable +class BitTorrentBroadcast[T] (@transient var value_ : T, isLocal: Boolean) +extends Broadcast[T] with Logging { + + def value = value_ + + BitTorrentBroadcast.synchronized { + BitTorrentBroadcast.values.put (uuid, value_) + } + + @transient var arrayOfBlocks: Array[BroadcastBlock] = null + @transient var hasBlocksBitVector: BitSet = null + @transient var numCopiesSent: Array[Int] = null + @transient var totalBytes = -1 + @transient var totalBlocks = -1 + @transient var hasBlocks = 0 + + @transient var listenPortLock = new Object + @transient var guidePortLock = new Object + @transient var totalBlocksLock = new Object + + @transient var listOfSources = ListBuffer[SourceInfo] () + + @transient var serveMR: ServeMultipleRequests = null + + // Used only in Master + @transient var guideMR: GuideMultipleRequests = null + + // Used only in Workers + @transient var ttGuide: TalkToGuide = null + + @transient var rxSpeeds = new SpeedTracker + @transient var txSpeeds = new SpeedTracker + + @transient var hostAddress = InetAddress.getLocalHost.getHostAddress + @transient var listenPort = -1 + @transient var guidePort = -1 + + @transient var hasCopyInHDFS = false + @transient var stopBroadcast = false + + // Must call this after all the variables have been created/initialized + if (!isLocal) { + sendBroadcast + } + + def sendBroadcast (): Unit = { + logInfo ("Local host address: " + hostAddress) + + // Store a persistent copy in HDFS + // TODO: Turned OFF for now + // val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) + // out.writeObject (value_) + // out.close + // TODO: Fix this at some point + hasCopyInHDFS = true + + // Create a variableInfo object and store it in valueInfos + var variableInfo = blockifyObject (value_, BitTorrentBroadcast.BlockSize) + + // Prepare the value being broadcasted + // TODO: Refactoring and clean-up required here + arrayOfBlocks = variableInfo.arrayOfBlocks + totalBytes = variableInfo.totalBytes + totalBlocks = variableInfo.totalBlocks + hasBlocks = variableInfo.totalBlocks + + // Guide has all the blocks + hasBlocksBitVector = new BitSet (totalBlocks) + hasBlocksBitVector.set (0, totalBlocks) + + // Guide still hasn't sent any block + numCopiesSent = new Array[Int] (totalBlocks) + + guideMR = new GuideMultipleRequests + guideMR.setDaemon (true) + guideMR.start + logInfo ("GuideMultipleRequests started...") + + // Must always come AFTER guideMR is created + while (guidePort == -1) { + guidePortLock.synchronized { + guidePortLock.wait + } + } + + serveMR = new ServeMultipleRequests + serveMR.setDaemon (true) + serveMR.start + logInfo ("ServeMultipleRequests started...") + + // Must always come AFTER serveMR is created + while (listenPort == -1) { + listenPortLock.synchronized { + listenPortLock.wait + } + } + + // Must always come AFTER listenPort is created + val masterSource = + SourceInfo (hostAddress, listenPort, totalBlocks, totalBytes) + hasBlocksBitVector.synchronized { + masterSource.hasBlocksBitVector = hasBlocksBitVector + } + + // In the beginning, this is the only known source to Guide + listOfSources = listOfSources + masterSource + + // Register with the Tracker + BitTorrentBroadcast.registerValue (uuid, + SourceInfo (hostAddress, guidePort, totalBlocks, totalBytes)) + } + + private def readObject (in: ObjectInputStream): Unit = { + in.defaultReadObject + BitTorrentBroadcast.synchronized { + val cachedVal = BitTorrentBroadcast.values.get (uuid) + + if (cachedVal != null) { + value_ = cachedVal.asInstanceOf[T] + } else { + // Only the first worker in a node can ever be inside this 'else' + initializeWorkerVariables + + logInfo ("Local host address: " + hostAddress) + + // Start local ServeMultipleRequests thread first + serveMR = new ServeMultipleRequests + serveMR.setDaemon (true) + serveMR.start + logInfo ("ServeMultipleRequests started...") + + val start = System.nanoTime + + val receptionSucceeded = receiveBroadcast (uuid) + // If does not succeed, then get from HDFS copy + if (receptionSucceeded) { + value_ = unBlockifyObject[T] + BitTorrentBroadcast.values.put (uuid, value_) + } else { + // TODO: This part won't work, cause HDFS writing is turned OFF + val fileIn = new ObjectInputStream(DfsBroadcast.openFileForReading(uuid)) + value_ = fileIn.readObject.asInstanceOf[T] + BitTorrentBroadcast.values.put(uuid, value_) + fileIn.close + } + + val time = (System.nanoTime - start) / 1e9 + logInfo("Reading Broadcasted variable " + uuid + " took " + time + " s") + } + } + } + + // Initialize variables in the worker node. Master sends everything as 0/null + private def initializeWorkerVariables: Unit = { + arrayOfBlocks = null + hasBlocksBitVector = null + numCopiesSent = null + totalBytes = -1 + totalBlocks = -1 + hasBlocks = 0 + + listenPortLock = new Object + totalBlocksLock = new Object + + serveMR = null + ttGuide = null + + rxSpeeds = new SpeedTracker + txSpeeds = new SpeedTracker + + hostAddress = InetAddress.getLocalHost.getHostAddress + listenPort = -1 + + listOfSources = ListBuffer[SourceInfo] () + + stopBroadcast = false + } + + private def blockifyObject (obj: T, blockSize: Int): VariableInfo = { + val baos = new ByteArrayOutputStream + val oos = new ObjectOutputStream (baos) + oos.writeObject (obj) + oos.close + baos.close + val byteArray = baos.toByteArray + val bais = new ByteArrayInputStream (byteArray) + + var blockNum = (byteArray.length / blockSize) + if (byteArray.length % blockSize != 0) + blockNum += 1 + + var retVal = new Array[BroadcastBlock] (blockNum) + var blockID = 0 + + for (i <- 0 until (byteArray.length, blockSize)) { + val thisBlockSize = Math.min (blockSize, byteArray.length - i) + var tempByteArray = new Array[Byte] (thisBlockSize) + val hasRead = bais.read (tempByteArray, 0, thisBlockSize) + + retVal (blockID) = new BroadcastBlock (blockID, tempByteArray) + blockID += 1 + } + bais.close + + var variableInfo = VariableInfo (retVal, blockNum, byteArray.length) + variableInfo.hasBlocks = blockNum + + return variableInfo + } + + private def unBlockifyObject[A]: A = { + var retByteArray = new Array[Byte] (totalBytes) + for (i <- 0 until totalBlocks) { + System.arraycopy (arrayOfBlocks(i).byteArray, 0, retByteArray, + i * BitTorrentBroadcast.BlockSize, arrayOfBlocks(i).byteArray.length) + } + byteArrayToObject (retByteArray) + } + + private def byteArrayToObject[A] (bytes: Array[Byte]): A = { + val in = new ObjectInputStream (new ByteArrayInputStream (bytes)) + val retVal = in.readObject.asInstanceOf[A] + in.close + return retVal + } + + private def getLocalSourceInfo: SourceInfo = { + // Wait till hostName and listenPort are OK + while (listenPort == -1) { + listenPortLock.synchronized { + listenPortLock.wait + } + } + + // Wait till totalBlocks and totalBytes are OK + while (totalBlocks == -1) { + totalBlocksLock.synchronized { + totalBlocksLock.wait + } + } + + var localSourceInfo = SourceInfo (hostAddress, listenPort, totalBlocks, + totalBytes) + + localSourceInfo.hasBlocks = hasBlocks + + hasBlocksBitVector.synchronized { + localSourceInfo.hasBlocksBitVector = hasBlocksBitVector + } + + return localSourceInfo + } + + // Add new SourceInfo to the listOfSources. Update if it exists already. + // TODO: Optimizing just by OR-ing the BitVectors was BAD for performance + private def addToListOfSources (newSourceInfo: SourceInfo): Unit = { + listOfSources.synchronized { + if (listOfSources.contains(newSourceInfo)) { + listOfSources = listOfSources - newSourceInfo + } + listOfSources = listOfSources + newSourceInfo + } + } + + private def addToListOfSources (newSourceInfos: ListBuffer[SourceInfo]): Unit = { + newSourceInfos.foreach { newSourceInfo => + addToListOfSources (newSourceInfo) + } + } + + class TalkToGuide (gInfo: SourceInfo) + extends Thread with Logging { + override def run: Unit = { + + // Keep exchaning information until all blocks have been received + while (hasBlocks < totalBlocks) { + talkOnce + Thread.sleep (BitTorrentBroadcast.ranGen.nextInt ( + BitTorrentBroadcast.MaxKnockInterval - BitTorrentBroadcast.MinKnockInterval) + + BitTorrentBroadcast.MinKnockInterval) + } + + // Talk one more time to let the Guide know of reception completion + talkOnce + } + + // Connect to Guide and send this worker's information + private def talkOnce: Unit = { + var clientSocketToGuide: Socket = null + var oosGuide: ObjectOutputStream = null + var oisGuide: ObjectInputStream = null + + clientSocketToGuide = new Socket(gInfo.hostAddress, gInfo.listenPort) + oosGuide = new ObjectOutputStream (clientSocketToGuide.getOutputStream) + oosGuide.flush + oisGuide = new ObjectInputStream (clientSocketToGuide.getInputStream) + + // Send local information + oosGuide.writeObject(getLocalSourceInfo) + oosGuide.flush + + // Receive source information from Guide + var suitableSources = + oisGuide.readObject.asInstanceOf[ListBuffer[SourceInfo]] + logInfo("Received suitableSources from Master " + suitableSources) + + addToListOfSources (suitableSources) + + oisGuide.close + oosGuide.close + clientSocketToGuide.close + } + } + + def getGuideInfo (variableUUID: UUID): SourceInfo = { + var clientSocketToTracker: Socket = null + var oosTracker: ObjectOutputStream = null + var oisTracker: ObjectInputStream = null + + var gInfo: SourceInfo = SourceInfo ("", SourceInfo.TxOverGoToHDFS, + SourceInfo.UnusedParam, SourceInfo.UnusedParam) + + var retriesLeft = BitTorrentBroadcast.MaxRetryCount + do { + try { + // Connect to the tracker to find out GuideInfo + val clientSocketToTracker = + new Socket(BitTorrentBroadcast.MasterHostAddress, BitTorrentBroadcast.MasterTrackerPort) + val oosTracker = + new ObjectOutputStream (clientSocketToTracker.getOutputStream) + oosTracker.flush + val oisTracker = + new ObjectInputStream (clientSocketToTracker.getInputStream) + + // Send UUID and receive GuideInfo + oosTracker.writeObject (uuid) + oosTracker.flush + gInfo = oisTracker.readObject.asInstanceOf[SourceInfo] + } catch { + case e: Exception => { + logInfo ("getGuideInfo had a " + e) + } + } finally { + if (oisTracker != null) { + oisTracker.close + } + if (oosTracker != null) { + oosTracker.close + } + if (clientSocketToTracker != null) { + clientSocketToTracker.close + } + } + + Thread.sleep (BitTorrentBroadcast.ranGen.nextInt ( + BitTorrentBroadcast.MaxKnockInterval - BitTorrentBroadcast.MinKnockInterval) + + BitTorrentBroadcast.MinKnockInterval) + + retriesLeft -= 1 + } while (retriesLeft > 0 && gInfo.listenPort == SourceInfo.TxNotStartedRetry) + + logInfo ("Got this guidePort from Tracker: " + gInfo.listenPort) + return gInfo + } + + def receiveBroadcast (variableUUID: UUID): Boolean = { + val gInfo = getGuideInfo (variableUUID) + + if (gInfo.listenPort == SourceInfo.TxOverGoToHDFS || + gInfo.listenPort == SourceInfo.TxNotStartedRetry) { + // TODO: SourceInfo.TxNotStartedRetry is not really in use because we go + // to HDFS anyway when receiveBroadcast returns false + return false + } + + // Wait until hostAddress and listenPort are created by the + // ServeMultipleRequests thread + while (listenPort == -1) { + listenPortLock.synchronized { + listenPortLock.wait + } + } + + // Setup initial states of variables + totalBlocks = gInfo.totalBlocks + arrayOfBlocks = new Array[BroadcastBlock] (totalBlocks) + hasBlocksBitVector = new BitSet (totalBlocks) + numCopiesSent = new Array[Int] (totalBlocks) + totalBlocksLock.synchronized { + totalBlocksLock.notifyAll + } + totalBytes = gInfo.totalBytes + + // Start ttGuide to periodically talk to the Guide + var ttGuide = new TalkToGuide (gInfo) + ttGuide.setDaemon (true) + ttGuide.start + logInfo ("TalkToGuide started...") + + // Start pController to run TalkToPeer threads + var pcController = new PeerChatterController + pcController.setDaemon (true) + pcController.start + logInfo ("PeerChatterController started...") + + // TODO: Must fix this. This might never break if broadcast fails. + // We should be able to break and send false. Also need to kill threads + while (hasBlocks < totalBlocks) { + Thread.sleep(BitTorrentBroadcast.MaxKnockInterval) + } + + return true + } + + class PeerChatterController + extends Thread with Logging { + private var peersNowTalking = ListBuffer[SourceInfo] () + // TODO: There is a possible bug with blocksInRequestBitVector when a + // certain bit is NOT unset upon failure resulting in an infinite loop. + private var blocksInRequestBitVector = new BitSet (totalBlocks) + + override def run: Unit = { + var threadPool = + Broadcast.newDaemonFixedThreadPool (BitTorrentBroadcast.MaxTxPeers) + + while (hasBlocks < totalBlocks) { + var numThreadsToCreate = + Math.min (listOfSources.size, BitTorrentBroadcast.MaxTxPeers) - + threadPool.getActiveCount + + while (hasBlocks < totalBlocks && numThreadsToCreate > 0) { + var peerToTalkTo = pickPeerToTalkTo + if (peerToTalkTo != null) { + threadPool.execute (new TalkToPeer (peerToTalkTo)) + + // Add to peersNowTalking. Remove in the thread. We have to do this + // ASAP, otherwise pickPeerToTalkTo picks the same peer more than once + peersNowTalking.synchronized { + peersNowTalking = peersNowTalking + peerToTalkTo + } + } + + numThreadsToCreate = numThreadsToCreate - 1 + } + + // Sleep for a while before starting some more threads + Thread.sleep (BitTorrentBroadcast.MinKnockInterval) + } + // Shutdown the thread pool + threadPool.shutdown + } + + // Right now picking the one that has the most blocks this peer wants + // Also picking peer randomly if no one has anything interesting + private def pickPeerToTalkTo: SourceInfo = { + var curPeer: SourceInfo = null + var curMax = 0 + + logInfo ("Picking peers to talk to...") + + // Find peers that are not connected right now + var peersNotInUse = ListBuffer[SourceInfo] () + synchronized { + peersNotInUse = listOfSources -- peersNowTalking + } + + peersNotInUse.foreach { eachSource => + var tempHasBlocksBitVector: BitSet = null + hasBlocksBitVector.synchronized { + tempHasBlocksBitVector = hasBlocksBitVector.clone.asInstanceOf[BitSet] + } + tempHasBlocksBitVector.flip (0, tempHasBlocksBitVector.size) + tempHasBlocksBitVector.and (eachSource.hasBlocksBitVector) + + if (tempHasBlocksBitVector.cardinality > curMax) { + curPeer = eachSource + curMax = tempHasBlocksBitVector.cardinality + } + } + + // Always pick randomly or randomly pick randomly? + // Now always picking randomly + if (curPeer == null && peersNotInUse.size > 0) { + // Pick uniformly the i'th required peer + var i = BitTorrentBroadcast.ranGen.nextInt (peersNotInUse.size) + + var peerIter = peersNotInUse.iterator + curPeer = peerIter.next + + while (i > 0) { + curPeer = peerIter.next + i = i - 1 + } + } + + if (curPeer != null) + logInfo ("Peer chosen: " + curPeer + " with " + curPeer.hasBlocksBitVector) + else + logInfo ("No peer chosen...") + + return curPeer + } + + class TalkToPeer (peerToTalkTo: SourceInfo) + extends Thread with Logging { + private var peerSocketToSource: Socket = null + private var oosSource: ObjectOutputStream = null + private var oisSource: ObjectInputStream = null + + override def run: Unit = { + // TODO: There is a possible bug here regarding blocksInRequestBitVector + var blockToAskFor = -1 + + // Setup the timeout mechanism + var timeOutTask = new TimerTask { + override def run: Unit = { + cleanUpConnections + } + } + + var timeOutTimer = new Timer + timeOutTimer.schedule (timeOutTask, BitTorrentBroadcast.MaxKnockInterval) + + logInfo ("TalkToPeer started... => " + peerToTalkTo) + + try { + // Connect to the source + peerSocketToSource = + new Socket (peerToTalkTo.hostAddress, peerToTalkTo.listenPort) + oosSource = + new ObjectOutputStream (peerSocketToSource.getOutputStream) + oosSource.flush + oisSource = + new ObjectInputStream (peerSocketToSource.getInputStream) + + // Receive latest SourceInfo from peerToTalkTo + var newPeerToTalkTo = oisSource.readObject.asInstanceOf[SourceInfo] + // Update listOfSources + addToListOfSources (newPeerToTalkTo) + + // Turn the timer OFF, if the sender responds before timeout + timeOutTimer.cancel + + // Send the latest SourceInfo + oosSource.writeObject(getLocalSourceInfo) + oosSource.flush + + var keepReceiving = true + + while (hasBlocks < totalBlocks && keepReceiving) { + blockToAskFor = + pickBlockToRequest (newPeerToTalkTo.hasBlocksBitVector) + + // No block to request + if (blockToAskFor < 0) { + // Nothing to receive from newPeerToTalkTo + keepReceiving = false + } else { + // Let other thread know that blockToAskFor is being requested + blocksInRequestBitVector.synchronized { + blocksInRequestBitVector.set (blockToAskFor) + } + + // Start with sending the blockID + oosSource.writeObject(blockToAskFor) + oosSource.flush + + // Receive the requested block + val recvStartTime = System.currentTimeMillis + val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock] + val receptionTime = (System.currentTimeMillis - recvStartTime) + + // Expecting sender to send the block that was asked for + assert (bcBlock.blockID == blockToAskFor) + + logInfo ("Received block: " + bcBlock.blockID + " from " + peerToTalkTo + " in " + receptionTime + " millis.") + + if (!hasBlocksBitVector.get(bcBlock.blockID)) { + arrayOfBlocks(bcBlock.blockID) = bcBlock + + // Update the hasBlocksBitVector first + hasBlocksBitVector.synchronized { + hasBlocksBitVector.set (bcBlock.blockID) + } + hasBlocks += 1 + + rxSpeeds.addDataPoint (peerToTalkTo, receptionTime) + + // blockToAskFor has arrived. Not in request any more + // Probably no need to update it though + blocksInRequestBitVector.synchronized { + blocksInRequestBitVector.set (bcBlock.blockID, false) + } + + // Reset blockToAskFor to -1. Else it will be considered missing + blockToAskFor = -1 + } + + // Send the latest SourceInfo + oosSource.writeObject(getLocalSourceInfo) + oosSource.flush + } + } + } catch { + // EOFException is expected to happen because sender can break + // connection due to timeout + case eofe: java.io.EOFException => { } + case e: Exception => { + logInfo ("TalktoPeer had a " + e) + // TODO: Remove 'newPeerToTalkTo' from listOfSources + // We probably should have the following in some form, but not + // really here. This exception can happen if the sender just breaks connection + // listOfSources.synchronized { + // logInfo ("Exception in TalkToPeer. Removing source: " + peerToTalkTo) + // listOfSources = listOfSources - peerToTalkTo + // } + } + } finally { + // blockToAskFor != -1 => there was an exception + if (blockToAskFor != -1) { + blocksInRequestBitVector.synchronized { + blocksInRequestBitVector.set (blockToAskFor, false) + } + } + + cleanUpConnections + } + } + + // Right now it picks a block uniformly that this peer does not have + // TODO: Implement more intelligent block selection policies + private def pickBlockToRequest (txHasBlocksBitVector: BitSet): Int = { + var needBlocksBitVector: BitSet = null + + // Blocks already present + hasBlocksBitVector.synchronized { + needBlocksBitVector = hasBlocksBitVector.clone.asInstanceOf[BitSet] + } + + // Include blocks already in transmission ONLY IF + // BitTorrentBroadcast.EndGameFraction has NOT been achieved + if ((1.0 * hasBlocks / totalBlocks) < BitTorrentBroadcast.EndGameFraction) { + blocksInRequestBitVector.synchronized { + needBlocksBitVector.or (blocksInRequestBitVector) + } + } + + // Find blocks that are neither here nor in transit + needBlocksBitVector.flip (0, needBlocksBitVector.size) + + // Blocks that should be requested + needBlocksBitVector.and (txHasBlocksBitVector) + + if (needBlocksBitVector.cardinality == 0) { + return -1 + } else { + // Pick uniformly the i'th required block + var i = BitTorrentBroadcast.ranGen.nextInt (needBlocksBitVector.cardinality) + var pickedBlockIndex = needBlocksBitVector.nextSetBit (0) + + while (i > 0) { + pickedBlockIndex = + needBlocksBitVector.nextSetBit (pickedBlockIndex + 1) + i = i - 1 + } + + return pickedBlockIndex + } + } + + private def cleanUpConnections: Unit = { + if (oisSource != null) { + oisSource.close + } + if (oosSource != null) { + oosSource.close + } + if (peerSocketToSource != null) { + peerSocketToSource.close + } + + // Delete from peersNowTalking + peersNowTalking.synchronized { + peersNowTalking = peersNowTalking - peerToTalkTo + } + } + } + } + + class GuideMultipleRequests + extends Thread with Logging { + // Keep track of sources that have completed reception + private var setOfCompletedSources = Set[SourceInfo] () + + override def run: Unit = { + var threadPool = Broadcast.newDaemonCachedThreadPool + var serverSocket: ServerSocket = null + + serverSocket = new ServerSocket (0) + guidePort = serverSocket.getLocalPort + logInfo ("GuideMultipleRequests => " + serverSocket + " " + guidePort) + + guidePortLock.synchronized { + guidePortLock.notifyAll + } + + try { + // Don't stop until there is a copy in HDFS + while (!stopBroadcast || !hasCopyInHDFS) { + var clientSocket: Socket = null + try { + serverSocket.setSoTimeout (BitTorrentBroadcast.ServerSocketTimeout) + clientSocket = serverSocket.accept + } catch { + case e: Exception => { + logInfo ("GuideMultipleRequests Timeout.") + + // Stop broadcast if at least one worker has connected and + // everyone connected so far are done. Comparing with + // listOfSources.size - 1, because it includes the Guide itself + if (listOfSources.size > 1 && + setOfCompletedSources.size == listOfSources.size - 1) { + stopBroadcast = true + } + } + } + if (clientSocket != null) { + logInfo ("Guide: Accepted new client connection:" + clientSocket) + try { + threadPool.execute (new GuideSingleRequest (clientSocket)) + } catch { + // In failure, close the socket here; else, thread will close it + case ioe: IOException => { + clientSocket.close + } + } + } + } + + // Shutdown the thread pool + threadPool.shutdown + + logInfo ("Sending stopBroadcast notifications...") + sendStopBroadcastNotifications + + BitTorrentBroadcast.unregisterValue (uuid) + } finally { + if (serverSocket != null) { + logInfo ("GuideMultipleRequests now stopping...") + serverSocket.close + } + } + } + + private def sendStopBroadcastNotifications: Unit = { + listOfSources.synchronized { + listOfSources.foreach { sourceInfo => + + var guideSocketToSource: Socket = null + var gosSource: ObjectOutputStream = null + var gisSource: ObjectInputStream = null + + try { + // Connect to the source + guideSocketToSource = + new Socket (sourceInfo.hostAddress, sourceInfo.listenPort) + gosSource = + new ObjectOutputStream (guideSocketToSource.getOutputStream) + gosSource.flush + gisSource = + new ObjectInputStream (guideSocketToSource.getInputStream) + + // Throw away whatever comes in + gisSource.readObject.asInstanceOf[SourceInfo] + + // Send stopBroadcast signal. listenPort = SourceInfo.StopBroadcast + gosSource.writeObject(SourceInfo("", SourceInfo.StopBroadcast, + SourceInfo.UnusedParam, SourceInfo.UnusedParam)) + gosSource.flush + } catch { + case e: Exception => { + logInfo ("sendStopBroadcastNotifications had a " + e) + } + } finally { + if (gisSource != null) { + gisSource.close + } + if (gosSource != null) { + gosSource.close + } + if (guideSocketToSource != null) { + guideSocketToSource.close + } + } + } + } + } + + class GuideSingleRequest (val clientSocket: Socket) + extends Thread with Logging { + private val oos = new ObjectOutputStream (clientSocket.getOutputStream) + oos.flush + private val ois = new ObjectInputStream (clientSocket.getInputStream) + + private var sourceInfo: SourceInfo = null + private var selectedSources: ListBuffer[SourceInfo] = null + + override def run: Unit = { + try { + logInfo ("new GuideSingleRequest is running") + // Connecting worker is sending in its information + sourceInfo = ois.readObject.asInstanceOf[SourceInfo] + + // Select a suitable source and send it back to the worker + selectedSources = selectSuitableSources (sourceInfo) + logInfo ("Sending selectedSources:" + selectedSources) + oos.writeObject (selectedSources) + oos.flush + + // Add this source to the listOfSources + addToListOfSources (sourceInfo) + } catch { + case e: Exception => { + // Assuming exception caused by receiver failure: remove + if (listOfSources != null) { + listOfSources.synchronized { + listOfSources = listOfSources - sourceInfo + } + } + } + } finally { + ois.close + oos.close + clientSocket.close + } + } + + // Randomly select some sources to send back + private def selectSuitableSources(skipSourceInfo: SourceInfo): ListBuffer[SourceInfo] = { + var selectedSources = ListBuffer[SourceInfo] () + + // If skipSourceInfo.hasBlocksBitVector has all bits set to 'true' + // then add skipSourceInfo to setOfCompletedSources. Return blank. + if (skipSourceInfo.hasBlocks == totalBlocks) { + setOfCompletedSources += skipSourceInfo + return selectedSources + } + + listOfSources.synchronized { + if (listOfSources.size <= BitTorrentBroadcast.MaxPeersInGuideResponse) { + selectedSources = listOfSources.clone + } else { + var picksLeft = BitTorrentBroadcast.MaxPeersInGuideResponse + var alreadyPicked = new BitSet (listOfSources.size) + + while (picksLeft > 0) { + var i = -1 + + do { + i = BitTorrentBroadcast.ranGen.nextInt (listOfSources.size) + } while (alreadyPicked.get(i)) + + var peerIter = listOfSources.iterator + var curPeer = peerIter.next + + while (i > 0) { + curPeer = peerIter.next + i = i - 1 + } + + selectedSources = selectedSources + curPeer + alreadyPicked.set (i) + + picksLeft = picksLeft - 1 + } + } + } + + // Remove the receiving source (if present) + selectedSources = selectedSources - skipSourceInfo + + return selectedSources + } + } + } + + class ServeMultipleRequests + extends Thread with Logging { + // Server at most BitTorrentBroadcast.MaxRxPeers peers + var threadPool = + Broadcast.newDaemonFixedThreadPool(BitTorrentBroadcast.MaxRxPeers) + + override def run: Unit = { + var serverSocket = new ServerSocket (0) + listenPort = serverSocket.getLocalPort + + logInfo ("ServeMultipleRequests started with " + serverSocket) + + listenPortLock.synchronized { + listenPortLock.notifyAll + } + + try { + while (!stopBroadcast) { + var clientSocket: Socket = null + try { + serverSocket.setSoTimeout (BitTorrentBroadcast.ServerSocketTimeout) + clientSocket = serverSocket.accept + } catch { + case e: Exception => { + logInfo ("ServeMultipleRequests Timeout.") + } + } + if (clientSocket != null) { + logInfo ("Serve: Accepted new client connection:" + clientSocket) + try { + threadPool.execute (new ServeSingleRequest (clientSocket)) + } catch { + // In failure, close socket here; else, the thread will close it + case ioe: IOException => { + clientSocket.close + } + } + } + } + } finally { + if (serverSocket != null) { + logInfo ("ServeMultipleRequests now stopping...") + serverSocket.close + } + } + // Shutdown the thread pool + threadPool.shutdown + } + + class ServeSingleRequest (val clientSocket: Socket) + extends Thread with Logging { + private val oos = new ObjectOutputStream (clientSocket.getOutputStream) + oos.flush + private val ois = new ObjectInputStream (clientSocket.getInputStream) + + logInfo ("new ServeSingleRequest is running") + + override def run: Unit = { + try { + // Send latest local SourceInfo to the receiver + // In the case of receiver timeout and connection close, this will + // throw a java.net.SocketException: Broken pipe + oos.writeObject(getLocalSourceInfo) + oos.flush + + // Receive latest SourceInfo from the receiver + var rxSourceInfo = ois.readObject.asInstanceOf[SourceInfo] + // logInfo("rxSourceInfo: " + rxSourceInfo + " with " + rxSourceInfo.hasBlocksBitVector) + + if (rxSourceInfo.listenPort == SourceInfo.StopBroadcast) { + stopBroadcast = true + } else { + // Carry on + addToListOfSources (rxSourceInfo) + } + + val startTime = System.currentTimeMillis + var curTime = startTime + var keepSending = true + var numBlocksToSend = BitTorrentBroadcast.MaxChatBlocks + + while (!stopBroadcast && keepSending && numBlocksToSend > 0) { + // Receive which block to send + val blockToSend = ois.readObject.asInstanceOf[Int] + + // Send the block + sendBlock (blockToSend) + rxSourceInfo.hasBlocksBitVector.set (blockToSend) + + numBlocksToSend = numBlocksToSend - 1 + + // Receive latest SourceInfo from the receiver + rxSourceInfo = ois.readObject.asInstanceOf[SourceInfo] + // logInfo("rxSourceInfo: " + rxSourceInfo + " with " + rxSourceInfo.hasBlocksBitVector) + addToListOfSources (rxSourceInfo) + + curTime = System.currentTimeMillis + // Revoke sending only if there is anyone waiting in the queue + if (curTime - startTime >= BitTorrentBroadcast.MaxChatTime && + threadPool.getQueue.size > 0) { + keepSending = false + } + } + } catch { + // If something went wrong, e.g., the worker at the other end died etc. + // then close everything up + // Exception can happen if the receiver stops receiving + case e: Exception => { + logInfo ("ServeSingleRequest had a " + e) + } + } finally { + logInfo ("ServeSingleRequest is closing streams and sockets") + ois.close + // TODO: The following line causes a "java.net.SocketException: Socket closed" + oos.close + clientSocket.close + } + } + + private def sendBlock (blockToSend: Int): Unit = { + try { + oos.writeObject (arrayOfBlocks(blockToSend)) + oos.flush + } catch { + case e: Exception => { + logInfo ("sendBlock had a " + e) + } + } + logInfo ("Sent block: " + blockToSend + " to " + clientSocket) + } + } + } +} + +class BitTorrentBroadcastFactory +extends BroadcastFactory { + def initialize (isMaster: Boolean) = BitTorrentBroadcast.initialize (isMaster) + def newBroadcast[T] (value_ : T, isLocal: Boolean) = + new BitTorrentBroadcast[T] (value_, isLocal) +} + +private object BitTorrentBroadcast +extends Logging { + val values = Cache.newKeySpace() + + var valueToGuideMap = Map[UUID, SourceInfo] () + + // Random number generator + var ranGen = new Random + + private var initialized = false + private var isMaster_ = false + + private var MasterHostAddress_ = InetAddress.getLocalHost.getHostAddress + private var MasterTrackerPort_ : Int = 11111 + private var BlockSize_ : Int = 512 * 1024 + private var MaxRetryCount_ : Int = 2 + + private var TrackerSocketTimeout_ : Int = 50000 + private var ServerSocketTimeout_ : Int = 10000 + + private var trackMV: TrackMultipleValues = null + + // A peer syncs back to Guide after waiting randomly within following limits + // Also used thoughout the code for small and large waits/timeouts + private var MinKnockInterval_ = 500 + private var MaxKnockInterval_ = 999 + + private var MaxPeersInGuideResponse_ = 4 + + // Maximum number of receiving and sending threads of a peer + private var MaxRxPeers_ = 4 + private var MaxTxPeers_ = 4 + + // Peers can char at most this milliseconds or transfer this number of blocks + private var MaxChatTime_ = 250 + private var MaxChatBlocks_ = 1024 + + // Fraction of blocks to receive before entering the end game + private var EndGameFraction_ = 1.0 + + + def initialize (isMaster__ : Boolean): Unit = { + synchronized { + if (!initialized) { + // Fix for issue #42 + MasterHostAddress_ = + System.getProperty ("spark.broadcast.masterHostAddress", "") + MasterTrackerPort_ = + System.getProperty ("spark.broadcast.masterTrackerPort", "11111").toInt + BlockSize_ = + System.getProperty ("spark.broadcast.blockSize", "512").toInt * 1024 + MaxRetryCount_ = + System.getProperty ("spark.broadcast.maxRetryCount", "2").toInt + + TrackerSocketTimeout_ = + System.getProperty ("spark.broadcast.trackerSocketTimeout", "50000").toInt + ServerSocketTimeout_ = + System.getProperty ("spark.broadcast.serverSocketTimeout", "10000").toInt + + MinKnockInterval_ = + System.getProperty ("spark.broadcast.minKnockInterval", "500").toInt + MaxKnockInterval_ = + System.getProperty ("spark.broadcast.maxKnockInterval", "999").toInt + + MaxPeersInGuideResponse_ = + System.getProperty ("spark.broadcast.maxPeersInGuideResponse", "4").toInt + + MaxRxPeers_ = + System.getProperty ("spark.broadcast.maxRxPeers", "4").toInt + MaxTxPeers_ = + System.getProperty ("spark.broadcast.maxTxPeers", "4").toInt + + MaxChatTime_ = + System.getProperty ("spark.broadcast.maxChatTime", "250").toInt + MaxChatBlocks_ = + System.getProperty ("spark.broadcast.maxChatBlocks", "1024").toInt + + EndGameFraction_ = + System.getProperty ("spark.broadcast.endGameFraction", "1.0").toDouble + + isMaster_ = isMaster__ + + if (isMaster) { + trackMV = new TrackMultipleValues + trackMV.setDaemon (true) + trackMV.start + logInfo ("TrackMultipleValues started...") + } + + // Initialize DfsBroadcast to be used for broadcast variable persistence + DfsBroadcast.initialize + + initialized = true + } + } + } + + def MasterHostAddress = MasterHostAddress_ + def MasterTrackerPort = MasterTrackerPort_ + def BlockSize = BlockSize_ + def MaxRetryCount = MaxRetryCount_ + + def TrackerSocketTimeout = TrackerSocketTimeout_ + def ServerSocketTimeout = ServerSocketTimeout_ + + def isMaster = isMaster_ + + def MinKnockInterval = MinKnockInterval_ + def MaxKnockInterval = MaxKnockInterval_ + + def MaxPeersInGuideResponse = MaxPeersInGuideResponse_ + + def MaxRxPeers = MaxRxPeers_ + def MaxTxPeers = MaxTxPeers_ + + def MaxChatTime = MaxChatTime_ + def MaxChatBlocks = MaxChatBlocks_ + + def EndGameFraction = EndGameFraction_ + + def registerValue (uuid: UUID, gInfo: SourceInfo): Unit = { + valueToGuideMap.synchronized { + valueToGuideMap += (uuid -> gInfo) + logInfo ("New value registered with the Tracker " + valueToGuideMap) + } + } + + def unregisterValue (uuid: UUID): Unit = { + valueToGuideMap.synchronized { + valueToGuideMap (uuid) = SourceInfo ("", SourceInfo.TxOverGoToHDFS, + SourceInfo.UnusedParam, SourceInfo.UnusedParam) + logInfo ("Value unregistered from the Tracker " + valueToGuideMap) + } + } + + class TrackMultipleValues + extends Thread with Logging { + override def run: Unit = { + var threadPool = Broadcast.newDaemonCachedThreadPool + var serverSocket: ServerSocket = null + + serverSocket = new ServerSocket (BitTorrentBroadcast.MasterTrackerPort) + logInfo ("TrackMultipleValues" + serverSocket) + + try { + while (true) { + var clientSocket: Socket = null + try { + serverSocket.setSoTimeout (TrackerSocketTimeout) + clientSocket = serverSocket.accept + } catch { + case e: Exception => { + logInfo ("TrackMultipleValues Timeout. Stopping listening...") + } + } + + if (clientSocket != null) { + try { + threadPool.execute (new Thread { + override def run: Unit = { + val oos = new ObjectOutputStream (clientSocket.getOutputStream) + oos.flush + val ois = new ObjectInputStream (clientSocket.getInputStream) + try { + val uuid = ois.readObject.asInstanceOf[UUID] + var gInfo = + if (valueToGuideMap.contains (uuid)) { + valueToGuideMap (uuid) + } else SourceInfo ("", SourceInfo.TxNotStartedRetry, + SourceInfo.UnusedParam, SourceInfo.UnusedParam) + logInfo ("TrackMultipleValues: Got new request: " + clientSocket + " for " + uuid + " : " + gInfo.listenPort) + oos.writeObject (gInfo) + } catch { + case e: Exception => { + logInfo ("TrackMultipleValues had a " + e) + } + } finally { + ois.close + oos.close + clientSocket.close + } + } + }) + } catch { + // In failure, close socket here; else, client thread will close + case ioe: IOException => { + clientSocket.close + } + } + } + } + } finally { + serverSocket.close + } + // Shutdown the thread pool + threadPool.shutdown + } + } +} diff --git a/src/scala/spark/BoundedMemoryCache.scala b/core/src/main/scala/spark/BoundedMemoryCache.scala similarity index 100% rename from src/scala/spark/BoundedMemoryCache.scala rename to core/src/main/scala/spark/BoundedMemoryCache.scala diff --git a/core/src/main/scala/spark/Broadcast.scala b/core/src/main/scala/spark/Broadcast.scala new file mode 100644 index 0000000000..051e197db6 --- /dev/null +++ b/core/src/main/scala/spark/Broadcast.scala @@ -0,0 +1,140 @@ +package spark + +import java.util.{BitSet, UUID} +import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} + +@serializable +trait Broadcast[T] { + val uuid = UUID.randomUUID + + def value: T + + // We cannot have an abstract readObject here due to some weird issues with + // readObject having to be 'private' in sub-classes. Possibly a Scala bug! + + override def toString = "spark.Broadcast(" + uuid + ")" +} + +trait BroadcastFactory { + def initialize (isMaster: Boolean): Unit + def newBroadcast[T] (value_ : T, isLocal: Boolean): Broadcast[T] +} + +private object Broadcast +extends Logging { + private var initialized = false + private var broadcastFactory: BroadcastFactory = null + + // Called by SparkContext or Executor before using Broadcast + def initialize (isMaster: Boolean): Unit = synchronized { + if (!initialized) { + val broadcastFactoryClass = System.getProperty("spark.broadcast.factory", + "spark.DfsBroadcastFactory") + val booleanArgs = Array[AnyRef] (isMaster.asInstanceOf[AnyRef]) + + broadcastFactory = + Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] + + // Initialize appropriate BroadcastFactory and BroadcastObject + broadcastFactory.initialize(isMaster) + + initialized = true + } + } + + def getBroadcastFactory: BroadcastFactory = { + if (broadcastFactory == null) { + throw new SparkException ("Broadcast.getBroadcastFactory called before initialize") + } + broadcastFactory + } + + // Returns a standard ThreadFactory except all threads are daemons + private def newDaemonThreadFactory: ThreadFactory = { + new ThreadFactory { + def newThread(r: Runnable): Thread = { + var t = Executors.defaultThreadFactory.newThread (r) + t.setDaemon (true) + return t + } + } + } + + // Wrapper over newCachedThreadPool + def newDaemonCachedThreadPool: ThreadPoolExecutor = { + var threadPool = + Executors.newCachedThreadPool.asInstanceOf[ThreadPoolExecutor] + + threadPool.setThreadFactory (newDaemonThreadFactory) + + return threadPool + } + + // Wrapper over newFixedThreadPool + def newDaemonFixedThreadPool (nThreads: Int): ThreadPoolExecutor = { + var threadPool = + Executors.newFixedThreadPool (nThreads).asInstanceOf[ThreadPoolExecutor] + + threadPool.setThreadFactory (newDaemonThreadFactory) + + return threadPool + } +} + +@serializable +case class SourceInfo (val hostAddress: String, val listenPort: Int, + val totalBlocks: Int, val totalBytes: Int) +extends Comparable[SourceInfo] with Logging { + + var currentLeechers = 0 + var receptionFailed = false + + var hasBlocks = 0 + var hasBlocksBitVector: BitSet = new BitSet (totalBlocks) + + // Ascending sort based on leecher count + def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers)} + +object SourceInfo { + // Constants for special values of listenPort + val TxNotStartedRetry = -1 + val TxOverGoToHDFS = 0 + // Other constants + val StopBroadcast = -2 + val UnusedParam = 0 +} + +@serializable +case class BroadcastBlock (val blockID: Int, val byteArray: Array[Byte]) { } + +@serializable +case class VariableInfo (@transient val arrayOfBlocks : Array[BroadcastBlock], + val totalBlocks: Int, val totalBytes: Int) { + @transient var hasBlocks = 0 +} + +@serializable +class SpeedTracker { + // Mapping 'source' to '(totalTime, numBlocks)' + private var sourceToSpeedMap = Map[SourceInfo, (Long, Int)] () + + def addDataPoint (srcInfo: SourceInfo, timeInMillis: Long): Unit = { + sourceToSpeedMap.synchronized { + if (!sourceToSpeedMap.contains(srcInfo)) { + sourceToSpeedMap += (srcInfo -> (timeInMillis, 1)) + } else { + val tTnB = sourceToSpeedMap (srcInfo) + sourceToSpeedMap += (srcInfo -> (tTnB._1 + timeInMillis, tTnB._2 + 1)) + } + } + } + + def getTimePerBlock (srcInfo: SourceInfo): Double = { + sourceToSpeedMap.synchronized { + val tTnB = sourceToSpeedMap (srcInfo) + return tTnB._1 / tTnB._2 + } + } + + override def toString = sourceToSpeedMap.toString +} \ No newline at end of file diff --git a/src/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala similarity index 100% rename from src/scala/spark/Cache.scala rename to core/src/main/scala/spark/Cache.scala diff --git a/core/src/main/scala/spark/ChainedBroadcast.scala b/core/src/main/scala/spark/ChainedBroadcast.scala new file mode 100644 index 0000000000..d13553afde --- /dev/null +++ b/core/src/main/scala/spark/ChainedBroadcast.scala @@ -0,0 +1,873 @@ +package spark + +import java.io._ +import java.net._ +import java.util.{Comparator, PriorityQueue, Random, UUID} + +import scala.collection.mutable.{Map, Set} + +@serializable +class ChainedBroadcast[T] (@transient var value_ : T, isLocal: Boolean) +extends Broadcast[T] with Logging { + + def value = value_ + + ChainedBroadcast.synchronized { + ChainedBroadcast.values.put (uuid, value_) + } + + @transient var arrayOfBlocks: Array[BroadcastBlock] = null + @transient var totalBytes = -1 + @transient var totalBlocks = -1 + @transient var hasBlocks = 0 + + @transient var listenPortLock = new Object + @transient var guidePortLock = new Object + @transient var totalBlocksLock = new Object + @transient var hasBlocksLock = new Object + + @transient var pqOfSources = new PriorityQueue[SourceInfo] + + @transient var serveMR: ServeMultipleRequests = null + @transient var guideMR: GuideMultipleRequests = null + + @transient var hostAddress = InetAddress.getLocalHost.getHostAddress + @transient var listenPort = -1 + @transient var guidePort = -1 + + @transient var hasCopyInHDFS = false + @transient var stopBroadcast = false + + // Must call this after all the variables have been created/initialized + if (!isLocal) { + sendBroadcast + } + + def sendBroadcast (): Unit = { + logInfo ("Local host address: " + hostAddress) + + // Store a persistent copy in HDFS + // TODO: Turned OFF for now + // val out = new ObjectOutputStream (DfsBroadcast.openFileForWriting(uuid)) + // out.writeObject (value_) + // out.close + // TODO: Fix this at some point + hasCopyInHDFS = true + + // Create a variableInfo object and store it in valueInfos + var variableInfo = blockifyObject (value_, ChainedBroadcast.BlockSize) + + guideMR = new GuideMultipleRequests + guideMR.setDaemon (true) + guideMR.start + logInfo ("GuideMultipleRequests started...") + + serveMR = new ServeMultipleRequests + serveMR.setDaemon (true) + serveMR.start + logInfo ("ServeMultipleRequests started...") + + // Prepare the value being broadcasted + // TODO: Refactoring and clean-up required here + arrayOfBlocks = variableInfo.arrayOfBlocks + totalBytes = variableInfo.totalBytes + totalBlocks = variableInfo.totalBlocks + hasBlocks = variableInfo.totalBlocks + + while (listenPort == -1) { + listenPortLock.synchronized { + listenPortLock.wait + } + } + + pqOfSources = new PriorityQueue[SourceInfo] + val masterSource_0 = + SourceInfo (hostAddress, listenPort, totalBlocks, totalBytes) + pqOfSources.add (masterSource_0) + + // Register with the Tracker + while (guidePort == -1) { + guidePortLock.synchronized { + guidePortLock.wait + } + } + ChainedBroadcast.registerValue (uuid, guidePort) + } + + private def readObject (in: ObjectInputStream): Unit = { + in.defaultReadObject + ChainedBroadcast.synchronized { + val cachedVal = ChainedBroadcast.values.get (uuid) + if (cachedVal != null) { + value_ = cachedVal.asInstanceOf[T] + } else { + // Initializing everything because Master will only send null/0 values + initializeSlaveVariables + + logInfo ("Local host address: " + hostAddress) + + serveMR = new ServeMultipleRequests + serveMR.setDaemon (true) + serveMR.start + logInfo ("ServeMultipleRequests started...") + + val start = System.nanoTime + + val receptionSucceeded = receiveBroadcast (uuid) + // If does not succeed, then get from HDFS copy + if (receptionSucceeded) { + value_ = unBlockifyObject[T] + ChainedBroadcast.values.put (uuid, value_) + } else { + val fileIn = new ObjectInputStream(DfsBroadcast.openFileForReading(uuid)) + value_ = fileIn.readObject.asInstanceOf[T] + ChainedBroadcast.values.put(uuid, value_) + fileIn.close + } + + val time = (System.nanoTime - start) / 1e9 + logInfo("Reading Broadcasted variable " + uuid + " took " + time + " s") + } + } + } + + private def initializeSlaveVariables: Unit = { + arrayOfBlocks = null + totalBytes = -1 + totalBlocks = -1 + hasBlocks = 0 + + listenPortLock = new Object + totalBlocksLock = new Object + hasBlocksLock = new Object + + serveMR = null + + hostAddress = InetAddress.getLocalHost.getHostAddress + listenPort = -1 + + stopBroadcast = false + } + + private def blockifyObject (obj: T, blockSize: Int): VariableInfo = { + val baos = new ByteArrayOutputStream + val oos = new ObjectOutputStream (baos) + oos.writeObject (obj) + oos.close + baos.close + val byteArray = baos.toByteArray + val bais = new ByteArrayInputStream (byteArray) + + var blockNum = (byteArray.length / blockSize) + if (byteArray.length % blockSize != 0) + blockNum += 1 + + var retVal = new Array[BroadcastBlock] (blockNum) + var blockID = 0 + + for (i <- 0 until (byteArray.length, blockSize)) { + val thisBlockSize = Math.min (blockSize, byteArray.length - i) + var tempByteArray = new Array[Byte] (thisBlockSize) + val hasRead = bais.read (tempByteArray, 0, thisBlockSize) + + retVal (blockID) = new BroadcastBlock (blockID, tempByteArray) + blockID += 1 + } + bais.close + + var variableInfo = VariableInfo (retVal, blockNum, byteArray.length) + variableInfo.hasBlocks = blockNum + + return variableInfo + } + + private def unBlockifyObject[A]: A = { + var retByteArray = new Array[Byte] (totalBytes) + for (i <- 0 until totalBlocks) { + System.arraycopy (arrayOfBlocks(i).byteArray, 0, retByteArray, + i * ChainedBroadcast.BlockSize, arrayOfBlocks(i).byteArray.length) + } + byteArrayToObject (retByteArray) + } + + private def byteArrayToObject[A] (bytes: Array[Byte]): A = { + val in = new ObjectInputStream (new ByteArrayInputStream (bytes)) + val retVal = in.readObject.asInstanceOf[A] + in.close + return retVal + } + + def getMasterListenPort (variableUUID: UUID): Int = { + var clientSocketToTracker: Socket = null + var oosTracker: ObjectOutputStream = null + var oisTracker: ObjectInputStream = null + + var masterListenPort: Int = SourceInfo.TxOverGoToHDFS + + var retriesLeft = ChainedBroadcast.MaxRetryCount + do { + try { + // Connect to the tracker to find out the guide + val clientSocketToTracker = + new Socket(ChainedBroadcast.MasterHostAddress, ChainedBroadcast.MasterTrackerPort) + val oosTracker = + new ObjectOutputStream (clientSocketToTracker.getOutputStream) + oosTracker.flush + val oisTracker = + new ObjectInputStream (clientSocketToTracker.getInputStream) + + // Send UUID and receive masterListenPort + oosTracker.writeObject (uuid) + oosTracker.flush + masterListenPort = oisTracker.readObject.asInstanceOf[Int] + } catch { + case e: Exception => { + logInfo ("getMasterListenPort had a " + e) + } + } finally { + if (oisTracker != null) { + oisTracker.close + } + if (oosTracker != null) { + oosTracker.close + } + if (clientSocketToTracker != null) { + clientSocketToTracker.close + } + } + retriesLeft -= 1 + + Thread.sleep (ChainedBroadcast.ranGen.nextInt ( + ChainedBroadcast.MaxKnockInterval - ChainedBroadcast.MinKnockInterval) + + ChainedBroadcast.MinKnockInterval) + + } while (retriesLeft > 0 && masterListenPort == SourceInfo.TxNotStartedRetry) + + logInfo ("Got this guidePort from Tracker: " + masterListenPort) + return masterListenPort + } + + def receiveBroadcast (variableUUID: UUID): Boolean = { + val masterListenPort = getMasterListenPort (variableUUID) + + if (masterListenPort == SourceInfo.TxOverGoToHDFS || + masterListenPort == SourceInfo.TxNotStartedRetry) { + // TODO: SourceInfo.TxNotStartedRetry is not really in use because we go + // to HDFS anyway when receiveBroadcast returns false + return false + } + + // Wait until hostAddress and listenPort are created by the + // ServeMultipleRequests thread + while (listenPort == -1) { + listenPortLock.synchronized { + listenPortLock.wait + } + } + + var clientSocketToMaster: Socket = null + var oosMaster: ObjectOutputStream = null + var oisMaster: ObjectInputStream = null + + // Connect and receive broadcast from the specified source, retrying the + // specified number of times in case of failures + var retriesLeft = ChainedBroadcast.MaxRetryCount + do { + // Connect to Master and send this worker's Information + clientSocketToMaster = + new Socket(ChainedBroadcast.MasterHostAddress, masterListenPort) + // TODO: Guiding object connection is reusable + oosMaster = + new ObjectOutputStream (clientSocketToMaster.getOutputStream) + oosMaster.flush + oisMaster = + new ObjectInputStream (clientSocketToMaster.getInputStream) + + logInfo ("Connected to Master's guiding object") + + // Send local source information + oosMaster.writeObject(SourceInfo (hostAddress, listenPort, + SourceInfo.UnusedParam, SourceInfo.UnusedParam)) + oosMaster.flush + + // Receive source information from Master + var sourceInfo = oisMaster.readObject.asInstanceOf[SourceInfo] + totalBlocks = sourceInfo.totalBlocks + arrayOfBlocks = new Array[BroadcastBlock] (totalBlocks) + totalBlocksLock.synchronized { + totalBlocksLock.notifyAll + } + totalBytes = sourceInfo.totalBytes + + logInfo ("Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort) + + val start = System.nanoTime + val receptionSucceeded = receiveSingleTransmission (sourceInfo) + val time = (System.nanoTime - start) / 1e9 + + // Updating some statistics in sourceInfo. Master will be using them later + if (!receptionSucceeded) { + sourceInfo.receptionFailed = true + } + + // Send back statistics to the Master + oosMaster.writeObject (sourceInfo) + + if (oisMaster != null) { + oisMaster.close + } + if (oosMaster != null) { + oosMaster.close + } + if (clientSocketToMaster != null) { + clientSocketToMaster.close + } + + retriesLeft -= 1 + } while (retriesLeft > 0 && hasBlocks < totalBlocks) + + return (hasBlocks == totalBlocks) + } + + // Tries to receive broadcast from the source and returns Boolean status. + // This might be called multiple times to retry a defined number of times. + private def receiveSingleTransmission(sourceInfo: SourceInfo): Boolean = { + var clientSocketToSource: Socket = null + var oosSource: ObjectOutputStream = null + var oisSource: ObjectInputStream = null + + var receptionSucceeded = false + try { + // Connect to the source to get the object itself + clientSocketToSource = + new Socket (sourceInfo.hostAddress, sourceInfo.listenPort) + oosSource = + new ObjectOutputStream (clientSocketToSource.getOutputStream) + oosSource.flush + oisSource = + new ObjectInputStream (clientSocketToSource.getInputStream) + + logInfo ("Inside receiveSingleTransmission") + logInfo ("totalBlocks: "+ totalBlocks + " " + "hasBlocks: " + hasBlocks) + + // Send the range + oosSource.writeObject((hasBlocks, totalBlocks)) + oosSource.flush + + for (i <- hasBlocks until totalBlocks) { + val recvStartTime = System.currentTimeMillis + val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock] + val receptionTime = (System.currentTimeMillis - recvStartTime) + + logInfo ("Received block: " + bcBlock.blockID + " from " + sourceInfo + " in " + receptionTime + " millis.") + + arrayOfBlocks(hasBlocks) = bcBlock + hasBlocks += 1 + // Set to true if at least one block is received + receptionSucceeded = true + hasBlocksLock.synchronized { + hasBlocksLock.notifyAll + } + } + } catch { + case e: Exception => { + logInfo ("receiveSingleTransmission had a " + e) + } + } finally { + if (oisSource != null) { + oisSource.close + } + if (oosSource != null) { + oosSource.close + } + if (clientSocketToSource != null) { + clientSocketToSource.close + } + } + + return receptionSucceeded + } + + class GuideMultipleRequests + extends Thread with Logging { + // Keep track of sources that have completed reception + private var setOfCompletedSources = Set[SourceInfo] () + + override def run: Unit = { + var threadPool = Broadcast.newDaemonCachedThreadPool + var serverSocket: ServerSocket = null + + serverSocket = new ServerSocket (0) + guidePort = serverSocket.getLocalPort + logInfo ("GuideMultipleRequests => " + serverSocket + " " + guidePort) + + guidePortLock.synchronized { + guidePortLock.notifyAll + } + + try { + // Don't stop until there is a copy in HDFS + while (!stopBroadcast || !hasCopyInHDFS) { + var clientSocket: Socket = null + try { + serverSocket.setSoTimeout (ChainedBroadcast.ServerSocketTimeout) + clientSocket = serverSocket.accept + } catch { + case e: Exception => { + logInfo ("GuideMultipleRequests Timeout.") + + // Stop broadcast if at least one worker has connected and + // everyone connected so far are done. Comparing with + // pqOfSources.size - 1, because it includes the Guide itself + if (pqOfSources.size > 1 && + setOfCompletedSources.size == pqOfSources.size - 1) { + stopBroadcast = true + } + } + } + if (clientSocket != null) { + logInfo ("Guide: Accepted new client connection: " + clientSocket) + try { + threadPool.execute (new GuideSingleRequest (clientSocket)) + } catch { + // In failure, close the socket here; else, the thread will close it + case ioe: IOException => clientSocket.close + } + } + } + + logInfo ("Sending stopBroadcast notifications...") + sendStopBroadcastNotifications + + ChainedBroadcast.unregisterValue (uuid) + } finally { + if (serverSocket != null) { + logInfo ("GuideMultipleRequests now stopping...") + serverSocket.close + } + } + + // Shutdown the thread pool + threadPool.shutdown + } + + private def sendStopBroadcastNotifications: Unit = { + pqOfSources.synchronized { + var pqIter = pqOfSources.iterator + while (pqIter.hasNext) { + var sourceInfo = pqIter.next + + var guideSocketToSource: Socket = null + var gosSource: ObjectOutputStream = null + var gisSource: ObjectInputStream = null + + try { + // Connect to the source + guideSocketToSource = + new Socket (sourceInfo.hostAddress, sourceInfo.listenPort) + gosSource = + new ObjectOutputStream (guideSocketToSource.getOutputStream) + gosSource.flush + gisSource = + new ObjectInputStream (guideSocketToSource.getInputStream) + + // Send stopBroadcast signal. Range = SourceInfo.StopBroadcast*2 + gosSource.writeObject ((SourceInfo.StopBroadcast, + SourceInfo.StopBroadcast)) + gosSource.flush + } catch { + case e: Exception => { + logInfo ("sendStopBroadcastNotifications had a " + e) + } + } finally { + if (gisSource != null) { + gisSource.close + } + if (gosSource != null) { + gosSource.close + } + if (guideSocketToSource != null) { + guideSocketToSource.close + } + } + } + } + } + + class GuideSingleRequest (val clientSocket: Socket) + extends Thread with Logging { + private val oos = new ObjectOutputStream (clientSocket.getOutputStream) + oos.flush + private val ois = new ObjectInputStream (clientSocket.getInputStream) + + private var selectedSourceInfo: SourceInfo = null + private var thisWorkerInfo:SourceInfo = null + + override def run: Unit = { + try { + logInfo ("new GuideSingleRequest is running") + // Connecting worker is sending in its hostAddress and listenPort it will + // be listening to. Other fields are invalid (SourceInfo.UnusedParam) + var sourceInfo = ois.readObject.asInstanceOf[SourceInfo] + + pqOfSources.synchronized { + // Select a suitable source and send it back to the worker + selectedSourceInfo = selectSuitableSource (sourceInfo) + logInfo ("Sending selectedSourceInfo: " + selectedSourceInfo) + oos.writeObject (selectedSourceInfo) + oos.flush + + // Add this new (if it can finish) source to the PQ of sources + thisWorkerInfo = SourceInfo (sourceInfo.hostAddress, + sourceInfo.listenPort, totalBlocks, totalBytes) + logInfo ("Adding possible new source to pqOfSources: " + thisWorkerInfo) + pqOfSources.add (thisWorkerInfo) + } + + // Wait till the whole transfer is done. Then receive and update source + // statistics in pqOfSources + sourceInfo = ois.readObject.asInstanceOf[SourceInfo] + + pqOfSources.synchronized { + // This should work since SourceInfo is a case class + assert (pqOfSources.contains (selectedSourceInfo)) + + // Remove first + pqOfSources.remove (selectedSourceInfo) + // TODO: Removing a source based on just one failure notification! + + // Update sourceInfo and put it back in, IF reception succeeded + if (!sourceInfo.receptionFailed) { + // Add thisWorkerInfo to sources that have completed reception + setOfCompletedSources += thisWorkerInfo + + selectedSourceInfo.currentLeechers -= 1 + + // Put it back + pqOfSources.add (selectedSourceInfo) + } + } + } catch { + // If something went wrong, e.g., the worker at the other end died etc. + // then close everything up + case e: Exception => { + // Assuming that exception caused due to receiver worker failure. + // Remove failed worker from pqOfSources and update leecherCount of + // corresponding source worker + pqOfSources.synchronized { + if (selectedSourceInfo != null) { + // Remove first + pqOfSources.remove (selectedSourceInfo) + // Update leecher count and put it back in + selectedSourceInfo.currentLeechers -= 1 + pqOfSources.add (selectedSourceInfo) + } + + // Remove thisWorkerInfo + if (pqOfSources != null) { + pqOfSources.remove (thisWorkerInfo) + } + } + } + } finally { + ois.close + oos.close + clientSocket.close + } + } + + // TODO: Caller must have a synchronized block on pqOfSources + // TODO: If a worker fails to get the broadcasted variable from a source and + // comes back to Master, this function might choose the worker itself as a + // source tp create a dependency cycle (this worker was put into pqOfSources + // as a streming source when it first arrived). The length of this cycle can + // be arbitrarily long. + private def selectSuitableSource(skipSourceInfo: SourceInfo): SourceInfo = { + // Select one based on the ordering strategy (e.g., least leechers etc.) + // take is a blocking call removing the element from PQ + var selectedSource = pqOfSources.poll + assert (selectedSource != null) + // Update leecher count + selectedSource.currentLeechers += 1 + // Add it back and then return + pqOfSources.add (selectedSource) + return selectedSource + } + } + } + + class ServeMultipleRequests + extends Thread with Logging { + override def run: Unit = { + var threadPool = Broadcast.newDaemonCachedThreadPool + var serverSocket: ServerSocket = null + + serverSocket = new ServerSocket (0) + listenPort = serverSocket.getLocalPort + logInfo ("ServeMultipleRequests started with " + serverSocket) + + listenPortLock.synchronized { + listenPortLock.notifyAll + } + + try { + while (!stopBroadcast) { + var clientSocket: Socket = null + try { + serverSocket.setSoTimeout (ChainedBroadcast.ServerSocketTimeout) + clientSocket = serverSocket.accept + } catch { + case e: Exception => { + logInfo ("ServeMultipleRequests Timeout.") + } + } + if (clientSocket != null) { + logInfo ("Serve: Accepted new client connection: " + clientSocket) + try { + threadPool.execute (new ServeSingleRequest (clientSocket)) + } catch { + // In failure, close socket here; else, the thread will close it + case ioe: IOException => clientSocket.close + } + } + } + } finally { + if (serverSocket != null) { + logInfo ("ServeMultipleRequests now stopping...") + serverSocket.close + } + } + + // Shutdown the thread pool + threadPool.shutdown + } + + class ServeSingleRequest (val clientSocket: Socket) + extends Thread with Logging { + private val oos = new ObjectOutputStream (clientSocket.getOutputStream) + oos.flush + private val ois = new ObjectInputStream (clientSocket.getInputStream) + + private var sendFrom = 0 + private var sendUntil = totalBlocks + + override def run: Unit = { + try { + logInfo ("new ServeSingleRequest is running") + + // Receive range to send + var rangeToSend = ois.readObject.asInstanceOf[(Int, Int)] + sendFrom = rangeToSend._1 + sendUntil = rangeToSend._2 + + if (sendFrom == SourceInfo.StopBroadcast && + sendUntil == SourceInfo.StopBroadcast) { + stopBroadcast = true + } else { + // Carry on + sendObject + } + } catch { + // If something went wrong, e.g., the worker at the other end died etc. + // then close everything up + case e: Exception => { + logInfo ("ServeSingleRequest had a " + e) + } + } finally { + logInfo ("ServeSingleRequest is closing streams and sockets") + ois.close + oos.close + clientSocket.close + } + } + + private def sendObject: Unit = { + // Wait till receiving the SourceInfo from Master + while (totalBlocks == -1) { + totalBlocksLock.synchronized { + totalBlocksLock.wait + } + } + + for (i <- sendFrom until sendUntil) { + while (i == hasBlocks) { + hasBlocksLock.synchronized { + hasBlocksLock.wait + } + } + try { + oos.writeObject (arrayOfBlocks(i)) + oos.flush + } catch { + case e: Exception => { + logInfo ("sendObject had a " + e) + } + } + logInfo ("Sent block: " + i + " to " + clientSocket) + } + } + } + } +} + +class ChainedBroadcastFactory +extends BroadcastFactory { + def initialize (isMaster: Boolean) = ChainedBroadcast.initialize (isMaster) + def newBroadcast[T] (value_ : T, isLocal: Boolean) = + new ChainedBroadcast[T] (value_, isLocal) +} + +private object ChainedBroadcast +extends Logging { + val values = Cache.newKeySpace() + + var valueToGuidePortMap = Map[UUID, Int] () + + // Random number generator + var ranGen = new Random + + private var initialized = false + private var isMaster_ = false + + private var MasterHostAddress_ = InetAddress.getLocalHost.getHostAddress + private var MasterTrackerPort_ : Int = 22222 + private var BlockSize_ : Int = 512 * 1024 + private var MaxRetryCount_ : Int = 2 + + private var TrackerSocketTimeout_ : Int = 50000 + private var ServerSocketTimeout_ : Int = 10000 + + private var trackMV: TrackMultipleValues = null + + private var MinKnockInterval_ = 500 + private var MaxKnockInterval_ = 999 + + def initialize (isMaster__ : Boolean): Unit = { + synchronized { + if (!initialized) { + // Fix for issue #42 + MasterHostAddress_ = + System.getProperty ("spark.broadcast.masterHostAddress", "") + MasterTrackerPort_ = + System.getProperty ("spark.broadcast.masterTrackerPort", "22222").toInt + BlockSize_ = + System.getProperty ("spark.broadcast.blockSize", "512").toInt * 1024 + MaxRetryCount_ = + System.getProperty ("spark.broadcast.maxRetryCount", "2").toInt + + TrackerSocketTimeout_ = + System.getProperty ("spark.broadcast.trackerSocketTimeout", "50000").toInt + ServerSocketTimeout_ = + System.getProperty ("spark.broadcast.serverSocketTimeout", "10000").toInt + + MinKnockInterval_ = + System.getProperty ("spark.broadcast.minKnockInterval", "500").toInt + MaxKnockInterval_ = + System.getProperty ("spark.broadcast.maxKnockInterval", "999").toInt + + isMaster_ = isMaster__ + + if (isMaster) { + trackMV = new TrackMultipleValues + trackMV.setDaemon (true) + trackMV.start + logInfo ("TrackMultipleValues started...") + } + + // Initialize DfsBroadcast to be used for broadcast variable persistence + DfsBroadcast.initialize + + initialized = true + } + } + } + + def MasterHostAddress = MasterHostAddress_ + def MasterTrackerPort = MasterTrackerPort_ + def BlockSize = BlockSize_ + def MaxRetryCount = MaxRetryCount_ + + def TrackerSocketTimeout = TrackerSocketTimeout_ + def ServerSocketTimeout = ServerSocketTimeout_ + + def isMaster = isMaster_ + + def MinKnockInterval = MinKnockInterval_ + def MaxKnockInterval = MaxKnockInterval_ + + def registerValue (uuid: UUID, guidePort: Int): Unit = { + valueToGuidePortMap.synchronized { + valueToGuidePortMap += (uuid -> guidePort) + logInfo ("New value registered with the Tracker " + valueToGuidePortMap) + } + } + + def unregisterValue (uuid: UUID): Unit = { + valueToGuidePortMap.synchronized { + valueToGuidePortMap (uuid) = SourceInfo.TxOverGoToHDFS + logInfo ("Value unregistered from the Tracker " + valueToGuidePortMap) + } + } + + class TrackMultipleValues + extends Thread with Logging { + override def run: Unit = { + var threadPool = Broadcast.newDaemonCachedThreadPool + var serverSocket: ServerSocket = null + + serverSocket = new ServerSocket (ChainedBroadcast.MasterTrackerPort) + logInfo ("TrackMultipleValues" + serverSocket) + + try { + while (true) { + var clientSocket: Socket = null + try { + serverSocket.setSoTimeout (TrackerSocketTimeout) + clientSocket = serverSocket.accept + } catch { + case e: Exception => { + logInfo ("TrackMultipleValues Timeout. Stopping listening...") + } + } + + if (clientSocket != null) { + try { + threadPool.execute (new Thread { + override def run: Unit = { + val oos = new ObjectOutputStream (clientSocket.getOutputStream) + oos.flush + val ois = new ObjectInputStream (clientSocket.getInputStream) + try { + val uuid = ois.readObject.asInstanceOf[UUID] + var guidePort = + if (valueToGuidePortMap.contains (uuid)) { + valueToGuidePortMap (uuid) + } else SourceInfo.TxNotStartedRetry + logInfo ("TrackMultipleValues: Got new request: " + clientSocket + " for " + uuid + " : " + guidePort) + oos.writeObject (guidePort) + } catch { + case e: Exception => { + logInfo ("TrackMultipleValues had a " + e) + } + } finally { + ois.close + oos.close + clientSocket.close + } + } + }) + } catch { + // In failure, close socket here; else, client thread will close + case ioe: IOException => clientSocket.close + } + } + } + } finally { + serverSocket.close + } + + // Shutdown the thread pool + threadPool.shutdown + } + } +} diff --git a/src/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala similarity index 100% rename from src/scala/spark/ClosureCleaner.scala rename to core/src/main/scala/spark/ClosureCleaner.scala diff --git a/src/scala/spark/CustomBlockedInMemoryShuffle.scala b/core/src/main/scala/spark/CustomBlockedInMemoryShuffle.scala similarity index 100% rename from src/scala/spark/CustomBlockedInMemoryShuffle.scala rename to core/src/main/scala/spark/CustomBlockedInMemoryShuffle.scala diff --git a/src/scala/spark/CustomBlockedLocalFileShuffle.scala b/core/src/main/scala/spark/CustomBlockedLocalFileShuffle.scala similarity index 100% rename from src/scala/spark/CustomBlockedLocalFileShuffle.scala rename to core/src/main/scala/spark/CustomBlockedLocalFileShuffle.scala diff --git a/src/scala/spark/CustomParallelFakeShuffle.scala b/core/src/main/scala/spark/CustomParallelFakeShuffle.scala similarity index 100% rename from src/scala/spark/CustomParallelFakeShuffle.scala rename to core/src/main/scala/spark/CustomParallelFakeShuffle.scala diff --git a/src/scala/spark/CustomParallelInMemoryShuffle.scala b/core/src/main/scala/spark/CustomParallelInMemoryShuffle.scala similarity index 100% rename from src/scala/spark/CustomParallelInMemoryShuffle.scala rename to core/src/main/scala/spark/CustomParallelInMemoryShuffle.scala diff --git a/src/scala/spark/CustomParallelLocalFileShuffle.scala b/core/src/main/scala/spark/CustomParallelLocalFileShuffle.scala similarity index 100% rename from src/scala/spark/CustomParallelLocalFileShuffle.scala rename to core/src/main/scala/spark/CustomParallelLocalFileShuffle.scala diff --git a/core/src/main/scala/spark/DfsBroadcast.scala b/core/src/main/scala/spark/DfsBroadcast.scala new file mode 100644 index 0000000000..480d6dd9b1 --- /dev/null +++ b/core/src/main/scala/spark/DfsBroadcast.scala @@ -0,0 +1,132 @@ +package spark + +import java.io._ +import java.net._ +import java.util.UUID + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path, RawLocalFileSystem} + +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} + +@serializable +class DfsBroadcast[T](@transient var value_ : T, isLocal: Boolean) +extends Broadcast[T] with Logging { + + def value = value_ + + DfsBroadcast.synchronized { + DfsBroadcast.values.put(uuid, value_) + } + + if (!isLocal) { + sendBroadcast + } + + def sendBroadcast (): Unit = { + val out = new ObjectOutputStream (DfsBroadcast.openFileForWriting(uuid)) + out.writeObject (value_) + out.close + } + + // Called by JVM when deserializing an object + private def readObject(in: ObjectInputStream): Unit = { + in.defaultReadObject + DfsBroadcast.synchronized { + val cachedVal = DfsBroadcast.values.get(uuid) + if (cachedVal != null) { + value_ = cachedVal.asInstanceOf[T] + } else { + logInfo( "Started reading Broadcasted variable " + uuid) + val start = System.nanoTime + + val fileIn = new ObjectInputStream(DfsBroadcast.openFileForReading(uuid)) + value_ = fileIn.readObject.asInstanceOf[T] + DfsBroadcast.values.put(uuid, value_) + fileIn.close + + val time = (System.nanoTime - start) / 1e9 + logInfo( "Reading Broadcasted variable " + uuid + " took " + time + " s") + } + } + } +} + +class DfsBroadcastFactory +extends BroadcastFactory { + def initialize (isMaster: Boolean) = DfsBroadcast.initialize + def newBroadcast[T] (value_ : T, isLocal: Boolean) = + new DfsBroadcast[T] (value_, isLocal) +} + +private object DfsBroadcast +extends Logging { + val values = Cache.newKeySpace() + + private var initialized = false + + private var fileSystem: FileSystem = null + private var workDir: String = null + private var compress: Boolean = false + private var bufferSize: Int = 65536 + + def initialize (): Unit = { + synchronized { + if (!initialized) { + bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val dfs = System.getProperty("spark.dfs", "file:///") + if (!dfs.startsWith("file://")) { + val conf = new Configuration() + conf.setInt("io.file.buffer.size", bufferSize) + val rep = System.getProperty("spark.dfs.replication", "3").toInt + conf.setInt("dfs.replication", rep) + fileSystem = FileSystem.get(new URI(dfs), conf) + } + workDir = System.getProperty("spark.dfs.workDir", "/tmp") + compress = System.getProperty("spark.compress", "false").toBoolean + + initialized = true + } + } + } + + private def getPath(uuid: UUID) = new Path(workDir + "/broadcast-" + uuid) + + def openFileForReading(uuid: UUID): InputStream = { + val fileStream = if (fileSystem != null) { + fileSystem.open(getPath(uuid)) + } else { + // Local filesystem + new FileInputStream(getPath(uuid).toString) + } + + if (compress) { + // LZF stream does its own buffering + new LZFInputStream(fileStream) + } else if (fileSystem == null) { + new BufferedInputStream(fileStream, bufferSize) + } else { + // Hadoop streams do their own buffering + fileStream + } + } + + def openFileForWriting(uuid: UUID): OutputStream = { + val fileStream = if (fileSystem != null) { + fileSystem.create(getPath(uuid)) + } else { + // Local filesystem + new FileOutputStream(getPath(uuid).toString) + } + + if (compress) { + // LZF stream does its own buffering + new LZFOutputStream(fileStream) + } else if (fileSystem == null) { + new BufferedOutputStream(fileStream, bufferSize) + } else { + // Hadoop streams do their own buffering + fileStream + } + } +} diff --git a/src/scala/spark/DfsShuffle.scala b/core/src/main/scala/spark/DfsShuffle.scala similarity index 100% rename from src/scala/spark/DfsShuffle.scala rename to core/src/main/scala/spark/DfsShuffle.scala diff --git a/src/scala/spark/Executor.scala b/core/src/main/scala/spark/Executor.scala similarity index 100% rename from src/scala/spark/Executor.scala rename to core/src/main/scala/spark/Executor.scala diff --git a/src/scala/spark/HadoopFile.scala b/core/src/main/scala/spark/HadoopFile.scala similarity index 84% rename from src/scala/spark/HadoopFile.scala rename to core/src/main/scala/spark/HadoopFile.scala index a63c9d8a94..f5e80d5432 100644 --- a/src/scala/spark/HadoopFile.scala +++ b/core/src/main/scala/spark/HadoopFile.scala @@ -34,7 +34,7 @@ class HadoopFile[K, V]( keyClass: Class[K], valueClass: Class[V]) extends RDD[(K, V)](sc) { - @transient val splits_ : Array[Split] = ConfigureLock.synchronized { + @transient val splits_ : Array[Split] = { val conf = new JobConf() FileInputFormat.setInputPaths(conf, path) val inputFormat = createInputFormat(conf) @@ -53,13 +53,11 @@ extends RDD[(K, V)](sc) { val split = theSplit.asInstanceOf[HadoopSplit] var reader: RecordReader[K, V] = null - ConfigureLock.synchronized { - val conf = new JobConf() - val bufferSize = System.getProperty("spark.buffer.size", "65536") - conf.set("io.file.buffer.size", bufferSize) - val fmt = createInputFormat(conf) - reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) - } + val conf = new JobConf() + val bufferSize = System.getProperty("spark.buffer.size", "65536") + conf.set("io.file.buffer.size", bufferSize) + val fmt = createInputFormat(conf) + reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) val key: K = keyClass.newInstance() val value: V = valueClass.newInstance() @@ -76,6 +74,9 @@ extends RDD[(K, V)](sc) { } gotNext = true } + if (finished) { + reader.close() + } !finished } @@ -109,10 +110,3 @@ extends MappedRDD[String, (LongWritable, Text)]( classOf[LongWritable], classOf[Text]), { pair: (LongWritable, Text) => pair._2.toString } ) - - -/** - * Object used to ensure that only one thread at a time is configuring Hadoop - * InputFormat classes. Apparently configuring them is not thread safe! - */ -object ConfigureLock {} diff --git a/src/scala/spark/HttpBlockedLocalFileShuffle.scala b/core/src/main/scala/spark/HttpBlockedLocalFileShuffle.scala similarity index 100% rename from src/scala/spark/HttpBlockedLocalFileShuffle.scala rename to core/src/main/scala/spark/HttpBlockedLocalFileShuffle.scala diff --git a/src/scala/spark/HttpParallelLocalFileShuffle.scala b/core/src/main/scala/spark/HttpParallelLocalFileShuffle.scala similarity index 100% rename from src/scala/spark/HttpParallelLocalFileShuffle.scala rename to core/src/main/scala/spark/HttpParallelLocalFileShuffle.scala diff --git a/src/scala/spark/HttpServer.scala b/core/src/main/scala/spark/HttpServer.scala similarity index 100% rename from src/scala/spark/HttpServer.scala rename to core/src/main/scala/spark/HttpServer.scala diff --git a/src/scala/spark/Job.scala b/core/src/main/scala/spark/Job.scala similarity index 100% rename from src/scala/spark/Job.scala rename to core/src/main/scala/spark/Job.scala diff --git a/src/scala/spark/LocalScheduler.scala b/core/src/main/scala/spark/LocalScheduler.scala similarity index 100% rename from src/scala/spark/LocalScheduler.scala rename to core/src/main/scala/spark/LocalScheduler.scala diff --git a/src/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala similarity index 100% rename from src/scala/spark/Logging.scala rename to core/src/main/scala/spark/Logging.scala diff --git a/src/scala/spark/ManualBlockedLocalFileShuffle.scala b/core/src/main/scala/spark/ManualBlockedLocalFileShuffle.scala similarity index 100% rename from src/scala/spark/ManualBlockedLocalFileShuffle.scala rename to core/src/main/scala/spark/ManualBlockedLocalFileShuffle.scala diff --git a/src/scala/spark/MesosScheduler.scala b/core/src/main/scala/spark/MesosScheduler.scala similarity index 97% rename from src/scala/spark/MesosScheduler.scala rename to core/src/main/scala/spark/MesosScheduler.scala index c45eff64d4..1cfc7052c6 100644 --- a/src/scala/spark/MesosScheduler.scala +++ b/core/src/main/scala/spark/MesosScheduler.scala @@ -98,7 +98,7 @@ extends MScheduler with spark.Scheduler with Logging params("env." + key) = System.getenv(key) } } - new ExecutorInfo(execScript, createExecArg()) + new ExecutorInfo(execScript, createExecArg(), params) } /** @@ -253,9 +253,11 @@ extends MScheduler with spark.Scheduler with Logging // Copy each JAR to a unique filename in the jarDir for ((path, index) <- sc.jars.zipWithIndex) { val file = new File(path) - val filename = index + "_" + file.getName - copyFile(file, new File(jarDir, filename)) - filenames += filename + if (file.exists) { + val filename = index + "_" + file.getName + copyFile(file, new File(jarDir, filename)) + filenames += filename + } } // Create the server jarServer = new HttpServer(jarDir) diff --git a/src/scala/spark/NumberedSplitRDD.scala b/core/src/main/scala/spark/NumberedSplitRDD.scala similarity index 100% rename from src/scala/spark/NumberedSplitRDD.scala rename to core/src/main/scala/spark/NumberedSplitRDD.scala diff --git a/src/scala/spark/ParallelArray.scala b/core/src/main/scala/spark/ParallelArray.scala similarity index 100% rename from src/scala/spark/ParallelArray.scala rename to core/src/main/scala/spark/ParallelArray.scala diff --git a/src/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala similarity index 99% rename from src/scala/spark/RDD.scala rename to core/src/main/scala/spark/RDD.scala index bac59319a0..39f2dc4458 100644 --- a/src/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -359,7 +359,7 @@ extends RDD[Pair[T, U]](sc) { : RDD[(K, C)] = { val shufClass = Class.forName(System.getProperty( - "spark.shuffle.class", "spark.DfsShuffle")) + "spark.shuffle.class", "spark.LocalFileShuffle")) val shuf = shufClass.newInstance().asInstanceOf[Shuffle[K, V, C]] shuf.compute(self, numSplits, createCombiner, mergeValue, mergeCombiners) } diff --git a/src/scala/spark/Scheduler.scala b/core/src/main/scala/spark/Scheduler.scala similarity index 100% rename from src/scala/spark/Scheduler.scala rename to core/src/main/scala/spark/Scheduler.scala diff --git a/src/scala/spark/SerializableWritable.scala b/core/src/main/scala/spark/SerializableWritable.scala similarity index 100% rename from src/scala/spark/SerializableWritable.scala rename to core/src/main/scala/spark/SerializableWritable.scala diff --git a/src/scala/spark/Shuffle.scala b/core/src/main/scala/spark/Shuffle.scala similarity index 100% rename from src/scala/spark/Shuffle.scala rename to core/src/main/scala/spark/Shuffle.scala diff --git a/src/scala/spark/ShuffleTrackerStrategy.scala b/core/src/main/scala/spark/ShuffleTrackerStrategy.scala similarity index 100% rename from src/scala/spark/ShuffleTrackerStrategy.scala rename to core/src/main/scala/spark/ShuffleTrackerStrategy.scala diff --git a/src/scala/spark/SimpleJob.scala b/core/src/main/scala/spark/SimpleJob.scala similarity index 100% rename from src/scala/spark/SimpleJob.scala rename to core/src/main/scala/spark/SimpleJob.scala diff --git a/src/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala similarity index 89% rename from src/scala/spark/SizeEstimator.scala rename to core/src/main/scala/spark/SizeEstimator.scala index 12dd19d704..a3774fb055 100644 --- a/src/scala/spark/SizeEstimator.scala +++ b/core/src/main/scala/spark/SizeEstimator.scala @@ -5,6 +5,7 @@ import java.lang.reflect.Modifier import java.lang.reflect.{Array => JArray} import java.util.IdentityHashMap import java.util.concurrent.ConcurrentHashMap +import java.util.Random import scala.collection.mutable.ArrayBuffer @@ -98,8 +99,20 @@ object SizeEstimator { state.size += length * primitiveSize(elementClass) } else { state.size += length * POINTER_SIZE - for (i <- 0 until length) { - state.enqueue(JArray.get(array, i)) + if (length <= 100) { + for (i <- 0 until length) { + state.enqueue(JArray.get(array, i)) + } + } else { + // Estimate the size of a large array by sampling elements. + // TODO: Add a config setting for turning this off? + var size = 0.0 + val rand = new Random(42) + for (i <- 0 until 100) { + val elem = JArray.get(array, rand.nextInt(length)) + size += SizeEstimator.estimate(elem) + } + state.size += ((length / 100.0) * size).toLong } } } diff --git a/src/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala similarity index 100% rename from src/scala/spark/SoftReferenceCache.scala rename to core/src/main/scala/spark/SoftReferenceCache.scala diff --git a/src/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala similarity index 96% rename from src/scala/spark/SparkContext.scala rename to core/src/main/scala/spark/SparkContext.scala index 02e80c7756..bf70b5fcb1 100644 --- a/src/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -97,9 +97,9 @@ extends Logging { def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = new Accumulator(initialValue, param) - // TODO: Keep around a weak hash map of values to Cached versions? - def broadcast[T](value: T) = new CentralizedHDFSBroadcast(value, isLocal) - //def broadcast[T](value: T) = new ChainedStreamingBroadcast(value, isLocal) + // Keep around a weak hash map of values to Cached versions? + def broadcast[T](value: T) = + Broadcast.getBroadcastFactory.newBroadcast[T] (value, isLocal) // Stop the SparkContext def stop() { diff --git a/src/scala/spark/SparkException.scala b/core/src/main/scala/spark/SparkException.scala similarity index 100% rename from src/scala/spark/SparkException.scala rename to core/src/main/scala/spark/SparkException.scala diff --git a/src/scala/spark/Split.scala b/core/src/main/scala/spark/Split.scala similarity index 100% rename from src/scala/spark/Split.scala rename to core/src/main/scala/spark/Split.scala diff --git a/src/scala/spark/Task.scala b/core/src/main/scala/spark/Task.scala similarity index 100% rename from src/scala/spark/Task.scala rename to core/src/main/scala/spark/Task.scala diff --git a/src/scala/spark/TaskResult.scala b/core/src/main/scala/spark/TaskResult.scala similarity index 100% rename from src/scala/spark/TaskResult.scala rename to core/src/main/scala/spark/TaskResult.scala diff --git a/src/scala/spark/TrackedCustomBlockedInMemoryShuffle.scala b/core/src/main/scala/spark/TrackedCustomBlockedInMemoryShuffle.scala similarity index 100% rename from src/scala/spark/TrackedCustomBlockedInMemoryShuffle.scala rename to core/src/main/scala/spark/TrackedCustomBlockedInMemoryShuffle.scala diff --git a/src/scala/spark/TrackedCustomBlockedLocalFileShuffle.scala b/core/src/main/scala/spark/TrackedCustomBlockedLocalFileShuffle.scala similarity index 100% rename from src/scala/spark/TrackedCustomBlockedLocalFileShuffle.scala rename to core/src/main/scala/spark/TrackedCustomBlockedLocalFileShuffle.scala diff --git a/src/scala/spark/TrackedCustomParallelLocalFileShuffle.scala b/core/src/main/scala/spark/TrackedCustomParallelLocalFileShuffle.scala similarity index 100% rename from src/scala/spark/TrackedCustomParallelLocalFileShuffle.scala rename to core/src/main/scala/spark/TrackedCustomParallelLocalFileShuffle.scala diff --git a/src/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala similarity index 100% rename from src/scala/spark/Utils.scala rename to core/src/main/scala/spark/Utils.scala diff --git a/src/scala/spark/WeakReferenceCache.scala b/core/src/main/scala/spark/WeakReferenceCache.scala similarity index 100% rename from src/scala/spark/WeakReferenceCache.scala rename to core/src/main/scala/spark/WeakReferenceCache.scala diff --git a/src/scala/spark/repl/ExecutorClassLoader.scala b/core/src/main/scala/spark/repl/ExecutorClassLoader.scala similarity index 100% rename from src/scala/spark/repl/ExecutorClassLoader.scala rename to core/src/main/scala/spark/repl/ExecutorClassLoader.scala diff --git a/src/scala/spark/repl/Main.scala b/core/src/main/scala/spark/repl/Main.scala similarity index 100% rename from src/scala/spark/repl/Main.scala rename to core/src/main/scala/spark/repl/Main.scala diff --git a/src/scala/spark/repl/SparkCompletion.scala b/core/src/main/scala/spark/repl/SparkCompletion.scala similarity index 99% rename from src/scala/spark/repl/SparkCompletion.scala rename to core/src/main/scala/spark/repl/SparkCompletion.scala index d67438445b..9fa41736f3 100644 --- a/src/scala/spark/repl/SparkCompletion.scala +++ b/core/src/main/scala/spark/repl/SparkCompletion.scala @@ -310,7 +310,7 @@ class SparkCompletion(val repl: SparkInterpreter) extends SparkCompletionOutput else xs.reduceLeft(_ zip _ takeWhile (x => x._1 == x._2) map (_._1) mkString) // This is jline's entry point for completion. - override def complete(_buf: String, cursor: Int, candidates: JList[String]): Int = { + override def complete(_buf: String, cursor: Int, candidates: java.util.List[java.lang.String]): Int = { val buf = onull(_buf) verbosity = if (isConsecutiveTabs(buf, cursor)) verbosity + 1 else 0 DBG("complete(%s, %d) last = (%s, %d), verbosity: %s".format(buf, cursor, lastBuf, lastCursor, verbosity)) @@ -321,7 +321,7 @@ class SparkCompletion(val repl: SparkInterpreter) extends SparkCompletionOutput case Nil => None case xs => // modify in place and return the position - xs foreach (candidates add _) + xs.foreach(x => candidates.add(x)) // update the last buffer unless this is an alternatives list if (xs contains "") Some(p.cursor) diff --git a/src/scala/spark/repl/SparkCompletionOutput.scala b/core/src/main/scala/spark/repl/SparkCompletionOutput.scala similarity index 100% rename from src/scala/spark/repl/SparkCompletionOutput.scala rename to core/src/main/scala/spark/repl/SparkCompletionOutput.scala diff --git a/src/scala/spark/repl/SparkInteractiveReader.scala b/core/src/main/scala/spark/repl/SparkInteractiveReader.scala similarity index 100% rename from src/scala/spark/repl/SparkInteractiveReader.scala rename to core/src/main/scala/spark/repl/SparkInteractiveReader.scala diff --git a/src/scala/spark/repl/SparkInterpreter.scala b/core/src/main/scala/spark/repl/SparkInterpreter.scala similarity index 100% rename from src/scala/spark/repl/SparkInterpreter.scala rename to core/src/main/scala/spark/repl/SparkInterpreter.scala diff --git a/src/scala/spark/repl/SparkInterpreterLoop.scala b/core/src/main/scala/spark/repl/SparkInterpreterLoop.scala similarity index 99% rename from src/scala/spark/repl/SparkInterpreterLoop.scala rename to core/src/main/scala/spark/repl/SparkInterpreterLoop.scala index 5bad0a37da..d4974009ce 100644 --- a/src/scala/spark/repl/SparkInterpreterLoop.scala +++ b/core/src/main/scala/spark/repl/SparkInterpreterLoop.scala @@ -129,7 +129,8 @@ extends InterpreterControl { settings.classpath append addedClasspath interpreter = new SparkInterpreter(settings, out) { - override protected def parentClassLoader = classOf[SparkInterpreterLoop].getClassLoader + override protected def parentClassLoader = + classOf[SparkInterpreterLoop].getClassLoader } interpreter.setContextClassLoader() // interpreter.quietBind("settings", "spark.repl.SparkInterpreterSettings", interpreter.isettings) diff --git a/src/scala/spark/repl/SparkInterpreterSettings.scala b/core/src/main/scala/spark/repl/SparkInterpreterSettings.scala similarity index 100% rename from src/scala/spark/repl/SparkInterpreterSettings.scala rename to core/src/main/scala/spark/repl/SparkInterpreterSettings.scala diff --git a/src/scala/spark/repl/SparkJLineReader.scala b/core/src/main/scala/spark/repl/SparkJLineReader.scala similarity index 100% rename from src/scala/spark/repl/SparkJLineReader.scala rename to core/src/main/scala/spark/repl/SparkJLineReader.scala diff --git a/src/scala/spark/repl/SparkSimpleReader.scala b/core/src/main/scala/spark/repl/SparkSimpleReader.scala similarity index 100% rename from src/scala/spark/repl/SparkSimpleReader.scala rename to core/src/main/scala/spark/repl/SparkSimpleReader.scala diff --git a/src/test/spark/ParallelArraySplitSuite.scala b/core/src/test/scala/spark/ParallelArraySplitSuite.scala similarity index 100% rename from src/test/spark/ParallelArraySplitSuite.scala rename to core/src/test/scala/spark/ParallelArraySplitSuite.scala diff --git a/src/test/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala similarity index 100% rename from src/test/spark/ShuffleSuite.scala rename to core/src/test/scala/spark/ShuffleSuite.scala diff --git a/src/test/spark/repl/ReplSuite.scala b/core/src/test/scala/spark/repl/ReplSuite.scala similarity index 89% rename from src/test/spark/repl/ReplSuite.scala rename to core/src/test/scala/spark/repl/ReplSuite.scala index 8b38cde85f..225e766c71 100644 --- a/src/test/spark/repl/ReplSuite.scala +++ b/core/src/test/scala/spark/repl/ReplSuite.scala @@ -1,6 +1,10 @@ package spark.repl import java.io._ +import java.net.URLClassLoader + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ import org.scalatest.FunSuite @@ -8,9 +12,20 @@ class ReplSuite extends FunSuite { def runInterpreter(master: String, input: String): String = { val in = new BufferedReader(new StringReader(input + "\n")) val out = new StringWriter() + val cl = getClass.getClassLoader + var paths = new ArrayBuffer[String] + if (cl.isInstanceOf[URLClassLoader]) { + val urlLoader = cl.asInstanceOf[URLClassLoader] + for (url <- urlLoader.getURLs) { + if (url.getProtocol == "file") { + paths += url.getFile + } + } + } val interp = new SparkInterpreterLoop(in, new PrintWriter(out), master) spark.repl.Main.interp = interp - interp.main(new Array[String](0)) + val separator = System.getProperty("path.separator") + interp.main(Array("-classpath", paths.mkString(separator))) spark.repl.Main.interp = null return out.toString } diff --git a/src/examples/BroadcastTest.scala b/examples/src/main/scala/spark/examples/BroadcastTest.scala similarity index 97% rename from src/examples/BroadcastTest.scala rename to examples/src/main/scala/spark/examples/BroadcastTest.scala index 40c2be8f6d..2506de5ae5 100644 --- a/src/examples/BroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/BroadcastTest.scala @@ -1,3 +1,5 @@ +package spark.examples + import spark.SparkContext object BroadcastTest { diff --git a/src/examples/CpuHog.scala b/examples/src/main/scala/spark/examples/CpuHog.scala similarity index 95% rename from src/examples/CpuHog.scala rename to examples/src/main/scala/spark/examples/CpuHog.scala index f37c6f7824..94b3709850 100644 --- a/src/examples/CpuHog.scala +++ b/examples/src/main/scala/spark/examples/CpuHog.scala @@ -1,3 +1,5 @@ +package spark.examples + import spark._ object CpuHog { diff --git a/src/examples/GroupByTest.scala b/examples/src/main/scala/spark/examples/GroupByTest.scala similarity index 97% rename from src/examples/GroupByTest.scala rename to examples/src/main/scala/spark/examples/GroupByTest.scala index 6f591faa6e..48c02a52c6 100644 --- a/src/examples/GroupByTest.scala +++ b/examples/src/main/scala/spark/examples/GroupByTest.scala @@ -1,3 +1,5 @@ +package spark.examples + import spark.SparkContext import spark.SparkContext._ import java.util.Random diff --git a/src/examples/HdfsTest.scala b/examples/src/main/scala/spark/examples/HdfsTest.scala similarity index 95% rename from src/examples/HdfsTest.scala rename to examples/src/main/scala/spark/examples/HdfsTest.scala index e678154aab..072b4ce417 100644 --- a/src/examples/HdfsTest.scala +++ b/examples/src/main/scala/spark/examples/HdfsTest.scala @@ -1,3 +1,5 @@ +package spark.examples + import spark._ object HdfsTest { diff --git a/src/examples/LocalALS.scala b/examples/src/main/scala/spark/examples/LocalALS.scala similarity index 99% rename from src/examples/LocalALS.scala rename to examples/src/main/scala/spark/examples/LocalALS.scala index a976a5e1c5..10360dab3d 100644 --- a/src/examples/LocalALS.scala +++ b/examples/src/main/scala/spark/examples/LocalALS.scala @@ -1,3 +1,5 @@ +package spark.examples + import java.util.Random import scala.math.sqrt import cern.jet.math._ diff --git a/src/examples/LocalFileLR.scala b/examples/src/main/scala/spark/examples/LocalFileLR.scala similarity index 97% rename from src/examples/LocalFileLR.scala rename to examples/src/main/scala/spark/examples/LocalFileLR.scala index 3d3bb60677..cc14aa7090 100644 --- a/src/examples/LocalFileLR.scala +++ b/examples/src/main/scala/spark/examples/LocalFileLR.scala @@ -1,3 +1,5 @@ +package spark.examples + import java.util.Random import Vector._ diff --git a/src/examples/LocalLR.scala b/examples/src/main/scala/spark/examples/LocalLR.scala similarity index 97% rename from src/examples/LocalLR.scala rename to examples/src/main/scala/spark/examples/LocalLR.scala index 175907e551..3fd3f88fa8 100644 --- a/src/examples/LocalLR.scala +++ b/examples/src/main/scala/spark/examples/LocalLR.scala @@ -1,3 +1,5 @@ +package spark.examples + import java.util.Random import Vector._ diff --git a/src/examples/LocalPi.scala b/examples/src/main/scala/spark/examples/LocalPi.scala similarity index 93% rename from src/examples/LocalPi.scala rename to examples/src/main/scala/spark/examples/LocalPi.scala index c61b3e53d4..9457472f2d 100644 --- a/src/examples/LocalPi.scala +++ b/examples/src/main/scala/spark/examples/LocalPi.scala @@ -1,3 +1,5 @@ +package spark.examples + import scala.math.random import spark._ import SparkContext._ diff --git a/src/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala similarity index 98% rename from src/examples/SimpleSkewedGroupByTest.scala rename to examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala index d37efcd566..c8edb7d8b4 100644 --- a/src/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala @@ -1,3 +1,5 @@ +package spark.examples + import spark.SparkContext import spark.SparkContext._ import java.util.Random diff --git a/src/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala similarity index 98% rename from src/examples/SkewedGroupByTest.scala rename to examples/src/main/scala/spark/examples/SkewedGroupByTest.scala index 18edc1c77b..e6dec44bed 100644 --- a/src/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala @@ -1,3 +1,5 @@ +package spark.examples + import spark.SparkContext import spark.SparkContext._ import java.util.Random diff --git a/src/examples/SleepJob.scala b/examples/src/main/scala/spark/examples/SleepJob.scala similarity index 95% rename from src/examples/SleepJob.scala rename to examples/src/main/scala/spark/examples/SleepJob.scala index a5e0ea0dc2..02673a5f88 100644 --- a/src/examples/SleepJob.scala +++ b/examples/src/main/scala/spark/examples/SleepJob.scala @@ -1,3 +1,5 @@ +package spark.examples + import spark._ object SleepJob { diff --git a/src/examples/SparkALS.scala b/examples/src/main/scala/spark/examples/SparkALS.scala similarity index 99% rename from src/examples/SparkALS.scala rename to examples/src/main/scala/spark/examples/SparkALS.scala index 6fae3c0940..08e0420371 100644 --- a/src/examples/SparkALS.scala +++ b/examples/src/main/scala/spark/examples/SparkALS.scala @@ -1,3 +1,5 @@ +package spark.examples + import java.io.Serializable import java.util.Random import scala.math.sqrt diff --git a/src/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala similarity index 98% rename from src/examples/SparkHdfsLR.scala rename to examples/src/main/scala/spark/examples/SparkHdfsLR.scala index f14d48b17c..4c71fd0845 100644 --- a/src/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala @@ -1,3 +1,5 @@ +package spark.examples + import java.util.Random import scala.math.exp import Vector._ diff --git a/src/examples/SparkLR.scala b/examples/src/main/scala/spark/examples/SparkLR.scala similarity index 98% rename from src/examples/SparkLR.scala rename to examples/src/main/scala/spark/examples/SparkLR.scala index 71f9aea624..d08f5d3f01 100644 --- a/src/examples/SparkLR.scala +++ b/examples/src/main/scala/spark/examples/SparkLR.scala @@ -1,3 +1,5 @@ +package spark.examples + import java.util.Random import scala.math.exp import Vector._ diff --git a/src/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala similarity index 96% rename from src/examples/SparkPi.scala rename to examples/src/main/scala/spark/examples/SparkPi.scala index f055614125..31c6c5b9b1 100644 --- a/src/examples/SparkPi.scala +++ b/examples/src/main/scala/spark/examples/SparkPi.scala @@ -1,3 +1,5 @@ +package spark.examples + import scala.math.random import spark._ import SparkContext._ diff --git a/src/examples/Vector.scala b/examples/src/main/scala/spark/examples/Vector.scala similarity index 98% rename from src/examples/Vector.scala rename to examples/src/main/scala/spark/examples/Vector.scala index e9fbdca752..ea70626e71 100644 --- a/src/examples/Vector.scala +++ b/examples/src/main/scala/spark/examples/Vector.scala @@ -1,3 +1,5 @@ +package spark.examples + @serializable class Vector(val elements: Array[Double]) { def length = elements.length diff --git a/project/build.properties b/project/build.properties new file mode 100644 index 0000000000..b048c01e2a --- /dev/null +++ b/project/build.properties @@ -0,0 +1,8 @@ +#Project properties +#Sat Nov 13 21:57:32 PST 2010 +project.organization=UC Berkeley +project.name=Spark +sbt.version=0.7.5.RC0 +project.version=0.0.0 +build.scala.versions=2.8.1 +project.initialize=false diff --git a/project/build/SparkProject.scala b/project/build/SparkProject.scala new file mode 100644 index 0000000000..484daf5c50 --- /dev/null +++ b/project/build/SparkProject.scala @@ -0,0 +1,76 @@ +import sbt._ +import sbt.Process._ + +import assembly._ + +import de.element34.sbteclipsify._ + + +class SparkProject(info: ProjectInfo) +extends ParentProject(info) with IdeaProject +{ + lazy val core = project("core", "Spark Core", new CoreProject(_)) + + lazy val examples = + project("examples", "Spark Examples", new ExamplesProject(_), core) + + class CoreProject(info: ProjectInfo) + extends DefaultProject(info) with Eclipsify with IdeaProject with DepJar with XmlTestReport + {} + + class ExamplesProject(info: ProjectInfo) + extends DefaultProject(info) with Eclipsify with IdeaProject + {} +} + + +// Project mixin for an XML-based ScalaTest report. Unfortunately +// there is currently no way to call this directly from SBT without +// executing a subprocess. +trait XmlTestReport extends BasicScalaProject { + def testReportDir = outputPath / "test-report" + + lazy val testReport = task { + log.info("Creating " + testReportDir + "...") + if (!testReportDir.exists) { + testReportDir.asFile.mkdirs() + } + log.info("Executing org.scalatest.tools.Runner...") + val command = ("scala -classpath " + testClasspath.absString + + " org.scalatest.tools.Runner -o " + + " -u " + testReportDir.absolutePath + + " -p " + (outputPath / "test-classes").absolutePath) + Process(command, path("."), "JAVA_OPTS" -> "-Xmx500m") ! + + None + }.dependsOn(compile, testCompile).describedAs("Generate XML test report.") +} + + +// Project mixin for creating a JAR with a project's dependencies. This is based +// on the AssemblyBuilder plugin, but because this plugin attempts to package Scala +// and our project too, we leave that out using our own exclude filter (depJarExclude). +trait DepJar extends AssemblyBuilder { + def depJarExclude(base: PathFinder) = { + (base / "scala" ** "*") +++ // exclude scala library + (base / "spark" ** "*") +++ // exclude Spark classes + ((base / "META-INF" ** "*") --- // generally ignore the hell out of META-INF + (base / "META-INF" / "services" ** "*") --- // include all service providers + (base / "META-INF" / "maven" ** "*")) // include all Maven POMs and such + } + + def depJarTempDir = outputPath / "dep-classes" + + def depJarOutputPath = + outputPath / (name.toLowerCase.replace(" ", "-") + "-dep-" + version.toString + ".jar") + + lazy val depJar = { + packageTask( + Path.lazyPathFinder(assemblyPaths(depJarTempDir, + assemblyClasspath, + assemblyExtraJars, + depJarExclude)), + depJarOutputPath, + packageOptions) + }.dependsOn(compile).describedAs("Bundle project's dependencies into a JAR.") +} diff --git a/project/plugins/SparkProjectPlugins.scala b/project/plugins/SparkProjectPlugins.scala new file mode 100644 index 0000000000..b07dfafcfb --- /dev/null +++ b/project/plugins/SparkProjectPlugins.scala @@ -0,0 +1,11 @@ +import sbt._ + +class SparkProjectPlugins(info: ProjectInfo) extends PluginDefinition(info) { + val eclipse = "de.element34" % "sbt-eclipsify" % "0.7.0" + + val sbtIdeaRepo = "sbt-idea-repo" at "http://mpeltonen.github.com/maven/" + val sbtIdea = "com.github.mpeltonen" % "sbt-idea-plugin" % "0.2.0" + + val codaRepo = "Coda Hale's Repository" at "http://repo.codahale.com/" + val assemblySBT = "com.codahale" % "assembly-sbt" % "0.1.1" +} diff --git a/run b/run index d6f7d920c5..9fb815987d 100755 --- a/run +++ b/run @@ -25,7 +25,7 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$SPARK_JAVA_OPTS" -JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH:$FWDIR/third_party:$FWDIR/src/native:$MESOS_LIBRARY_PATH" +JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH:$FWDIR/lib:$FWDIR/src/main/native:$MESOS_LIBRARY_PATH" JAVA_OPTS+=" -Xms$SPARK_MEM -Xmx$SPARK_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e $FWDIR/conf/java-opts ] ; then @@ -33,22 +33,27 @@ if [ -e $FWDIR/conf/java-opts ] ; then fi export JAVA_OPTS +CORE_DIR=$FWDIR/core +EXAMPLES_DIR=$FWDIR/examples + # Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$FWDIR/build/classes:$MESOS_CLASSPATH" +CLASSPATH="$SPARK_CLASSPATH:$CORE_DIR/target/scala_2.8.1/classes:$MESOS_CLASSPATH" CLASSPATH+=:$FWDIR/conf -CLASSPATH+=:$FWDIR/third_party/mesos.jar -CLASSPATH+=:$FWDIR/third_party/asm-3.2/lib/all/asm-all-3.2.jar -CLASSPATH+=:$FWDIR/third_party/colt.jar -CLASSPATH+=:$FWDIR/third_party/guava-r07/guava-r07.jar -CLASSPATH+=:$FWDIR/third_party/hadoop-0.20.0/hadoop-0.20.0-core.jar -CLASSPATH+=:$FWDIR/third_party/scalatest-1.2/scalatest-1.2.jar -CLASSPATH+=:$FWDIR/third_party/scalacheck_2.8.0-1.7.jar -CLASSPATH+=:$FWDIR/third_party/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar -CLASSPATH+=:$FWDIR/third_party/jetty-7.1.6.v20100715/servlet-api-2.5.jar -CLASSPATH+=:$FWDIR/third_party/apache-log4j-1.2.16/log4j-1.2.16.jar -CLASSPATH+=:$FWDIR/third_party/slf4j-1.6.1/slf4j-api-1.6.1.jar -CLASSPATH+=:$FWDIR/third_party/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar -for jar in $FWDIR/third_party/hadoop-0.20.0/lib/*.jar; do +CLASSPATH+=:$CORE_DIR/lib/mesos.jar +CLASSPATH+=:$CORE_DIR/lib/asm-3.2/lib/all/asm-all-3.2.jar +CLASSPATH+=:$CORE_DIR/lib/colt.jar +CLASSPATH+=:$CORE_DIR/lib/guava-r07/guava-r07.jar +CLASSPATH+=:$CORE_DIR/lib/hadoop-0.20.2/hadoop-0.20.2-core.jar +CLASSPATH+=:$CORE_DIR/lib/scalatest-1.2/scalatest-1.2.jar +CLASSPATH+=:$CORE_DIR/lib/scalacheck_2.8.0-1.7.jar +CLASSPATH+=:$CORE_DIR/lib/jetty-7.1.6.v20100715/jetty-server-7.1.6.v20100715.jar +CLASSPATH+=:$CORE_DIR/lib/jetty-7.1.6.v20100715/servlet-api-2.5.jar +CLASSPATH+=:$CORE_DIR/lib/apache-log4j-1.2.16/log4j-1.2.16.jar +CLASSPATH+=:$CORE_DIR/lib/slf4j-1.6.1/slf4j-api-1.6.1.jar +CLASSPATH+=:$CORE_DIR/lib/slf4j-1.6.1/slf4j-log4j12-1.6.1.jar +CLASSPATH+=:$CORE_DIR/lib/compress-lzf-0.6.0/compress-lzf-0.6.0.jar +CLASSPATH+=:$EXAMPLES_DIR/target/scala_2.8.1/classes +for jar in $CORE_DIR/lib/hadoop-0.20.2/lib/*.jar; do CLASSPATH+=:$jar done export CLASSPATH # Needed for spark-shell diff --git a/sbt/sbt b/sbt/sbt new file mode 100755 index 0000000000..87395fa28b --- /dev/null +++ b/sbt/sbt @@ -0,0 +1,2 @@ +#!/bin/bash +java -Xmx700M -jar `dirname $0`/sbt-launch-*.jar "$@" diff --git a/sbt/sbt-launch-0.7.5.RC0.jar b/sbt/sbt-launch-0.7.5.RC0.jar new file mode 100644 index 0000000000..e8a5c00f80 Binary files /dev/null and b/sbt/sbt-launch-0.7.5.RC0.jar differ diff --git a/src/java/spark/compress/lzf/LZF.java b/src/java/spark/compress/lzf/LZF.java deleted file mode 100644 index 294a0494ec..0000000000 --- a/src/java/spark/compress/lzf/LZF.java +++ /dev/null @@ -1,27 +0,0 @@ -package spark.compress.lzf; - -public class LZF { - private static boolean loaded; - - static { - try { - System.loadLibrary("spark_native"); - loaded = true; - } catch(Throwable t) { - System.out.println("Failed to load native LZF library: " + t.toString()); - loaded = false; - } - } - - public static boolean isLoaded() { - return loaded; - } - - public static native int compress( - byte[] in, int inOff, int inLen, - byte[] out, int outOff, int outLen); - - public static native int decompress( - byte[] in, int inOff, int inLen, - byte[] out, int outOff, int outLen); -} diff --git a/src/java/spark/compress/lzf/LZFInputStream.java b/src/java/spark/compress/lzf/LZFInputStream.java deleted file mode 100644 index 16bc687489..0000000000 --- a/src/java/spark/compress/lzf/LZFInputStream.java +++ /dev/null @@ -1,180 +0,0 @@ -package spark.compress.lzf; - -import java.io.EOFException; -import java.io.FilterInputStream; -import java.io.IOException; -import java.io.InputStream; - -public class LZFInputStream extends FilterInputStream { - private static final int MAX_BLOCKSIZE = 1024 * 64 - 1; - private static final int MAX_HDR_SIZE = 7; - - private byte[] inBuf; // Holds data to decompress (including header) - private byte[] outBuf; // Holds decompressed data to output - private int outPos; // Current position in outBuf - private int outSize; // Total amount of data in outBuf - - private boolean closed; - private boolean reachedEof; - - private byte[] singleByte = new byte[1]; - - public LZFInputStream(InputStream in) { - super(in); - if (in == null) - throw new NullPointerException(); - inBuf = new byte[MAX_BLOCKSIZE + MAX_HDR_SIZE]; - outBuf = new byte[MAX_BLOCKSIZE + MAX_HDR_SIZE]; - outPos = 0; - outSize = 0; - } - - private void ensureOpen() throws IOException { - if (closed) throw new IOException("Stream closed"); - } - - @Override - public int read() throws IOException { - ensureOpen(); - int count = read(singleByte, 0, 1); - return (count == -1 ? -1 : singleByte[0] & 0xFF); - } - - @Override - public int read(byte[] b, int off, int len) throws IOException { - ensureOpen(); - if ((off | len | (off + len) | (b.length - (off + len))) < 0) - throw new IndexOutOfBoundsException(); - - int totalRead = 0; - - // Start with the current block in outBuf, and read and decompress any - // further blocks necessary. Instead of trying to decompress directly to b - // when b is large, we always use outBuf as an intermediate holding space - // in case GetPrimitiveArrayCritical decides to copy arrays instead of - // pinning them, which would cause b to be copied repeatedly into C-land. - while (len > 0) { - if (outPos == outSize) { - readNextBlock(); - if (reachedEof) - return totalRead == 0 ? -1 : totalRead; - } - int amtToCopy = Math.min(outSize - outPos, len); - System.arraycopy(outBuf, outPos, b, off, amtToCopy); - off += amtToCopy; - len -= amtToCopy; - outPos += amtToCopy; - totalRead += amtToCopy; - } - - return totalRead; - } - - // Read len bytes from this.in to a buffer, stopping only if EOF is reached - private int readFully(byte[] b, int off, int len) throws IOException { - int totalRead = 0; - while (len > 0) { - int amt = in.read(b, off, len); - if (amt == -1) - break; - off += amt; - len -= amt; - totalRead += amt; - } - return totalRead; - } - - // Read the next block from the underlying InputStream into outBuf, - // setting outPos and outSize, or set reachedEof if the stream ends. - private void readNextBlock() throws IOException { - // Read first 5 bytes of header - int count = readFully(inBuf, 0, 5); - if (count == 0) { - reachedEof = true; - return; - } else if (count < 5) { - throw new EOFException("Truncated LZF block header"); - } - - // Check magic bytes - if (inBuf[0] != 'Z' || inBuf[1] != 'V') - throw new IOException("Wrong magic bytes in LZF block header"); - - // Read the block - if (inBuf[2] == 0) { - // Uncompressed block - read directly to outBuf - int size = ((inBuf[3] & 0xFF) << 8) | (inBuf[4] & 0xFF); - if (readFully(outBuf, 0, size) != size) - throw new EOFException("EOF inside LZF block"); - outPos = 0; - outSize = size; - } else if (inBuf[2] == 1) { - // Compressed block - read to inBuf and decompress - if (readFully(inBuf, 5, 2) != 2) - throw new EOFException("Truncated LZF block header"); - int csize = ((inBuf[3] & 0xFF) << 8) | (inBuf[4] & 0xFF); - int usize = ((inBuf[5] & 0xFF) << 8) | (inBuf[6] & 0xFF); - if (readFully(inBuf, 7, csize) != csize) - throw new EOFException("Truncated LZF block"); - if (LZF.decompress(inBuf, 7, csize, outBuf, 0, usize) != usize) - throw new IOException("Corrupt LZF data stream"); - outPos = 0; - outSize = usize; - } else { - throw new IOException("Unknown block type in LZF block header"); - } - } - - /** - * Returns 0 after EOF has been reached, otherwise always return 1. - * - * Programs should not count on this method to return the actual number - * of bytes that could be read without blocking. - */ - @Override - public int available() throws IOException { - ensureOpen(); - return reachedEof ? 0 : 1; - } - - // TODO: Skip complete chunks without decompressing them? - @Override - public long skip(long n) throws IOException { - ensureOpen(); - if (n < 0) - throw new IllegalArgumentException("negative skip length"); - byte[] buf = new byte[512]; - long skipped = 0; - while (skipped < n) { - int len = (int) Math.min(n - skipped, buf.length); - len = read(buf, 0, len); - if (len == -1) { - reachedEof = true; - break; - } - skipped += len; - } - return skipped; - } - - @Override - public void close() throws IOException { - if (!closed) { - in.close(); - closed = true; - } - } - - @Override - public boolean markSupported() { - return false; - } - - @Override - public void mark(int readLimit) {} - - @Override - public void reset() throws IOException { - throw new IOException("mark/reset not supported"); - } -} diff --git a/src/java/spark/compress/lzf/LZFOutputStream.java b/src/java/spark/compress/lzf/LZFOutputStream.java deleted file mode 100644 index 5f65e95d2a..0000000000 --- a/src/java/spark/compress/lzf/LZFOutputStream.java +++ /dev/null @@ -1,85 +0,0 @@ -package spark.compress.lzf; - -import java.io.FilterOutputStream; -import java.io.IOException; -import java.io.OutputStream; - -public class LZFOutputStream extends FilterOutputStream { - private static final int BLOCKSIZE = 1024 * 64 - 1; - private static final int MAX_HDR_SIZE = 7; - - private byte[] inBuf; // Holds input data to be compressed - private byte[] outBuf; // Holds compressed data to be written - private int inPos; // Current position in inBuf - - public LZFOutputStream(OutputStream out) { - super(out); - inBuf = new byte[BLOCKSIZE + MAX_HDR_SIZE]; - outBuf = new byte[BLOCKSIZE + MAX_HDR_SIZE]; - inPos = MAX_HDR_SIZE; - } - - @Override - public void write(int b) throws IOException { - inBuf[inPos++] = (byte) b; - if (inPos == inBuf.length) - compressAndSendBlock(); - } - - @Override - public void write(byte[] b, int off, int len) throws IOException { - if ((off | len | (off + len) | (b.length - (off + len))) < 0) - throw new IndexOutOfBoundsException(); - - // If we're given a large array, copy it piece by piece into inBuf and - // write one BLOCKSIZE at a time. This is done to prevent the JNI code - // from copying the whole array repeatedly if GetPrimitiveArrayCritical - // decides to copy instead of pinning. - while (inPos + len >= inBuf.length) { - int amtToCopy = inBuf.length - inPos; - System.arraycopy(b, off, inBuf, inPos, amtToCopy); - inPos += amtToCopy; - compressAndSendBlock(); - off += amtToCopy; - len -= amtToCopy; - } - - // Copy the remaining (incomplete) block into inBuf - System.arraycopy(b, off, inBuf, inPos, len); - inPos += len; - } - - @Override - public void flush() throws IOException { - if (inPos > MAX_HDR_SIZE) - compressAndSendBlock(); - out.flush(); - } - - // Send the data in inBuf, and reset inPos to start writing a new block. - private void compressAndSendBlock() throws IOException { - int us = inPos - MAX_HDR_SIZE; - int maxcs = us > 4 ? us - 4 : us; - int cs = LZF.compress(inBuf, MAX_HDR_SIZE, us, outBuf, MAX_HDR_SIZE, maxcs); - if (cs != 0) { - // Compression made the data smaller; use type 1 header - outBuf[0] = 'Z'; - outBuf[1] = 'V'; - outBuf[2] = 1; - outBuf[3] = (byte) (cs >> 8); - outBuf[4] = (byte) (cs & 0xFF); - outBuf[5] = (byte) (us >> 8); - outBuf[6] = (byte) (us & 0xFF); - out.write(outBuf, 0, 7 + cs); - } else { - // Compression didn't help; use type 0 header and uncompressed data - inBuf[2] = 'Z'; - inBuf[3] = 'V'; - inBuf[4] = 0; - inBuf[5] = (byte) (us >> 8); - inBuf[6] = (byte) (us & 0xFF); - out.write(inBuf, 2, 5 + us); - } - inPos = MAX_HDR_SIZE; - } -} diff --git a/src/native/Makefile b/src/native/Makefile deleted file mode 100644 index 6236e26f3d..0000000000 --- a/src/native/Makefile +++ /dev/null @@ -1,30 +0,0 @@ -CC = gcc -#JAVA_HOME = /usr/lib/jvm/java-6-sun -OS_NAME = linux - -CFLAGS = -fPIC -O3 -funroll-all-loops - -SPARK = ../.. - -LZF = $(SPARK)/third_party/liblzf-3.5 - -LIB = libspark_native.so - -all: $(LIB) - -spark_compress_lzf_LZF.h: $(SPARK)/build/classes/spark/compress/lzf/LZF.class -ifeq ($(JAVA_HOME),) - $(error JAVA_HOME is not set) -else - $(JAVA_HOME)/bin/javah -classpath $(SPARK)/build/classes spark.compress.lzf.LZF -endif - -$(LIB): spark_compress_lzf_LZF.h spark_compress_lzf_LZF.c - $(CC) $(CFLAGS) -shared -o $@ spark_compress_lzf_LZF.c \ - -I $(JAVA_HOME)/include -I $(JAVA_HOME)/include/$(OS_NAME) \ - -I $(LZF) $(LZF)/lzf_c.c $(LZF)/lzf_d.c - -clean: - rm -f spark_compress_lzf_LZF.h $(LIB) - -.PHONY: all clean diff --git a/src/native/spark_compress_lzf_LZF.c b/src/native/spark_compress_lzf_LZF.c deleted file mode 100644 index c2a59def3e..0000000000 --- a/src/native/spark_compress_lzf_LZF.c +++ /dev/null @@ -1,90 +0,0 @@ -#include "spark_compress_lzf_LZF.h" -#include - - -/* Helper function to throw an exception */ -static void throwException(JNIEnv *env, const char* className) { - jclass cls = (*env)->FindClass(env, className); - if (cls != 0) /* If cls is null, an exception was already thrown */ - (*env)->ThrowNew(env, cls, ""); -} - - -/* - * Since LZF.compress() and LZF.decompress() have the same signatures - * and differ only in which lzf_ function they call, implement both in a - * single function and pass it a pointer to the correct lzf_ function. - */ -static jint callCompressionFunction - (unsigned int (*func)(const void *const, unsigned int, void *, unsigned int), - JNIEnv *env, jclass cls, jbyteArray inArray, jint inOff, jint inLen, - jbyteArray outArray, jint outOff, jint outLen) -{ - jint inCap; - jint outCap; - jbyte *inData = 0; - jbyte *outData = 0; - jint ret; - jint s; - - if (!inArray || !outArray) { - throwException(env, "java/lang/NullPointerException"); - goto cleanup; - } - - inCap = (*env)->GetArrayLength(env, inArray); - outCap = (*env)->GetArrayLength(env, outArray); - - // Check if any of the offset/length pairs is invalid; we do this by OR'ing - // things we don't want to be negative and seeing if the result is negative - s = inOff | inLen | (inOff + inLen) | (inCap - (inOff + inLen)) | - outOff | outLen | (outOff + outLen) | (outCap - (outOff + outLen)); - if (s < 0) { - throwException(env, "java/lang/IndexOutOfBoundsException"); - goto cleanup; - } - - inData = (*env)->GetPrimitiveArrayCritical(env, inArray, 0); - outData = (*env)->GetPrimitiveArrayCritical(env, outArray, 0); - - if (!inData || !outData) { - // Out of memory - JVM will throw OutOfMemoryError - goto cleanup; - } - - ret = func(inData + inOff, inLen, outData + outOff, outLen); - -cleanup: - if (inData) - (*env)->ReleasePrimitiveArrayCritical(env, inArray, inData, 0); - if (outData) - (*env)->ReleasePrimitiveArrayCritical(env, outArray, outData, 0); - - return ret; -} - -/* - * Class: spark_compress_lzf_LZF - * Method: compress - * Signature: ([B[B)I - */ -JNIEXPORT jint JNICALL Java_spark_compress_lzf_LZF_compress - (JNIEnv *env, jclass cls, jbyteArray inArray, jint inOff, jint inLen, - jbyteArray outArray, jint outOff, jint outLen) -{ - return callCompressionFunction(lzf_compress, env, cls, - inArray, inOff, inLen, outArray,outOff, outLen); -} - -/* - * Class: spark_compress_lzf_LZF - * Method: decompress - * Signature: ([B[B)I - */ -JNIEXPORT jint JNICALL Java_spark_compress_lzf_LZF_decompress - (JNIEnv *env, jclass cls, jbyteArray inArray, jint inOff, jint inLen, - jbyteArray outArray, jint outOff, jint outLen) -{ - return callCompressionFunction(lzf_decompress, env, cls, - inArray, inOff, inLen, outArray,outOff, outLen); -} diff --git a/src/scala/spark/Broadcast.scala b/src/scala/spark/Broadcast.scala deleted file mode 100644 index 2fbe31b1ea..0000000000 --- a/src/scala/spark/Broadcast.scala +++ /dev/null @@ -1,799 +0,0 @@ -package spark - -import java.io._ -import java.net._ -import java.util.{UUID, PriorityQueue, Comparator} - -import java.util.concurrent.{Executors, ExecutorService} - -import scala.actors.Actor -import scala.actors.Actor._ - -import scala.collection.mutable.Map - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path, RawLocalFileSystem} - -import spark.compress.lzf.{LZFInputStream, LZFOutputStream} - -@serializable -trait BroadcastRecipe { - val uuid = UUID.randomUUID - - // We cannot have an abstract readObject here due to some weird issues with - // readObject having to be 'private' in sub-classes. Possibly a Scala bug! - def sendBroadcast: Unit - - override def toString = "spark.Broadcast(" + uuid + ")" -} - -// TODO: Right, now no parallelization between multiple broadcasts -@serializable -class ChainedStreamingBroadcast[T] (@transient var value_ : T, local: Boolean) -extends BroadcastRecipe with Logging { - - def value = value_ - - BroadcastCS.synchronized { BroadcastCS.values.put (uuid, value_) } - - if (!local) { sendBroadcast } - - def sendBroadcast () { - // Create a variableInfo object and store it in valueInfos - var variableInfo = blockifyObject (value_, BroadcastCS.blockSize) - // TODO: Even though this part is not in use now, there is problem in the - // following statement. Shouldn't use constant port and hostAddress anymore? - // val masterSource = - // new SourceInfo (BroadcastCS.masterHostAddress, BroadcastCS.masterListenPort, - // variableInfo.totalBlocks, variableInfo.totalBytes, 0) - // variableInfo.pqOfSources.add (masterSource) - - BroadcastCS.synchronized { - // BroadcastCS.valueInfos.put (uuid, variableInfo) - - // TODO: Not using variableInfo in current implementation. Manually - // setting all the variables inside BroadcastCS object - - BroadcastCS.initializeVariable (variableInfo) - } - - // Now store a persistent copy in HDFS, just in case - val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) - out.writeObject (value_) - out.close - } - - // Called by Java when deserializing an object - private def readObject (in: ObjectInputStream) { - in.defaultReadObject - BroadcastCS.synchronized { - val cachedVal = BroadcastCS.values.get (uuid) - if (cachedVal != null) { - value_ = cachedVal.asInstanceOf[T] - } else { - // Only a single worker (the first one) in the same node can ever be - // here. The rest will always get the value ready. - val start = System.nanoTime - - val retByteArray = BroadcastCS.receiveBroadcast (uuid) - // If does not succeed, then get from HDFS copy - if (retByteArray != null) { - value_ = byteArrayToObject[T] (retByteArray) - BroadcastCS.values.put (uuid, value_) - // val variableInfo = blockifyObject (value_, BroadcastCS.blockSize) - // BroadcastCS.valueInfos.put (uuid, variableInfo) - } else { - val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) - value_ = fileIn.readObject.asInstanceOf[T] - BroadcastCH.values.put(uuid, value_) - fileIn.close - } - - val time = (System.nanoTime - start) / 1e9 - logInfo("Reading Broadcasted variable " + uuid + " took " + time + " s") - } - } - } - - private def blockifyObject (obj: T, blockSize: Int): VariableInfo = { - val baos = new ByteArrayOutputStream - val oos = new ObjectOutputStream (baos) - oos.writeObject (obj) - oos.close - baos.close - val byteArray = baos.toByteArray - val bais = new ByteArrayInputStream (byteArray) - - var blockNum = (byteArray.length / blockSize) - if (byteArray.length % blockSize != 0) - blockNum += 1 - - var retVal = new Array[BroadcastBlock] (blockNum) - var blockID = 0 - - // TODO: What happens in byteArray.length == 0 => blockNum == 0 - for (i <- 0 until (byteArray.length, blockSize)) { - val thisBlockSize = Math.min (blockSize, byteArray.length - i) - var tempByteArray = new Array[Byte] (thisBlockSize) - val hasRead = bais.read (tempByteArray, 0, thisBlockSize) - - retVal (blockID) = new BroadcastBlock (blockID, tempByteArray) - blockID += 1 - } - bais.close - - var variableInfo = VariableInfo (retVal, blockNum, byteArray.length) - variableInfo.hasBlocks = blockNum - - return variableInfo - } - - private def byteArrayToObject[A] (bytes: Array[Byte]): A = { - val in = new ObjectInputStream (new ByteArrayInputStream (bytes)) - val retVal = in.readObject.asInstanceOf[A] - in.close - return retVal - } - - private def getByteArrayOutputStream (obj: T): ByteArrayOutputStream = { - val bOut = new ByteArrayOutputStream - val out = new ObjectOutputStream (bOut) - out.writeObject (obj) - out.close - bOut.close - return bOut - } -} - -@serializable -class CentralizedHDFSBroadcast[T](@transient var value_ : T, local: Boolean) -extends BroadcastRecipe with Logging { - - def value = value_ - - BroadcastCH.synchronized { BroadcastCH.values.put(uuid, value_) } - - if (!local) { sendBroadcast } - - def sendBroadcast () { - val out = new ObjectOutputStream (BroadcastCH.openFileForWriting(uuid)) - out.writeObject (value_) - out.close - } - - // Called by Java when deserializing an object - private def readObject(in: ObjectInputStream) { - in.defaultReadObject - BroadcastCH.synchronized { - val cachedVal = BroadcastCH.values.get(uuid) - if (cachedVal != null) { - value_ = cachedVal.asInstanceOf[T] - } else { - val start = System.nanoTime - - val fileIn = new ObjectInputStream(BroadcastCH.openFileForReading(uuid)) - value_ = fileIn.readObject.asInstanceOf[T] - BroadcastCH.values.put(uuid, value_) - fileIn.close - - val time = (System.nanoTime - start) / 1e9 - logInfo("Reading Broadcasted variable " + uuid + " took " + time + " s") - } - } - } -} - -@serializable -case class SourceInfo (val hostAddress: String, val listenPort: Int, - val totalBlocks: Int, val totalBytes: Int, val replicaID: Int) -extends Comparable[SourceInfo]{ - - var currentLeechers = 0 - var receptionFailed = false - - def compareTo (o: SourceInfo): Int = (currentLeechers - o.currentLeechers) -} - -@serializable -case class BroadcastBlock (val blockID: Int, val byteArray: Array[Byte]) { } - -@serializable -case class VariableInfo (@transient val arrayOfBlocks : Array[BroadcastBlock], - val totalBlocks: Int, val totalBytes: Int) { - - @transient var hasBlocks = 0 - - val listenPortLock = new AnyRef - val totalBlocksLock = new AnyRef - val hasBlocksLock = new AnyRef - - @transient var pqOfSources = new PriorityQueue[SourceInfo] -} - -private object Broadcast { - private var initialized = false - - // Will be called by SparkContext or Executor before using Broadcast - // Calls all other initializers here - def initialize (isMaster: Boolean) { - synchronized { - if (!initialized) { - // Initialization for CentralizedHDFSBroadcast - BroadcastCH.initialize - // Initialization for ChainedStreamingBroadcast - // BroadcastCS.initialize (isMaster) - - initialized = true - } - } - } -} - -private object BroadcastCS extends Logging { - val values = Cache.newKeySpace() - - // private var valueToPort = Map[UUID, Int] () - - private var initialized = false - private var isMaster_ = false - - private var masterHostAddress_ = "127.0.0.1" - private var masterListenPort_ : Int = 11111 - private var blockSize_ : Int = 512 * 1024 - private var maxRetryCount_ : Int = 2 - private var serverSocketTimout_ : Int = 50000 - private var dualMode_ : Boolean = false - - private val hostAddress = InetAddress.getLocalHost.getHostAddress - private var listenPort = -1 - - var arrayOfBlocks: Array[BroadcastBlock] = null - var totalBytes = -1 - var totalBlocks = -1 - var hasBlocks = 0 - - val listenPortLock = new Object - val totalBlocksLock = new Object - val hasBlocksLock = new Object - - var pqOfSources = new PriorityQueue[SourceInfo] - - private var serveMR: ServeMultipleRequests = null - private var guideMR: GuideMultipleRequests = null - - def initialize (isMaster__ : Boolean) { - synchronized { - if (!initialized) { - masterHostAddress_ = - System.getProperty ("spark.broadcast.masterHostAddress", "127.0.0.1") - masterListenPort_ = - System.getProperty ("spark.broadcast.masterListenPort", "11111").toInt - blockSize_ = - System.getProperty ("spark.broadcast.blockSize", "512").toInt * 1024 - maxRetryCount_ = - System.getProperty ("spark.broadcast.maxRetryCount", "2").toInt - serverSocketTimout_ = - System.getProperty ("spark.broadcast.serverSocketTimout", "50000").toInt - dualMode_ = - System.getProperty ("spark.broadcast.dualMode", "false").toBoolean - - isMaster_ = isMaster__ - - if (isMaster) { - guideMR = new GuideMultipleRequests - guideMR.setDaemon (true) - guideMR.start - logInfo("GuideMultipleRequests started") - } - - serveMR = new ServeMultipleRequests - serveMR.setDaemon (true) - serveMR.start - logInfo("ServeMultipleRequests started") - - logInfo("BroadcastCS object has been initialized") - - initialized = true - } - } - } - - // TODO: This should change in future implementation. - // Called from the Master constructor to setup states for this particular that - // is being broadcasted - def initializeVariable (variableInfo: VariableInfo) { - arrayOfBlocks = variableInfo.arrayOfBlocks - totalBytes = variableInfo.totalBytes - totalBlocks = variableInfo.totalBlocks - hasBlocks = variableInfo.totalBlocks - - // listenPort should already be valid - assert (listenPort != -1) - - pqOfSources = new PriorityQueue[SourceInfo] - val masterSource_0 = - new SourceInfo (hostAddress, listenPort, totalBlocks, totalBytes, 0) - BroadcastCS.pqOfSources.add (masterSource_0) - // Add one more time to have two replicas of any seeds in the PQ - if (BroadcastCS.dualMode) { - val masterSource_1 = - new SourceInfo (hostAddress, listenPort, totalBlocks, totalBytes, 1) - BroadcastCS.pqOfSources.add (masterSource_1) - } - } - - def masterHostAddress = masterHostAddress_ - def masterListenPort = masterListenPort_ - def blockSize = blockSize_ - def maxRetryCount = maxRetryCount_ - def serverSocketTimout = serverSocketTimout_ - def dualMode = dualMode_ - - def isMaster = isMaster_ - - def receiveBroadcast (variableUUID: UUID): Array[Byte] = { - // Wait until hostAddress and listenPort are created by the - // ServeMultipleRequests thread - // NO need to wait; ServeMultipleRequests is created much further ahead - while (listenPort == -1) { - listenPortLock.synchronized { - listenPortLock.wait - } - } - - // Connect and receive broadcast from the specified source, retrying the - // specified number of times in case of failures - var retriesLeft = BroadcastCS.maxRetryCount - var retByteArray: Array[Byte] = null - do { - // Connect to Master and send this worker's Information - val clientSocketToMaster = - new Socket(BroadcastCS.masterHostAddress, BroadcastCS.masterListenPort) - logInfo("Connected to Master's guiding object") - // TODO: Guiding object connection is reusable - val oisMaster = - new ObjectInputStream (clientSocketToMaster.getInputStream) - val oosMaster = - new ObjectOutputStream (clientSocketToMaster.getOutputStream) - - oosMaster.writeObject(new SourceInfo (hostAddress, listenPort, -1, -1, 0)) - oosMaster.flush - - // Receive source information from Master - var sourceInfo = oisMaster.readObject.asInstanceOf[SourceInfo] - totalBlocks = sourceInfo.totalBlocks - arrayOfBlocks = new Array[BroadcastBlock] (totalBlocks) - totalBlocksLock.synchronized { - totalBlocksLock.notifyAll - } - totalBytes = sourceInfo.totalBytes - logInfo("Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort) - - retByteArray = receiveSingleTransmission (sourceInfo) - - logInfo("I got this from receiveSingleTransmission: " + retByteArray) - - // TODO: Update sourceInfo to add error notifactions for Master - if (retByteArray == null) { sourceInfo.receptionFailed = true } - - // TODO: Supposed to update values here, but we don't support advanced - // statistics right now. Master can handle leecherCount by itself. - - // Send back statistics to the Master - oosMaster.writeObject (sourceInfo) - - oisMaster.close - oosMaster.close - clientSocketToMaster.close - - retriesLeft -= 1 - } while (retriesLeft > 0 && retByteArray == null) - - return retByteArray - } - - // Tries to receive broadcast from the Master and returns Boolean status. - // This might be called multiple times to retry a defined number of times. - private def receiveSingleTransmission(sourceInfo: SourceInfo): Array[Byte] = { - var clientSocketToSource: Socket = null - var oisSource: ObjectInputStream = null - var oosSource: ObjectOutputStream = null - - var retByteArray:Array[Byte] = null - - try { - // Connect to the source to get the object itself - clientSocketToSource = - new Socket (sourceInfo.hostAddress, sourceInfo.listenPort) - oosSource = - new ObjectOutputStream (clientSocketToSource.getOutputStream) - oisSource = - new ObjectInputStream (clientSocketToSource.getInputStream) - - logInfo("Inside receiveSingleTransmission") - logInfo("totalBlocks: " + totalBlocks + " " + "hasBlocks: " + hasBlocks) - retByteArray = new Array[Byte] (totalBytes) - for (i <- 0 until totalBlocks) { - val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock] - System.arraycopy (bcBlock.byteArray, 0, retByteArray, - i * BroadcastCS.blockSize, bcBlock.byteArray.length) - arrayOfBlocks(hasBlocks) = bcBlock - hasBlocks += 1 - hasBlocksLock.synchronized { - hasBlocksLock.notifyAll - } - logInfo("Received block: " + i + " " + bcBlock) - } - assert (hasBlocks == totalBlocks) - logInfo("After the receive loop") - } catch { - case e: Exception => { - retByteArray = null - logInfo("receiveSingleTransmission had a " + e) - } - } finally { - if (oisSource != null) { oisSource.close } - if (oosSource != null) { oosSource.close } - if (clientSocketToSource != null) { clientSocketToSource.close } - } - - return retByteArray - } - -// class TrackMultipleValues extends Thread with Logging { -// override def run = { -// var threadPool = Executors.newCachedThreadPool -// var serverSocket: ServerSocket = null -// -// serverSocket = new ServerSocket (BroadcastCS.masterListenPort) -// logInfo("TrackMultipleVariables" + serverSocket + " " + listenPort) -// -// var keepAccepting = true -// try { -// while (keepAccepting) { -// var clientSocket: Socket = null -// try { -// serverSocket.setSoTimeout (serverSocketTimout) -// clientSocket = serverSocket.accept -// } catch { -// case e: Exception => { -// logInfo("TrackMultipleValues Timeout. Stopping listening...") -// keepAccepting = false -// } -// } -// logInfo("TrackMultipleValues:Got new request:" + clientSocket) -// if (clientSocket != null) { -// try { -// threadPool.execute (new Runnable { -// def run = { -// val oos = new ObjectOutputStream (clientSocket.getOutputStream) -// val ois = new ObjectInputStream (clientSocket.getInputStream) -// try { -// val variableUUID = ois.readObject.asInstanceOf[UUID] -// var contactPort = 0 -// // TODO: Add logic and data structures to find out UUID->port -// // mapping. 0 = missed the broadcast, read from HDFS; <0 = -// // Haven't started yet, wait & retry; >0 = Read from this port -// oos.writeObject (contactPort) -// } catch { -// case e: Exception => { } -// } finally { -// ois.close -// oos.close -// clientSocket.close -// } -// } -// }) -// } catch { -// // In failure, close the socket here; else, the thread will close it -// case ioe: IOException => clientSocket.close -// } -// } -// } -// } finally { -// serverSocket.close -// } -// } -// } -// -// class TrackSingleValue { -// -// } - -// public static ExecutorService newCachedThreadPool() { -// return new ThreadPoolExecutor(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, -// new SynchronousQueue()); -// } - - - class GuideMultipleRequests extends Thread with Logging { - override def run = { - var threadPool = Executors.newCachedThreadPool - var serverSocket: ServerSocket = null - - serverSocket = new ServerSocket (BroadcastCS.masterListenPort) - // listenPort = BroadcastCS.masterListenPort - logInfo("GuideMultipleRequests" + serverSocket + " " + listenPort) - - var keepAccepting = true - try { - while (keepAccepting) { - var clientSocket: Socket = null - try { - serverSocket.setSoTimeout (serverSocketTimout) - clientSocket = serverSocket.accept - } catch { - case e: Exception => { - logInfo("GuideMultipleRequests Timeout. Stopping listening...") - keepAccepting = false - } - } - if (clientSocket != null) { - logInfo("Guide:Accepted new client connection:" + clientSocket) - try { - threadPool.execute (new GuideSingleRequest (clientSocket)) - } catch { - // In failure, close the socket here; else, the thread will close it - case ioe: IOException => clientSocket.close - } - } - } - } finally { - serverSocket.close - } - } - - class GuideSingleRequest (val clientSocket: Socket) - extends Runnable with Logging { - private val oos = new ObjectOutputStream (clientSocket.getOutputStream) - private val ois = new ObjectInputStream (clientSocket.getInputStream) - - private var selectedSourceInfo: SourceInfo = null - private var thisWorkerInfo:SourceInfo = null - - def run = { - try { - logInfo("new GuideSingleRequest is running") - // Connecting worker is sending in its hostAddress and listenPort it will - // be listening to. ReplicaID is 0 and other fields are invalid (-1) - var sourceInfo = ois.readObject.asInstanceOf[SourceInfo] - - // Select a suitable source and send it back to the worker - selectedSourceInfo = selectSuitableSource (sourceInfo) - logInfo("Sending selectedSourceInfo:" + selectedSourceInfo) - oos.writeObject (selectedSourceInfo) - oos.flush - - // Add this new (if it can finish) source to the PQ of sources - thisWorkerInfo = new SourceInfo(sourceInfo.hostAddress, - sourceInfo.listenPort, totalBlocks, totalBytes, 0) - logInfo("Adding possible new source to pqOfSources: " + thisWorkerInfo) - pqOfSources.synchronized { - pqOfSources.add (thisWorkerInfo) - } - - // Wait till the whole transfer is done. Then receive and update source - // statistics in pqOfSources - sourceInfo = ois.readObject.asInstanceOf[SourceInfo] - - pqOfSources.synchronized { - // This should work since SourceInfo is a case class - assert (pqOfSources.contains (selectedSourceInfo)) - - // Remove first - pqOfSources.remove (selectedSourceInfo) - // TODO: Removing a source based on just one failure notification! - // Update leecher count and put it back in IF reception succeeded - if (!sourceInfo.receptionFailed) { - selectedSourceInfo.currentLeechers -= 1 - pqOfSources.add (selectedSourceInfo) - - // No need to find and update thisWorkerInfo, but add its replica - if (BroadcastCS.dualMode) { - pqOfSources.add (new SourceInfo (thisWorkerInfo.hostAddress, - thisWorkerInfo.listenPort, totalBlocks, totalBytes, 1)) - } - } - } - } catch { - // If something went wrong, e.g., the worker at the other end died etc. - // then close everything up - case e: Exception => { - // Assuming that exception caused due to receiver worker failure - // Remove failed worker from pqOfSources and update leecherCount of - // corresponding source worker - pqOfSources.synchronized { - if (selectedSourceInfo != null) { - // Remove first - pqOfSources.remove (selectedSourceInfo) - // Update leecher count and put it back in - selectedSourceInfo.currentLeechers -= 1 - pqOfSources.add (selectedSourceInfo) - } - - // Remove thisWorkerInfo - if (pqOfSources != null) { pqOfSources.remove (thisWorkerInfo) } - } - } - } finally { - ois.close - oos.close - clientSocket.close - } - } - - // TODO: If a worker fails to get the broadcasted variable from a source and - // comes back to Master, this function might choose the worker itself as a - // source tp create a dependency cycle (this worker was put into pqOfSources - // as a streming source when it first arrived). The length of this cycle can - // be arbitrarily long. - private def selectSuitableSource(skipSourceInfo: SourceInfo): SourceInfo = { - // Select one with the lowest number of leechers - pqOfSources.synchronized { - // take is a blocking call removing the element from PQ - var selectedSource = pqOfSources.poll - assert (selectedSource != null) - // Update leecher count - selectedSource.currentLeechers += 1 - // Add it back and then return - pqOfSources.add (selectedSource) - return selectedSource - } - } - } - } - - class ServeMultipleRequests extends Thread with Logging { - override def run = { - var threadPool = Executors.newCachedThreadPool - var serverSocket: ServerSocket = null - - serverSocket = new ServerSocket (0) - listenPort = serverSocket.getLocalPort - logInfo("ServeMultipleRequests" + serverSocket + " " + listenPort) - - listenPortLock.synchronized { - listenPortLock.notifyAll - } - - var keepAccepting = true - try { - while (keepAccepting) { - var clientSocket: Socket = null - try { - serverSocket.setSoTimeout (serverSocketTimout) - clientSocket = serverSocket.accept - } catch { - case e: Exception => { - logInfo("ServeMultipleRequests Timeout. Stopping listening...") - keepAccepting = false - } - } - if (clientSocket != null) { - logInfo("Serve:Accepted new client connection:" + clientSocket) - try { - threadPool.execute (new ServeSingleRequest (clientSocket)) - } catch { - // In failure, close socket here; else, the thread will close it - case ioe: IOException => clientSocket.close - } - } - } - } finally { - serverSocket.close - } - } - - class ServeSingleRequest (val clientSocket: Socket) - extends Runnable with Logging { - private val oos = new ObjectOutputStream (clientSocket.getOutputStream) - private val ois = new ObjectInputStream (clientSocket.getInputStream) - - def run = { - try { - logInfo("new ServeSingleRequest is running") - sendObject - } catch { - // TODO: Need to add better exception handling here - // If something went wrong, e.g., the worker at the other end died etc. - // then close everything up - case e: Exception => { - logInfo("ServeSingleRequest had a " + e) - } - } finally { - logInfo("ServeSingleRequest is closing streams and sockets") - ois.close - oos.close - clientSocket.close - } - } - - private def sendObject = { - // Wait till receiving the SourceInfo from Master - while (totalBlocks == -1) { - totalBlocksLock.synchronized { - totalBlocksLock.wait - } - } - - for (i <- 0 until totalBlocks) { - while (i == hasBlocks) { - hasBlocksLock.synchronized { - hasBlocksLock.wait - } - } - try { - oos.writeObject (arrayOfBlocks(i)) - oos.flush - } catch { - case e: Exception => { } - } - logInfo("Send block: " + i + " " + arrayOfBlocks(i)) - } - } - } - } -} - -private object BroadcastCH extends Logging { - val values = Cache.newKeySpace() - - private var initialized = false - - private var fileSystem: FileSystem = null - private var workDir: String = null - private var compress: Boolean = false - private var bufferSize: Int = 65536 - - def initialize () { - synchronized { - if (!initialized) { - bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - val dfs = System.getProperty("spark.dfs", "file:///") - if (!dfs.startsWith("file://")) { - val conf = new Configuration() - conf.setInt("io.file.buffer.size", bufferSize) - val rep = System.getProperty("spark.dfs.replication", "3").toInt - conf.setInt("dfs.replication", rep) - fileSystem = FileSystem.get(new URI(dfs), conf) - } - workDir = System.getProperty("spark.dfs.workdir", "/tmp") - compress = System.getProperty("spark.compress", "false").toBoolean - - initialized = true - } - } - } - - private def getPath(uuid: UUID) = new Path(workDir + "/broadcast-" + uuid) - - def openFileForReading(uuid: UUID): InputStream = { - val fileStream = if (fileSystem != null) { - fileSystem.open(getPath(uuid)) - } else { - // Local filesystem - new FileInputStream(getPath(uuid).toString) - } - if (compress) - new LZFInputStream(fileStream) // LZF stream does its own buffering - else if (fileSystem == null) - new BufferedInputStream(fileStream, bufferSize) - else - fileStream // Hadoop streams do their own buffering - } - - def openFileForWriting(uuid: UUID): OutputStream = { - val fileStream = if (fileSystem != null) { - fileSystem.create(getPath(uuid)) - } else { - // Local filesystem - new FileOutputStream(getPath(uuid).toString) - } - if (compress) - new LZFOutputStream(fileStream) // LZF stream does its own buffering - else if (fileSystem == null) - new BufferedOutputStream(fileStream, bufferSize) - else - fileStream // Hadoop streams do their own buffering - } -} diff --git a/third_party/hadoop-0.20.0/.DS_Store b/third_party/hadoop-0.20.0/.DS_Store deleted file mode 100644 index 81f4e05e09..0000000000 Binary files a/third_party/hadoop-0.20.0/.DS_Store and /dev/null differ diff --git a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/Pipes.hh b/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/Pipes.hh deleted file mode 100644 index 9a785d966a..0000000000 --- a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/Pipes.hh +++ /dev/null @@ -1,258 +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. - */ -#ifndef HADOOP_PIPES_HH -#define HADOOP_PIPES_HH - -#ifdef SWIG -%module (directors="1") HadoopPipes -%include "std_string.i" -%feature("director") Mapper; -%feature("director") Reducer; -%feature("director") Partitioner; -%feature("director") RecordReader; -%feature("director") RecordWriter; -%feature("director") Factory; -#else -#include -#endif - -namespace HadoopPipes { - -/** - * This interface defines the interface between application code and the - * foreign code interface to Hadoop Map/Reduce. - */ - -/** - * A JobConf defines the properties for a job. - */ -class JobConf { -public: - virtual bool hasKey(const std::string& key) const = 0; - virtual const std::string& get(const std::string& key) const = 0; - virtual int getInt(const std::string& key) const = 0; - virtual float getFloat(const std::string& key) const = 0; - virtual bool getBoolean(const std::string&key) const = 0; - virtual ~JobConf() {} -}; - -/** - * Task context provides the information about the task and job. - */ -class TaskContext { -public: - /** - * Counter to keep track of a property and its value. - */ - class Counter { - private: - int id; - public: - Counter(int counterId) : id(counterId) {} - Counter(const Counter& counter) : id(counter.id) {} - - int getId() const { return id; } - }; - - /** - * Get the JobConf for the current task. - */ - virtual const JobConf* getJobConf() = 0; - - /** - * Get the current key. - * @return the current key - */ - virtual const std::string& getInputKey() = 0; - - /** - * Get the current value. - * @return the current value - */ - virtual const std::string& getInputValue() = 0; - - /** - * Generate an output record - */ - virtual void emit(const std::string& key, const std::string& value) = 0; - - /** - * Mark your task as having made progress without changing the status - * message. - */ - virtual void progress() = 0; - - /** - * Set the status message and call progress. - */ - virtual void setStatus(const std::string& status) = 0; - - /** - * Register a counter with the given group and name. - */ - virtual Counter* - getCounter(const std::string& group, const std::string& name) = 0; - - /** - * Increment the value of the counter with the given amount. - */ - virtual void incrementCounter(const Counter* counter, uint64_t amount) = 0; - - virtual ~TaskContext() {} -}; - -class MapContext: public TaskContext { -public: - - /** - * Access the InputSplit of the mapper. - */ - virtual const std::string& getInputSplit() = 0; - - /** - * Get the name of the key class of the input to this task. - */ - virtual const std::string& getInputKeyClass() = 0; - - /** - * Get the name of the value class of the input to this task. - */ - virtual const std::string& getInputValueClass() = 0; - -}; - -class ReduceContext: public TaskContext { -public: - /** - * Advance to the next value. - */ - virtual bool nextValue() = 0; -}; - -class Closable { -public: - virtual void close() {} - virtual ~Closable() {} -}; - -/** - * The application's mapper class to do map. - */ -class Mapper: public Closable { -public: - virtual void map(MapContext& context) = 0; -}; - -/** - * The application's reducer class to do reduce. - */ -class Reducer: public Closable { -public: - virtual void reduce(ReduceContext& context) = 0; -}; - -/** - * User code to decide where each key should be sent. - */ -class Partitioner { -public: - virtual int partition(const std::string& key, int numOfReduces) = 0; - virtual ~Partitioner() {} -}; - -/** - * For applications that want to read the input directly for the map function - * they can define RecordReaders in C++. - */ -class RecordReader: public Closable { -public: - virtual bool next(std::string& key, std::string& value) = 0; - - /** - * The progress of the record reader through the split as a value between - * 0.0 and 1.0. - */ - virtual float getProgress() = 0; -}; - -/** - * An object to write key/value pairs as they are emited from the reduce. - */ -class RecordWriter: public Closable { -public: - virtual void emit(const std::string& key, - const std::string& value) = 0; -}; - -/** - * A factory to create the necessary application objects. - */ -class Factory { -public: - virtual Mapper* createMapper(MapContext& context) const = 0; - virtual Reducer* createReducer(ReduceContext& context) const = 0; - - /** - * Create a combiner, if this application has one. - * @return the new combiner or NULL, if one is not needed - */ - virtual Reducer* createCombiner(MapContext& context) const { - return NULL; - } - - /** - * Create an application partitioner object. - * @return the new partitioner or NULL, if the default partitioner should be - * used. - */ - virtual Partitioner* createPartitioner(MapContext& context) const { - return NULL; - } - - /** - * Create an application record reader. - * @return the new RecordReader or NULL, if the Java RecordReader should be - * used. - */ - virtual RecordReader* createRecordReader(MapContext& context) const { - return NULL; - } - - /** - * Create an application record writer. - * @return the new RecordWriter or NULL, if the Java RecordWriter should be - * used. - */ - virtual RecordWriter* createRecordWriter(ReduceContext& context) const { - return NULL; - } - - virtual ~Factory() {} -}; - -/** - * Run the assigned task in the framework. - * The user's main function should set the various functions using the - * set* functions above and then call this. - * @return true, if the task succeeded. - */ -bool runTask(const Factory& factory); - -} - -#endif diff --git a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/SerialUtils.hh b/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/SerialUtils.hh deleted file mode 100644 index 16cbab65b2..0000000000 --- a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/SerialUtils.hh +++ /dev/null @@ -1,169 +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. - */ -#ifndef HADOOP_SERIAL_UTILS_HH -#define HADOOP_SERIAL_UTILS_HH - -#include - -namespace HadoopUtils { - - /** - * A simple exception class that records a message for the user. - */ - class Error { - private: - std::string error; - public: - - /** - * Create an error object with the given message. - */ - Error(const std::string& msg); - - /** - * Construct an error object with the given message that was created on - * the given file, line, and functino. - */ - Error(const std::string& msg, - const std::string& file, int line, const std::string& function); - - /** - * Get the error message. - */ - const std::string& getMessage() const; - }; - - /** - * Check to make sure that the condition is true, and throw an exception - * if it is not. The exception will contain the message and a description - * of the source location. - */ - #define HADOOP_ASSERT(CONDITION, MESSAGE) \ - { \ - if (!(CONDITION)) { \ - throw HadoopUtils::Error((MESSAGE), __FILE__, __LINE__, \ - __PRETTY_FUNCTION__); \ - } \ - } - - /** - * An interface for an input stream. - */ - class InStream { - public: - /** - * Reads len bytes from the stream into the buffer. - * @param buf the buffer to read into - * @param buflen the length of the buffer - * @throws Error if there are problems reading - */ - virtual void read(void *buf, size_t len) = 0; - virtual ~InStream() {} - }; - - /** - * An interface for an output stream. - */ - class OutStream { - public: - /** - * Write the given buffer to the stream. - * @param buf the data to write - * @param len the number of bytes to write - * @throws Error if there are problems writing - */ - virtual void write(const void *buf, size_t len) = 0; - /** - * Flush the data to the underlying store. - */ - virtual void flush() = 0; - virtual ~OutStream() {} - }; - - /** - * A class to read a file as a stream. - */ - class FileInStream : public InStream { - public: - FileInStream(); - bool open(const std::string& name); - bool open(FILE* file); - void read(void *buf, size_t buflen); - bool skip(size_t nbytes); - bool close(); - virtual ~FileInStream(); - private: - /** - * The file to write to. - */ - FILE *mFile; - /** - * Does is this class responsible for closing the FILE*? - */ - bool isOwned; - }; - - /** - * A class to write a stream to a file. - */ - class FileOutStream: public OutStream { - public: - - /** - * Create a stream that isn't bound to anything. - */ - FileOutStream(); - - /** - * Create the given file, potentially overwriting an existing file. - */ - bool open(const std::string& name, bool overwrite); - bool open(FILE* file); - void write(const void* buf, size_t len); - bool advance(size_t nbytes); - void flush(); - bool close(); - virtual ~FileOutStream(); - private: - FILE *mFile; - bool isOwned; - }; - - /** - * A stream that reads from a string. - */ - class StringInStream: public InStream { - public: - StringInStream(const std::string& str); - virtual void read(void *buf, size_t buflen); - private: - const std::string& buffer; - std::string::const_iterator itr; - }; - - void serializeInt(int32_t t, OutStream& stream); - int32_t deserializeInt(InStream& stream); - void serializeLong(int64_t t, OutStream& stream); - int64_t deserializeLong(InStream& stream); - void serializeFloat(float t, OutStream& stream); - float deserializeFloat(InStream& stream); - void serializeString(const std::string& t, OutStream& stream); - void deserializeString(std::string& t, InStream& stream); -} - -#endif diff --git a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/StringUtils.hh b/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/StringUtils.hh deleted file mode 100644 index 4720172725..0000000000 --- a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/StringUtils.hh +++ /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. - */ -#ifndef HADOOP_STRING_UTILS_HH -#define HADOOP_STRING_UTILS_HH - -#include -#include -#include - -namespace HadoopUtils { - - /** - * Convert an integer to a string. - */ - std::string toString(int32_t x); - - /** - * Convert a string to an integer. - * @throws Error if the string is not a valid integer - */ - int32_t toInt(const std::string& val); - - /** - * Convert the string to a float. - * @throws Error if the string is not a valid float - */ - float toFloat(const std::string& val); - - /** - * Convert the string to a boolean. - * @throws Error if the string is not a valid boolean value - */ - bool toBool(const std::string& val); - - /** - * Get the current time in the number of milliseconds since 1970. - */ - uint64_t getCurrentMillis(); - - /** - * Split a string into "words". Multiple deliminators are treated as a single - * word break, so no zero-length words are returned. - * @param str the string to split - * @param separator a list of characters that divide words - */ - std::vector splitString(const std::string& str, - const char* separator); - - /** - * Quote a string to avoid "\", non-printable characters, and the - * deliminators. - * @param str the string to quote - * @param deliminators the set of characters to always quote - */ - std::string quoteString(const std::string& str, - const char* deliminators); - - /** - * Unquote the given string to return the original string. - * @param str the string to unquote - */ - std::string unquoteString(const std::string& str); - -} - -#endif diff --git a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/TemplateFactory.hh b/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/TemplateFactory.hh deleted file mode 100644 index 22e10ae56f..0000000000 --- a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/include/hadoop/TemplateFactory.hh +++ /dev/null @@ -1,96 +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. - */ -#ifndef HADOOP_PIPES_TEMPLATE_FACTORY_HH -#define HADOOP_PIPES_TEMPLATE_FACTORY_HH - -namespace HadoopPipes { - - template - class TemplateFactory2: public Factory { - public: - Mapper* createMapper(MapContext& context) const { - return new mapper(context); - } - Reducer* createReducer(ReduceContext& context) const { - return new reducer(context); - } - }; - - template - class TemplateFactory3: public TemplateFactory2 { - public: - Partitioner* createPartitioner(MapContext& context) const { - return new partitioner(context); - } - }; - - template - class TemplateFactory3 - : public TemplateFactory2 { - }; - - template - class TemplateFactory4 - : public TemplateFactory3{ - public: - Reducer* createCombiner(MapContext& context) const { - return new combiner(context); - } - }; - - template - class TemplateFactory4 - : public TemplateFactory3{ - }; - - template - class TemplateFactory5 - : public TemplateFactory4{ - public: - RecordReader* createRecordReader(MapContext& context) const { - return new recordReader(context); - } - }; - - template - class TemplateFactory5 - : public TemplateFactory4{ - }; - - template - class TemplateFactory - : public TemplateFactory5{ - public: - RecordWriter* createRecordWriter(ReduceContext& context) const { - return new recordWriter(context); - } - }; - - template - class TemplateFactory - : public TemplateFactory5{ - }; - -} - -#endif diff --git a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/lib/libhadooppipes.a b/third_party/hadoop-0.20.0/c++/Linux-amd64-64/lib/libhadooppipes.a deleted file mode 100644 index be303140cb..0000000000 Binary files a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/lib/libhadooppipes.a and /dev/null differ diff --git a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/lib/libhadooputils.a b/third_party/hadoop-0.20.0/c++/Linux-amd64-64/lib/libhadooputils.a deleted file mode 100644 index 8a0aded98e..0000000000 Binary files a/third_party/hadoop-0.20.0/c++/Linux-amd64-64/lib/libhadooputils.a and /dev/null differ diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/Pipes.hh b/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/Pipes.hh deleted file mode 100644 index 9a785d966a..0000000000 --- a/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/Pipes.hh +++ /dev/null @@ -1,258 +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. - */ -#ifndef HADOOP_PIPES_HH -#define HADOOP_PIPES_HH - -#ifdef SWIG -%module (directors="1") HadoopPipes -%include "std_string.i" -%feature("director") Mapper; -%feature("director") Reducer; -%feature("director") Partitioner; -%feature("director") RecordReader; -%feature("director") RecordWriter; -%feature("director") Factory; -#else -#include -#endif - -namespace HadoopPipes { - -/** - * This interface defines the interface between application code and the - * foreign code interface to Hadoop Map/Reduce. - */ - -/** - * A JobConf defines the properties for a job. - */ -class JobConf { -public: - virtual bool hasKey(const std::string& key) const = 0; - virtual const std::string& get(const std::string& key) const = 0; - virtual int getInt(const std::string& key) const = 0; - virtual float getFloat(const std::string& key) const = 0; - virtual bool getBoolean(const std::string&key) const = 0; - virtual ~JobConf() {} -}; - -/** - * Task context provides the information about the task and job. - */ -class TaskContext { -public: - /** - * Counter to keep track of a property and its value. - */ - class Counter { - private: - int id; - public: - Counter(int counterId) : id(counterId) {} - Counter(const Counter& counter) : id(counter.id) {} - - int getId() const { return id; } - }; - - /** - * Get the JobConf for the current task. - */ - virtual const JobConf* getJobConf() = 0; - - /** - * Get the current key. - * @return the current key - */ - virtual const std::string& getInputKey() = 0; - - /** - * Get the current value. - * @return the current value - */ - virtual const std::string& getInputValue() = 0; - - /** - * Generate an output record - */ - virtual void emit(const std::string& key, const std::string& value) = 0; - - /** - * Mark your task as having made progress without changing the status - * message. - */ - virtual void progress() = 0; - - /** - * Set the status message and call progress. - */ - virtual void setStatus(const std::string& status) = 0; - - /** - * Register a counter with the given group and name. - */ - virtual Counter* - getCounter(const std::string& group, const std::string& name) = 0; - - /** - * Increment the value of the counter with the given amount. - */ - virtual void incrementCounter(const Counter* counter, uint64_t amount) = 0; - - virtual ~TaskContext() {} -}; - -class MapContext: public TaskContext { -public: - - /** - * Access the InputSplit of the mapper. - */ - virtual const std::string& getInputSplit() = 0; - - /** - * Get the name of the key class of the input to this task. - */ - virtual const std::string& getInputKeyClass() = 0; - - /** - * Get the name of the value class of the input to this task. - */ - virtual const std::string& getInputValueClass() = 0; - -}; - -class ReduceContext: public TaskContext { -public: - /** - * Advance to the next value. - */ - virtual bool nextValue() = 0; -}; - -class Closable { -public: - virtual void close() {} - virtual ~Closable() {} -}; - -/** - * The application's mapper class to do map. - */ -class Mapper: public Closable { -public: - virtual void map(MapContext& context) = 0; -}; - -/** - * The application's reducer class to do reduce. - */ -class Reducer: public Closable { -public: - virtual void reduce(ReduceContext& context) = 0; -}; - -/** - * User code to decide where each key should be sent. - */ -class Partitioner { -public: - virtual int partition(const std::string& key, int numOfReduces) = 0; - virtual ~Partitioner() {} -}; - -/** - * For applications that want to read the input directly for the map function - * they can define RecordReaders in C++. - */ -class RecordReader: public Closable { -public: - virtual bool next(std::string& key, std::string& value) = 0; - - /** - * The progress of the record reader through the split as a value between - * 0.0 and 1.0. - */ - virtual float getProgress() = 0; -}; - -/** - * An object to write key/value pairs as they are emited from the reduce. - */ -class RecordWriter: public Closable { -public: - virtual void emit(const std::string& key, - const std::string& value) = 0; -}; - -/** - * A factory to create the necessary application objects. - */ -class Factory { -public: - virtual Mapper* createMapper(MapContext& context) const = 0; - virtual Reducer* createReducer(ReduceContext& context) const = 0; - - /** - * Create a combiner, if this application has one. - * @return the new combiner or NULL, if one is not needed - */ - virtual Reducer* createCombiner(MapContext& context) const { - return NULL; - } - - /** - * Create an application partitioner object. - * @return the new partitioner or NULL, if the default partitioner should be - * used. - */ - virtual Partitioner* createPartitioner(MapContext& context) const { - return NULL; - } - - /** - * Create an application record reader. - * @return the new RecordReader or NULL, if the Java RecordReader should be - * used. - */ - virtual RecordReader* createRecordReader(MapContext& context) const { - return NULL; - } - - /** - * Create an application record writer. - * @return the new RecordWriter or NULL, if the Java RecordWriter should be - * used. - */ - virtual RecordWriter* createRecordWriter(ReduceContext& context) const { - return NULL; - } - - virtual ~Factory() {} -}; - -/** - * Run the assigned task in the framework. - * The user's main function should set the various functions using the - * set* functions above and then call this. - * @return true, if the task succeeded. - */ -bool runTask(const Factory& factory); - -} - -#endif diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/SerialUtils.hh b/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/SerialUtils.hh deleted file mode 100644 index 16cbab65b2..0000000000 --- a/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/SerialUtils.hh +++ /dev/null @@ -1,169 +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. - */ -#ifndef HADOOP_SERIAL_UTILS_HH -#define HADOOP_SERIAL_UTILS_HH - -#include - -namespace HadoopUtils { - - /** - * A simple exception class that records a message for the user. - */ - class Error { - private: - std::string error; - public: - - /** - * Create an error object with the given message. - */ - Error(const std::string& msg); - - /** - * Construct an error object with the given message that was created on - * the given file, line, and functino. - */ - Error(const std::string& msg, - const std::string& file, int line, const std::string& function); - - /** - * Get the error message. - */ - const std::string& getMessage() const; - }; - - /** - * Check to make sure that the condition is true, and throw an exception - * if it is not. The exception will contain the message and a description - * of the source location. - */ - #define HADOOP_ASSERT(CONDITION, MESSAGE) \ - { \ - if (!(CONDITION)) { \ - throw HadoopUtils::Error((MESSAGE), __FILE__, __LINE__, \ - __PRETTY_FUNCTION__); \ - } \ - } - - /** - * An interface for an input stream. - */ - class InStream { - public: - /** - * Reads len bytes from the stream into the buffer. - * @param buf the buffer to read into - * @param buflen the length of the buffer - * @throws Error if there are problems reading - */ - virtual void read(void *buf, size_t len) = 0; - virtual ~InStream() {} - }; - - /** - * An interface for an output stream. - */ - class OutStream { - public: - /** - * Write the given buffer to the stream. - * @param buf the data to write - * @param len the number of bytes to write - * @throws Error if there are problems writing - */ - virtual void write(const void *buf, size_t len) = 0; - /** - * Flush the data to the underlying store. - */ - virtual void flush() = 0; - virtual ~OutStream() {} - }; - - /** - * A class to read a file as a stream. - */ - class FileInStream : public InStream { - public: - FileInStream(); - bool open(const std::string& name); - bool open(FILE* file); - void read(void *buf, size_t buflen); - bool skip(size_t nbytes); - bool close(); - virtual ~FileInStream(); - private: - /** - * The file to write to. - */ - FILE *mFile; - /** - * Does is this class responsible for closing the FILE*? - */ - bool isOwned; - }; - - /** - * A class to write a stream to a file. - */ - class FileOutStream: public OutStream { - public: - - /** - * Create a stream that isn't bound to anything. - */ - FileOutStream(); - - /** - * Create the given file, potentially overwriting an existing file. - */ - bool open(const std::string& name, bool overwrite); - bool open(FILE* file); - void write(const void* buf, size_t len); - bool advance(size_t nbytes); - void flush(); - bool close(); - virtual ~FileOutStream(); - private: - FILE *mFile; - bool isOwned; - }; - - /** - * A stream that reads from a string. - */ - class StringInStream: public InStream { - public: - StringInStream(const std::string& str); - virtual void read(void *buf, size_t buflen); - private: - const std::string& buffer; - std::string::const_iterator itr; - }; - - void serializeInt(int32_t t, OutStream& stream); - int32_t deserializeInt(InStream& stream); - void serializeLong(int64_t t, OutStream& stream); - int64_t deserializeLong(InStream& stream); - void serializeFloat(float t, OutStream& stream); - float deserializeFloat(InStream& stream); - void serializeString(const std::string& t, OutStream& stream); - void deserializeString(std::string& t, InStream& stream); -} - -#endif diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/StringUtils.hh b/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/StringUtils.hh deleted file mode 100644 index 4720172725..0000000000 --- a/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/StringUtils.hh +++ /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. - */ -#ifndef HADOOP_STRING_UTILS_HH -#define HADOOP_STRING_UTILS_HH - -#include -#include -#include - -namespace HadoopUtils { - - /** - * Convert an integer to a string. - */ - std::string toString(int32_t x); - - /** - * Convert a string to an integer. - * @throws Error if the string is not a valid integer - */ - int32_t toInt(const std::string& val); - - /** - * Convert the string to a float. - * @throws Error if the string is not a valid float - */ - float toFloat(const std::string& val); - - /** - * Convert the string to a boolean. - * @throws Error if the string is not a valid boolean value - */ - bool toBool(const std::string& val); - - /** - * Get the current time in the number of milliseconds since 1970. - */ - uint64_t getCurrentMillis(); - - /** - * Split a string into "words". Multiple deliminators are treated as a single - * word break, so no zero-length words are returned. - * @param str the string to split - * @param separator a list of characters that divide words - */ - std::vector splitString(const std::string& str, - const char* separator); - - /** - * Quote a string to avoid "\", non-printable characters, and the - * deliminators. - * @param str the string to quote - * @param deliminators the set of characters to always quote - */ - std::string quoteString(const std::string& str, - const char* deliminators); - - /** - * Unquote the given string to return the original string. - * @param str the string to unquote - */ - std::string unquoteString(const std::string& str); - -} - -#endif diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/TemplateFactory.hh b/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/TemplateFactory.hh deleted file mode 100644 index 22e10ae56f..0000000000 --- a/third_party/hadoop-0.20.0/c++/Linux-i386-32/include/hadoop/TemplateFactory.hh +++ /dev/null @@ -1,96 +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. - */ -#ifndef HADOOP_PIPES_TEMPLATE_FACTORY_HH -#define HADOOP_PIPES_TEMPLATE_FACTORY_HH - -namespace HadoopPipes { - - template - class TemplateFactory2: public Factory { - public: - Mapper* createMapper(MapContext& context) const { - return new mapper(context); - } - Reducer* createReducer(ReduceContext& context) const { - return new reducer(context); - } - }; - - template - class TemplateFactory3: public TemplateFactory2 { - public: - Partitioner* createPartitioner(MapContext& context) const { - return new partitioner(context); - } - }; - - template - class TemplateFactory3 - : public TemplateFactory2 { - }; - - template - class TemplateFactory4 - : public TemplateFactory3{ - public: - Reducer* createCombiner(MapContext& context) const { - return new combiner(context); - } - }; - - template - class TemplateFactory4 - : public TemplateFactory3{ - }; - - template - class TemplateFactory5 - : public TemplateFactory4{ - public: - RecordReader* createRecordReader(MapContext& context) const { - return new recordReader(context); - } - }; - - template - class TemplateFactory5 - : public TemplateFactory4{ - }; - - template - class TemplateFactory - : public TemplateFactory5{ - public: - RecordWriter* createRecordWriter(ReduceContext& context) const { - return new recordWriter(context); - } - }; - - template - class TemplateFactory - : public TemplateFactory5{ - }; - -} - -#endif diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhadooppipes.a b/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhadooppipes.a deleted file mode 100644 index 73debcec92..0000000000 Binary files a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhadooppipes.a and /dev/null differ diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhadooputils.a b/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhadooputils.a deleted file mode 100644 index 6753169b8a..0000000000 Binary files a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhadooputils.a and /dev/null differ diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.la b/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.la deleted file mode 100644 index b6ce94229f..0000000000 --- a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.la +++ /dev/null @@ -1,41 +0,0 @@ -# libhdfs.la - a libtool library file -# Generated by ltmain.sh (GNU libtool) 2.2 -# -# Please DO NOT delete this file! -# It is necessary for linking the library. - -# The name that we can dlopen(3). -dlname='libhdfs.so.0' - -# Names of this library. -library_names='libhdfs.so.0.0.0 libhdfs.so.0 libhdfs.so' - -# The name of the static archive. -old_library='' - -# Linker flags that can not go in dependency_libs. -inherited_linker_flags='' - -# Libraries that this one depends upon. -dependency_libs=' -L/home/hadoopqa/tools/java/latest1.6-32/jre/lib/i386/server -ljvm -ldl -lpthread' - -# Names of additional weak libraries provided by this library -weak_library_names='' - -# Version information for libhdfs. -current=0 -age=0 -revision=0 - -# Is this an already installed library? -installed=yes - -# Should we warn about portability when linking against -modules? -shouldnotlink=no - -# Files to dlopen/dlpreopen -dlopen='' -dlpreopen='' - -# Directory that this library needs to be installed in: -libdir='/home/ndaley/hadoop/branch-0.20/build/c++/Linux-i386-32/lib' diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.so b/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.so deleted file mode 100644 index 358d582d43..0000000000 Binary files a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.so and /dev/null differ diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.so.0 b/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.so.0 deleted file mode 100644 index 358d582d43..0000000000 Binary files a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.so.0 and /dev/null differ diff --git a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.so.0.0.0 b/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.so.0.0.0 deleted file mode 100644 index 358d582d43..0000000000 Binary files a/third_party/hadoop-0.20.0/c++/Linux-i386-32/lib/libhdfs.so.0.0.0 and /dev/null differ diff --git a/third_party/hadoop-0.20.0/contrib/capacity-scheduler/hadoop-0.20.0-capacity-scheduler.jar b/third_party/hadoop-0.20.0/contrib/capacity-scheduler/hadoop-0.20.0-capacity-scheduler.jar deleted file mode 100644 index b4900e565e..0000000000 Binary files a/third_party/hadoop-0.20.0/contrib/capacity-scheduler/hadoop-0.20.0-capacity-scheduler.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/contrib/datajoin/hadoop-0.20.0-datajoin.jar b/third_party/hadoop-0.20.0/contrib/datajoin/hadoop-0.20.0-datajoin.jar deleted file mode 100644 index 21294d4d1d..0000000000 Binary files a/third_party/hadoop-0.20.0/contrib/datajoin/hadoop-0.20.0-datajoin.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/contrib/eclipse-plugin/hadoop-0.20.0-eclipse-plugin.jar b/third_party/hadoop-0.20.0/contrib/eclipse-plugin/hadoop-0.20.0-eclipse-plugin.jar deleted file mode 100644 index 7b316393f6..0000000000 Binary files a/third_party/hadoop-0.20.0/contrib/eclipse-plugin/hadoop-0.20.0-eclipse-plugin.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/contrib/fairscheduler/hadoop-0.20.0-fairscheduler.jar b/third_party/hadoop-0.20.0/contrib/fairscheduler/hadoop-0.20.0-fairscheduler.jar deleted file mode 100644 index 758b98367c..0000000000 Binary files a/third_party/hadoop-0.20.0/contrib/fairscheduler/hadoop-0.20.0-fairscheduler.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/README b/third_party/hadoop-0.20.0/contrib/hdfsproxy/README deleted file mode 100644 index 2c33988926..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/README +++ /dev/null @@ -1,30 +0,0 @@ -HDFSPROXY is an HTTPS proxy server that exposes the same HSFTP interface as a -real cluster. It authenticates users via user certificates and enforce access -control based on configuration files. - -Starting up an HDFSPROXY server is similar to starting up an HDFS cluster. -Simply run "hdfsproxy" shell command. The main configuration file is -hdfsproxy-default.xml, which should be on the classpath. hdfsproxy-env.sh -can be used to set up environmental variables. In particular, JAVA_HOME should -be set. Additional configuration files include user-certs.xml, -user-permissions.xml and ssl-server.xml, which are used to specify allowed user -certs, allowed directories/files, and ssl keystore information for the proxy, -respectively. The location of these files can be specified in -hdfsproxy-default.xml. Environmental variable HDFSPROXY_CONF_DIR can be used to -point to the directory where these configuration files are located. The -configuration files of the proxied HDFS cluster should also be available on the -classpath (hdfs-default.xml and hdfs-site.xml). - -Mirroring those used in HDFS, a few shell scripts are provided to start and -stop a group of proxy servers. The hosts to run hdfsproxy on are specified in -hdfsproxy-hosts file, one host per line. All hdfsproxy servers are stateless -and run independently from each other. Simple load balancing can be set up by -mapping all hdfsproxy server IP addresses to a single hostname. Users should -use that hostname to access the proxy. If an IP address look up for that -hostname returns more than one IP addresses, an HFTP/HSFTP client will randomly -pick one to use. - -Command "hdfsproxy -reloadPermFiles" can be used to trigger reloading of -user-certs.xml and user-permissions.xml files on all proxy servers listed in -the hdfsproxy-hosts file. Similarly, "hdfsproxy -clearUgiCache" command can be -used to clear the UGI caches on all proxy servers. diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy b/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy deleted file mode 100755 index 1b1e597891..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy +++ /dev/null @@ -1,170 +0,0 @@ -#!/usr/bin/env bash - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -# The HdfsProxy command script -# -# Environment Variables -# -# JAVA_HOME The java implementation to use. Overrides JAVA_HOME. -# -# HDFSPROXY_CLASSPATH Extra Java CLASSPATH entries. -# -# HDFSPROXY_HEAPSIZE The maximum amount of heap to use, in MB. -# Default is 1000. -# -# HDFSPROXY_OPTS Extra Java runtime options. -# -# HDFSPROXY_NAMENODE_OPTS These options are added to HDFSPROXY_OPTS -# HDFSPROXY_CLIENT_OPTS when the respective command is run. -# HDFSPROXY_{COMMAND}_OPTS etc HDFSPROXY_JT_OPTS applies to JobTracker -# for e.g. HDFSPROXY_CLIENT_OPTS applies to -# more than one command (fs, dfs, fsck, -# dfsadmin etc) -# -# HDFSPROXY_CONF_DIR Alternate conf dir. Default is ${HDFSPROXY_HOME}/conf. -# -# HDFSPROXY_ROOT_LOGGER The root appender. Default is INFO,console -# - -bin=`dirname "$0"` -bin=`cd "$bin"; pwd` - -. "$bin"/hdfsproxy-config.sh - -cygwin=false -case "`uname`" in -CYGWIN*) cygwin=true;; -esac - -if [ -f "${HDFSPROXY_CONF_DIR}/hdfsproxy-env.sh" ]; then - . "${HDFSPROXY_CONF_DIR}/hdfsproxy-env.sh" -fi - -# some Java parameters -if [ "$JAVA_HOME" != "" ]; then - #echo "run java in $JAVA_HOME" - JAVA_HOME=$JAVA_HOME -fi - -if [ "$JAVA_HOME" = "" ]; then - echo "Error: JAVA_HOME is not set." - exit 1 -fi - -JAVA=$JAVA_HOME/bin/java -JAVA_HEAP_MAX=-Xmx1000m - -# check envvars which might override default args -if [ "$HDFSPROXY_HEAPSIZE" != "" ]; then - #echo "run with heapsize $HDFSPROXY_HEAPSIZE" - JAVA_HEAP_MAX="-Xmx""$HDFSPROXY_HEAPSIZE""m" - #echo $JAVA_HEAP_MAX -fi - -# CLASSPATH initially contains $HDFSPROXY_CONF_DIR -CLASSPATH="${HDFSPROXY_CONF_DIR}" -CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/tools.jar - -# for developers, add HdfsProxy classes to CLASSPATH -if [ -d "$HDFSPROXY_HOME/build/classes" ]; then - CLASSPATH=${CLASSPATH}:$HDFSPROXY_HOME/build/classes -fi -if [ -d "$HDFSPROXY_HOME/build/webapps" ]; then - CLASSPATH=${CLASSPATH}:$HDFSPROXY_HOME/build -fi -if [ -d "$HDFSPROXY_HOME/build/test/classes" ]; then - CLASSPATH=${CLASSPATH}:$HDFSPROXY_HOME/build/test/classes -fi - -# so that filenames w/ spaces are handled correctly in loops below -IFS= - -# for releases, add hdfsproxy jar & webapps to CLASSPATH -if [ -d "$HDFSPROXY_HOME/webapps" ]; then - CLASSPATH=${CLASSPATH}:$HDFSPROXY_HOME -fi -for f in $HDFSPROXY_HOME/hdfsproxy-*.jar; do - CLASSPATH=${CLASSPATH}:$f; -done - -# add libs to CLASSPATH -if [ -d "$HDFSPROXY_HOME/lib" ]; then - for f in $HDFSPROXY_HOME/lib/*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done -fi - -if [ -d "$HDFSPROXY_HOME/../../" ]; then - for f in $HDFSPROXY_HOME/../../*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done -fi -if [ -d "$HDFSPROXY_HOME/../../lib" ]; then - for f in $HDFSPROXY_HOME/../../lib/*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done -fi -if [ -d "$HDFSPROXY_HOME/../../lib/jsp-2.1" ]; then - for f in $HDFSPROXY_HOME/../../lib/jsp-2.1/*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done -fi - - -# add user-specified CLASSPATH last -if [ "$HDFSPROXY_CLASSPATH" != "" ]; then - CLASSPATH=${CLASSPATH}:${HDFSPROXY_CLASSPATH} -fi - -# default log directory & file -if [ "$HDFSPROXY_LOG_DIR" = "" ]; then - HDFSPROXY_LOG_DIR="$HDFSPROXY_HOME/logs" -fi -if [ "$HDFSPROXY_LOGFILE" = "" ]; then - HDFSPROXY_LOGFILE='hdfsproxy.log' -fi - -# restore ordinary behaviour -unset IFS - -# figure out which class to run -CLASS='org.apache.hadoop.hdfsproxy.HdfsProxy' - -# cygwin path translation -if $cygwin; then - CLASSPATH=`cygpath -p -w "$CLASSPATH"` - HDFSPROXY_HOME=`cygpath -d "$HDFSPROXY_HOME"` - HDFSPROXY_LOG_DIR=`cygpath -d "$HDFSPROXY_LOG_DIR"` -fi - -# cygwin path translation -if $cygwin; then - JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"` -fi - -HDFSPROXY_OPTS="$HDFSPROXY_OPTS -Dhdfsproxy.log.dir=$HDFSPROXY_LOG_DIR" -HDFSPROXY_OPTS="$HDFSPROXY_OPTS -Dhdfsproxy.log.file=$HDFSPROXY_LOGFILE" -HDFSPROXY_OPTS="$HDFSPROXY_OPTS -Dhdfsproxy.home.dir=$HDFSPROXY_HOME" -HDFSPROXY_OPTS="$HDFSPROXY_OPTS -Dhdfsproxy.id.str=$HDFSPROXY_IDENT_STRING" -HDFSPROXY_OPTS="$HDFSPROXY_OPTS -Dhdfsproxy.root.logger=${HDFSPROXY_ROOT_LOGGER:-INFO,console}" -if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then - HDFSPROXY_OPTS="$HDFSPROXY_OPTS -Djava.library.path=$JAVA_LIBRARY_PATH" -fi - -# run it -exec "$JAVA" $JAVA_HEAP_MAX $HDFSPROXY_OPTS -classpath "$CLASSPATH" $CLASS "$@" diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-config.sh b/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-config.sh deleted file mode 100755 index 8fe6aac68b..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-config.sh +++ /dev/null @@ -1,67 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# included in all the hadoop scripts with source command -# should not be executable directly -# also should not be passed any arguments, since we need original $* - -# resolve links - $0 may be a softlink - -this="$0" -while [ -h "$this" ]; do - ls=`ls -ld "$this"` - link=`expr "$ls" : '.*-> \(.*\)$'` - if expr "$link" : '.*/.*' > /dev/null; then - this="$link" - else - this=`dirname "$this"`/"$link" - fi -done - -# convert relative path to absolute path -bin=`dirname "$this"` -script=`basename "$this"` -bin=`cd "$bin"; pwd` -this="$bin/$script" - -# the root of the HdfsProxy installation -export HDFSPROXY_HOME=`dirname "$this"`/.. - -#check to see if the conf dir is given as an optional argument -if [ $# -gt 1 ] -then - if [ "--config" = "$1" ] - then - shift - confdir=$1 - shift - HDFSPROXY_CONF_DIR=$confdir - fi -fi - -# Allow alternate conf dir location. -HDFSPROXY_CONF_DIR="${HDFSPROXY_CONF_DIR:-$HDFSPROXY_HOME/conf}" - -#check to see it is specified whether to use the slaves file -if [ $# -gt 1 ] -then - if [ "--hosts" = "$1" ] - then - shift - slavesfile=$1 - shift - export HDFSPROXY_SLAVES="${HDFSPROXY_CONF_DIR}/$slavesfile" - fi -fi diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-daemon.sh b/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-daemon.sh deleted file mode 100755 index 6d5a75247f..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-daemon.sh +++ /dev/null @@ -1,141 +0,0 @@ -#!/usr/bin/env bash - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -# Runs a HdfsProxy as a daemon. -# -# Environment Variables -# -# HDFSPROXY_CONF_DIR Alternate conf dir. Default is ${HDFSPROXY_HOME}/conf. -# HDFSPROXY_LOG_DIR Where log files are stored. PWD by default. -# HDFSPROXY_MASTER host:path where hdfsproxy code should be rsync'd from -# HDFSPROXY_PID_DIR The pid files are stored. /tmp by default. -# HDFSPROXY_IDENT_STRING A string representing this instance of hdfsproxy. $USER by default -# HDFSPROXY_NICENESS The scheduling priority for daemons. Defaults to 0. -## - -usage="Usage: hdfsproxy-daemon.sh [--config ] [--hosts hostlistfile] (start|stop) " - -# if no args specified, show usage -if [ $# -le 1 ]; then - echo $usage - exit 1 -fi - -bin=`dirname "$0"` -bin=`cd "$bin"; pwd` - -. "$bin"/hdfsproxy-config.sh - -# get arguments -startStop=$1 -shift - -hdfsproxy_rotate_log () -{ - log=$1; - num=5; - if [ -n "$2" ]; then - num=$2 - fi - if [ -f "$log" ]; then # rotate logs - while [ $num -gt 1 ]; do - prev=`expr $num - 1` - [ -f "$log.$prev" ] && mv "$log.$prev" "$log.$num" - num=$prev - done - mv "$log" "$log.$num"; - fi -} - -if [ -f "${HDFSPROXY_CONF_DIR}/hdfsproxy-env.sh" ]; then - . "${HDFSPROXY_CONF_DIR}/hdfsproxy-env.sh" -fi - -# get log directory -if [ "$HDFSPROXY_LOG_DIR" = "" ]; then - export HDFSPROXY_LOG_DIR="$HDFSPROXY_HOME/logs" -fi -mkdir -p "$HDFSPROXY_LOG_DIR" - -if [ "$HDFSPROXY_PID_DIR" = "" ]; then - HDFSPROXY_PID_DIR=/tmp -fi - -if [ "$HDFSPROXY_IDENT_STRING" = "" ]; then - export HDFSPROXY_IDENT_STRING="$USER" -fi - -# some variables -export HDFSPROXY_LOGFILE=hdfsproxy-$HDFSPROXY_IDENT_STRING-$HOSTNAME.log -export HDFSPROXY_ROOT_LOGGER="INFO,DRFA" -log=$HDFSPROXY_LOG_DIR/hdfsproxy-$HDFSPROXY_IDENT_STRING-$HOSTNAME.out -pid=$HDFSPROXY_PID_DIR/hdfsproxy-$HDFSPROXY_IDENT_STRING.pid - -# Set default scheduling priority -if [ "$HDFSPROXY_NICENESS" = "" ]; then - export HDFSPROXY_NICENESS=0 -fi - -case $startStop in - - (start) - - mkdir -p "$HDFSPROXY_PID_DIR" - - if [ -f $pid ]; then - if kill -0 `cat $pid` > /dev/null 2>&1; then - echo hdfsproxy running as process `cat $pid`. Stop it first. - exit 1 - fi - fi - - if [ "$HDFSPROXY_MASTER" != "" ]; then - echo rsync from $HDFSPROXY_MASTER - rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $HDFSPROXY_MASTER/ "$HDFSPROXY_HOME" - fi - - hdfsproxy_rotate_log $log - echo starting hdfsproxy, logging to $log - cd "$HDFSPROXY_HOME" - nohup nice -n $HDFSPROXY_NICENESS "$HDFSPROXY_HOME"/bin/hdfsproxy --config $HDFSPROXY_CONF_DIR "$@" > "$log" 2>&1 < /dev/null & - echo $! > $pid - sleep 1; head "$log" - ;; - - (stop) - - if [ -f $pid ]; then - if kill -0 `cat $pid` > /dev/null 2>&1; then - echo stopping hdfsproxy - kill `cat $pid` - else - echo no hdfsproxy to stop - fi - else - echo no hdfsproxy to stop - fi - ;; - - (*) - echo $usage - exit 1 - ;; - -esac - - diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-daemons.sh b/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-daemons.sh deleted file mode 100755 index 7dd8568a3b..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-daemons.sh +++ /dev/null @@ -1,34 +0,0 @@ -#!/usr/bin/env bash - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -# Run a HdfsProxy command on all slave hosts. - -usage="Usage: hdfsproxy-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] " - -# if no args specified, show usage -if [ $# -le 1 ]; then - echo $usage - exit 1 -fi - -bin=`dirname "$0"` -bin=`cd "$bin"; pwd` - -. $bin/hdfsproxy-config.sh - -exec "$bin/hdfsproxy-slaves.sh" --config $HDFSPROXY_CONF_DIR cd "$HDFSPROXY_HOME" \; "$bin/hdfsproxy-daemon.sh" --config $HDFSPROXY_CONF_DIR "$@" diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-slaves.sh b/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-slaves.sh deleted file mode 100755 index db54bd5b38..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/hdfsproxy-slaves.sh +++ /dev/null @@ -1,68 +0,0 @@ -#!/usr/bin/env bash - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -# Run a shell command on all slave hosts. -# -# Environment Variables -# -# HDFSPROXY_SLAVES File naming remote hosts. -# Default is ${HDFSPROXY_CONF_DIR}/hdfsproxy-hosts. -# HDFSPROXY_CONF_DIR Alternate conf dir. Default is ${HDFSPROXY_HOME}/conf. -# HDFSPROXY_SLAVE_SLEEP Seconds to sleep between spawning remote commands. -# HDFSPROXY_SSH_OPTS Options passed to ssh when running remote commands. -## - -usage="Usage: hdfsproxy-slaves.sh [--config confdir] command..." - -# if no args specified, show usage -if [ $# -le 0 ]; then - echo $usage - exit 1 -fi - -bin=`dirname "$0"` -bin=`cd "$bin"; pwd` - -. "$bin"/hdfsproxy-config.sh - -# If the slaves file is specified in the command line, -# then it takes precedence over the definition in -# hdfsproxy-env.sh. Save it here. -HOSTLIST=$HDFSPROXY_SLAVES - -if [ -f "${HDFSPROXY_CONF_DIR}/hdfsproxy-env.sh" ]; then - . "${HDFSPROXY_CONF_DIR}/hdfsproxy-env.sh" -fi - -if [ "$HOSTLIST" = "" ]; then - if [ "$HDFSPROXY_SLAVES" = "" ]; then - export HOSTLIST="${HDFSPROXY_CONF_DIR}/hdfsproxy-hosts" - else - export HOSTLIST="${HDFSPROXY_SLAVES}" - fi -fi - -for slave in `cat "$HOSTLIST"`; do - ssh $HDFSPROXY_SSH_OPTS $slave $"${@// /\\ }" \ - 2>&1 | sed "s/^/$slave: /" & - if [ "$HDFSPROXY_SLAVE_SLEEP" != "" ]; then - sleep $HDFSPROXY_SLAVE_SLEEP - fi -done - -wait diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/start-hdfsproxy.sh b/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/start-hdfsproxy.sh deleted file mode 100755 index 2592d9c8cc..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/start-hdfsproxy.sh +++ /dev/null @@ -1,37 +0,0 @@ -#!/usr/bin/env bash - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -# Start hdfsproxy daemons. -# Run this on master node. - -usage="Usage: start-hdfsproxy.sh" - -bin=`dirname "$0"` -bin=`cd "$bin"; pwd` - -. "$bin"/hdfsproxy-config.sh - -# get arguments -if [ $# -ge 1 ]; then - echo $usage - exit 1 -fi - -# start hdfsproxy daemons -# "$bin"/hdfsproxy-daemon.sh --config $HDFSPROXY_CONF_DIR start -"$bin"/hdfsproxy-daemons.sh --config $HDFSPROXY_CONF_DIR --hosts hdfsproxy-hosts start diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/stop-hdfsproxy.sh b/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/stop-hdfsproxy.sh deleted file mode 100755 index 78089e31cf..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/bin/stop-hdfsproxy.sh +++ /dev/null @@ -1,28 +0,0 @@ -#!/usr/bin/env bash - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - -# Stop hdfsproxy daemons. Run this on master node. - -bin=`dirname "$0"` -bin=`cd "$bin"; pwd` - -. "$bin"/hdfsproxy-config.sh - -# "$bin"/hdfsproxy-daemon.sh --config $HDFSPROXY_CONF_DIR stop -"$bin"/hdfsproxy-daemons.sh --config $HDFSPROXY_CONF_DIR --hosts hdfsproxy-hosts stop - diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/build.xml b/third_party/hadoop-0.20.0/contrib/hdfsproxy/build.xml deleted file mode 100644 index e62b2f279a..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/build.xml +++ /dev/null @@ -1,183 +0,0 @@ - - - - - - - - - - - - - - - - Building the .jar files. - - - -

    - - - -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/configuration.xsl b/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/configuration.xsl deleted file mode 100644 index 377cdbeb93..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/configuration.xsl +++ /dev/null @@ -1,24 +0,0 @@ - - - - - - - - - - - - - - - - - - - -
    namevaluedescription
    - - -
    -
    diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-default.xml b/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-default.xml deleted file mode 100644 index 0d2a006c8e..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-default.xml +++ /dev/null @@ -1,59 +0,0 @@ - - - - - - - - - hdfsproxy.https.address - 0.0.0.0:50479 - the SSL port that hdfsproxy listens on - - - - - hdfsproxy.hosts - hdfsproxy-hosts - location of hdfsproxy-hosts file - - - - - hdfsproxy.dfs.namenode.address - - namenode address of the HDFS cluster being proxied - - - - - hdfsproxy.https.server.keystore.resource - ssl-server.xml - location of the resource from which ssl server keystore - information will be extracted - - - - - hdfsproxy.user.permissions.file.location - user-permissions.xml - location of the user permissions file - - - - - hdfsproxy.user.certs.file.location - user-certs.xml - location of the user certs file - - - - - hdfsproxy.ugi.cache.ugi.lifetime - 15 - The lifetime (in minutes) of a cached ugi - - - - - diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-env.sh b/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-env.sh deleted file mode 100644 index a0ff7a5d27..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-env.sh +++ /dev/null @@ -1,44 +0,0 @@ -# Set HdfsProxy-specific environment variables here. - -# The only required environment variable is JAVA_HOME. All others are -# optional. When running a distributed configuration it is best to -# set JAVA_HOME in this file, so that it is correctly defined on -# remote nodes. - -# The java implementation to use. Required. -# export JAVA_HOME=/usr/lib/j2sdk1.5-sun - -# Extra Java CLASSPATH elements. Optional. -# export HDFSPROXY_CLASSPATH= - -# The maximum amount of heap to use, in MB. Default is 1000. -# export HDFSPROXY_HEAPSIZE=2000 - -# Extra Java runtime options. Empty by default. -# export HDFSPROXY_OPTS= - -# Extra ssh options. Empty by default. -# export HDFSPROXY_SSH_OPTS="-o ConnectTimeout=1 -o SendEnv=HDFSPROXY_CONF_DIR" - -# Where log files are stored. $HDFSPROXY_HOME/logs by default. -# export HDFSPROXY_LOG_DIR=${HDFSPROXY_HOME}/logs - -# File naming remote slave hosts. $HDFSPROXY_HOME/conf/slaves by default. -# export HDFSPROXY_SLAVES=${HDFSPROXY_HOME}/conf/slaves - -# host:path where hdfsproxy code should be rsync'd from. Unset by default. -# export HDFSPROXY_MASTER=master:/home/$USER/src/hdfsproxy - -# Seconds to sleep between slave commands. Unset by default. This -# can be useful in large clusters, where, e.g., slave rsyncs can -# otherwise arrive faster than the master can service them. -# export HDFSPROXY_SLAVE_SLEEP=0.1 - -# The directory where pid files are stored. /tmp by default. -# export HDFSPROXY_PID_DIR=/var/hdfsproxy/pids - -# A string representing this instance of hdfsproxy. $USER by default. -# export HDFSPROXY_IDENT_STRING=$USER - -# The scheduling priority for daemon processes. See 'man nice'. -# export HDFSPROXY_NICENESS=10 diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-env.sh.template b/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-env.sh.template deleted file mode 100644 index a0ff7a5d27..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-env.sh.template +++ /dev/null @@ -1,44 +0,0 @@ -# Set HdfsProxy-specific environment variables here. - -# The only required environment variable is JAVA_HOME. All others are -# optional. When running a distributed configuration it is best to -# set JAVA_HOME in this file, so that it is correctly defined on -# remote nodes. - -# The java implementation to use. Required. -# export JAVA_HOME=/usr/lib/j2sdk1.5-sun - -# Extra Java CLASSPATH elements. Optional. -# export HDFSPROXY_CLASSPATH= - -# The maximum amount of heap to use, in MB. Default is 1000. -# export HDFSPROXY_HEAPSIZE=2000 - -# Extra Java runtime options. Empty by default. -# export HDFSPROXY_OPTS= - -# Extra ssh options. Empty by default. -# export HDFSPROXY_SSH_OPTS="-o ConnectTimeout=1 -o SendEnv=HDFSPROXY_CONF_DIR" - -# Where log files are stored. $HDFSPROXY_HOME/logs by default. -# export HDFSPROXY_LOG_DIR=${HDFSPROXY_HOME}/logs - -# File naming remote slave hosts. $HDFSPROXY_HOME/conf/slaves by default. -# export HDFSPROXY_SLAVES=${HDFSPROXY_HOME}/conf/slaves - -# host:path where hdfsproxy code should be rsync'd from. Unset by default. -# export HDFSPROXY_MASTER=master:/home/$USER/src/hdfsproxy - -# Seconds to sleep between slave commands. Unset by default. This -# can be useful in large clusters, where, e.g., slave rsyncs can -# otherwise arrive faster than the master can service them. -# export HDFSPROXY_SLAVE_SLEEP=0.1 - -# The directory where pid files are stored. /tmp by default. -# export HDFSPROXY_PID_DIR=/var/hdfsproxy/pids - -# A string representing this instance of hdfsproxy. $USER by default. -# export HDFSPROXY_IDENT_STRING=$USER - -# The scheduling priority for daemon processes. See 'man nice'. -# export HDFSPROXY_NICENESS=10 diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-hosts b/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-hosts deleted file mode 100644 index 2fbb50c4a8..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/hdfsproxy-hosts +++ /dev/null @@ -1 +0,0 @@ -localhost diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/log4j.properties b/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/log4j.properties deleted file mode 100644 index 2520ab3795..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/log4j.properties +++ /dev/null @@ -1,61 +0,0 @@ -# Define some default values that can be overridden by system properties -hdfsproxy.root.logger=INFO,console -hdfsproxy.log.dir=. -hdfsproxy.log.file=hdfsproxy.log - -# Define the root logger to the system property "hdfsproxy.root.logger". -log4j.rootLogger=${hdfsproxy.root.logger} - -# Logging Threshold -log4j.threshhold=ALL - -# -# Daily Rolling File Appender -# - -log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender -log4j.appender.DRFA.File=${hdfsproxy.log.dir}/${hdfsproxy.log.file} - -# Rollver at midnight -log4j.appender.DRFA.DatePattern=.yyyy-MM-dd - -# 30-day backup -#log4j.appender.DRFA.MaxBackupIndex=30 -log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout - -# Pattern format: Date LogLevel LoggerName LogMessage -log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n -# Debugging Pattern format -#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n - - -# -# console -# Add "console" to rootlogger above if you want to use this -# - -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.err -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n - -# -# Rolling File Appender -# - -#log4j.appender.RFA=org.apache.log4j.RollingFileAppender -#log4j.appender.RFA.File=${hdfsproxy.log.dir}/${hdfsproxy.log.file} - -# Logfile size and and 30-day backups -#log4j.appender.RFA.MaxFileSize=1MB -#log4j.appender.RFA.MaxBackupIndex=30 - -#log4j.appender.RFA.layout=org.apache.log4j.PatternLayout -#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} - %m%n -#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n - -# Custom Logging levels - -#log4j.logger.org.apache.hadoop.hdfsproxy.HttpsProxy=DEBUG -#log4j.logger.org.apache.hadoop.hdfsproxy.ProxyFilter=DEBUG - diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/user-certs.xml b/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/user-certs.xml deleted file mode 100644 index f572a55294..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/user-certs.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - - - - - - Admin - - Special hdfsproxy admin user - - - - diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/user-permissions.xml b/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/user-permissions.xml deleted file mode 100644 index b7373751bd..0000000000 --- a/third_party/hadoop-0.20.0/contrib/hdfsproxy/conf/user-permissions.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - - - - - - - - - - diff --git a/third_party/hadoop-0.20.0/contrib/hdfsproxy/hdfsproxy-1.0.jar b/third_party/hadoop-0.20.0/contrib/hdfsproxy/hdfsproxy-1.0.jar deleted file mode 100644 index a313391dfb..0000000000 Binary files a/third_party/hadoop-0.20.0/contrib/hdfsproxy/hdfsproxy-1.0.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/contrib/index/hadoop-0.20.0-index.jar b/third_party/hadoop-0.20.0/contrib/index/hadoop-0.20.0-index.jar deleted file mode 100644 index f1f850fcd3..0000000000 Binary files a/third_party/hadoop-0.20.0/contrib/index/hadoop-0.20.0-index.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/contrib/streaming/hadoop-0.20.0-streaming.jar b/third_party/hadoop-0.20.0/contrib/streaming/hadoop-0.20.0-streaming.jar deleted file mode 100644 index 84251e3a3c..0000000000 Binary files a/third_party/hadoop-0.20.0/contrib/streaming/hadoop-0.20.0-streaming.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/contrib/thriftfs/hadoop-0.20.0-thriftfs.jar b/third_party/hadoop-0.20.0/contrib/thriftfs/hadoop-0.20.0-thriftfs.jar deleted file mode 100644 index bf10c05e1d..0000000000 Binary files a/third_party/hadoop-0.20.0/contrib/thriftfs/hadoop-0.20.0-thriftfs.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/contrib/vaidya/bin/vaidya.sh b/third_party/hadoop-0.20.0/contrib/vaidya/bin/vaidya.sh deleted file mode 100755 index ada6715342..0000000000 --- a/third_party/hadoop-0.20.0/contrib/vaidya/bin/vaidya.sh +++ /dev/null @@ -1,47 +0,0 @@ -#!/bin/sh -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -this="$0" -while [ -h "$this" ]; do - ls=`ls -ld "$this"` - link=`expr "$ls" : '.*-> \(.*\)$'` - if expr "$link" : '.*/.*' > /dev/null; then - this="$link" - else - this=`dirname "$this"`/"$link" - fi -done - -# convert relative path to absolute path -bin=`dirname "$this"` -script=`basename "$this"` -bin=`cd "$bin"; pwd` -this="$bin/$script" - -# Check if HADOOP_HOME AND JAVA_HOME is set. -if [ -z $HADOOP_HOME ] ; then - echo "HADOOP_HOME environment variable not defined" - exit -1; -fi - -if [ -z $JAVA_HOME ] ; then - echo "JAVA_HOME environment variable not defined" - exit -1; -fi - -hadoopVersion=`$HADOOP_HOME/bin/hadoop version | awk 'BEGIN { RS = "" ; FS = "\n" } ; { print $1 }' | awk '{print $2}'` - -$JAVA_HOME/bin/java -classpath $HADOOP_HOME/hadoop-${hadoopVersion}-core.jar:$HADOOP_HOME/contrib/vaidya/hadoop-${hadoopVersion}-vaidya.jar:$HADOOP_HOME/lib/commons-logging-1.0.4.jar:${CLASSPATH} org.apache.hadoop.vaidya.postexdiagnosis.PostExPerformanceDiagnoser $@ diff --git a/third_party/hadoop-0.20.0/contrib/vaidya/conf/postex_diagnosis_tests.xml b/third_party/hadoop-0.20.0/contrib/vaidya/conf/postex_diagnosis_tests.xml deleted file mode 100644 index f30d5d9cc8..0000000000 --- a/third_party/hadoop-0.20.0/contrib/vaidya/conf/postex_diagnosis_tests.xml +++ /dev/null @@ -1,104 +0,0 @@ - - - - - - - <![CDATA[Balanaced Reduce Partitioning]]> - - - - - - - - - - - - <![CDATA[Impact of Map tasks Re-Execution]]> - - - - - - - - - - - <![CDATA[Impact of Reduce tasks Re-Execution]]> - - - - - - - - - - - <![CDATA[Map and/or Reduce tasks reading HDFS data as a side effect]]> - - - - - - - - - - - <![CDATA[Map side disk spill]]> - - - - - - - 3.0 - - - - diff --git a/third_party/hadoop-0.20.0/contrib/vaidya/hadoop-0.20.0-vaidya.jar b/third_party/hadoop-0.20.0/contrib/vaidya/hadoop-0.20.0-vaidya.jar deleted file mode 100644 index 534b18d974..0000000000 Binary files a/third_party/hadoop-0.20.0/contrib/vaidya/hadoop-0.20.0-vaidya.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/.DS_Store b/third_party/hadoop-0.20.0/lib/.DS_Store deleted file mode 100644 index e0d363a012..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/.DS_Store and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/commons-cli-2.0-SNAPSHOT.jar b/third_party/hadoop-0.20.0/lib/commons-cli-2.0-SNAPSHOT.jar deleted file mode 100644 index 0b1d51072a..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/commons-cli-2.0-SNAPSHOT.jar and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/native/.DS_Store b/third_party/hadoop-0.20.0/lib/native/.DS_Store deleted file mode 100644 index ff86c6f7f2..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/native/.DS_Store and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.a b/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.a deleted file mode 100644 index d8d90cf067..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.a and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.so b/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.so deleted file mode 100644 index fb2cbad0b5..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.so and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.so.1 b/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.so.1 deleted file mode 100644 index fb2cbad0b5..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.so.1 and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.so.1.0.0 b/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.so.1.0.0 deleted file mode 100644 index fb2cbad0b5..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/native/Linux-amd64-64/libhadoop.so.1.0.0 and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.a b/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.a deleted file mode 100644 index 068d2d6bf6..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.a and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.so b/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.so deleted file mode 100644 index e3acc2b220..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.so and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.so.1 b/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.so.1 deleted file mode 100644 index e3acc2b220..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.so.1 and /dev/null differ diff --git a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.so.1.0.0 b/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.so.1.0.0 deleted file mode 100644 index e3acc2b220..0000000000 Binary files a/third_party/hadoop-0.20.0/lib/native/Linux-i386-32/libhadoop.so.1.0.0 and /dev/null differ diff --git a/third_party/hadoop-0.20.0/librecordio/librecordio.a b/third_party/hadoop-0.20.0/librecordio/librecordio.a deleted file mode 100644 index 49f7c22d1e..0000000000 Binary files a/third_party/hadoop-0.20.0/librecordio/librecordio.a and /dev/null differ diff --git a/third_party/hadoop-0.20.0/webapps/datanode/WEB-INF/web.xml b/third_party/hadoop-0.20.0/webapps/datanode/WEB-INF/web.xml deleted file mode 100644 index c271b62815..0000000000 --- a/third_party/hadoop-0.20.0/webapps/datanode/WEB-INF/web.xml +++ /dev/null @@ -1,40 +0,0 @@ - - - - - - - org.apache.hadoop.hdfs.server.datanode.browseDirectory_jsp - org.apache.hadoop.hdfs.server.datanode.browseDirectory_jsp - - - - org.apache.hadoop.hdfs.server.datanode.tail_jsp - org.apache.hadoop.hdfs.server.datanode.tail_jsp - - - - org.apache.hadoop.hdfs.server.datanode.browseBlock_jsp - org.apache.hadoop.hdfs.server.datanode.browseBlock_jsp - - - - org.apache.hadoop.hdfs.server.datanode.browseDirectory_jsp - /browseDirectory.jsp - - - - org.apache.hadoop.hdfs.server.datanode.tail_jsp - /tail.jsp - - - - org.apache.hadoop.hdfs.server.datanode.browseBlock_jsp - /browseBlock.jsp - - - - diff --git a/third_party/hadoop-0.20.0/webapps/hdfs/WEB-INF/web.xml b/third_party/hadoop-0.20.0/webapps/hdfs/WEB-INF/web.xml deleted file mode 100644 index 40a73d5293..0000000000 --- a/third_party/hadoop-0.20.0/webapps/hdfs/WEB-INF/web.xml +++ /dev/null @@ -1,40 +0,0 @@ - - - - - - - org.apache.hadoop.hdfs.server.namenode.nn_005fbrowsedfscontent_jsp - org.apache.hadoop.hdfs.server.namenode.nn_005fbrowsedfscontent_jsp - - - - org.apache.hadoop.hdfs.server.namenode.dfsnodelist_jsp - org.apache.hadoop.hdfs.server.namenode.dfsnodelist_jsp - - - - org.apache.hadoop.hdfs.server.namenode.dfshealth_jsp - org.apache.hadoop.hdfs.server.namenode.dfshealth_jsp - - - - org.apache.hadoop.hdfs.server.namenode.nn_005fbrowsedfscontent_jsp - /nn_browsedfscontent.jsp - - - - org.apache.hadoop.hdfs.server.namenode.dfsnodelist_jsp - /dfsnodelist.jsp - - - - org.apache.hadoop.hdfs.server.namenode.dfshealth_jsp - /dfshealth.jsp - - - - diff --git a/third_party/hadoop-0.20.0/webapps/hdfs/index.html b/third_party/hadoop-0.20.0/webapps/hdfs/index.html deleted file mode 100644 index b9ad74218a..0000000000 --- a/third_party/hadoop-0.20.0/webapps/hdfs/index.html +++ /dev/null @@ -1,20 +0,0 @@ - - - - -Hadoop Administration - - - - -

    Hadoop Administration

    - - - - - - diff --git a/third_party/hadoop-0.20.0/webapps/job/WEB-INF/web.xml b/third_party/hadoop-0.20.0/webapps/job/WEB-INF/web.xml deleted file mode 100644 index e124ff744b..0000000000 --- a/third_party/hadoop-0.20.0/webapps/job/WEB-INF/web.xml +++ /dev/null @@ -1,180 +0,0 @@ - - - - - - - org.apache.hadoop.mapred.jobqueue_005fdetails_jsp - org.apache.hadoop.mapred.jobqueue_005fdetails_jsp - - - - org.apache.hadoop.mapred.jobtracker_jsp - org.apache.hadoop.mapred.jobtracker_jsp - - - - org.apache.hadoop.mapred.machines_jsp - org.apache.hadoop.mapred.machines_jsp - - - - org.apache.hadoop.mapred.taskdetailshistory_jsp - org.apache.hadoop.mapred.taskdetailshistory_jsp - - - - org.apache.hadoop.mapred.jobhistory_jsp - org.apache.hadoop.mapred.jobhistory_jsp - - - - org.apache.hadoop.mapred.jobconf_005fhistory_jsp - org.apache.hadoop.mapred.jobconf_005fhistory_jsp - - - - org.apache.hadoop.mapred.loadhistory_jsp - org.apache.hadoop.mapred.loadhistory_jsp - - - - org.apache.hadoop.mapred.jobdetailshistory_jsp - org.apache.hadoop.mapred.jobdetailshistory_jsp - - - - org.apache.hadoop.mapred.jobtaskshistory_jsp - org.apache.hadoop.mapred.jobtaskshistory_jsp - - - - org.apache.hadoop.mapred.jobfailures_jsp - org.apache.hadoop.mapred.jobfailures_jsp - - - - org.apache.hadoop.mapred.taskdetails_jsp - org.apache.hadoop.mapred.taskdetails_jsp - - - - org.apache.hadoop.mapred.analysejobhistory_jsp - org.apache.hadoop.mapred.analysejobhistory_jsp - - - - org.apache.hadoop.mapred.jobblacklistedtrackers_jsp - org.apache.hadoop.mapred.jobblacklistedtrackers_jsp - - - - org.apache.hadoop.mapred.jobdetails_jsp - org.apache.hadoop.mapred.jobdetails_jsp - - - - org.apache.hadoop.mapred.jobtasks_jsp - org.apache.hadoop.mapred.jobtasks_jsp - - - - org.apache.hadoop.mapred.jobconf_jsp - org.apache.hadoop.mapred.jobconf_jsp - - - - org.apache.hadoop.mapred.taskstats_jsp - org.apache.hadoop.mapred.taskstats_jsp - - - - org.apache.hadoop.mapred.jobqueue_005fdetails_jsp - /jobqueue_details.jsp - - - - org.apache.hadoop.mapred.jobtracker_jsp - /jobtracker.jsp - - - - org.apache.hadoop.mapred.machines_jsp - /machines.jsp - - - - org.apache.hadoop.mapred.taskdetailshistory_jsp - /taskdetailshistory.jsp - - - - org.apache.hadoop.mapred.jobhistory_jsp - /jobhistory.jsp - - - - org.apache.hadoop.mapred.jobconf_005fhistory_jsp - /jobconf_history.jsp - - - - org.apache.hadoop.mapred.loadhistory_jsp - /loadhistory.jsp - - - - org.apache.hadoop.mapred.jobdetailshistory_jsp - /jobdetailshistory.jsp - - - - org.apache.hadoop.mapred.jobtaskshistory_jsp - /jobtaskshistory.jsp - - - - org.apache.hadoop.mapred.jobfailures_jsp - /jobfailures.jsp - - - - org.apache.hadoop.mapred.taskdetails_jsp - /taskdetails.jsp - - - - org.apache.hadoop.mapred.analysejobhistory_jsp - /analysejobhistory.jsp - - - - org.apache.hadoop.mapred.jobblacklistedtrackers_jsp - /jobblacklistedtrackers.jsp - - - - org.apache.hadoop.mapred.jobdetails_jsp - /jobdetails.jsp - - - - org.apache.hadoop.mapred.jobtasks_jsp - /jobtasks.jsp - - - - org.apache.hadoop.mapred.jobconf_jsp - /jobconf.jsp - - - - org.apache.hadoop.mapred.taskstats_jsp - /taskstats.jsp - - - - diff --git a/third_party/hadoop-0.20.0/webapps/job/index.html b/third_party/hadoop-0.20.0/webapps/job/index.html deleted file mode 100644 index 0cc16dce3d..0000000000 --- a/third_party/hadoop-0.20.0/webapps/job/index.html +++ /dev/null @@ -1,20 +0,0 @@ - - - - -Hadoop Administration - - - - -

    Hadoop Administration

    - - - - - - diff --git a/third_party/hadoop-0.20.0/webapps/static/hadoop-logo.jpg b/third_party/hadoop-0.20.0/webapps/static/hadoop-logo.jpg deleted file mode 100644 index 809525d9f1..0000000000 Binary files a/third_party/hadoop-0.20.0/webapps/static/hadoop-logo.jpg and /dev/null differ diff --git a/third_party/hadoop-0.20.0/webapps/static/hadoop.css b/third_party/hadoop-0.20.0/webapps/static/hadoop.css deleted file mode 100644 index 0560cb3075..0000000000 --- a/third_party/hadoop-0.20.0/webapps/static/hadoop.css +++ /dev/null @@ -1,134 +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. -*/ - -body { - background-color : #ffffff; - font-family : sans-serif; -} - -.small { - font-size : smaller; -} - -div#dfsnodetable tr#row1, div#dfstable td#col1 { - font-weight : bolder; -} - -div#dfstable td#col1 { - vertical-align : top; -} - -div#dfstable td#col3 { - text-align : right; -} - -div#dfsnodetable caption { - text-align : left; -} - -div#dfsnodetable a#title { - font-size : larger; - font-weight : bolder; -} - -div#dfsnodetable td, th { - border-bottom-style : none; - padding-bottom : 4px; - padding-top : 4px; -} - -div#dfsnodetable A:link, A:visited { - text-decoration : none; -} - -div#dfsnodetable th.header, th.headerASC, th.headerDSC { - padding-bottom : 8px; - padding-top : 8px; -} -div#dfsnodetable th.header:hover, th.headerASC:hover, th.headerDSC:hover, - td.name:hover { - text-decoration : underline; - cursor : pointer; -} - -div#dfsnodetable td.blocks, td.size, td.pcused, td.adminstate, td.lastcontact { - text-align : right; -} - -div#dfsnodetable .rowNormal .header { - background-color : #ffffff; -} -div#dfsnodetable .rowAlt, .headerASC, .headerDSC { - background-color : lightyellow; -} - -.warning { - font-weight : bolder; - color : red; -} - -div#dfstable table { - white-space : pre; -} - -div#dfsnodetable td, div#dfsnodetable th, div#dfstable td { - padding-left : 10px; - padding-right : 10px; -} - -td.perc_filled { - background-color:#AAAAFF; -} - -td.perc_nonfilled { - background-color:#FFFFFF; -} - -line.taskgraphline { - stroke-width:1;stroke-linecap:round; -} - -#quicklinks { - margin: 0; - padding: 2px 4px; - position: fixed; - top: 0; - right: 0; - text-align: right; - background-color: #eee; - font-weight: bold; -} - -#quicklinks ul { - margin: 0; - padding: 0; - list-style-type: none; - font-weight: normal; -} - -#quicklinks ul { - display: none; -} - -#quicklinks a { - font-size: smaller; - text-decoration: none; -} - -#quicklinks ul a { - text-decoration: underline; -} diff --git a/third_party/hadoop-0.20.0/webapps/static/jobconf.xsl b/third_party/hadoop-0.20.0/webapps/static/jobconf.xsl deleted file mode 100644 index f3c2e33cef..0000000000 --- a/third_party/hadoop-0.20.0/webapps/static/jobconf.xsl +++ /dev/null @@ -1,18 +0,0 @@ - - - - - - - - - - - - - - - -
    namevalue
    -
    -
    diff --git a/third_party/hadoop-0.20.0/webapps/static/jobtracker.js b/third_party/hadoop-0.20.0/webapps/static/jobtracker.js deleted file mode 100644 index 7da16c1fcc..0000000000 --- a/third_party/hadoop-0.20.0/webapps/static/jobtracker.js +++ /dev/null @@ -1,151 +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. -*/ - -function checkButtonVerbage() -{ - var inputs = document.getElementsByName("jobCheckBox"); - var check = getCheckStatus(inputs); - - setCheckButtonVerbage(! check); -} - -function selectAll() -{ - var inputs = document.getElementsByName("jobCheckBox"); - var check = getCheckStatus(inputs); - - for (var i in inputs) { - if ('jobCheckBox' == inputs[i].name) { - if ( inputs[i].parentNode.parentNode.style.display != 'none') { - inputs[i].checked = ! check; - } - } - } - - setCheckButtonVerbage(check); -} - -function getCheckStatus(inputs) -{ - var check = true; - - for (var i in inputs) { - if ('jobCheckBox' == inputs[i].name) { - if ( inputs[i].parentNode.parentNode.style.display != 'none') { - check = (inputs[i].checked && check); - } - } - } - - return check; -} - - -function setCheckButtonVerbage(check) -{ - var op = document.getElementById("checkEm"); - op.value = check ? "Select All" : "Deselect All"; -} - -function applyfilter() -{ - var cols = ["job","priority","user","name"]; - var nodes = []; - var filters = []; - - for (var i = 0; i < cols.length; ++i) { - nodes[i] = document.getElementById(cols[i] + "_0" ); - } - - var filter = document.getElementById("filter"); - filters = filter.value.split(' '); - - var row = 0; - while ( nodes[0] != null ) { - //default display status - var display = true; - - // for each filter - for (var filter_idx = 0; filter_idx < filters.length; ++filter_idx) { - - // go check each column - if ((getDisplayStatus(nodes, filters[filter_idx], cols)) == 0) { - display = false; - break; - } - } - - // set the display status - nodes[0].parentNode.style.display = display ? '' : 'none'; - - // next row - ++row; - - // next set of controls - for (var i = 0; i < cols.length; ++i) { - nodes[i] = document.getElementById(cols[i] + "_" + row); - } - } // while -} - -function getDisplayStatus(nodes, filter, cols) -{ - var offset = filter.indexOf(':'); - - var search = offset != -1 ? filter.substring(offset + 1).toLowerCase() : filter.toLowerCase(); - - for (var col = 0; col < cols.length; ++col) { - // a column specific filter - if (offset != -1 ) { - var searchCol = filter.substring(0, offset).toLowerCase(); - - if (searchCol == cols[col]) { - // special case jobs to remove unnecessary stuff - return containsIgnoreCase(stripHtml(nodes[col].innerHTML), search); - } - } else if (containsIgnoreCase(stripHtml(nodes[col].innerHTML), filter)) { - return true; - } - } - - return false; -} - -function stripHtml(text) -{ - return text.replace(/<[^>]*>/g,'').replace(/&[^;]*;/g,''); -} - -function containsIgnoreCase(haystack, needle) -{ - return haystack.toLowerCase().indexOf(needle.toLowerCase()) != -1; -} - -function confirmAction() -{ - return confirm("Are you sure?"); -} - -function toggle(id) -{ - if ( document.getElementById(id).style.display != 'block') { - document.getElementById(id).style.display = 'block'; - } - else { - document.getElementById(id).style.display = 'none'; - } -} diff --git a/third_party/hadoop-0.20.0/webapps/task/WEB-INF/web.xml b/third_party/hadoop-0.20.0/webapps/task/WEB-INF/web.xml deleted file mode 100644 index 44a03bed6b..0000000000 --- a/third_party/hadoop-0.20.0/webapps/task/WEB-INF/web.xml +++ /dev/null @@ -1,20 +0,0 @@ - - - - - - - org.apache.hadoop.mapred.tasktracker_jsp - org.apache.hadoop.mapred.tasktracker_jsp - - - - org.apache.hadoop.mapred.tasktracker_jsp - /tasktracker.jsp - - - - diff --git a/third_party/hadoop-0.20.0/webapps/task/index.html b/third_party/hadoop-0.20.0/webapps/task/index.html deleted file mode 100644 index ab3d56ba46..0000000000 --- a/third_party/hadoop-0.20.0/webapps/task/index.html +++ /dev/null @@ -1 +0,0 @@ -