Merge trunk into auto-HA branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3042@1333291 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2012-05-03 02:14:01 +00:00
commit f6c01e2f8e
630 changed files with 18935 additions and 1760 deletions

View File

@ -87,4 +87,8 @@ Create source and binary distributions with native code and documentation:
$ mvn package -Pdist,native,docs,src -DskipTests -Dtar
Create a local staging version of the website (in /tmp/hadoop-site)
$ mvn clean site; mvn site:stage -DstagingDirectory=/tmp/hadoop-site
----------------------------------------------------------------------------------

View File

@ -39,6 +39,7 @@ WGET=${WGET:-wget}
SVN=${SVN:-svn}
GREP=${GREP:-grep}
PATCH=${PATCH:-patch}
DIFF=${DIFF:-diff}
JIRACLI=${JIRA:-jira}
FINDBUGS_HOME=${FINDBUGS_HOME}
FORREST_HOME=${FORREST_HOME}
@ -61,6 +62,7 @@ printUsage() {
echo "--svn-cmd=<cmd> The 'svn' command to use (default 'svn')"
echo "--grep-cmd=<cmd> The 'grep' command to use (default 'grep')"
echo "--patch-cmd=<cmd> The 'patch' command to use (default 'patch')"
echo "--diff-cmd=<cmd> The 'diff' command to use (default 'diff')"
echo "--findbugs-home=<path> Findbugs home directory (default FINDBUGS_HOME environment variable)"
echo "--forrest-home=<path> Forrest home directory (default FORREST_HOME environment variable)"
echo "--dirty-workspace Allow the local SVN workspace to have uncommitted changes"
@ -113,6 +115,9 @@ parseArgs() {
--patch-cmd=*)
PATCH=${i#*=}
;;
--diff-cmd=*)
DIFF=${i#*=}
;;
--jira-cmd=*)
JIRACLI=${i#*=}
;;
@ -235,15 +240,6 @@ setup () {
cleanupAndExit 0
fi
fi
. $BASEDIR/dev-support/test-patch.properties
### exit if warnings are NOT defined in the properties file
if [ -z "$OK_FINDBUGS_WARNINGS" ] || [[ -z "$OK_JAVADOC_WARNINGS" ]] || [[ -z $OK_RELEASEAUDIT_WARNINGS ]]; then
echo "Please define the following properties in test-patch.properties file"
echo "OK_FINDBUGS_WARNINGS"
echo "OK_RELEASEAUDIT_WARNINGS"
echo "OK_JAVADOC_WARNINGS"
cleanupAndExit 1
fi
echo ""
echo ""
echo "======================================================================"
@ -384,10 +380,10 @@ checkJavadocWarnings () {
echo ""
echo "$MVN clean test javadoc:javadoc -DskipTests -Pdocs -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchJavadocWarnings.txt 2>&1"
if [ -d hadoop-project ]; then
(cd hadoop-project; $MVN install)
(cd hadoop-project; $MVN install > /dev/null 2>&1)
fi
if [ -d hadoop-common-project/hadoop-annotations ]; then
(cd hadoop-common-project/hadoop-annotations; $MVN install)
(cd hadoop-common-project/hadoop-annotations; $MVN install > /dev/null 2>&1)
fi
$MVN clean test javadoc:javadoc -DskipTests -Pdocs -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchJavadocWarnings.txt 2>&1
javadocWarnings=`$GREP '\[WARNING\]' $PATCH_DIR/patchJavadocWarnings.txt | $AWK '/Javadoc Warnings/,EOF' | $GREP warning | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
@ -395,8 +391,10 @@ checkJavadocWarnings () {
echo ""
echo "There appear to be $javadocWarnings javadoc warnings generated by the patched build."
#There are 6 warnings that are caused by things that are caused by using sun internal APIs.
OK_JAVADOC_WARNINGS=6;
### if current warnings greater than OK_JAVADOC_WARNINGS
if [[ $javadocWarnings -gt $OK_JAVADOC_WARNINGS ]] ; then
if [[ $javadocWarnings -ne $OK_JAVADOC_WARNINGS ]] ; then
JIRA_COMMENT="$JIRA_COMMENT
-1 javadoc. The javadoc tool appears to have generated `expr $(($javadocWarnings-$OK_JAVADOC_WARNINGS))` warning messages."
@ -430,14 +428,21 @@ checkJavacWarnings () {
fi
### Compare trunk and patch javac warning numbers
if [[ -f $PATCH_DIR/patchJavacWarnings.txt ]] ; then
trunkJavacWarnings=`$GREP '\[WARNING\]' $PATCH_DIR/trunkJavacWarnings.txt | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
patchJavacWarnings=`$GREP '\[WARNING\]' $PATCH_DIR/patchJavacWarnings.txt | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
$GREP '\[WARNING\]' $PATCH_DIR/trunkJavacWarnings.txt > $PATCH_DIR/filteredTrunkJavacWarnings.txt
$GREP '\[WARNING\]' $PATCH_DIR/patchJavacWarnings.txt > $PATCH_DIR/filteredPatchJavacWarnings.txt
trunkJavacWarnings=`cat $PATCH_DIR/filteredTrunkJavacWarnings.txt | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
patchJavacWarnings=`cat $PATCH_DIR/filteredPatchJavacWarnings.txt | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
echo "There appear to be $trunkJavacWarnings javac compiler warnings before the patch and $patchJavacWarnings javac compiler warnings after applying the patch."
if [[ $patchJavacWarnings != "" && $trunkJavacWarnings != "" ]] ; then
if [[ $patchJavacWarnings -gt $trunkJavacWarnings ]] ; then
JIRA_COMMENT="$JIRA_COMMENT
-1 javac. The applied patch generated $patchJavacWarnings javac compiler warnings (more than the trunk's current $trunkJavacWarnings warnings)."
$DIFF $PATCH_DIR/filteredTrunkJavacWarnings.txt $PATCH_DIR/filteredPatchJavacWarnings.txt > $PATCH_DIR/diffJavacWarnings.txt
JIRA_COMMENT_FOOTER="Javac warnings: $BUILD_URL/artifact/trunk/$(basename $BASEDIR)/patchprocess/diffJavacWarnings.txt
$JIRA_COMMENT_FOOTER"
return 1
fi
fi
@ -460,8 +465,8 @@ checkReleaseAuditWarnings () {
echo "======================================================================"
echo ""
echo ""
echo "$MVN apache-rat:check -D${PROJECT_NAME}PatchProcess 2>&1"
$MVN apache-rat:check -D${PROJECT_NAME}PatchProcess 2>&1
echo "$MVN apache-rat:check -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchReleaseAuditOutput.txt 2>&1"
$MVN apache-rat:check -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchReleaseAuditOutput.txt 2>&1
find $BASEDIR -name rat.txt | xargs cat > $PATCH_DIR/patchReleaseAuditWarnings.txt
### Compare trunk and patch release audit warning numbers
@ -469,12 +474,12 @@ checkReleaseAuditWarnings () {
patchReleaseAuditWarnings=`$GREP -c '\!?????' $PATCH_DIR/patchReleaseAuditWarnings.txt`
echo ""
echo ""
echo "There appear to be $OK_RELEASEAUDIT_WARNINGS release audit warnings before the patch and $patchReleaseAuditWarnings release audit warnings after applying the patch."
if [[ $patchReleaseAuditWarnings != "" && $OK_RELEASEAUDIT_WARNINGS != "" ]] ; then
if [[ $patchReleaseAuditWarnings -gt $OK_RELEASEAUDIT_WARNINGS ]] ; then
echo "There appear to be $patchReleaseAuditWarnings release audit warnings after applying the patch."
if [[ $patchReleaseAuditWarnings != "" ]] ; then
if [[ $patchReleaseAuditWarnings -gt 0 ]] ; then
JIRA_COMMENT="$JIRA_COMMENT
-1 release audit. The applied patch generated $patchReleaseAuditWarnings release audit warnings (more than the trunk's current $OK_RELEASEAUDIT_WARNINGS warnings)."
-1 release audit. The applied patch generated $patchReleaseAuditWarnings release audit warnings."
$GREP '\!?????' $PATCH_DIR/patchReleaseAuditWarnings.txt > $PATCH_DIR/patchReleaseAuditProblems.txt
echo "Lines that start with ????? in the release audit report indicate files that do not have an Apache license header." >> $PATCH_DIR/patchReleaseAuditProblems.txt
JIRA_COMMENT_FOOTER="Release audit warnings: $BUILD_URL/artifact/trunk/patchprocess/patchReleaseAuditProblems.txt
@ -536,10 +541,21 @@ checkFindbugsWarnings () {
echo "======================================================================"
echo ""
echo ""
echo "$MVN clean test findbugs:findbugs -DskipTests -D${PROJECT_NAME}PatchProcess"
$MVN clean test findbugs:findbugs -DskipTests -D${PROJECT_NAME}PatchProcess < /dev/null
modules=$(findModules)
rc=0
for module in $modules;
do
cd $module
echo " Running findbugs in $module"
module_suffix=`basename ${module}`
echo "$MVN clean test findbugs:findbugs -DskipTests -D${PROJECT_NAME}PatchProcess < /dev/null > $PATCH_DIR/patchFindBugsOutput${module_suffix}.txt 2>&1"
$MVN clean test findbugs:findbugs -DskipTests -D${PROJECT_NAME}PatchProcess < /dev/null > $PATCH_DIR/patchFindBugsOutput${module_suffix}.txt 2>&1
(( rc = rc + $? ))
cd -
done
if [ $? != 0 ] ; then
if [ $rc != 0 ] ; then
JIRA_COMMENT="$JIRA_COMMENT
-1 findbugs. The patch appears to cause Findbugs (version ${findbugs_version}) to fail."
@ -572,11 +588,10 @@ $JIRA_COMMENT_FOOTER"
fi
done
### if current warnings greater than OK_FINDBUGS_WARNINGS
if [[ $findbugsWarnings -gt $OK_FINDBUGS_WARNINGS ]] ; then
if [[ $findbugsWarnings -gt 0 ]] ; then
JIRA_COMMENT="$JIRA_COMMENT
-1 findbugs. The patch appears to introduce `expr $(($findbugsWarnings-$OK_FINDBUGS_WARNINGS))` new Findbugs (version ${findbugs_version}) warnings."
-1 findbugs. The patch appears to introduce $findbugsWarnings new Findbugs (version ${findbugs_version}) warnings."
return 1
fi
JIRA_COMMENT="$JIRA_COMMENT
@ -598,8 +613,8 @@ checkEclipseGeneration () {
echo ""
echo ""
echo "$MVN eclipse:eclipse -D${PROJECT_NAME}PatchProcess"
$MVN eclipse:eclipse -D${PROJECT_NAME}PatchProcess
echo "$MVN eclipse:eclipse -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchEclipseOutput.txt 2>&1"
$MVN eclipse:eclipse -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchEclipseOutput.txt 2>&1
if [[ $? != 0 ]] ; then
JIRA_COMMENT="$JIRA_COMMENT
@ -627,16 +642,28 @@ runTests () {
echo ""
echo ""
echo "$MVN clean install -fn -Pnative -D${PROJECT_NAME}PatchProcess"
$MVN clean install -fn -Pnative -D${PROJECT_NAME}PatchProcess
failed_tests=`find . -name 'TEST*.xml' | xargs $GREP -l -E "<failure|<error" | sed -e "s|.*target/surefire-reports/TEST-| |g" | sed -e "s|\.xml||g"`
# With -fn mvn always exits with a 0 exit code. Because of this we need to
# find the errors instead of using the exit code. We assume that if the build
# failed a -1 is already given for that case
failed_tests=""
modules=$(findModules)
for module in $modules;
do
cd $module
echo " Running tests in $module"
echo " $MVN clean install -fn -Pnative -D${PROJECT_NAME}PatchProcess"
$MVN clean install -fn -Pnative -D${PROJECT_NAME}PatchProcess
module_failed_tests=`find . -name 'TEST*.xml' | xargs $GREP -l -E "<failure|<error" | sed -e "s|.*target/surefire-reports/TEST-| |g" | sed -e "s|\.xml||g"`
# With -fn mvn always exits with a 0 exit code. Because of this we need to
# find the errors instead of using the exit code. We assume that if the build
# failed a -1 is already given for that case
if [[ -n "$module_failed_tests" ]] ; then
failed_tests="${failed_tests}
${module_failed_tests}"
fi
cd -
done
if [[ -n "$failed_tests" ]] ; then
JIRA_COMMENT="$JIRA_COMMENT
-1 core tests. The patch failed these unit tests:
-1 core tests. The patch failed these unit tests in $modules:
$failed_tests"
return 1
fi
@ -646,6 +673,51 @@ $failed_tests"
return 0
}
###############################################################################
# Find the maven module containing the given file.
findModule (){
dir=`dirname $1`
while [ 1 ]
do
if [ -f "$dir/pom.xml" ]
then
echo $dir
return
else
dir=`dirname $dir`
fi
done
}
findModules () {
# Come up with a list of changed files into $TMP
TMP=/tmp/tmp.paths.$$
$GREP '^+++\|^---' $PATCH_DIR/patch | cut -c '5-' | $GREP -v /dev/null | sort | uniq > $TMP
# if all of the lines start with a/ or b/, then this is a git patch that
# was generated without --no-prefix
if ! $GREP -qv '^a/\|^b/' $TMP ; then
sed -i -e 's,^[ab]/,,' $TMP
fi
# Now find all the modules that were changed
TMP_MODULES=/tmp/tmp.modules.$$
for file in $(cut -f 1 $TMP | sort | uniq); do
echo $(findModule $file) >> $TMP_MODULES
done
rm $TMP
# Filter out modules without code
CHANGED_MODULES=""
for module in $(cat $TMP_MODULES | sort | uniq); do
$GREP "<packaging>pom</packaging>" $module/pom.xml > /dev/null
if [ "$?" != 0 ]; then
CHANGED_MODULES="$CHANGED_MODULES $module"
fi
done
rm $TMP_MODULES
echo $CHANGED_MODULES
}
###############################################################################
### Run the test-contrib target
runContribTests () {

View File

@ -171,10 +171,6 @@
<groupId>junit</groupId>
<artifactId>junit</artifactId>
</exclusion>
<exclusion>
<groupId>com.cenqua.clover</groupId>
<artifactId>clover</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
@ -211,10 +207,6 @@
<groupId>com.sun.jersey.contribs</groupId>
<artifactId>jersey-guice</artifactId>
</exclusion>
<exclusion>
<groupId>com.cenqua.clover</groupId>
<artifactId>clover</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-servlet</artifactId>
@ -263,10 +255,6 @@
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-servlet</artifactId>
</exclusion>
<exclusion>
<groupId>com.cenqua.clover</groupId>
<artifactId>clover</artifactId>
</exclusion>
</exclusions>
</dependency>
@ -291,10 +279,6 @@
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-servlet</artifactId>
</exclusion>
<exclusion>
<groupId>com.cenqua.clover</groupId>
<artifactId>clover</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>

View File

@ -1,21 +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.
# The number of acceptable warning for this module
# Please update the root test-patch.properties if you update this file.
OK_RELEASEAUDIT_WARNINGS=0
OK_FINDBUGS_WARNINGS=0
OK_JAVADOC_WARNINGS=13

View File

@ -288,7 +288,7 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
String clientPrincipal = gssContext.getSrcName().toString();
KerberosName kerberosName = new KerberosName(clientPrincipal);
String userName = kerberosName.getShortName();
token = new AuthenticationToken(userName, clientPrincipal, TYPE);
token = new AuthenticationToken(userName, clientPrincipal, getType());
response.setStatus(HttpServletResponse.SC_OK);
LOG.trace("SPNEGO completed for principal [{}]", clientPrincipal);
}

View File

@ -126,7 +126,7 @@ public class PseudoAuthenticationHandler implements AuthenticationHandler {
throw new AuthenticationException("Anonymous requests are disallowed");
}
} else {
token = new AuthenticationToken(userName, userName, TYPE);
token = new AuthenticationToken(userName, userName, getType());
}
return token;
}

View File

@ -63,6 +63,10 @@ Trunk (unreleased changes)
HADOOP-8290. Remove remaining references to hadoop.native.lib (harsh)
HADOOP-8285 Use ProtoBuf for RpcPayLoadHeader (sanjay radia)
HADOOP-8308. Support cross-project Jenkins builds. (tomwhite)
BUG FIXES
HADOOP-8177. MBeans shouldn't try to register when it fails to create MBeanName.
@ -119,6 +123,12 @@ Trunk (unreleased changes)
HADOOP-7788. Add simple HealthMonitor class to watch an HAService (todd)
HADOOP-8312. testpatch.sh should provide a simpler way to see which
warnings changed (bobby)
HADOOP-8339. jenkins complaining about 16 javadoc warnings
(Tom White and Robert Evans via tgraves)
OPTIMIZATIONS
HADOOP-7761. Improve the performance of raw comparisons. (todd)
@ -265,6 +275,15 @@ Release 2.0.0 - UNRELEASED
HADOOP-8117. Upgrade test build to Surefire 2.12 (todd)
HADOOP-8152. Expand public APIs for security library classes. (atm via eli)
HADOOP-7549. Use JDK ServiceLoader mechanism to find FileSystem implementations. (tucu)
HADOOP-8185. Update namenode -format documentation and add -nonInteractive
and -force. (Arpit Gupta via atm)
HADOOP-8214. make hadoop script recognize a full set of deprecated commands (rvs via tucu)
OPTIMIZATIONS
BUG FIXES
@ -360,6 +379,33 @@ Release 2.0.0 - UNRELEASED
HADOOP-8282. start-all.sh refers incorrectly start-dfs.sh
existence for starting start-yarn.sh. (Devaraj K via eli)
HADOOP-7350. Use ServiceLoader to discover compression codec classes.
(tomwhite)
HADOOP-8284. clover integration broken, also mapreduce poms are pulling
in clover as a dependency. (phunt via tucu)
HADOOP-8309. Pseudo & Kerberos AuthenticationHandler should use
getType() to create token (tucu)
HADOOP-8314. HttpServer#hasAdminAccess should return false if
authorization is enabled but user is not authenticated. (tucu)
HADOOP-8296. hadoop/yarn daemonlog usage wrong (Devaraj K via tgraves)
HADOOP-8310. FileContext#checkPath should handle URIs with no port. (atm)
HADOOP-8321. TestUrlStreamHandler fails. (tucu)
HADOOP-8325. Add a ShutdownHookManager to be used by different
components instead of the JVM shutdownhook (tucu)
HADOOP-8275. Range check DelegationKey length.
(Colin Patrick McCabe via eli)
HADOOP-8342. HDFS command fails with exception following merge of
HADOOP-8325 (tucu)
BREAKDOWN OF HADOOP-7454 SUBTASKS
HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
@ -412,6 +458,12 @@ Release 2.0.0 - UNRELEASED
HADOOP-8116. RetriableCommand is using RetryPolicy incorrectly after
HADOOP-7896. (atm)
HADOOP-8317. Update maven-assembly-plugin to 2.3 - fix build on FreeBSD
(Radim Kolar via bobby)
HADOOP-8172. Configuration no longer sets all keys in a deprecated key
list. (Anupam Seth via bobby)
Release 0.23.3 - UNRELEASED
INCOMPATIBLE CHANGES
@ -423,6 +475,9 @@ Release 0.23.3 - UNRELEASED
HADOOP-8108. Move method getHostPortString() from NameNode to NetUtils.
(Brandon Li via jitendra)
HADOOP-8288. Remove references of mapred.child.ulimit etc. since they are
not being used any more (Ravi Prakash via bobby)
OPTIMIZATIONS
BUG FIXES
@ -453,6 +508,17 @@ Release 0.23.3 - UNRELEASED
HADOOP-8227. Allow RPC to limit ephemeral port range. (bobby)
HADOOP-8305. distcp over viewfs is broken (John George via bobby)
HADOOP-8334. HttpServer sometimes returns incorrect port (Daryn Sharp via
bobby)
HADOOP-8330. Update TestSequenceFile.testCreateUsesFsArg() for HADOOP-8305.
(John George via szetszwo)
HADOOP-8335. Improve Configuration's address handling (Daryn Sharp via
bobby)
Release 0.23.2 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -281,9 +281,14 @@
<Match>
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.ipc\.protobuf\.IpcConnectionContextProtos.*"/>
</Match>
<Match>
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.ipc\.protobuf\.RpcPayloadHeaderProtos.*"/>
</Match>
<Match>
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.ha\.proto\.HAServiceProtocolProtos.*"/>
</Match>
</FindBugsFilter>

View File

@ -50,15 +50,16 @@ fi
COMMAND=$1
case $COMMAND in
#hdfs commands
namenode|secondarynamenode|datanode|dfs|dfsadmin|fsck|balancer|fetchdt)
namenode|secondarynamenode|datanode|dfs|dfsadmin|fsck|balancer|fetchdt|oiv|dfsgroups)
echo "DEPRECATED: Use of this script to execute hdfs command is deprecated." 1>&2
echo "Instead use the hdfs command for it." 1>&2
echo "" 1>&2
#try to locate hdfs and if present, delegate to it.
shift
if [ -f "${HADOOP_HDFS_HOME}"/bin/hdfs ]; then
exec "${HADOOP_HDFS_HOME}"/bin/hdfs $*
exec "${HADOOP_HDFS_HOME}"/bin/hdfs ${COMMAND/dfsgroups/groups} $*
elif [ -f "${HADOOP_PREFIX}"/bin/hdfs ]; then
exec "${HADOOP_PREFIX}"/bin/hdfs $*
exec "${HADOOP_PREFIX}"/bin/hdfs ${COMMAND/dfsgroups/groups} $*
else
echo "HADOOP_HDFS_HOME not found!"
exit 1
@ -66,15 +67,16 @@ case $COMMAND in
;;
#mapred commands for backwards compatibility
pipes|job|queue)
pipes|job|queue|mrgroups|mradmin|jobtracker|tasktracker)
echo "DEPRECATED: Use of this script to execute mapred command is deprecated." 1>&2
echo "Instead use the mapred command for it." 1>&2
echo "" 1>&2
#try to locate mapred and if present, delegate to it.
shift
if [ -f "${HADOOP_MAPRED_HOME}"/bin/mapred ]; then
exec "${HADOOP_MAPRED_HOME}"/bin/mapred $*
exec "${HADOOP_MAPRED_HOME}"/bin/mapred ${COMMAND/mrgroups/groups} $*
elif [ -f "${HADOOP_PREFIX}"/bin/mapred ]; then
exec "${HADOOP_PREFIX}"/bin/mapred $*
exec "${HADOOP_PREFIX}"/bin/mapred ${COMMAND/mrgroups/groups} $*
else
echo "HADOOP_MAPRED_HOME not found!"
exit 1

View File

@ -696,7 +696,7 @@
<a href="http://hadoop.apache.org/hdfs/docs/current/hdfs_user_guide.html#Upgrade+and+Rollback">Upgrade and Rollback</a>.
</p>
<p>
<code>Usage: hadoop namenode [-format] | [-upgrade] | [-rollback] | [-finalize] | [-importCheckpoint] | [-checkpoint] | [-backup]</code>
<code>Usage: hadoop namenode [-format [-force] [-nonInteractive] [-clusterid someid]] | [-upgrade] | [-rollback] | [-finalize] | [-importCheckpoint] | [-checkpoint] | [-backup]</code>
</p>
<table>
<tr><th> COMMAND_OPTION </th><th> Description </th></tr>
@ -714,8 +714,11 @@
<td>Start namenode in backup role, maintaining an up-to-date in-memory copy of the namespace and creating periodic checkpoints.</td>
</tr>
<tr>
<td><code>-format</code></td>
<td>Formats the namenode. It starts the namenode, formats it and then shut it down.</td>
<td><code>-format [-force] [-nonInteractive] [-clusterid someid]</code></td>
<td>Formats the namenode. It starts the namenode, formats it and then shuts it down. User will be prompted before formatting any non empty name directories in the local filesystem.<br/>
-nonInteractive: User will not be prompted for input if non empty name directories exist in the local filesystem and the format will fail.<br/>
-force: Formats the namenode and the user will NOT be prompted to confirm formatting of the name directories in the local filesystem. If -nonInteractive option is specified it will be ignored.<br/>
-clusterid: Associates the namenode with the id specified. When formatting federated namenodes use this option to make sure all namenodes are associated with the same id.</td>
</tr>
<tr>
<td><code>-upgrade</code></td>

View File

@ -33,6 +33,7 @@ import java.io.Writer;
import java.net.InetSocketAddress;
import java.net.URL;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Enumeration;
@ -269,10 +270,18 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
* This is to be used only by the developers in order to add deprecation of
* keys, and attempts to call this method after loading resources once,
* would lead to <tt>UnsupportedOperationException</tt>
*
* If a key is deprecated in favor of multiple keys, they are all treated as
* aliases of each other, and setting any one of them resets all the others
* to the new value.
*
* @param key
* @param newKeys
* @param customMessage
* @deprecated use {@link addDeprecation(String key, String newKey,
String customMessage)} instead
*/
@Deprecated
public synchronized static void addDeprecation(String key, String[] newKeys,
String customMessage) {
if (key == null || key.length() == 0 ||
@ -288,6 +297,22 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
}
}
}
/**
* Adds the deprecated key to the deprecation map.
* It does not override any existing entries in the deprecation map.
* This is to be used only by the developers in order to add deprecation of
* keys, and attempts to call this method after loading resources once,
* would lead to <tt>UnsupportedOperationException</tt>
*
* @param key
* @param newKey
* @param customMessage
*/
public synchronized static void addDeprecation(String key, String newKey,
String customMessage) {
addDeprecation(key, new String[] {newKey}, customMessage);
}
/**
* Adds the deprecated key to the deprecation map when no custom message
@ -297,13 +322,34 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
* keys, and attempts to call this method after loading resources once,
* would lead to <tt>UnsupportedOperationException</tt>
*
* If a key is deprecated in favor of multiple keys, they are all treated as
* aliases of each other, and setting any one of them resets all the others
* to the new value.
*
* @param key Key that is to be deprecated
* @param newKeys list of keys that take up the values of deprecated key
* @deprecated use {@link addDeprecation(String key, String newKey)} instead
*/
@Deprecated
public synchronized static void addDeprecation(String key, String[] newKeys) {
addDeprecation(key, newKeys, null);
}
/**
* Adds the deprecated key to the deprecation map when no custom message
* is provided.
* It does not override any existing entries in the deprecation map.
* This is to be used only by the developers in order to add deprecation of
* keys, and attempts to call this method after loading resources once,
* would lead to <tt>UnsupportedOperationException</tt>
*
* @param key Key that is to be deprecated
* @param newKey key that takes up the value of deprecated key
*/
public synchronized static void addDeprecation(String key, String newKey) {
addDeprecation(key, new String[] {newKey}, null);
}
/**
* checks whether the given <code>key</code> is deprecated.
*
@ -322,16 +368,26 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
* @param name property name.
* @return alternate name.
*/
private String getAlternateName(String name) {
String altName;
private String[] getAlternateNames(String name) {
String oldName, altNames[] = null;
DeprecatedKeyInfo keyInfo = deprecatedKeyMap.get(name);
if (keyInfo != null) {
altName = (keyInfo.newKeys.length > 0) ? keyInfo.newKeys[0] : null;
if (keyInfo == null) {
altNames = (reverseDeprecatedKeyMap.get(name) != null ) ?
new String [] {reverseDeprecatedKeyMap.get(name)} : null;
if(altNames != null && altNames.length > 0) {
//To help look for other new configs for this deprecated config
keyInfo = deprecatedKeyMap.get(altNames[0]);
}
}
if(keyInfo != null && keyInfo.newKeys.length > 0) {
List<String> list = new ArrayList<String>();
if(altNames != null) {
list.addAll(Arrays.asList(altNames));
}
list.addAll(Arrays.asList(keyInfo.newKeys));
altNames = list.toArray(new String[list.size()]);
}
else {
altName = reverseDeprecatedKeyMap.get(name);
}
return altName;
return altNames;
}
/**
@ -346,24 +402,29 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
* @return the first property in the list of properties mapping
* the <code>name</code> or the <code>name</code> itself.
*/
private String handleDeprecation(String name) {
if (isDeprecated(name)) {
private String[] handleDeprecation(String name) {
ArrayList<String > names = new ArrayList<String>();
if (isDeprecated(name)) {
DeprecatedKeyInfo keyInfo = deprecatedKeyMap.get(name);
warnOnceIfDeprecated(name);
for (String newKey : keyInfo.newKeys) {
if(newKey != null) {
name = newKey;
break;
names.add(newKey);
}
}
}
String deprecatedKey = reverseDeprecatedKeyMap.get(name);
if (deprecatedKey != null && !getOverlay().containsKey(name) &&
getOverlay().containsKey(deprecatedKey)) {
getProps().setProperty(name, getOverlay().getProperty(deprecatedKey));
getOverlay().setProperty(name, getOverlay().getProperty(deprecatedKey));
if(names.size() == 0) {
names.add(name);
}
return name;
for(String n : names) {
String deprecatedKey = reverseDeprecatedKeyMap.get(n);
if (deprecatedKey != null && !getOverlay().containsKey(n) &&
getOverlay().containsKey(deprecatedKey)) {
getProps().setProperty(n, getOverlay().getProperty(deprecatedKey));
getOverlay().setProperty(n, getOverlay().getProperty(deprecatedKey));
}
}
return names.toArray(new String[names.size()]);
}
private void handleDeprecation() {
@ -595,8 +656,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
* or null if no such property exists.
*/
public String get(String name) {
name = handleDeprecation(name);
return substituteVars(getProps().getProperty(name));
String[] names = handleDeprecation(name);
String result = null;
for(String n : names) {
result = substituteVars(getProps().getProperty(n));
}
return result;
}
/**
@ -633,8 +698,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
* its replacing property and null if no such property exists.
*/
public String getRaw(String name) {
name = handleDeprecation(name);
return getProps().getProperty(name);
String[] names = handleDeprecation(name);
String result = null;
for(String n : names) {
result = getProps().getProperty(n);
}
return result;
}
/**
@ -652,10 +721,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
getOverlay().setProperty(name, value);
getProps().setProperty(name, value);
updatingResource.put(name, UNKNOWN_RESOURCE);
String altName = getAlternateName(name);
if (altName != null) {
getOverlay().setProperty(altName, value);
getProps().setProperty(altName, value);
String[] altNames = getAlternateNames(name);
if (altNames != null && altNames.length > 0) {
for(String altName : altNames) {
getOverlay().setProperty(altName, value);
getProps().setProperty(altName, value);
}
}
warnOnceIfDeprecated(name);
}
@ -671,12 +742,14 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
* Unset a previously set property.
*/
public synchronized void unset(String name) {
String altName = getAlternateName(name);
String[] altNames = getAlternateNames(name);
getOverlay().remove(name);
getProps().remove(name);
if (altName !=null) {
getOverlay().remove(altName);
getProps().remove(altName);
if (altNames !=null && altNames.length > 0) {
for(String altName : altNames) {
getOverlay().remove(altName);
getProps().remove(altName);
}
}
}
@ -711,8 +784,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
* doesn't exist.
*/
public String get(String name, String defaultValue) {
name = handleDeprecation(name);
return substituteVars(getProps().getProperty(name, defaultValue));
String[] names = handleDeprecation(name);
String result = null;
for(String n : names) {
result = substituteVars(getProps().getProperty(n, defaultValue));
}
return result;
}
/**
@ -1236,6 +1313,29 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
final String address = get(name, defaultAddress);
return NetUtils.createSocketAddr(address, defaultPort, name);
}
/**
* Set the socket address for the <code>name</code> property as
* a <code>host:port</code>.
*/
public void setSocketAddr(String name, InetSocketAddress addr) {
set(name, NetUtils.getHostPortString(addr));
}
/**
* Set the socket address a client can use to connect for the
* <code>name</code> property as a <code>host:port</code>. The wildcard
* address is replaced with the local host's address.
* @param name property name.
* @param addr InetSocketAddress of a listener to store in the given property
* @return InetSocketAddress for clients to connect
*/
public InetSocketAddress updateConnectAddr(String name,
InetSocketAddress addr) {
final InetSocketAddress connectAddr = NetUtils.getConnectAddress(addr);
setSocketAddr(name, connectAddr);
return connectAddr;
}
/**
* Load a class by name.

View File

@ -350,20 +350,23 @@ public abstract class AbstractFileSystem {
}
}
String thisScheme = this.getUri().getScheme();
String thisAuthority = this.getUri().getAuthority();
String thisHost = this.getUri().getHost();
String thatHost = uri.getHost();
// Schemes and authorities must match.
// Schemes and hosts must match.
// Allow for null Authority for file:///
if (!thisScheme.equalsIgnoreCase(thatScheme) ||
(thisAuthority != null &&
!thisAuthority.equalsIgnoreCase(thatAuthority)) ||
(thisAuthority == null && thatAuthority != null)) {
(thisHost != null &&
!thisHost.equalsIgnoreCase(thatHost)) ||
(thisHost == null && thatHost != null)) {
throw new InvalidPathException("Wrong FS: " + path + ", expected: "
+ this.getUri());
}
// Ports must match, unless this FS instance is using the default port, in
// which case the port may be omitted from the given URI
int thisPort = this.getUri().getPort();
int thatPort = path.toUri().getPort();
int thatPort = uri.getPort();
if (thatPort == -1) { // -1 => defaultPort of Uri scheme
thatPort = this.getUriDefaultPort();
}

View File

@ -54,6 +54,7 @@ import org.apache.hadoop.fs.InvalidPathException;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.ShutdownHookManager;
/**
* The FileContext class provides an interface to the application writer for
@ -171,7 +172,12 @@ public final class FileContext {
public static final Log LOG = LogFactory.getLog(FileContext.class);
public static final FsPermission DEFAULT_PERM = FsPermission.getDefault();
/**
* Priority of the FileContext shutdown hook.
*/
public static final int SHUTDOWN_HOOK_PRIORITY = 20;
/**
* List of files that should be deleted on JVM shutdown.
*/
@ -1456,8 +1462,8 @@ public final class FileContext {
return false;
}
synchronized (DELETE_ON_EXIT) {
if (DELETE_ON_EXIT.isEmpty() && !FINALIZER.isAlive()) {
Runtime.getRuntime().addShutdownHook(FINALIZER);
if (DELETE_ON_EXIT.isEmpty()) {
ShutdownHookManager.get().addShutdownHook(FINALIZER, SHUTDOWN_HOOK_PRIORITY);
}
Set<Path> set = DELETE_ON_EXIT.get(this);
@ -2215,7 +2221,7 @@ public final class FileContext {
/**
* Deletes all the paths in deleteOnExit on JVM shutdown.
*/
static class FileContextFinalizer extends Thread {
static class FileContextFinalizer implements Runnable {
public synchronized void run() {
processDeleteOnExit();
}

View File

@ -32,6 +32,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.ServiceLoader;
import java.util.Set;
import java.util.Stack;
import java.util.TreeSet;
@ -54,6 +55,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.ShutdownHookManager;
/****************************************************************
* An abstract base class for a fairly generic filesystem. It
@ -83,6 +85,11 @@ public abstract class FileSystem extends Configured implements Closeable {
public static final Log LOG = LogFactory.getLog(FileSystem.class);
/**
* Priority of the FileSystem shutdown hook.
*/
public static final int SHUTDOWN_HOOK_PRIORITY = 10;
/** FileSystem cache */
static final Cache CACHE = new Cache();
@ -184,6 +191,17 @@ public abstract class FileSystem extends Configured implements Closeable {
statistics = getStatistics(name.getScheme(), getClass());
}
/**
* Return the protocol scheme for the FileSystem.
* <p/>
* This implementation throws an <code>UnsupportedOperationException</code>.
*
* @return the protocol scheme for the FileSystem.
*/
public String getScheme() {
throw new UnsupportedOperationException("Not implemented by the FileSystem implementation");
}
/** Returns a URI whose scheme and authority identify this FileSystem.*/
public abstract URI getUri();
@ -2078,9 +2096,45 @@ public abstract class FileSystem extends Configured implements Closeable {
) throws IOException {
}
// making it volatile to be able to do a double checked locking
private volatile static boolean FILE_SYSTEMS_LOADED = false;
private static final Map<String, Class<? extends FileSystem>>
SERVICE_FILE_SYSTEMS = new HashMap<String, Class<? extends FileSystem>>();
private static void loadFileSystems() {
synchronized (FileSystem.class) {
if (!FILE_SYSTEMS_LOADED) {
ServiceLoader<FileSystem> serviceLoader = ServiceLoader.load(FileSystem.class);
for (FileSystem fs : serviceLoader) {
SERVICE_FILE_SYSTEMS.put(fs.getScheme(), fs.getClass());
}
FILE_SYSTEMS_LOADED = true;
}
}
}
public static Class<? extends FileSystem> getFileSystemClass(String scheme,
Configuration conf) throws IOException {
if (!FILE_SYSTEMS_LOADED) {
loadFileSystems();
}
Class<? extends FileSystem> clazz = null;
if (conf != null) {
clazz = (Class<? extends FileSystem>) conf.getClass("fs." + scheme + ".impl", null);
}
if (clazz == null) {
clazz = SERVICE_FILE_SYSTEMS.get(scheme);
}
if (clazz == null) {
throw new IOException("No FileSystem for scheme: " + scheme);
}
return clazz;
}
private static FileSystem createFileSystem(URI uri, Configuration conf
) throws IOException {
Class<?> clazz = conf.getClass("fs." + uri.getScheme() + ".impl", null);
Class<?> clazz = getFileSystemClass(uri.getScheme(), conf);
if (clazz == null) {
throw new IOException("No FileSystem for scheme: " + uri.getScheme());
}
@ -2128,8 +2182,8 @@ public abstract class FileSystem extends Configured implements Closeable {
}
// now insert the new file system into the map
if (map.isEmpty() && !clientFinalizer.isAlive()) {
Runtime.getRuntime().addShutdownHook(clientFinalizer);
if (map.isEmpty() ) {
ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY);
}
fs.key = key;
map.put(key, fs);
@ -2144,13 +2198,7 @@ public abstract class FileSystem extends Configured implements Closeable {
if (map.containsKey(key) && fs == map.get(key)) {
map.remove(key);
toAutoClose.remove(key);
if (map.isEmpty() && !clientFinalizer.isAlive()) {
if (!Runtime.getRuntime().removeShutdownHook(clientFinalizer)) {
LOG.info("Could not cancel cleanup thread, though no " +
"FileSystems are open");
}
}
}
}
synchronized void closeAll() throws IOException {
@ -2194,7 +2242,7 @@ public abstract class FileSystem extends Configured implements Closeable {
}
}
private class ClientFinalizer extends Thread {
private class ClientFinalizer implements Runnable {
public synchronized void run() {
try {
closeAll(true);

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.fs;
import java.io.IOException;
import java.net.URLStreamHandlerFactory;
import java.util.HashMap;
import java.util.Map;
@ -50,25 +51,23 @@ public class FsUrlStreamHandlerFactory implements
private java.net.URLStreamHandler handler;
public FsUrlStreamHandlerFactory() {
this.conf = new Configuration();
// force the resolution of the configuration files
// this is required if we want the factory to be able to handle
// file:// URLs
this.conf.getClass("fs.file.impl", null);
this.handler = new FsUrlStreamHandler(this.conf);
this(new Configuration());
}
public FsUrlStreamHandlerFactory(Configuration conf) {
this.conf = new Configuration(conf);
// force the resolution of the configuration files
this.conf.getClass("fs.file.impl", null);
this.handler = new FsUrlStreamHandler(this.conf);
}
public java.net.URLStreamHandler createURLStreamHandler(String protocol) {
if (!protocols.containsKey(protocol)) {
boolean known =
(conf.getClass("fs." + protocol + ".impl", null) != null);
boolean known = true;
try {
FileSystem.getFileSystemClass(protocol, conf);
}
catch (IOException ex) {
known = false;
}
protocols.put(protocol, known);
}
if (protocols.get(protocol)) {

View File

@ -71,7 +71,18 @@ public class HarFileSystem extends FilterFileSystem {
*/
public HarFileSystem() {
}
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>har</code>
*/
@Override
public String getScheme() {
return "har";
}
/**
* Constructor to create a HarFileSystem with an
* underlying filesystem.

View File

@ -39,7 +39,18 @@ public class LocalFileSystem extends ChecksumFileSystem {
public LocalFileSystem() {
this(new RawLocalFileSystem());
}
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>file</code>
*/
@Override
public String getScheme() {
return "file";
}
public FileSystem getRaw() {
return getRawFileSystem();
}

View File

@ -59,6 +59,17 @@ public class FTPFileSystem extends FileSystem {
private URI uri;
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>ftp</code>
*/
@Override
public String getScheme() {
return "ftp";
}
@Override
public void initialize(URI uri, Configuration conf) throws IOException { // get
super.initialize(uri, conf);

View File

@ -57,6 +57,17 @@ public class KosmosFileSystem extends FileSystem {
this.kfsImpl = fsimpl;
}
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>kfs</code>
*/
@Override
public String getScheme() {
return "kfs";
}
@Override
public URI getUri() {
return uri;

View File

@ -67,6 +67,17 @@ public class S3FileSystem extends FileSystem {
this.store = store;
}
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>s3</code>
*/
@Override
public String getScheme() {
return "s3";
}
@Override
public URI getUri() {
return uri;

View File

@ -251,7 +251,18 @@ public class NativeS3FileSystem extends FileSystem {
public NativeS3FileSystem(NativeFileSystemStore store) {
this.store = store;
}
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>s3n</code>
*/
@Override
public String getScheme() {
return "s3n";
}
@Override
public void initialize(URI uri, Configuration conf) throws IOException {
super.initialize(uri, conf);

View File

@ -149,6 +149,17 @@ public class ViewFileSystem extends FileSystem {
creationTime = System.currentTimeMillis();
}
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>viewfs</code>
*/
@Override
public String getScheme() {
return "viewfs";
}
/**
* Called after a new FileSystem instance is constructed.
* @param theUri a uri whose authority section names the host, port, etc. for

View File

@ -52,7 +52,7 @@ public interface FenceMethod {
/**
* Attempt to fence the target node.
* @param serviceAddr the address (host:ipcport) of the service to fence
* @param target the target of the service to fence
* @param args the configured arguments, which were checked at startup by
* {@link #checkArgs(String)}
* @return true if fencing was successful, false if unsuccessful or

View File

@ -144,12 +144,13 @@ public interface HAServiceProtocol {
/**
* Return the current status of the service. The status indicates
* the current <em>state</em> (e.g ACTIVE/STANDBY) as well as
* some additional information. {@see HAServiceStatus}
* some additional information.
*
* @throws AccessControlException
* if access is denied.
* @throws IOException
* if other errors happen
* @see HAServiceStatus
*/
public HAServiceStatus getServiceStatus() throws AccessControlException,
IOException;

View File

@ -636,80 +636,16 @@ public class HttpServer implements FilterContainer {
*/
public void start() throws IOException {
try {
if(listenerStartedExternally) { // Expect that listener was started securely
if(listener.getLocalPort() == -1) // ... and verify
throw new Exception("Exepected webserver's listener to be started " +
"previously but wasn't");
// And skip all the port rolling issues.
try {
openListener();
LOG.info("Jetty bound to port " + listener.getLocalPort());
webServer.start();
} else {
int port = 0;
int oriPort = listener.getPort(); // The original requested port
while (true) {
try {
port = webServer.getConnectors()[0].getLocalPort();
LOG.debug("Port returned by webServer.getConnectors()[0]." +
"getLocalPort() before open() is "+ port +
". Opening the listener on " + oriPort);
listener.open();
port = listener.getLocalPort();
LOG.debug("listener.getLocalPort() returned " + listener.getLocalPort() +
" webServer.getConnectors()[0].getLocalPort() returned " +
webServer.getConnectors()[0].getLocalPort());
//Workaround to handle the problem reported in HADOOP-4744
if (port < 0) {
Thread.sleep(100);
int numRetries = 1;
while (port < 0) {
LOG.warn("listener.getLocalPort returned " + port);
if (numRetries++ > MAX_RETRIES) {
throw new Exception(" listener.getLocalPort is returning " +
"less than 0 even after " +numRetries+" resets");
}
for (int i = 0; i < 2; i++) {
LOG.info("Retrying listener.getLocalPort()");
port = listener.getLocalPort();
if (port > 0) {
break;
}
Thread.sleep(200);
}
if (port > 0) {
break;
}
LOG.info("Bouncing the listener");
listener.close();
Thread.sleep(1000);
listener.setPort(oriPort == 0 ? 0 : (oriPort += 1));
listener.open();
Thread.sleep(100);
port = listener.getLocalPort();
}
} //Workaround end
LOG.info("Jetty bound to port " + port);
webServer.start();
break;
} catch (IOException ex) {
// if this is a bind exception,
// then try the next port number.
if (ex instanceof BindException) {
if (!findPort) {
BindException be = new BindException(
"Port in use: " + listener.getHost()
+ ":" + listener.getPort());
be.initCause(ex);
throw be;
}
} else {
LOG.info("HttpServer.start() threw a non Bind IOException");
throw ex;
}
} catch (MultiException ex) {
LOG.info("HttpServer.start() threw a MultiException");
throw ex;
}
listener.setPort((oriPort += 1));
}
} catch (IOException ex) {
LOG.info("HttpServer.start() threw a non Bind IOException", ex);
throw ex;
} catch (MultiException ex) {
LOG.info("HttpServer.start() threw a MultiException", ex);
throw ex;
}
// Make sure there is no handler failures.
Handler[] handlers = webServer.getHandlers();
@ -729,6 +665,52 @@ public class HttpServer implements FilterContainer {
}
}
/**
* Open the main listener for the server
* @throws Exception
*/
void openListener() throws Exception {
if (listener.getLocalPort() != -1) { // it's already bound
return;
}
if (listenerStartedExternally) { // Expect that listener was started securely
throw new Exception("Expected webserver's listener to be started " +
"previously but wasn't");
}
int port = listener.getPort();
while (true) {
// jetty has a bug where you can't reopen a listener that previously
// failed to open w/o issuing a close first, even if the port is changed
try {
listener.close();
listener.open();
break;
} catch (BindException ex) {
if (port == 0 || !findPort) {
BindException be = new BindException(
"Port in use: " + listener.getHost() + ":" + listener.getPort());
be.initCause(ex);
throw be;
}
}
// try the next port number
listener.setPort(++port);
Thread.sleep(100);
}
}
/**
* Return the bind address of the listener.
* @return InetSocketAddress of the listener
*/
public InetSocketAddress getListenerAddress() {
int port = listener.getLocalPort();
if (port == -1) { // not bound, return requested port
port = listener.getPort();
}
return new InetSocketAddress(listener.getHost(), port);
}
/**
* stop the server
*/
@ -821,7 +803,10 @@ public class HttpServer implements FilterContainer {
String remoteUser = request.getRemoteUser();
if (remoteUser == null) {
return true;
response.sendError(HttpServletResponse.SC_UNAUTHORIZED,
"Unauthenticated users are not " +
"authorized to access this page.");
return false;
}
AccessControlList adminsAcl = (AccessControlList) servletContext
.getAttribute(ADMINS_ACL);
@ -830,9 +815,7 @@ public class HttpServer implements FilterContainer {
if (adminsAcl != null) {
if (!adminsAcl.isUserAllowed(remoteUserUGI)) {
response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User "
+ remoteUser + " is unauthorized to access this page. "
+ "AccessControlList for accessing this page : "
+ adminsAcl.toString());
+ remoteUser + " is unauthorized to access this page.");
return false;
}
}

View File

@ -1050,9 +1050,9 @@ public class SequenceFile {
int bufferSize = bufferSizeOption == null ? getBufferSize(conf) :
bufferSizeOption.getValue();
short replication = replicationOption == null ?
fs.getDefaultReplication() :
fs.getDefaultReplication(p) :
(short) replicationOption.getValue();
long blockSize = blockSizeOption == null ? fs.getDefaultBlockSize() :
long blockSize = blockSizeOption == null ? fs.getDefaultBlockSize(p) :
blockSizeOption.getValue();
Progressable progress = progressOption == null ? null :
progressOption.getValue();

View File

@ -326,9 +326,41 @@ public final class WritableUtils {
* @return deserialized integer from stream.
*/
public static int readVInt(DataInput stream) throws IOException {
return (int) readVLong(stream);
long n = readVLong(stream);
if ((n > Integer.MAX_VALUE) || (n < Integer.MIN_VALUE)) {
throw new IOException("value too long to fit in integer");
}
return (int)n;
}
/**
* Reads an integer from the input stream and returns it.
*
* This function validates that the integer is between [lower, upper],
* inclusive.
*
* @param stream Binary input stream
* @throws java.io.IOException
* @return deserialized integer from stream
*/
public static int readVIntInRange(DataInput stream, int lower, int upper)
throws IOException {
long n = readVLong(stream);
if (n < lower) {
if (lower == 0) {
throw new IOException("expected non-negative integer, got " + n);
} else {
throw new IOException("expected integer greater than or equal to " +
lower + ", got " + n);
}
}
if (n > upper) {
throw new IOException("expected integer less or equal to " + upper +
", got " + n);
}
return (int)n;
}
/**
* Given the first byte of a vint/vlong, determine the sign
* @param value the first byte

View File

@ -36,6 +36,9 @@ public class CompressionCodecFactory {
public static final Log LOG =
LogFactory.getLog(CompressionCodecFactory.class.getName());
private static final ServiceLoader<CompressionCodec> CODEC_PROVIDERS =
ServiceLoader.load(CompressionCodec.class);
/**
* A map from the reversed filename suffixes to the codecs.
@ -95,16 +98,23 @@ public class CompressionCodecFactory {
}
/**
* Get the list of codecs listed in the configuration
* Get the list of codecs discovered via a Java ServiceLoader, or
* listed in the configuration. Codecs specified in configuration come
* later in the returned list, and are considered to override those
* from the ServiceLoader.
* @param conf the configuration to look in
* @return a list of the Configuration classes or null if the attribute
* was not set
* @return a list of the {@link CompressionCodec} classes
*/
public static List<Class<? extends CompressionCodec>> getCodecClasses(Configuration conf) {
List<Class<? extends CompressionCodec>> result
= new ArrayList<Class<? extends CompressionCodec>>();
// Add codec classes discovered via service loading
for (CompressionCodec codec : CODEC_PROVIDERS) {
result.add(codec.getClass());
}
// Add codec classes from configuration
String codecsString = conf.get("io.compression.codecs");
if (codecsString != null) {
List<Class<? extends CompressionCodec>> result
= new ArrayList<Class<? extends CompressionCodec>>();
StringTokenizer codecSplit = new StringTokenizer(codecsString, ",");
while (codecSplit.hasMoreElements()) {
String codecSubstring = codecSplit.nextToken();
@ -123,14 +133,14 @@ public class CompressionCodecFactory {
}
}
}
return result;
} else {
return null;
}
return result;
}
/**
* Sets a list of codec classes in the configuration.
* Sets a list of codec classes in the configuration. In addition to any
* classes specified using this method, {@link CompressionCodec} classes on
* the classpath are discovered using a Java ServiceLoader.
* @param conf the configuration to modify
* @param classes the list of classes to set
*/
@ -151,21 +161,19 @@ public class CompressionCodecFactory {
/**
* Find the codecs specified in the config value io.compression.codecs
* and register them. Defaults to gzip and zip.
* and register them. Defaults to gzip and deflate.
*/
public CompressionCodecFactory(Configuration conf) {
codecs = new TreeMap<String, CompressionCodec>();
codecsByClassName = new HashMap<String, CompressionCodec>();
codecsByName = new HashMap<String, CompressionCodec>();
List<Class<? extends CompressionCodec>> codecClasses = getCodecClasses(conf);
if (codecClasses == null) {
if (codecClasses == null || codecClasses.isEmpty()) {
addCodec(new GzipCodec());
addCodec(new DefaultCodec());
} else {
Iterator<Class<? extends CompressionCodec>> itr = codecClasses.iterator();
while (itr.hasNext()) {
CompressionCodec codec = ReflectionUtils.newInstance(itr.next(), conf);
addCodec(codec);
for (Class<? extends CompressionCodec> codecClass : codecClasses) {
addCodec(ReflectionUtils.newInstance(codecClass, conf));
}
}
}

View File

@ -50,8 +50,9 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.ipc.RpcPayloadHeader.*;
import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.RpcPayloadHeaderProto;
import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.RpcPayloadOperationProto;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
@ -163,10 +164,10 @@ public class Client {
final Writable rpcRequest; // the serialized rpc request - RpcPayload
Writable rpcResponse; // null if rpc has error
IOException error; // exception, null if success
final RpcKind rpcKind; // Rpc EngineKind
final RPC.RpcKind rpcKind; // Rpc EngineKind
boolean done; // true when call is done
protected Call(RpcKind rpcKind, Writable param) {
protected Call(RPC.RpcKind rpcKind, Writable param) {
this.rpcKind = rpcKind;
this.rpcRequest = param;
synchronized (Client.this) {
@ -613,7 +614,7 @@ public class Client {
this.in = new DataInputStream(new BufferedInputStream(inStream));
}
this.out = new DataOutputStream(new BufferedOutputStream(outStream));
writeHeader();
writeConnectionContext();
// update last activity time
touch();
@ -704,16 +705,17 @@ public class Client {
out.flush();
}
/* Write the protocol header for each connection
/* Write the connection context header for each connection
* Out is not synchronized because only the first thread does this.
*/
private void writeHeader() throws IOException {
private void writeConnectionContext() throws IOException {
// Write out the ConnectionHeader
DataOutputBuffer buf = new DataOutputBuffer();
connectionContext.writeTo(buf);
// Write out the payload length
int bufLen = buf.getLength();
out.writeInt(bufLen);
out.write(buf.getData(), 0, bufLen);
}
@ -806,21 +808,22 @@ public class Client {
if (LOG.isDebugEnabled())
LOG.debug(getName() + " sending #" + call.id);
//for serializing the
//data to be written
// Serializing the data to be written.
// Format:
// 0) Length of rest below (1 + 2)
// 1) PayloadHeader - is serialized Delimited hence contains length
// 2) the Payload - the RpcRequest
//
d = new DataOutputBuffer();
d.writeInt(0); // placeholder for data length
RpcPayloadHeader header = new RpcPayloadHeader(
call.rpcKind, RpcPayloadOperation.RPC_FINAL_PAYLOAD, call.id);
header.write(d);
RpcPayloadHeaderProto header = ProtoUtil.makeRpcPayloadHeader(
call.rpcKind, RpcPayloadOperationProto.RPC_FINAL_PAYLOAD, call.id);
header.writeDelimitedTo(d);
call.rpcRequest.write(d);
byte[] data = d.getData();
int dataLength = d.getLength() - 4;
data[0] = (byte)((dataLength >>> 24) & 0xff);
data[1] = (byte)((dataLength >>> 16) & 0xff);
data[2] = (byte)((dataLength >>> 8) & 0xff);
data[3] = (byte)(dataLength & 0xff);
out.write(data, 0, dataLength + 4);//write the data
int totalLength = d.getLength();
out.writeInt(totalLength); // Total Length
out.write(data, 0, totalLength);//PayloadHeader + RpcRequest
out.flush();
}
} catch(IOException e) {
@ -937,7 +940,7 @@ public class Client {
private int index;
public ParallelCall(Writable param, ParallelResults results, int index) {
super(RpcKind.RPC_WRITABLE, param);
super(RPC.RpcKind.RPC_WRITABLE, param);
this.results = results;
this.index = index;
}
@ -1022,22 +1025,22 @@ public class Client {
}
/**
* Same as {@link #call(RpcPayloadHeader.RpcKind, Writable, ConnectionId)}
* Same as {@link #call(RPC.RpcKind, Writable, ConnectionId)}
* for RPC_BUILTIN
*/
public Writable call(Writable param, InetSocketAddress address)
throws InterruptedException, IOException {
return call(RpcKind.RPC_BUILTIN, param, address);
return call(RPC.RpcKind.RPC_BUILTIN, param, address);
}
/** Make a call, passing <code>param</code>, to the IPC server running at
* <code>address</code>, returning the value. Throws exceptions if there are
* network problems or if the remote code threw an exception.
* @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
* @deprecated Use {@link #call(RPC.RpcKind, Writable,
* ConnectionId)} instead
*/
@Deprecated
public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress address)
public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress address)
throws InterruptedException, IOException {
return call(rpcKind, param, address, null);
}
@ -1047,11 +1050,11 @@ public class Client {
* the value.
* Throws exceptions if there are network problems or if the remote code
* threw an exception.
* @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
* @deprecated Use {@link #call(RPC.RpcKind, Writable,
* ConnectionId)} instead
*/
@Deprecated
public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr,
public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress addr,
UserGroupInformation ticket)
throws InterruptedException, IOException {
ConnectionId remoteId = ConnectionId.getConnectionId(addr, null, ticket, 0,
@ -1065,11 +1068,11 @@ public class Client {
* timeout, returning the value.
* Throws exceptions if there are network problems or if the remote code
* threw an exception.
* @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
* @deprecated Use {@link #call(RPC.RpcKind, Writable,
* ConnectionId)} instead
*/
@Deprecated
public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr,
public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress addr,
Class<?> protocol, UserGroupInformation ticket,
int rpcTimeout)
throws InterruptedException, IOException {
@ -1080,7 +1083,7 @@ public class Client {
/**
* Same as {@link #call(RpcPayloadHeader.RpcKind, Writable, InetSocketAddress,
* Same as {@link #call(RPC.RpcKind, Writable, InetSocketAddress,
* Class, UserGroupInformation, int, Configuration)}
* except that rpcKind is writable.
*/
@ -1090,7 +1093,7 @@ public class Client {
throws InterruptedException, IOException {
ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
ticket, rpcTimeout, conf);
return call(RpcKind.RPC_BUILTIN, param, remoteId);
return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
}
/**
@ -1101,7 +1104,7 @@ public class Client {
* value. Throws exceptions if there are network problems or if the remote
* code threw an exception.
*/
public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr,
public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress addr,
Class<?> protocol, UserGroupInformation ticket,
int rpcTimeout, Configuration conf)
throws InterruptedException, IOException {
@ -1111,12 +1114,12 @@ public class Client {
}
/**
* Same as {link {@link #call(RpcPayloadHeader.RpcKind, Writable, ConnectionId)}
* Same as {link {@link #call(RPC.RpcKind, Writable, ConnectionId)}
* except the rpcKind is RPC_BUILTIN
*/
public Writable call(Writable param, ConnectionId remoteId)
throws InterruptedException, IOException {
return call(RpcKind.RPC_BUILTIN, param, remoteId);
return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
}
/**
@ -1130,7 +1133,7 @@ public class Client {
* Throws exceptions if there are network problems or if the remote code
* threw an exception.
*/
public Writable call(RpcKind rpcKind, Writable rpcRequest,
public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
ConnectionId remoteId) throws InterruptedException, IOException {
Call call = new Call(rpcKind, rpcRequest);
Connection connection = getConnection(remoteId, call);

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.io.DataOutputOutputStream;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.ipc.Client.ConnectionId;
import org.apache.hadoop.ipc.RPC.RpcInvoker;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcRequestProto;
import org.apache.hadoop.security.UserGroupInformation;
@ -61,7 +60,7 @@ public class ProtobufRpcEngine implements RpcEngine {
static { // Register the rpcRequest deserializer for WritableRpcEngine
org.apache.hadoop.ipc.Server.registerProtocolEngine(
RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWritable.class,
RPC.RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWritable.class,
new Server.ProtoBufRpcInvoker());
}
@ -182,7 +181,7 @@ public class ProtobufRpcEngine implements RpcEngine {
HadoopRpcRequestProto rpcRequest = constructRpcRequest(method, args);
RpcResponseWritable val = null;
try {
val = (RpcResponseWritable) client.call(RpcKind.RPC_PROTOCOL_BUFFER,
val = (RpcResponseWritable) client.call(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
new RpcRequestWritable(rpcRequest), remoteId);
} catch (Throwable e) {
throw new ServiceException(e);
@ -351,7 +350,7 @@ public class ProtobufRpcEngine implements RpcEngine {
numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
.getClass().getName()), secretManager, portRangeConfig);
this.verbose = verbose;
registerProtocolAndImpl(RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
registerProtocolAndImpl(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
protocolImpl);
}
@ -363,10 +362,10 @@ public class ProtobufRpcEngine implements RpcEngine {
String protoName, long version) throws IOException {
ProtoNameVer pv = new ProtoNameVer(protoName, version);
ProtoClassProtoImpl impl =
server.getProtocolImplMap(RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
server.getProtocolImplMap(RPC.RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
if (impl == null) { // no match for Protocol AND Version
VerProtocolImpl highest =
server.getHighestSupportedProtocol(RpcKind.RPC_PROTOCOL_BUFFER,
server.getHighestSupportedProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
protoName);
if (highest == null) {
throw new IOException("Unknown protocol: " + protoName);

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.ipc;
import org.apache.hadoop.ipc.RPC.Server.VerProtocolImpl;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolVersionsRequestProto;
@ -49,7 +48,7 @@ public class ProtocolMetaInfoServerSideTranslatorPB implements
String protocol = request.getProtocol();
GetProtocolVersionsResponseProto.Builder builder =
GetProtocolVersionsResponseProto.newBuilder();
for (RpcKind r : RpcKind.values()) {
for (RPC.RpcKind r : RPC.RpcKind.values()) {
long[] versions;
try {
versions = getProtocolVersionForRpcKind(r, protocol);
@ -78,7 +77,7 @@ public class ProtocolMetaInfoServerSideTranslatorPB implements
String rpcKind = request.getRpcKind();
long[] versions;
try {
versions = getProtocolVersionForRpcKind(RpcKind.valueOf(rpcKind),
versions = getProtocolVersionForRpcKind(RPC.RpcKind.valueOf(rpcKind),
protocol);
} catch (ClassNotFoundException e1) {
throw new ServiceException(e1);
@ -104,7 +103,7 @@ public class ProtocolMetaInfoServerSideTranslatorPB implements
return builder.build();
}
private long[] getProtocolVersionForRpcKind(RpcKind rpcKind,
private long[] getProtocolVersionForRpcKind(RPC.RpcKind rpcKind,
String protocol) throws ClassNotFoundException {
Class<?> protocolClass = Class.forName(protocol);
String protocolName = RPC.getProtocolName(protocolClass);

View File

@ -42,7 +42,6 @@ import org.apache.commons.logging.*;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.io.*;
import org.apache.hadoop.ipc.Client.ConnectionId;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolInfoService;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SaslRpcServer;
@ -73,6 +72,18 @@ import com.google.protobuf.BlockingService;
* the protocol instance is transmitted.
*/
public class RPC {
public enum RpcKind {
RPC_BUILTIN ((short) 1), // Used for built in calls by tests
RPC_WRITABLE ((short) 2), // Use WritableRpcEngine
RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
private static final short FIRST_INDEX = RPC_BUILTIN.value;
public final short value; //TODO make it private
RpcKind(short val) {
this.value = val;
}
}
interface RpcInvoker {
/**
@ -777,7 +788,7 @@ public class RPC {
ArrayList<Map<ProtoNameVer, ProtoClassProtoImpl>> protocolImplMapArray =
new ArrayList<Map<ProtoNameVer, ProtoClassProtoImpl>>(RpcKind.MAX_INDEX);
Map<ProtoNameVer, ProtoClassProtoImpl> getProtocolImplMap(RpcKind rpcKind) {
Map<ProtoNameVer, ProtoClassProtoImpl> getProtocolImplMap(RPC.RpcKind rpcKind) {
if (protocolImplMapArray.size() == 0) {// initialize for all rpc kinds
for (int i=0; i <= RpcKind.MAX_INDEX; ++i) {
protocolImplMapArray.add(
@ -821,7 +832,7 @@ public class RPC {
@SuppressWarnings("unused") // will be useful later.
VerProtocolImpl[] getSupportedProtocolVersions(RpcKind rpcKind,
VerProtocolImpl[] getSupportedProtocolVersions(RPC.RpcKind rpcKind,
String protocolName) {
VerProtocolImpl[] resultk =
new VerProtocolImpl[getProtocolImplMap(rpcKind).size()];
@ -900,7 +911,7 @@ public class RPC {
}
@Override
public Writable call(RpcKind rpcKind, String protocol,
public Writable call(RPC.RpcKind rpcKind, String protocol,
Writable rpcRequest, long receiveTime) throws Exception {
return getRpcInvoker(rpcKind).call(this, protocol, rpcRequest,
receiveTime);

View File

@ -27,7 +27,6 @@ import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
@ -107,7 +106,7 @@ public class RpcClientUtil {
* @throws IOException
*/
public static boolean isMethodSupported(Object rpcProxy, Class<?> protocol,
RpcKind rpcKind, long version, String methodName) throws IOException {
RPC.RpcKind rpcKind, long version, String methodName) throws IOException {
InetSocketAddress serverAddress = RPC.getServerAddress(rpcProxy);
Map<Long, ProtocolSignature> versionMap = getVersionSignatureMap(
serverAddress, protocol.getName(), rpcKind.toString());

View File

@ -1,118 +0,0 @@
package org.apache.hadoop.ipc;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
/**
* This is the rpc payload header. It is sent with every rpc call
* <pre>
* The format of RPC call is as follows:
* +---------------------------------------------------+
* | Rpc length in bytes (header + payload length) |
* +---------------------------------------------------+
* | Rpc Header | Rpc Payload |
* +---------------------------------------------------+
*
* The format of Rpc Header is:
* +----------------------------------+
* | RpcKind (1 bytes) |
* +----------------------------------+
* | RpcPayloadOperation (1 bytes) |
* +----------------------------------+
* | Call ID (4 bytes) |
* +----------------------------------+
*
* {@link RpcKind} determines the type of serialization used for Rpc Payload.
* </pre>
* <p>
* <b>Note this header does NOT have its own version number,
* it used the version number from the connection header. </b>
*/
public class RpcPayloadHeader implements Writable {
public enum RpcPayloadOperation {
RPC_FINAL_PAYLOAD ((short)1),
RPC_CONTINUATION_PAYLOAD ((short)2), // not implemented yet
RPC_CLOSE_CONNECTION ((short)3); // close the rpc connection
private final short code;
private static final short FIRST_INDEX = RPC_FINAL_PAYLOAD.code;
RpcPayloadOperation(short val) {
this.code = val;
}
public void write(DataOutput out) throws IOException {
out.writeByte(code);
}
static RpcPayloadOperation readFields(DataInput in) throws IOException {
short inValue = in.readByte();
return RpcPayloadOperation.values()[inValue - FIRST_INDEX];
}
}
public enum RpcKind {
RPC_BUILTIN ((short) 1), // Used for built in calls by tests
RPC_WRITABLE ((short) 2), // Use WritableRpcEngine
RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
private static final short FIRST_INDEX = RPC_BUILTIN.value;
private final short value;
RpcKind(short val) {
this.value = val;
}
public void write(DataOutput out) throws IOException {
out.writeByte(value);
}
static RpcKind readFields(DataInput in) throws IOException {
short inValue = in.readByte();
return RpcKind.values()[inValue - FIRST_INDEX];
}
}
private RpcKind kind;
private RpcPayloadOperation operation;
private int callId;
public RpcPayloadHeader() {
kind = RpcKind.RPC_WRITABLE;
operation = RpcPayloadOperation.RPC_CLOSE_CONNECTION;
}
public RpcPayloadHeader(RpcKind kind, RpcPayloadOperation op, int callId) {
this.kind = kind;
this.operation = op;
this.callId = callId;
}
int getCallId() {
return callId;
}
RpcKind getkind() {
return kind;
}
RpcPayloadOperation getOperation() {
return operation;
}
@Override
public void write(DataOutput out) throws IOException {
kind.write(out);
operation.write(out);
out.writeInt(callId);
}
@Override
public void readFields(DataInput in) throws IOException {
kind = RpcKind.readFields(in);
operation = RpcPayloadOperation.readFields(in);
this.callId = in.readInt();
}
}

View File

@ -72,11 +72,10 @@ import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.ipc.RPC.RpcInvoker;
import org.apache.hadoop.ipc.RPC.VersionMismatch;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcPayloadOperation;
import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
import org.apache.hadoop.ipc.metrics.RpcMetrics;
import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.*;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.SaslRpcServer;
@ -170,8 +169,8 @@ public abstract class Server {
this.rpcRequestWrapperClass = rpcRequestWrapperClass;
}
}
static Map<RpcKind, RpcKindMapValue> rpcKindMap = new
HashMap<RpcKind, RpcKindMapValue>(4);
static Map<RPC.RpcKind, RpcKindMapValue> rpcKindMap = new
HashMap<RPC.RpcKind, RpcKindMapValue>(4);
@ -185,7 +184,7 @@ public abstract class Server {
* @param rpcInvoker - use to process the calls on SS.
*/
public static void registerProtocolEngine(RpcKind rpcKind,
public static void registerProtocolEngine(RPC.RpcKind rpcKind,
Class<? extends Writable> rpcRequestWrapperClass,
RpcInvoker rpcInvoker) {
RpcKindMapValue old =
@ -201,14 +200,14 @@ public abstract class Server {
}
public Class<? extends Writable> getRpcRequestWrapper(
RpcKind rpcKind) {
RpcKindProto rpcKind) {
if (rpcRequestClass != null)
return rpcRequestClass;
RpcKindMapValue val = rpcKindMap.get(rpcKind);
RpcKindMapValue val = rpcKindMap.get(ProtoUtil.convert(rpcKind));
return (val == null) ? null : val.rpcRequestWrapperClass;
}
public static RpcInvoker getRpcInvoker(RpcKind rpcKind) {
public static RpcInvoker getRpcInvoker(RPC.RpcKind rpcKind) {
RpcKindMapValue val = rpcKindMap.get(rpcKind);
return (val == null) ? null : val.rpcInvoker;
}
@ -403,12 +402,12 @@ public abstract class Server {
private long timestamp; // time received when response is null
// time served when response is not null
private ByteBuffer rpcResponse; // the response for this call
private final RpcKind rpcKind;
private final RPC.RpcKind rpcKind;
public Call(int id, Writable param, Connection connection) {
this( id, param, connection, RpcKind.RPC_BUILTIN );
this( id, param, connection, RPC.RpcKind.RPC_BUILTIN );
}
public Call(int id, Writable param, Connection connection, RpcKind kind) {
public Call(int id, Writable param, Connection connection, RPC.RpcKind kind) {
this.callId = id;
this.rpcRequest = param;
this.connection = connection;
@ -1366,7 +1365,6 @@ public abstract class Server {
if (data == null) {
dataLengthBuffer.flip();
dataLength = dataLengthBuffer.getInt();
if ((dataLength == Client.PING_CALL_ID) && (!useWrap)) {
// covers the !useSasl too
dataLengthBuffer.clear();
@ -1555,22 +1553,27 @@ public abstract class Server {
private void processData(byte[] buf) throws IOException, InterruptedException {
DataInputStream dis =
new DataInputStream(new ByteArrayInputStream(buf));
RpcPayloadHeader header = new RpcPayloadHeader();
header.readFields(dis); // Read the RpcPayload header
RpcPayloadHeaderProto header = RpcPayloadHeaderProto.parseDelimitedFrom(dis);
if (LOG.isDebugEnabled())
LOG.debug(" got #" + header.getCallId());
if (header.getOperation() != RpcPayloadOperation.RPC_FINAL_PAYLOAD) {
if (!header.hasRpcOp()) {
throw new IOException(" IPC Server: No rpc op in rpcPayloadHeader");
}
if (header.getRpcOp() != RpcPayloadOperationProto.RPC_FINAL_PAYLOAD) {
throw new IOException("IPC Server does not implement operation" +
header.getOperation());
header.getRpcOp());
}
// If we know the rpc kind, get its class so that we can deserialize
// (Note it would make more sense to have the handler deserialize but
// we continue with this original design.
if (!header.hasRpcKind()) {
throw new IOException(" IPC Server: No rpc kind in rpcPayloadHeader");
}
Class<? extends Writable> rpcRequestClass =
getRpcRequestWrapper(header.getkind());
getRpcRequestWrapper(header.getRpcKind());
if (rpcRequestClass == null) {
LOG.warn("Unknown rpc kind " + header.getkind() +
LOG.warn("Unknown rpc kind " + header.getRpcKind() +
" from client " + getHostAddress());
final Call readParamsFailedCall =
new Call(header.getCallId(), null, this);
@ -1578,7 +1581,7 @@ public abstract class Server {
setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, null,
IOException.class.getName(),
"Unknown rpc kind " + header.getkind());
"Unknown rpc kind " + header.getRpcKind());
responder.doRespond(readParamsFailedCall);
return;
}
@ -1589,7 +1592,7 @@ public abstract class Server {
} catch (Throwable t) {
LOG.warn("Unable to read call parameters for client " +
getHostAddress() + "on connection protocol " +
this.protocolName + " for rpcKind " + header.getkind(), t);
this.protocolName + " for rpcKind " + header.getRpcKind(), t);
final Call readParamsFailedCall =
new Call(header.getCallId(), null, this);
ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
@ -1601,7 +1604,8 @@ public abstract class Server {
return;
}
Call call = new Call(header.getCallId(), rpcRequest, this, header.getkind());
Call call = new Call(header.getCallId(), rpcRequest, this,
ProtoUtil.convert(header.getRpcKind()));
callQueue.put(call); // queue the call; maybe blocked here
incRpcCount(); // Increment the rpc count
}
@ -1772,7 +1776,7 @@ public abstract class Server {
* from configuration. Otherwise the configuration will be picked up.
*
* If rpcRequestClass is null then the rpcRequestClass must have been
* registered via {@link #registerProtocolEngine(RpcPayloadHeader.RpcKind,
* registered via {@link #registerProtocolEngine(RPC.RpcKind,
* Class, RPC.RpcInvoker)}
* This parameter has been retained for compatibility with existing tests
* and usage.
@ -1986,16 +1990,16 @@ public abstract class Server {
/**
* Called for each call.
* @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, String,
* @deprecated Use {@link #call(RPC.RpcKind, String,
* Writable, long)} instead
*/
@Deprecated
public Writable call(Writable param, long receiveTime) throws Exception {
return call(RpcKind.RPC_BUILTIN, null, param, receiveTime);
return call(RPC.RpcKind.RPC_BUILTIN, null, param, receiveTime);
}
/** Called for each call. */
public abstract Writable call(RpcKind rpcKind, String protocol,
public abstract Writable call(RPC.RpcKind rpcKind, String protocol,
Writable param, long receiveTime) throws Exception;
/**

View File

@ -33,7 +33,6 @@ import org.apache.commons.logging.*;
import org.apache.hadoop.io.*;
import org.apache.hadoop.ipc.Client.ConnectionId;
import org.apache.hadoop.ipc.RPC.RpcInvoker;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
@ -75,7 +74,7 @@ public class WritableRpcEngine implements RpcEngine {
* Register the rpcRequest deserializer for WritableRpcEngine
*/
private static synchronized void initialize() {
org.apache.hadoop.ipc.Server.registerProtocolEngine(RpcKind.RPC_WRITABLE,
org.apache.hadoop.ipc.Server.registerProtocolEngine(RPC.RpcKind.RPC_WRITABLE,
Invocation.class, new Server.WritableRpcInvoker());
isInitialized = true;
}
@ -223,7 +222,7 @@ public class WritableRpcEngine implements RpcEngine {
}
ObjectWritable value = (ObjectWritable)
client.call(RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId);
client.call(RPC.RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId);
if (LOG.isDebugEnabled()) {
long callTime = System.currentTimeMillis() - startTime;
LOG.debug("Call: " + method.getName() + " " + callTime);
@ -412,12 +411,12 @@ public class WritableRpcEngine implements RpcEngine {
protocolImpl.getClass());
}
// register protocol class and its super interfaces
registerProtocolAndImpl(RpcKind.RPC_WRITABLE, protocolClass, protocolImpl);
registerProtocolAndImpl(RPC.RpcKind.RPC_WRITABLE, protocolClass, protocolImpl);
protocols = RPC.getProtocolInterfaces(protocolClass);
}
for (Class<?> p : protocols) {
if (!p.equals(VersionedProtocol.class)) {
registerProtocolAndImpl(RpcKind.RPC_WRITABLE, p, protocolImpl);
registerProtocolAndImpl(RPC.RpcKind.RPC_WRITABLE, p, protocolImpl);
}
}
@ -461,7 +460,7 @@ public class WritableRpcEngine implements RpcEngine {
// registered directly.
// Send the call to the highest protocol version
VerProtocolImpl highest = server.getHighestSupportedProtocol(
RpcKind.RPC_WRITABLE, protocolName);
RPC.RpcKind.RPC_WRITABLE, protocolName);
if (highest == null) {
throw new IOException("Unknown protocol: " + protocolName);
}
@ -473,10 +472,10 @@ public class WritableRpcEngine implements RpcEngine {
ProtoNameVer pv =
new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
protocolImpl =
server.getProtocolImplMap(RpcKind.RPC_WRITABLE).get(pv);
server.getProtocolImplMap(RPC.RpcKind.RPC_WRITABLE).get(pv);
if (protocolImpl == null) { // no match for Protocol AND Version
VerProtocolImpl highest =
server.getHighestSupportedProtocol(RpcKind.RPC_WRITABLE,
server.getHighestSupportedProtocol(RPC.RpcKind.RPC_WRITABLE,
protoName);
if (highest == null) {
throw new IOException("Unknown protocol: " + protoName);

View File

@ -36,11 +36,9 @@ import org.apache.hadoop.util.ServletUtil;
*/
@InterfaceStability.Evolving
public class LogLevel {
public static final String USAGES = "\nUSAGES:\n"
+ "java " + LogLevel.class.getName()
+ " -getlevel <host:port> <name>\n"
+ "java " + LogLevel.class.getName()
+ " -setlevel <host:port> <name> <level>\n";
public static final String USAGES = "\nUsage: General options are:\n"
+ "\t[-getlevel <host:httpPort> <name>]\n"
+ "\t[-setlevel <host:httpPort> <name> <level>]\n";
/**
* A command line implementation

View File

@ -351,8 +351,19 @@ public class NetUtils {
* @return socket address that a client can use to connect to the server.
*/
public static InetSocketAddress getConnectAddress(Server server) {
InetSocketAddress addr = server.getListenerAddress();
if (addr.getAddress().isAnyLocalAddress()) {
return getConnectAddress(server.getListenerAddress());
}
/**
* Returns the InetSocketAddress that a client can use to connect to the
* given listening address. This returns "hostname:port" of the server,
* or "127.0.0.1:port" when given a wildcard address of "0.0.0.0:port".
*
* @param addr of a listener
* @return socket address that a client can use to connect to the server.
*/
public static InetSocketAddress getConnectAddress(InetSocketAddress addr) {
if (!addr.isUnresolved() && addr.getAddress().isAnyLocalAddress()) {
try {
addr = new InetSocketAddress(InetAddress.getLocalHost(), addr.getPort());
} catch (UnknownHostException uhe) {

View File

@ -220,6 +220,8 @@ public class SecurityUtil {
* @return converted Kerberos principal name
* @throws IOException if the client address cannot be determined
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static String getServerPrincipal(String principalConfig,
String hostname) throws IOException {
String[] components = getComponents(principalConfig);
@ -245,6 +247,8 @@ public class SecurityUtil {
* @return converted Kerberos principal name
* @throws IOException if the client address cannot be determined
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static String getServerPrincipal(String principalConfig,
InetAddress addr) throws IOException {
String[] components = getComponents(principalConfig);
@ -292,6 +296,8 @@ public class SecurityUtil {
* the key to look for user's Kerberos principal name in conf
* @throws IOException if login fails
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static void login(final Configuration conf,
final String keytabFileKey, final String userNameKey) throws IOException {
login(conf, keytabFileKey, userNameKey, getLocalHostName());
@ -312,6 +318,8 @@ public class SecurityUtil {
* hostname to use for substitution
* @throws IOException if the config doesn't specify a keytab
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static void login(final Configuration conf,
final String keytabFileKey, final String userNameKey, String hostname)
throws IOException {

View File

@ -69,7 +69,7 @@ import org.apache.hadoop.util.Shell;
* user's username and groups. It supports both the Windows, Unix and Kerberos
* login modules.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "HBase", "Hive", "Oozie"})
@InterfaceStability.Evolving
public class UserGroupInformation {
private static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
@ -258,6 +258,8 @@ public class UserGroupInformation {
* group look up service.
* @param conf the configuration to use
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static void setConfiguration(Configuration conf) {
initialize(conf, false);
}
@ -500,6 +502,8 @@ public class UserGroupInformation {
* @return the current user
* @throws IOException if login fails
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public synchronized
static UserGroupInformation getCurrentUser() throws IOException {
AccessControlContext context = AccessController.getContext();
@ -516,6 +520,8 @@ public class UserGroupInformation {
* @return the logged in user
* @throws IOException if login fails
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public synchronized
static UserGroupInformation getLoginUser() throws IOException {
if (loginUser == null) {
@ -652,6 +658,8 @@ public class UserGroupInformation {
* @param path the path to the keytab file
* @throws IOException if the keytab file can't be read
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public synchronized
static void loginUserFromKeytab(String user,
String path
@ -710,6 +718,8 @@ public class UserGroupInformation {
* the new credentials.
* @throws IOException on a failure
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public synchronized void reloginFromKeytab()
throws IOException {
if (!isSecurityEnabled() ||
@ -769,6 +779,8 @@ public class UserGroupInformation {
* the new credentials.
* @throws IOException on a failure
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public synchronized void reloginFromTicketCache()
throws IOException {
if (!isSecurityEnabled() ||
@ -867,6 +879,8 @@ public class UserGroupInformation {
* Did the login happen via keytab
* @return true or false
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public synchronized static boolean isLoginKeytabBased() throws IOException {
return getLoginUser().isKeytab;
}
@ -877,6 +891,8 @@ public class UserGroupInformation {
* @param user the full user principal name, must not be empty or null
* @return the UserGroupInformation for the remote user.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static UserGroupInformation createRemoteUser(String user) {
if (user == null || "".equals(user)) {
throw new IllegalArgumentException("Null user");
@ -891,6 +907,7 @@ public class UserGroupInformation {
/**
* existing types of authentications' methods
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static enum AuthenticationMethod {
SIMPLE,
@ -908,6 +925,8 @@ public class UserGroupInformation {
* @param realUser
* @return proxyUser ugi
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static UserGroupInformation createProxyUser(String user,
UserGroupInformation realUser) {
if (user == null || "".equals(user)) {
@ -929,6 +948,8 @@ public class UserGroupInformation {
* get RealUser (vs. EffectiveUser)
* @return realUser running over proxy user
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public UserGroupInformation getRealUser() {
for (RealUser p: subject.getPrincipals(RealUser.class)) {
return p.getRealUser();
@ -974,7 +995,8 @@ public class UserGroupInformation {
* @param userGroups the names of the groups that the user belongs to
* @return a fake user for running unit tests
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static UserGroupInformation createUserForTesting(String user,
String[] userGroups) {
ensureInitialized();
@ -1000,7 +1022,6 @@ public class UserGroupInformation {
* the names of the groups that the user belongs to
* @return a fake user for running unit tests
*/
@InterfaceAudience.LimitedPrivate( { "HDFS", "MapReduce" })
public static UserGroupInformation createProxyUserForTesting(String user,
UserGroupInformation realUser, String[] userGroups) {
ensureInitialized();
@ -1029,6 +1050,8 @@ public class UserGroupInformation {
* Get the user's full principal name.
* @return the user's full principal name.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public String getUserName() {
return user.getName();
}
@ -1182,6 +1205,8 @@ public class UserGroupInformation {
* @param action the method to execute
* @return the value from the run method
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public <T> T doAs(PrivilegedAction<T> action) {
logPrivilegedAction(subject, action);
return Subject.doAs(subject, action);
@ -1198,6 +1223,8 @@ public class UserGroupInformation {
* @throws InterruptedException if the action throws an InterruptedException
* @throws UndeclaredThrowableException if the action throws something else
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public <T> T doAs(PrivilegedExceptionAction<T> action
) throws IOException, InterruptedException {
try {

View File

@ -41,6 +41,7 @@ public class DelegationKey implements Writable {
private long expiryDate;
@Nullable
private byte[] keyBytes = null;
private static final int MAX_KEY_LEN = 1024 * 1024;
/** Default constructore required for Writable */
public DelegationKey() {
@ -55,6 +56,10 @@ public class DelegationKey implements Writable {
this.keyId = keyId;
this.expiryDate = expiryDate;
if (encodedKey != null) {
if (encodedKey.length > MAX_KEY_LEN) {
throw new RuntimeException("can't create " + encodedKey.length +
" byte long DelegationKey.");
}
this.keyBytes = encodedKey;
}
}
@ -102,7 +107,7 @@ public class DelegationKey implements Writable {
public void readFields(DataInput in) throws IOException {
keyId = WritableUtils.readVInt(in);
expiryDate = WritableUtils.readVLong(in);
int len = WritableUtils.readVInt(in);
int len = WritableUtils.readVIntInRange(in, -1, MAX_KEY_LEN);
if (len == -1) {
keyBytes = null;
} else {

View File

@ -21,8 +21,10 @@ package org.apache.hadoop.util;
import java.io.DataInput;
import java.io.IOException;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.UserInformationProto;
import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.*;
import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
import org.apache.hadoop.security.UserGroupInformation;
@ -135,4 +137,30 @@ public abstract class ProtoUtil {
}
return ugi;
}
static RpcKindProto convert(RPC.RpcKind kind) {
switch (kind) {
case RPC_BUILTIN: return RpcKindProto.RPC_BUILTIN;
case RPC_WRITABLE: return RpcKindProto.RPC_WRITABLE;
case RPC_PROTOCOL_BUFFER: return RpcKindProto.RPC_PROTOCOL_BUFFER;
}
return null;
}
public static RPC.RpcKind convert( RpcKindProto kind) {
switch (kind) {
case RPC_BUILTIN: return RPC.RpcKind.RPC_BUILTIN;
case RPC_WRITABLE: return RPC.RpcKind.RPC_WRITABLE;
case RPC_PROTOCOL_BUFFER: return RPC.RpcKind.RPC_PROTOCOL_BUFFER;
}
return null;
}
public static RpcPayloadHeaderProto makeRpcPayloadHeader(RPC.RpcKind rpcKind,
RpcPayloadOperationProto operation, int callId) {
RpcPayloadHeaderProto.Builder result = RpcPayloadHeaderProto.newBuilder();
result.setRpcKind(convert(rpcKind)).setRpcOp(operation).setCallId(callId);
return result.build();
}
}

View File

@ -50,6 +50,11 @@ public class RunJar {
/** Pattern that matches any string */
public static final Pattern MATCH_ANY = Pattern.compile(".*");
/**
* Priority of the RunJar shutdown hook.
*/
public static final int SHUTDOWN_HOOK_PRIORITY = 10;
/**
* Unpack a jar file into a directory.
*
@ -167,11 +172,14 @@ public class RunJar {
}
ensureDirectory(workDir);
Runtime.getRuntime().addShutdownHook(new Thread() {
ShutdownHookManager.get().addShutdownHook(
new Runnable() {
@Override
public void run() {
FileUtil.fullyDelete(workDir);
}
});
}, SHUTDOWN_HOOK_PRIORITY);
unJar(file, workDir);

View File

@ -81,64 +81,6 @@ abstract public class Shell {
/** If or not script timed out*/
private AtomicBoolean timedOut;
/** a Unix command to get ulimit of a process. */
public static final String ULIMIT_COMMAND = "ulimit";
/**
* Get the Unix command for setting the maximum virtual memory available
* to a given child process. This is only relevant when we are forking a
* process from within the Mapper or the Reducer implementations.
* Also see Hadoop Pipes and Hadoop Streaming.
*
* It also checks to ensure that we are running on a *nix platform else
* (e.g. in Cygwin/Windows) it returns <code>null</code>.
* @param memoryLimit virtual memory limit
* @return a <code>String[]</code> with the ulimit command arguments or
* <code>null</code> if we are running on a non *nix platform or
* if the limit is unspecified.
*/
public static String[] getUlimitMemoryCommand(int memoryLimit) {
// ulimit isn't supported on Windows
if (WINDOWS) {
return null;
}
return new String[] {ULIMIT_COMMAND, "-v", String.valueOf(memoryLimit)};
}
/**
* Get the Unix command for setting the maximum virtual memory available
* to a given child process. This is only relevant when we are forking a
* process from within the Mapper or the Reducer implementations.
* see also Hadoop Pipes and Streaming.
*
* It also checks to ensure that we are running on a *nix platform else
* (e.g. in Cygwin/Windows) it returns <code>null</code>.
* @param conf configuration
* @return a <code>String[]</code> with the ulimit command arguments or
* <code>null</code> if we are running on a non *nix platform or
* if the limit is unspecified.
* @deprecated Use {@link #getUlimitMemoryCommand(int)}
*/
@Deprecated
public static String[] getUlimitMemoryCommand(Configuration conf) {
// ulimit isn't supported on Windows
if (WINDOWS) {
return null;
}
// get the memory limit from the configuration
String ulimit = conf.get("mapred.child.ulimit");
if (ulimit == null) {
return null;
}
// Parse it to ensure it is legal/sane
int memoryLimit = Integer.valueOf(ulimit);
return getUlimitMemoryCommand(memoryLimit);
}
/** Set to true on Windows platforms */
public static final boolean WINDOWS /* borrowed from Path.WINDOWS */
= System.getProperty("os.name").startsWith("Windows");

View File

@ -0,0 +1,181 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.util;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* The <code>ShutdownHookManager</code> enables running shutdownHook
* in a determistic order, higher priority first.
* <p/>
* The JVM runs ShutdownHooks in a non-deterministic order or in parallel.
* This class registers a single JVM shutdownHook and run all the
* shutdownHooks registered to it (to this class) in order based on their
* priority.
*/
public class ShutdownHookManager {
private static final ShutdownHookManager MGR = new ShutdownHookManager();
private static final Log LOG = LogFactory.getLog(ShutdownHookManager.class);
static {
Runtime.getRuntime().addShutdownHook(
new Thread() {
@Override
public void run() {
MGR.shutdownInProgress.set(true);
for (Runnable hook: MGR.getShutdownHooksInOrder()) {
try {
hook.run();
} catch (Throwable ex) {
LOG.warn("ShutdownHook '" + hook.getClass().getSimpleName() +
"' failed, " + ex.toString(), ex);
}
}
}
}
);
}
/**
* Return <code>ShutdownHookManager</code> singleton.
*
* @return <code>ShutdownHookManager</code> singleton.
*/
public static ShutdownHookManager get() {
return MGR;
}
/**
* Private structure to store ShutdownHook and its priority.
*/
private static class HookEntry {
Runnable hook;
int priority;
public HookEntry(Runnable hook, int priority) {
this.hook = hook;
this.priority = priority;
}
@Override
public int hashCode() {
return hook.hashCode();
}
@Override
public boolean equals(Object obj) {
boolean eq = false;
if (obj != null) {
if (obj instanceof HookEntry) {
eq = (hook == ((HookEntry)obj).hook);
}
}
return eq;
}
}
private Set<HookEntry> hooks =
Collections.synchronizedSet(new HashSet<HookEntry>());
private AtomicBoolean shutdownInProgress = new AtomicBoolean(false);
//private to constructor to ensure singularity
private ShutdownHookManager() {
}
/**
* Returns the list of shutdownHooks in order of execution,
* Highest priority first.
*
* @return the list of shutdownHooks in order of execution.
*/
List<Runnable> getShutdownHooksInOrder() {
List<HookEntry> list;
synchronized (MGR.hooks) {
list = new ArrayList<HookEntry>(MGR.hooks);
}
Collections.sort(list, new Comparator<HookEntry>() {
//reversing comparison so highest priority hooks are first
@Override
public int compare(HookEntry o1, HookEntry o2) {
return o2.priority - o1.priority;
}
});
List<Runnable> ordered = new ArrayList<Runnable>();
for (HookEntry entry: list) {
ordered.add(entry.hook);
}
return ordered;
}
/**
* Adds a shutdownHook with a priority, the higher the priority
* the earlier will run. ShutdownHooks with same priority run
* in a non-deterministic order.
*
* @param shutdownHook shutdownHook <code>Runnable</code>
* @param priority priority of the shutdownHook.
*/
public void addShutdownHook(Runnable shutdownHook, int priority) {
if (shutdownHook == null) {
throw new IllegalArgumentException("shutdownHook cannot be NULL");
}
if (shutdownInProgress.get()) {
throw new IllegalStateException("Shutdown in progress, cannot add a shutdownHook");
}
hooks.add(new HookEntry(shutdownHook, priority));
}
/**
* Removes a shutdownHook.
*
* @param shutdownHook shutdownHook to remove.
* @return TRUE if the shutdownHook was registered and removed,
* FALSE otherwise.
*/
public boolean removeShutdownHook(Runnable shutdownHook) {
if (shutdownInProgress.get()) {
throw new IllegalStateException("Shutdown in progress, cannot remove a shutdownHook");
}
return hooks.remove(new HookEntry(shutdownHook, 0));
}
/**
* Indicates if a shutdownHook is registered or nt.
*
* @param shutdownHook shutdownHook to check if registered.
* @return TRUE/FALSE depending if the shutdownHook is is registered.
*/
public boolean hasShutdownHook(Runnable shutdownHook) {
return hooks.contains(new HookEntry(shutdownHook, 0));
}
}

View File

@ -46,6 +46,11 @@ import org.apache.hadoop.net.NetUtils;
@InterfaceStability.Unstable
public class StringUtils {
/**
* Priority of the StringUtils shutdown hook.
*/
public static final int SHUTDOWN_HOOK_PRIORITY = 0;
private static final DecimalFormat decimalFormat;
static {
NumberFormat numberFormat = NumberFormat.getNumberInstance(Locale.ENGLISH);
@ -600,12 +605,15 @@ public class StringUtils {
)
);
Runtime.getRuntime().addShutdownHook(new Thread() {
public void run() {
LOG.info(toStartupShutdownString("SHUTDOWN_MSG: ", new String[]{
"Shutting down " + classname + " at " + hostname}));
}
});
ShutdownHookManager.get().addShutdownHook(
new Runnable() {
@Override
public void run() {
LOG.info(toStartupShutdownString("SHUTDOWN_MSG: ", new String[]{
"Shutting down " + classname + " at " + hostname}));
}
}, SHUTDOWN_HOOK_PRIORITY);
}
/**

View File

@ -108,11 +108,6 @@
<value>-server -Xmx640m -Djava.net.preferIPv4Stack=true</value>
</property>
<property>
<name>mapred.child.ulimit</name>
<value>8388608</value>
</property>
<property>
<name>mapred.job.tracker.persist.jobstatus.active</name>
<value>true</value>

View File

@ -0,0 +1,58 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
option java_package = "org.apache.hadoop.ipc.protobuf";
option java_outer_classname = "RpcPayloadHeaderProtos";
option java_generate_equals_and_hash = true;
/**
* This is the rpc payload header. It is sent with every rpc call.
*
* The format of RPC call is as follows:
* +-----------------------------------------------------+
* | Rpc length in bytes |
* +-----------------------------------------------------+
* | RpcPayloadHeader - serialized delimited ie has len |
* +-----------------------------------------------------+
* | RpcRequest Payload |
* +-----------------------------------------------------+
*
*/
/**
* RpcKind determine the rpcEngine and the serialization of the rpc payload
*/
enum RpcKindProto {
RPC_BUILTIN = 0; // Used for built in calls by tests
RPC_WRITABLE = 1; // Use WritableRpcEngine
RPC_PROTOCOL_BUFFER = 2; // Use ProtobufRpcEngine
}
enum RpcPayloadOperationProto {
RPC_FINAL_PAYLOAD = 0; // The final payload
RPC_CONTINUATION_PAYLOAD = 1; // not implemented yet
RPC_CLOSE_CONNECTION = 2; // close the rpc connection
}
message RpcPayloadHeaderProto { // the header for the RpcRequest
optional RpcKindProto rpcKind = 1;
optional RpcPayloadOperationProto rpcOp = 2;
optional uint32 callId = 3; // each rpc has a callId that is also used in response
}

View File

@ -13,9 +13,10 @@
# See the License for the specific language governing permissions and
# limitations under the License.
# The number of acceptable warning for *all* modules
# Please update the per-module test-patch.properties if you update this file.
OK_RELEASEAUDIT_WARNINGS=0
OK_FINDBUGS_WARNINGS=0
OK_JAVADOC_WARNINGS=8
org.apache.hadoop.fs.LocalFileSystem
org.apache.hadoop.fs.viewfs.ViewFileSystem
org.apache.hadoop.fs.s3.S3FileSystem
org.apache.hadoop.fs.s3native.NativeS3FileSystem
org.apache.hadoop.fs.kfs.KosmosFileSystem
org.apache.hadoop.fs.ftp.FTPFileSystem
org.apache.hadoop.fs.HarFileSystem

View File

@ -0,0 +1,20 @@
#
# 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.
#
org.apache.hadoop.io.compress.BZip2Codec
org.apache.hadoop.io.compress.DefaultCodec
org.apache.hadoop.io.compress.DeflateCodec
org.apache.hadoop.io.compress.GzipCodec
org.apache.hadoop.io.compress.Lz4Codec
org.apache.hadoop.io.compress.SnappyCodec

View File

@ -268,9 +268,11 @@
<property>
<name>io.compression.codecs</name>
<value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.DeflateCodec,org.apache.hadoop.io.compress.SnappyCodec,org.apache.hadoop.io.compress.Lz4Codec</value>
<description>A list of the compression codec classes that can be used
for compression/decompression.</description>
<value></value>
<description>A comma-separated list of the compression codec classes that can
be used for compression/decompression. In addition to any classes specified
with this property (which take precedence), codec classes on the classpath
are discovered using a Java ServiceLoader.</description>
</property>
<property>
@ -350,25 +352,6 @@
</description>
</property>
<property>
<name>fs.file.impl</name>
<value>org.apache.hadoop.fs.LocalFileSystem</value>
<description>The FileSystem for file: uris.</description>
</property>
<property>
<name>fs.hdfs.impl</name>
<value>org.apache.hadoop.hdfs.DistributedFileSystem</value>
<description>The FileSystem for hdfs: uris.</description>
</property>
<property>
<name>fs.viewfs.impl</name>
<value>org.apache.hadoop.fs.viewfs.ViewFileSystem</value>
<description>The FileSystem for view file system for viewfs: uris
(ie client side mount table:).</description>
</property>
<property>
<name>fs.AbstractFileSystem.file.impl</name>
<value>org.apache.hadoop.fs.local.LocalFs</value>
@ -389,45 +372,6 @@
(ie client side mount table:).</description>
</property>
<property>
<name>fs.s3.impl</name>
<value>org.apache.hadoop.fs.s3.S3FileSystem</value>
<description>The FileSystem for s3: uris.</description>
</property>
<property>
<name>fs.s3n.impl</name>
<value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
<description>The FileSystem for s3n: (Native S3) uris.</description>
</property>
<property>
<name>fs.kfs.impl</name>
<value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value>
<description>The FileSystem for kfs: uris.</description>
</property>
<property>
<name>fs.hftp.impl</name>
<value>org.apache.hadoop.hdfs.HftpFileSystem</value>
</property>
<property>
<name>fs.hsftp.impl</name>
<value>org.apache.hadoop.hdfs.HsftpFileSystem</value>
</property>
<property>
<name>fs.webhdfs.impl</name>
<value>org.apache.hadoop.hdfs.web.WebHdfsFileSystem</value>
</property>
<property>
<name>fs.ftp.impl</name>
<value>org.apache.hadoop.fs.ftp.FTPFileSystem</value>
<description>The FileSystem for ftp: uris.</description>
</property>
<property>
<name>fs.ftp.host</name>
<value>0.0.0.0</value>
@ -442,18 +386,6 @@
</description>
</property>
<property>
<name>fs.har.impl</name>
<value>org.apache.hadoop.fs.HarFileSystem</value>
<description>The filesystem for Hadoop archives. </description>
</property>
<property>
<name>fs.har.impl.disable.cache</name>
<value>true</value>
<description>Don't cache 'har' filesystem instances.</description>
</property>
<property>
<name>fs.df.interval</name>
<value>60000</value>

View File

@ -314,8 +314,6 @@ Deprecated Properties
*---+---+
|mapred.map.child.log.level | mapreduce.map.log.level
*---+---+
|mapred.map.child.ulimit | mapreduce.map.ulimit
*---+---+
|mapred.map.max.attempts | mapreduce.map.maxattempts
*---+---+
|mapred.map.output.compression.codec | mapreduce.map.output.compress.codec
@ -378,8 +376,6 @@ Deprecated Properties
*---+---+
|mapred.reduce.child.log.level | mapreduce.reduce.log.level
*---+---+
|mapred.reduce.child.ulimit | mapreduce.reduce.ulimit
*---+---+
|mapred.reduce.max.attempts | mapreduce.reduce.maxattempts
*---+---+
|mapred.reduce.parallel.copies | mapreduce.reduce.shuffle.parallelcopies

View File

@ -23,6 +23,7 @@ import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.io.StringWriter;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Arrays;
@ -671,6 +672,27 @@ public class TestConfiguration extends TestCase {
}
}
public void testSetSocketAddress() throws IOException {
Configuration conf = new Configuration();
NetUtils.addStaticResolution("host", "127.0.0.1");
final String defaultAddr = "host:1";
InetSocketAddress addr = NetUtils.createSocketAddr(defaultAddr);
conf.setSocketAddr("myAddress", addr);
assertEquals(defaultAddr, NetUtils.getHostPortString(addr));
}
public void testUpdateSocketAddress() throws IOException {
InetSocketAddress addr = NetUtils.createSocketAddrForHost("host", 1);
InetSocketAddress connectAddr = conf.updateConnectAddr("myAddress", addr);
assertEquals(connectAddr.getHostName(), addr.getHostName());
addr = new InetSocketAddress(1);
connectAddr = conf.updateConnectAddr("myAddress", addr);
assertEquals(connectAddr.getHostName(),
InetAddress.getLocalHost().getHostName());
}
public void testReload() throws IOException {
out=new BufferedWriter(new FileWriter(CONFIG));
startConfig();

View File

@ -164,7 +164,7 @@ public class TestConfigurationDeprecation {
conf.set("Y", "y");
conf.set("Z", "z");
// get old key
assertEquals("y", conf.get("X"));
assertEquals("z", conf.get("X"));
}
/**
@ -305,7 +305,7 @@ public class TestConfigurationDeprecation {
assertTrue("deprecated Key not found", dKFound);
assertTrue("new Key not found", nKFound);
}
@Test
public void testUnsetWithDeprecatedKeys() {
Configuration conf = new Configuration();

View File

@ -18,10 +18,15 @@
package org.apache.hadoop.conf;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayOutputStream;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.junit.Test;
import junit.framework.TestCase;
@ -53,4 +58,49 @@ public class TestDeprecatedKeys extends TestCase {
assertTrue(fileContents.contains("old.config.yet.to.be.deprecated"));
assertTrue(fileContents.contains("new.conf.to.replace.deprecated.conf"));
}
@Test
public void testIteratorWithDeprecatedKeysMappedToMultipleNewKeys() {
Configuration conf = new Configuration();
Configuration.addDeprecation("dK", new String[]{"nK1", "nK2"});
conf.set("k", "v");
conf.set("dK", "V");
assertEquals("V", conf.get("dK"));
assertEquals("V", conf.get("nK1"));
assertEquals("V", conf.get("nK2"));
conf.set("nK1", "VV");
assertEquals("VV", conf.get("dK"));
assertEquals("VV", conf.get("nK1"));
assertEquals("VV", conf.get("nK2"));
conf.set("nK2", "VVV");
assertEquals("VVV", conf.get("dK"));
assertEquals("VVV", conf.get("nK2"));
assertEquals("VVV", conf.get("nK1"));
boolean kFound = false;
boolean dKFound = false;
boolean nK1Found = false;
boolean nK2Found = false;
for (Map.Entry<String, String> entry : conf) {
if (entry.getKey().equals("k")) {
assertEquals("v", entry.getValue());
kFound = true;
}
if (entry.getKey().equals("dK")) {
assertEquals("VVV", entry.getValue());
dKFound = true;
}
if (entry.getKey().equals("nK1")) {
assertEquals("VVV", entry.getValue());
nK1Found = true;
}
if (entry.getKey().equals("nK2")) {
assertEquals("VVV", entry.getValue());
nK2Found = true;
}
}
assertTrue("regular Key not found", kFound);
assertTrue("deprecated Key not found", dKFound);
assertTrue("new Key 1 not found", nK1Found);
assertTrue("new Key 2 not found", nK2Found);
}
}

View File

@ -0,0 +1,176 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.EnumSet;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.util.Progressable;
import org.junit.Test;
public class TestAfsCheckPath {
private static int DEFAULT_PORT = 1234;
private static int OTHER_PORT = 4321;
@Test
public void testCheckPathWithNoPorts() throws URISyntaxException {
URI uri = new URI("dummy://dummy-host");
AbstractFileSystem afs = new DummyFileSystem(uri);
afs.checkPath(new Path("dummy://dummy-host"));
}
@Test
public void testCheckPathWithDefaultPort() throws URISyntaxException {
URI uri = new URI("dummy://dummy-host:" + DEFAULT_PORT);
AbstractFileSystem afs = new DummyFileSystem(uri);
afs.checkPath(new Path("dummy://dummy-host:" + DEFAULT_PORT));
}
@Test
public void testCheckPathWithTheSameNonDefaultPort()
throws URISyntaxException {
URI uri = new URI("dummy://dummy-host:" + OTHER_PORT);
AbstractFileSystem afs = new DummyFileSystem(uri);
afs.checkPath(new Path("dummy://dummy-host:" + OTHER_PORT));
}
@Test(expected=InvalidPathException.class)
public void testCheckPathWithDifferentPorts() throws URISyntaxException {
URI uri = new URI("dummy://dummy-host:" + DEFAULT_PORT);
AbstractFileSystem afs = new DummyFileSystem(uri);
afs.checkPath(new Path("dummy://dummy-host:" + OTHER_PORT));
}
private static class DummyFileSystem extends AbstractFileSystem {
public DummyFileSystem(URI uri) throws URISyntaxException {
super(uri, "dummy", true, DEFAULT_PORT);
}
@Override
public int getUriDefaultPort() {
return DEFAULT_PORT;
}
@Override
public FSDataOutputStream createInternal(Path f, EnumSet<CreateFlag> flag,
FsPermission absolutePermission, int bufferSize, short replication,
long blockSize, Progressable progress, int bytesPerChecksum,
boolean createParent) throws IOException {
// deliberately empty
return null;
}
@Override
public boolean delete(Path f, boolean recursive)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
// deliberately empty
return false;
}
@Override
public BlockLocation[] getFileBlockLocations(Path f, long start, long len)
throws IOException {
// deliberately empty
return null;
}
@Override
public FileChecksum getFileChecksum(Path f) throws IOException {
// deliberately empty
return null;
}
@Override
public FileStatus getFileStatus(Path f) throws IOException {
// deliberately empty
return null;
}
@Override
public FsStatus getFsStatus() throws IOException {
// deliberately empty
return null;
}
@Override
public FsServerDefaults getServerDefaults() throws IOException {
// deliberately empty
return null;
}
@Override
public FileStatus[] listStatus(Path f) throws IOException {
// deliberately empty
return null;
}
@Override
public void mkdir(Path dir, FsPermission permission, boolean createParent)
throws IOException {
// deliberately empty
}
@Override
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
// deliberately empty
return null;
}
@Override
public void renameInternal(Path src, Path dst) throws IOException {
// deliberately empty
}
@Override
public void setOwner(Path f, String username, String groupname)
throws IOException {
// deliberately empty
}
@Override
public void setPermission(Path f, FsPermission permission)
throws IOException {
// deliberately empty
}
@Override
public boolean setReplication(Path f, short replication) throws IOException {
// deliberately empty
return false;
}
@Override
public void setTimes(Path f, long mtime, long atime) throws IOException {
// deliberately empty
}
@Override
public void setVerifyChecksum(boolean verifyChecksum) throws IOException {
// deliberately empty
}
}
}

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.Set;
import junit.framework.Assert;
import org.apache.hadoop.util.ShutdownHookManager;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -65,7 +66,7 @@ public class TestFileContextDeleteOnExit {
checkDeleteOnExitData(1, fc, file1);
// Ensure shutdown hook is added
Assert.assertTrue(Runtime.getRuntime().removeShutdownHook(FileContext.FINALIZER));
Assert.assertTrue(ShutdownHookManager.get().hasShutdownHook(FileContext.FINALIZER));
Path file2 = getTestRootPath(fc, "dir1/file2");
createFile(fc, file2, numBlocks, blockSize);
@ -79,8 +80,7 @@ public class TestFileContextDeleteOnExit {
// trigger deleteOnExit and ensure the registered
// paths are cleaned up
FileContext.FINALIZER.start();
FileContext.FINALIZER.join();
FileContext.FINALIZER.run();
checkDeleteOnExitData(0, fc, new Path[0]);
Assert.assertFalse(exists(fc, file1));
Assert.assertFalse(exists(fc, file2));

View File

@ -43,7 +43,7 @@ public class TestFileSystemCaching {
@Test
public void testCacheEnabled() throws Exception {
Configuration conf = new Configuration();
conf.set("fs.cachedfile.impl", conf.get("fs.file.impl"));
conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
FileSystem fs1 = FileSystem.get(new URI("cachedfile://a"), conf);
FileSystem fs2 = FileSystem.get(new URI("cachedfile://a"), conf);
assertSame(fs1, fs2);
@ -84,7 +84,7 @@ public class TestFileSystemCaching {
// wait for InitializeForeverFileSystem to start initialization
InitializeForeverFileSystem.sem.acquire();
conf.set("fs.cachedfile.impl", conf.get("fs.file.impl"));
conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
FileSystem.get(new URI("cachedfile://a"), conf);
t.interrupt();
t.join();
@ -93,7 +93,7 @@ public class TestFileSystemCaching {
@Test
public void testCacheDisabled() throws Exception {
Configuration conf = new Configuration();
conf.set("fs.uncachedfile.impl", conf.get("fs.file.impl"));
conf.set("fs.uncachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
conf.setBoolean("fs.uncachedfile.impl.disable.cache", true);
FileSystem fs1 = FileSystem.get(new URI("uncachedfile://a"), conf);
FileSystem fs2 = FileSystem.get(new URI("uncachedfile://a"), conf);
@ -104,7 +104,7 @@ public class TestFileSystemCaching {
@Test
public <T extends TokenIdentifier> void testCacheForUgi() throws Exception {
final Configuration conf = new Configuration();
conf.set("fs.cachedfile.impl", conf.get("fs.file.impl"));
conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
UserGroupInformation ugiA = UserGroupInformation.createRemoteUser("foo");
UserGroupInformation ugiB = UserGroupInformation.createRemoteUser("bar");
FileSystem fsA = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
@ -156,7 +156,7 @@ public class TestFileSystemCaching {
@Test
public void testUserFS() throws Exception {
final Configuration conf = new Configuration();
conf.set("fs.cachedfile.impl", conf.get("fs.file.impl"));
conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
FileSystem fsU1 = FileSystem.get(new URI("cachedfile://a"), conf, "bar");
FileSystem fsU2 = FileSystem.get(new URI("cachedfile://a"), conf, "foo");
@ -166,7 +166,7 @@ public class TestFileSystemCaching {
@Test
public void testFsUniqueness() throws Exception {
final Configuration conf = new Configuration();
conf.set("fs.cachedfile.impl", conf.get("fs.file.impl"));
conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
// multiple invocations of FileSystem.get return the same object.
FileSystem fs1 = FileSystem.get(conf);
FileSystem fs2 = FileSystem.get(conf);
@ -183,7 +183,7 @@ public class TestFileSystemCaching {
@Test
public void testCloseAllForUGI() throws Exception {
final Configuration conf = new Configuration();
conf.set("fs.cachedfile.impl", conf.get("fs.file.impl"));
conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
UserGroupInformation ugiA = UserGroupInformation.createRemoteUser("foo");
FileSystem fsA = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
public FileSystem run() throws Exception {

View File

@ -165,7 +165,10 @@ public class TestFilterFileSystem {
public Token<?> getDelegationToken(String renewer) throws IOException {
return null;
}
public String getScheme() {
return "dontcheck";
}
}
@Test

View File

@ -99,6 +99,19 @@ public class HttpServerFunctionalTest extends Assert {
}
}
/**
* Create an HttpServer instance on the given address for the given webapp
* @param host to bind
* @param port to bind
* @return the server
* @throws IOException if it could not be created
*/
public static HttpServer createServer(String host, int port)
throws IOException {
prepareTestWebapp();
return new HttpServer(TEST, host, port, true);
}
/**
* Create an HttpServer instance for the given webapp
* @param webapp the webapp to work with

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.http;
import java.io.IOException;
import java.io.PrintWriter;
import java.net.HttpURLConnection;
import java.net.InetSocketAddress;
import java.net.URL;
import java.util.Arrays;
import java.util.Enumeration;
@ -35,6 +36,7 @@ import java.util.concurrent.Executors;
import javax.servlet.Filter;
import javax.servlet.FilterChain;
import javax.servlet.FilterConfig;
import javax.servlet.ServletContext;
import javax.servlet.ServletException;
import javax.servlet.ServletRequest;
import javax.servlet.ServletResponse;
@ -53,10 +55,12 @@ import org.apache.hadoop.http.HttpServer.QuotingInputFilter.RequestQuoter;
import org.apache.hadoop.http.resource.JerseyResource;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mortbay.util.ajax.JSON;
@ -422,4 +426,96 @@ public class TestHttpServer extends HttpServerFunctionalTest {
assertEquals("bar", m.get(JerseyResource.OP));
LOG.info("END testJersey()");
}
@Test
public void testHasAdministratorAccess() throws Exception {
Configuration conf = new Configuration();
conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false);
ServletContext context = Mockito.mock(ServletContext.class);
Mockito.when(context.getAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE)).thenReturn(conf);
Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(null);
HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
Mockito.when(request.getRemoteUser()).thenReturn(null);
HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
//authorization OFF
Assert.assertTrue(HttpServer.hasAdministratorAccess(context, request, response));
//authorization ON & user NULL
response = Mockito.mock(HttpServletResponse.class);
conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
Assert.assertFalse(HttpServer.hasAdministratorAccess(context, request, response));
Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED), Mockito.anyString());
//authorization ON & user NOT NULL & ACLs NULL
response = Mockito.mock(HttpServletResponse.class);
Mockito.when(request.getRemoteUser()).thenReturn("foo");
Assert.assertTrue(HttpServer.hasAdministratorAccess(context, request, response));
//authorization ON & user NOT NULL & ACLs NOT NULL & user not in ACLs
response = Mockito.mock(HttpServletResponse.class);
AccessControlList acls = Mockito.mock(AccessControlList.class);
Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(false);
Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(acls);
Assert.assertFalse(HttpServer.hasAdministratorAccess(context, request, response));
Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED), Mockito.anyString());
//authorization ON & user NOT NULL & ACLs NOT NULL & user in in ACLs
response = Mockito.mock(HttpServletResponse.class);
Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(true);
Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(acls);
Assert.assertTrue(HttpServer.hasAdministratorAccess(context, request, response));
}
@Test public void testBindAddress() throws Exception {
checkBindAddress("0.0.0.0", 0, false).stop();
// hang onto this one for a bit more testing
HttpServer myServer = checkBindAddress("localhost", 0, false);
HttpServer myServer2 = null;
try {
int port = myServer.getListenerAddress().getPort();
// it's already in use, true = expect a higher port
myServer2 = checkBindAddress("localhost", port, true);
// try to reuse the port
port = myServer2.getListenerAddress().getPort();
myServer2.stop();
assertEquals(-1, myServer2.getPort()); // not bound
myServer2.openListener();
assertEquals(port, myServer2.getPort()); // expect same port
} finally {
myServer.stop();
if (myServer2 != null) {
myServer2.stop();
}
}
}
private HttpServer checkBindAddress(String host, int port, boolean findPort)
throws Exception {
HttpServer server = createServer(host, port);
try {
// not bound, ephemeral should return requested port (0 for ephemeral)
InetSocketAddress addr = server.getListenerAddress();
assertEquals(port, addr.getPort());
// verify hostname is what was given
server.openListener();
addr = server.getListenerAddress();
assertEquals(host, addr.getHostName());
int boundPort = addr.getPort();
if (port == 0) {
assertTrue(boundPort != 0); // ephemeral should now return bound port
} else if (findPort) {
assertTrue(boundPort > port);
// allow a little wiggle room to prevent random test failures if
// some consecutive ports are already in use
assertTrue(addr.getPort() - port < 8);
}
} catch (Exception e) {
server.stop();
throw e;
}
return server;
}
}

View File

@ -470,7 +470,7 @@ public class TestSequenceFile extends TestCase {
SequenceFile.Writer writer = SequenceFile.createWriter(
spyFs, conf, p, NullWritable.class, NullWritable.class);
writer.close();
Mockito.verify(spyFs).getDefaultReplication();
Mockito.verify(spyFs).getDefaultReplication(p);
}
private static class TestFSDataInputStream extends FSDataInputStream {

View File

@ -44,6 +44,26 @@ public class TestWritableUtils extends TestCase {
assertEquals(vintlen, WritableUtils.getVIntSize(val));
assertEquals(vintlen, WritableUtils.decodeVIntSize(buf.getData()[0]));
}
public static void testReadInRange(long val, int lower,
int upper, boolean expectSuccess) throws IOException {
DataOutputBuffer buf = new DataOutputBuffer();
DataInputBuffer inbuf = new DataInputBuffer();
WritableUtils.writeVLong(buf, val);
try {
inbuf.reset(buf.getData(), 0, buf.getLength());
long val2 = WritableUtils.readVIntInRange(inbuf, lower, upper);
if (!expectSuccess) {
fail("expected readVIntInRange to throw an exception");
}
assertEquals(val, val2);
} catch(IOException e) {
if (expectSuccess) {
LOG.error("unexpected exception:", e);
fail("readVIntInRange threw an unexpected exception");
}
}
}
public static void testVInt() throws Exception {
testValue(12, 1);
@ -61,5 +81,10 @@ public class TestWritableUtils extends TestCase {
testValue(-65536, 3);
testValue(65536, 4);
testValue(-65537, 4);
testReadInRange(123, 122, 123, true);
testReadInRange(123, 0, 100, false);
testReadInRange(0, 0, 100, true);
testReadInRange(-1, 0, 100, false);
testReadInRange(1099511627776L, 0, Integer.MAX_VALUE, false);
}
}

View File

@ -101,6 +101,12 @@ public class TestCodecFactory extends TestCase {
}
}
private static class NewGzipCodec extends BaseCodec {
public String getDefaultExtension() {
return ".gz";
}
}
/**
* Returns a factory for a given set of codecs
* @param classes the codec classes to include
@ -167,32 +173,43 @@ public class TestCodecFactory extends TestCase {
checkCodec("default factory for deflate codec", DeflateCodec.class, codec);
factory = setClasses(new Class[0]);
// gz, bz2, snappy, lz4 are picked up by service loader, but bar isn't
codec = factory.getCodec(new Path("/tmp/foo.bar"));
assertEquals("empty codec bar codec", null, codec);
assertEquals("empty factory bar codec", null, codec);
codec = factory.getCodecByClassName(BarCodec.class.getCanonicalName());
assertEquals("empty codec bar codec", null, codec);
assertEquals("empty factory bar codec", null, codec);
codec = factory.getCodec(new Path("/tmp/foo.gz"));
assertEquals("empty codec gz codec", null, codec);
checkCodec("empty factory gz codec", GzipCodec.class, codec);
codec = factory.getCodecByClassName(GzipCodec.class.getCanonicalName());
assertEquals("empty codec gz codec", null, codec);
checkCodec("empty factory gz codec", GzipCodec.class, codec);
codec = factory.getCodec(new Path("/tmp/foo.bz2"));
assertEquals("empty factory for .bz2", null, codec);
checkCodec("empty factory for .bz2", BZip2Codec.class, codec);
codec = factory.getCodecByClassName(BZip2Codec.class.getCanonicalName());
assertEquals("empty factory for bzip2 codec", null, codec);
checkCodec("empty factory for bzip2 codec", BZip2Codec.class, codec);
codec = factory.getCodec(new Path("/tmp/foo.snappy"));
checkCodec("empty factory snappy codec", SnappyCodec.class, codec);
codec = factory.getCodecByClassName(SnappyCodec.class.getCanonicalName());
checkCodec("empty factory snappy codec", SnappyCodec.class, codec);
codec = factory.getCodec(new Path("/tmp/foo.lz4"));
checkCodec("empty factory lz4 codec", Lz4Codec.class, codec);
codec = factory.getCodecByClassName(Lz4Codec.class.getCanonicalName());
checkCodec("empty factory lz4 codec", Lz4Codec.class, codec);
factory = setClasses(new Class[]{BarCodec.class, FooCodec.class,
FooBarCodec.class});
codec = factory.getCodec(new Path("/tmp/.foo.bar.gz"));
assertEquals("full factory gz codec", null, codec);
checkCodec("full factory gz codec", GzipCodec.class, codec);
codec = factory.getCodecByClassName(GzipCodec.class.getCanonicalName());
assertEquals("full codec gz codec", null, codec);
checkCodec("full codec gz codec", GzipCodec.class, codec);
codec = factory.getCodec(new Path("/tmp/foo.bz2"));
assertEquals("full factory for .bz2", null, codec);
checkCodec("full factory for .bz2", BZip2Codec.class, codec);
codec = factory.getCodecByClassName(BZip2Codec.class.getCanonicalName());
assertEquals("full codec bzip2 codec", null, codec);
checkCodec("full codec bzip2 codec", BZip2Codec.class, codec);
codec = factory.getCodec(new Path("/tmp/foo.bar"));
checkCodec("full factory bar codec", BarCodec.class, codec);
@ -220,5 +237,11 @@ public class TestCodecFactory extends TestCase {
checkCodec("full factory foo codec", FooCodec.class, codec);
codec = factory.getCodecByName("FOO");
checkCodec("full factory foo codec", FooCodec.class, codec);
factory = setClasses(new Class[]{NewGzipCodec.class});
codec = factory.getCodec(new Path("/tmp/foo.gz"));
checkCodec("overridden factory for .gz", NewGzipCodec.class, codec);
codec = factory.getCodecByClassName(NewGzipCodec.class.getCanonicalName());
checkCodec("overridden factory for gzip codec", NewGzipCodec.class, codec);
}
}

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.net.NetUtils;
@ -99,7 +98,7 @@ public class TestIPC {
}
@Override
public Writable call(RpcKind rpcKind, String protocol, Writable param,
public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param,
long receiveTime) throws IOException {
if (sleep) {
// sleep a bit

View File

@ -30,7 +30,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
/**
@ -73,7 +72,7 @@ public class TestIPCServerResponder extends TestCase {
}
@Override
public Writable call(RpcKind rpcKind, String protocol, Writable param,
public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param,
long receiveTime) throws IOException {
if (sleep) {
try {

View File

@ -23,7 +23,6 @@ import java.net.InetSocketAddress;
import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.TestProtoBufRpc.PBServerImpl;
import org.apache.hadoop.ipc.TestProtoBufRpc.TestRpcService;
import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
@ -178,9 +177,9 @@ public class TestMultipleProtocolServer {
// create a server with two handlers
server = RPC.getServer(Foo0.class,
new Foo0Impl(), ADDRESS, 0, 2, false, conf, null);
server.addProtocol(RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
server.addProtocol(RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
server.addProtocol(RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
// Add Protobuf server
@ -189,7 +188,7 @@ public class TestMultipleProtocolServer {
new PBServerImpl();
BlockingService service = TestProtobufRpcProto
.newReflectiveBlockingService(pbServerImpl);
server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
service);
server.start();
addr = NetUtils.getConnectAddress(server);

View File

@ -24,7 +24,6 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto;
@ -122,7 +121,7 @@ public class TestProtoBufRpc {
BlockingService service2 = TestProtobufRpc2Proto
.newReflectiveBlockingService(server2Impl);
server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService2.class,
server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService2.class,
service2);
server.start();
}

View File

@ -31,7 +31,6 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
@ -134,7 +133,7 @@ public class TestRPCCompatibility {
TestImpl1 impl = new TestImpl1();
server = RPC.getServer(TestProtocol1.class,
impl, ADDRESS, 0, 2, false, conf, null);
server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
addr = NetUtils.getConnectAddress(server);
@ -201,7 +200,7 @@ System.out.println("echo int is NOT supported");
TestImpl1 impl = new TestImpl1();
server = RPC.getServer(TestProtocol1.class,
impl, ADDRESS, 0, 2, false, conf, null);
server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
addr = NetUtils.getConnectAddress(server);
@ -222,7 +221,7 @@ System.out.println("echo int is NOT supported");
TestImpl2 impl = new TestImpl2();
server = RPC.getServer(TestProtocol2.class,
impl, ADDRESS, 0, 2, false, conf, null);
server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
addr = NetUtils.getConnectAddress(server);
@ -316,11 +315,11 @@ System.out.println("echo int is NOT supported");
TestProtocol2 proxy = RPC.getProxy(TestProtocol2.class,
TestProtocol2.versionID, addr, conf);
boolean supported = RpcClientUtil.isMethodSupported(proxy,
TestProtocol2.class, RpcKind.RPC_WRITABLE,
TestProtocol2.class, RPC.RpcKind.RPC_WRITABLE,
RPC.getProtocolVersion(TestProtocol2.class), "echo");
Assert.assertTrue(supported);
supported = RpcClientUtil.isMethodSupported(proxy,
TestProtocol2.class, RpcKind.RPC_PROTOCOL_BUFFER,
TestProtocol2.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(TestProtocol2.class), "echo");
Assert.assertFalse(supported);
}
@ -334,7 +333,7 @@ System.out.println("echo int is NOT supported");
TestImpl1 impl = new TestImpl1();
server = RPC.getServer(TestProtocol1.class, impl, ADDRESS, 0, 2, false,
conf, null);
server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
ProtocolMetaInfoServerSideTranslatorPB xlator =
@ -343,13 +342,13 @@ System.out.println("echo int is NOT supported");
GetProtocolSignatureResponseProto resp = xlator.getProtocolSignature(
null,
createGetProtocolSigRequestProto(TestProtocol1.class,
RpcKind.RPC_PROTOCOL_BUFFER));
RPC.RpcKind.RPC_PROTOCOL_BUFFER));
//No signatures should be found
Assert.assertEquals(0, resp.getProtocolSignatureCount());
resp = xlator.getProtocolSignature(
null,
createGetProtocolSigRequestProto(TestProtocol1.class,
RpcKind.RPC_WRITABLE));
RPC.RpcKind.RPC_WRITABLE));
Assert.assertEquals(1, resp.getProtocolSignatureCount());
ProtocolSignatureProto sig = resp.getProtocolSignatureList().get(0);
Assert.assertEquals(TestProtocol1.versionID, sig.getVersion());
@ -366,7 +365,7 @@ System.out.println("echo int is NOT supported");
}
private GetProtocolSignatureRequestProto createGetProtocolSigRequestProto(
Class<?> protocol, RpcKind rpcKind) {
Class<?> protocol, RPC.RpcKind rpcKind) {
GetProtocolSignatureRequestProto.Builder builder =
GetProtocolSignatureRequestProto.newBuilder();
builder.setProtocol(protocol.getName());

View File

@ -169,6 +169,19 @@ public class TestNetUtils {
assertInException(wrapped, "/UnknownHost");
}
@Test
public void testGetConnectAddress() throws IOException {
NetUtils.addStaticResolution("host", "127.0.0.1");
InetSocketAddress addr = NetUtils.createSocketAddrForHost("host", 1);
InetSocketAddress connectAddr = NetUtils.getConnectAddress(addr);
assertEquals(addr.getHostName(), connectAddr.getHostName());
addr = new InetSocketAddress(1);
connectAddr = NetUtils.getConnectAddress(addr);
assertEquals(InetAddress.getLocalHost().getHostName(),
connectAddr.getHostName());
}
@Test
public void testCreateSocketAddress() throws Throwable {
InetSocketAddress addr = NetUtils.createSocketAddr(

View File

@ -0,0 +1,62 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.util;
import org.junit.Assert;
import org.junit.Test;
public class TestShutdownHookManager {
@Test
public void shutdownHookManager() {
ShutdownHookManager mgr = ShutdownHookManager.get();
Assert.assertNotNull(mgr);
Assert.assertEquals(0, mgr.getShutdownHooksInOrder().size());
Runnable hook1 = new Runnable() {
@Override
public void run() {
}
};
Runnable hook2 = new Runnable() {
@Override
public void run() {
}
};
mgr.addShutdownHook(hook1, 0);
Assert.assertTrue(mgr.hasShutdownHook(hook1));
Assert.assertEquals(1, mgr.getShutdownHooksInOrder().size());
Assert.assertEquals(hook1, mgr.getShutdownHooksInOrder().get(0));
mgr.removeShutdownHook(hook1);
Assert.assertFalse(mgr.hasShutdownHook(hook1));
mgr.addShutdownHook(hook1, 0);
Assert.assertTrue(mgr.hasShutdownHook(hook1));
Assert.assertEquals(1, mgr.getShutdownHooksInOrder().size());
Assert.assertTrue(mgr.hasShutdownHook(hook1));
Assert.assertEquals(1, mgr.getShutdownHooksInOrder().size());
mgr.addShutdownHook(hook2, 1);
Assert.assertTrue(mgr.hasShutdownHook(hook1));
Assert.assertTrue(mgr.hasShutdownHook(hook2));
Assert.assertEquals(2, mgr.getShutdownHooksInOrder().size());
Assert.assertEquals(hook2, mgr.getShutdownHooksInOrder().get(0));
Assert.assertEquals(hook1, mgr.getShutdownHooksInOrder().get(1));
}
}

View File

@ -1,18 +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.
OK_RELEASEAUDIT_WARNINGS=1
OK_FINDBUGS_WARNINGS=0
OK_JAVADOC_WARNINGS=6

View File

@ -1,21 +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.
# The number of acceptable warning for this module
# Please update the root test-patch.properties if you update this file.
OK_RELEASEAUDIT_WARNINGS=0
OK_FINDBUGS_WARNINGS=0
OK_JAVADOC_WARNINGS=8

View File

@ -159,7 +159,7 @@ public class HttpFSFileSystem extends FileSystem {
* Get operations.
*/
public enum GetOpValues {
OPEN, GETFILESTATUS, LISTSTATUS, GETHOMEDIR, GETCONTENTSUMMARY, GETFILECHECKSUM,
OPEN, GETFILESTATUS, LISTSTATUS, GETHOMEDIRECTORY, GETCONTENTSUMMARY, GETFILECHECKSUM,
GETDELEGATIONTOKEN, GETFILEBLOCKLOCATIONS, INSTRUMENTATION
}
@ -684,7 +684,7 @@ public class HttpFSFileSystem extends FileSystem {
@Override
public Path getHomeDirectory() {
Map<String, String> params = new HashMap<String, String>();
params.put(OP_PARAM, GetOpValues.GETHOMEDIR.toString());
params.put(OP_PARAM, GetOpValues.GETHOMEDIRECTORY.toString());
try {
HttpURLConnection conn = getConnection(HTTP_GET, params, new Path(getUri().toString(), "/"), false);
validateResponse(conn, HttpURLConnection.HTTP_OK);

View File

@ -43,8 +43,8 @@ import java.util.Map;
public class FSOperations {
/**
* Converts a Unix permission octal & symbolic representation
* (i.e. 655 or -rwxr--r--) into a FileSystemAccess permission.
* Converts a Unix permission octal
* (i.e. 655 or 1777) into a FileSystemAccess permission.
*
* @param str Unix permission symbolic representation.
*
@ -55,10 +55,8 @@ public class FSOperations {
FsPermission permission;
if (str.equals(HttpFSFileSystem.DEFAULT_PERMISSION)) {
permission = FsPermission.getDefault();
} else if (str.length() == 3) {
permission = new FsPermission(Short.parseShort(str, 8));
} else {
permission = FsPermission.valueOf(str);
permission = new FsPermission(Short.parseShort(str, 8));
}
return permission;
}

View File

@ -446,7 +446,7 @@ public class HttpFSParams {
* Symbolic Unix permissions regular expression pattern.
*/
private static final Pattern PERMISSION_PATTERN =
Pattern.compile(DEFAULT + "|(-[-r][-w][-x][-r][-w][-x][-r][-w][-x])" + "|[0-7][0-7][0-7]");
Pattern.compile(DEFAULT + "|[0-1]?[0-7][0-7][0-7]");
/**
* Constructor.

View File

@ -291,7 +291,7 @@ public class HttpFSServer {
response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
break;
}
case GETHOMEDIR: {
case GETHOMEDIRECTORY: {
FSOperations.FSHomeDir command = new FSOperations.FSHomeDir();
JSONObject json = fsExecute(user, doAs.value(), command);
AUDIT_LOG.info("");

View File

@ -310,11 +310,8 @@ public class TestHttpFSFileSystem extends HFSTestCase {
private void testSetPermission() throws Exception {
FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
OutputStream os = fs.create(path);
os.write(1);
os.close();
fs.close();
Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foodir");
fs.mkdirs(path);
fs = getHttpFileSystem();
FsPermission permission1 = new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE);
@ -326,6 +323,19 @@ public class TestHttpFSFileSystem extends HFSTestCase {
fs.close();
FsPermission permission2 = status1.getPermission();
Assert.assertEquals(permission2, permission1);
//sticky bit
fs = getHttpFileSystem();
permission1 = new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE, true);
fs.setPermission(path, permission1);
fs.close();
fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
status1 = fs.getFileStatus(path);
fs.close();
permission2 = status1.getPermission();
Assert.assertTrue(permission2.getStickyBit());
Assert.assertEquals(permission2, permission1);
}
private void testSetOwner() throws Exception {

View File

@ -53,7 +53,7 @@ public class TestCheckUploadContentTypeFilter {
@Test
public void getOther() throws Exception {
test("GET", HttpFSFileSystem.GetOpValues.GETHOMEDIR.toString(), "plain/text", false, false);
test("GET", HttpFSFileSystem.GetOpValues.GETHOMEDIRECTORY.toString(), "plain/text", false, false);
}
@Test

View File

@ -65,8 +65,17 @@ Trunk (unreleased changes)
HDFS-3273. Refactor BackupImage and FSEditLog, and rename
JournalListener.rollLogs(..) to startLogSegment(..). (szetszwo)
HDFS-3292. Remove the deprecated DiskStatus, getDiskStatus(), getRawCapacity() and
getRawUsed() from DistributedFileSystem. (Arpit Gupta via szetszwo)
HDFS-3292. Remove the deprecated DiskStatus, getDiskStatus(), getRawUsed()
and getRawCapacity() from DistributedFileSystem. (Arpit Gupta via szetszwo)
HADOOP-8285. HDFS changes for Use ProtoBuf for RpcPayLoadHeader. (sanjay
radia)
HDFS-2743. Streamline usage of bookkeeper journal manager.
(Ivan Kelly via umamahesh)
HDFS-3293. Add toString(), equals(..) and hashCode() to JournalInfo.
(Hari Mankude via szetszwo)
OPTIMIZATIONS
@ -130,6 +139,8 @@ Trunk (unreleased changes)
(Henry Robinson via atm)
HDFS-3243. TestParallelRead timing out on jenkins. (Henry Robinson via todd)
HDFS-3265. PowerPc Build error. (Kumar Ravi via mattf)
Release 2.0.0 - UNRELEASED
@ -210,6 +221,10 @@ Release 2.0.0 - UNRELEASED
HDFS-3004. Implement Recovery Mode. (Colin Patrick McCabe via eli)
HDFS-3282. Add HdfsDataInputStream as a public API. (umamahesh)
HDFS-3298. Add HdfsDataOutputStream as a public API. (szetszwo)
IMPROVEMENTS
HDFS-2018. Move all journal stream management code into one place.
@ -390,6 +405,20 @@ Release 2.0.0 - UNRELEASED
HDFS-3263. HttpFS should read HDFS config from Hadoop site.xml files (tucu)
HDFS-3206. Miscellaneous xml cleanups for OEV.
(Colin Patrick McCabe via eli)
HDFS-3169. TestFsck should test multiple -move operations in a row.
(Colin Patrick McCabe via eli)
HDFS-3258. Test for HADOOP-8144 (pseudoSortByDistance in
NetworkTopology for first rack local node). (Junping Du via eli)
HDFS-3322. Use HdfsDataInputStream and HdfsDataOutputStream in Hdfs.
(szetszwo)
HDFS-3339. Change INode to package private. (John George via szetszwo)
OPTIMIZATIONS
HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
@ -535,6 +564,31 @@ Release 2.0.0 - UNRELEASED
HDFS-3165. HDFS Balancer scripts are refering to wrong path of
hadoop-daemon.sh (Amith D K via eli)
HDFS-891. DataNode no longer needs to check for dfs.network.script.
(harsh via eli)
HDFS-3305. GetImageServlet should consider SBN a valid requestor in a
secure HA setup. (atm)
HDFS-3314. HttpFS operation for getHomeDirectory is incorrect. (tucu)
HDFS-3319. Change DFSOutputStream to not to start a thread in constructors.
(szetszwo)
HDFS-3181. Fix a test case in TestLeaseRecovery2. (szetszwo)
HDFS-3309. HttpFS (Hoop) chmod not supporting octal and sticky bit
permissions. (tucu)
HDFS-3326. Append enabled log message uses the wrong variable.
(Matthew Jacobs via eli)
HDFS-3336. hdfs launcher script will be better off not special casing
namenode command with regards to hadoop.security.logger (rvs via tucu)
HDFS-3330. If GetImageServlet throws an Error or RTE, response should not
have HTTP "OK" status. (todd)
BREAKDOWN OF HDFS-1623 SUBTASKS
HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
@ -871,6 +925,23 @@ Release 0.23.3 - UNRELEASED
HDFS-2652. Add support for host-based delegation tokens. (Daryn Sharp via
szetszwo)
HDFS-3308. Uses canonical URI to select delegation tokens in HftpFileSystem
and WebHdfsFileSystem. (Daryn Sharp via szetszwo)
HDFS-3312. In HftpFileSystem, the namenode URI is non-secure but the
delegation tokens have to use secure URI. (Daryn Sharp via szetszwo)
HDFS-3318. Use BoundedInputStream in ByteRangeInputStream, otherwise, it
hangs on transfers >2 GB. (Daryn Sharp via szetszwo)
HDFS-3321. Fix safe mode turn off tip message. (Ravi Prakash via szetszwo)
HDFS-3334. Fix ByteRangeInputStream stream leakage. (Daryn Sharp via
szetszwo)
HDFS-3331. In namenode, check superuser privilege for setBalancerBandwidth
and acquire the write lock for finalizeUpgrade. (szetszwo)
Release 0.23.2 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -12,19 +12,25 @@ How do I build?
To generate the distribution packages for BK journal, do the
following.
$ mvn clean install -Pdist -Dtar
$ mvn clean package -Pdist
This will generate a tarball,
target/hadoop-hdfs-bkjournal-<VERSION>.tar.gz
This will generate a jar with all the dependencies needed by the journal
manager,
target/hadoop-hdfs-bkjournal-<VERSION>.jar
Note that the -Pdist part of the build command is important, as otherwise
the dependencies would not be packaged in the jar.
-------------------------------------------------------------------------------
How do I use the BookKeeper Journal?
To run a HDFS namenode using BookKeeper as a backend, extract the
distribution package on top of hdfs
To run a HDFS namenode using BookKeeper as a backend, copy the bkjournal
jar, generated above, into the lib directory of hdfs. In the standard
distribution of HDFS, this is at $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
cd hadoop-hdfs-<VERSION>/
tar --strip-components 1 -zxvf path/to/hadoop-hdfs-bkjournal-<VERSION>.tar.gz
cp target/hadoop-hdfs-bkjournal-<VERSION>.jar \
$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
Then, in hdfs-site.xml, set the following properties.

View File

@ -65,4 +65,50 @@
<scope>test</scope>
</dependency>
</dependencies>
<profiles>
<profile>
<id>dist</id>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<version>1.5</version>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<createDependencyReducedPom>false</createDependencyReducedPom>
<artifactSet>
<includes>
<include>org.apache.bookkeeper:bookkeeper-server</include>
<include>org.apache.zookeeper:zookeeper</include>
<include>org.jboss.netty:netty</include>
</includes>
</artifactSet>
<relocations>
<relocation>
<pattern>org.apache.bookkeeper</pattern>
<shadedPattern>hidden.bkjournal.org.apache.bookkeeper</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.zookeeper</pattern>
<shadedPattern>hidden.bkjournal.org.apache.zookeeper</shadedPattern>
</relocation>
<relocation>
<pattern>org.jboss.netty</pattern>
<shadedPattern>hidden.bkjournal.org.jboss.netty</shadedPattern>
</relocation>
</relocations>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View File

@ -122,12 +122,7 @@ if $cygwin; then
fi
export CLASSPATH=$CLASSPATH
#turn security logger on the namenode
if [ $COMMAND = "namenode" ]; then
HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,RFAS}"
else
HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,NullAppender}"
fi
HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,NullAppender}"
# Check to see if we should start a secure datanode
if [ "$starting_secure_dn" = "true" ]; then

View File

@ -35,6 +35,8 @@ import org.apache.hadoop.hdfs.CorruptFileBlockIterator;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@ -43,8 +45,8 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
import org.apache.hadoop.util.Progressable;
@ -88,11 +90,11 @@ public class Hdfs extends AbstractFileSystem {
}
@Override
public FSDataOutputStream createInternal(Path f,
public HdfsDataOutputStream createInternal(Path f,
EnumSet<CreateFlag> createFlag, FsPermission absolutePermission,
int bufferSize, short replication, long blockSize, Progressable progress,
int bytesPerChecksum, boolean createParent) throws IOException {
return new FSDataOutputStream(dfs.primitiveCreate(getUriPath(f),
return new HdfsDataOutputStream(dfs.primitiveCreate(getUriPath(f),
absolutePermission, createFlag, createParent, replication, blockSize,
progress, bufferSize, bytesPerChecksum), getStatistics());
}
@ -324,8 +326,9 @@ public class Hdfs extends AbstractFileSystem {
dfs.mkdirs(getUriPath(dir), permission, createParent);
}
@SuppressWarnings("deprecation")
@Override
public FSDataInputStream open(Path f, int bufferSize)
public HdfsDataInputStream open(Path f, int bufferSize)
throws IOException, UnresolvedLinkException {
return new DFSClient.DFSDataInputStream(dfs.open(getUriPath(f),
bufferSize, verifyChecksum));

View File

@ -23,9 +23,12 @@ import java.io.InputStream;
import java.net.HttpURLConnection;
import java.net.URL;
import org.apache.commons.io.input.BoundedInputStream;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
import com.google.common.annotations.VisibleForTesting;
/**
* To support HTTP byte streams, a new connection to an HTTP server needs to be
* created each time. This class hides the complexity of those multiple
@ -60,7 +63,7 @@ public abstract class ByteRangeInputStream extends FSInputStream {
}
enum StreamStatus {
NORMAL, SEEK
NORMAL, SEEK, CLOSED
}
protected InputStream in;
protected URLOpener originalURL;
@ -88,66 +91,93 @@ public abstract class ByteRangeInputStream extends FSInputStream {
protected abstract URL getResolvedUrl(final HttpURLConnection connection
) throws IOException;
private InputStream getInputStream() throws IOException {
if (status != StreamStatus.NORMAL) {
if (in != null) {
in.close();
in = null;
}
// Use the original url if no resolved url exists, eg. if
// it's the first time a request is made.
final URLOpener opener =
(resolvedURL.getURL() == null) ? originalURL : resolvedURL;
final HttpURLConnection connection = opener.openConnection(startPos);
connection.connect();
checkResponseCode(connection);
final String cl = connection.getHeaderField(StreamFile.CONTENT_LENGTH);
filelength = (cl == null) ? -1 : Long.parseLong(cl);
in = connection.getInputStream();
resolvedURL.setURL(getResolvedUrl(connection));
status = StreamStatus.NORMAL;
@VisibleForTesting
protected InputStream getInputStream() throws IOException {
switch (status) {
case NORMAL:
break;
case SEEK:
if (in != null) {
in.close();
}
in = openInputStream();
status = StreamStatus.NORMAL;
break;
case CLOSED:
throw new IOException("Stream closed");
}
return in;
}
private void update(final boolean isEOF, final int n)
throws IOException {
if (!isEOF) {
@VisibleForTesting
protected InputStream openInputStream() throws IOException {
// Use the original url if no resolved url exists, eg. if
// it's the first time a request is made.
final URLOpener opener =
(resolvedURL.getURL() == null) ? originalURL : resolvedURL;
final HttpURLConnection connection = opener.openConnection(startPos);
connection.connect();
checkResponseCode(connection);
final String cl = connection.getHeaderField(StreamFile.CONTENT_LENGTH);
if (cl == null) {
throw new IOException(StreamFile.CONTENT_LENGTH+" header is missing");
}
final long streamlength = Long.parseLong(cl);
filelength = startPos + streamlength;
// Java has a bug with >2GB request streams. It won't bounds check
// the reads so the transfer blocks until the server times out
InputStream is =
new BoundedInputStream(connection.getInputStream(), streamlength);
resolvedURL.setURL(getResolvedUrl(connection));
return is;
}
private int update(final int n) throws IOException {
if (n != -1) {
currentPos += n;
} else if (currentPos < filelength) {
throw new IOException("Got EOF but currentPos = " + currentPos
+ " < filelength = " + filelength);
}
return n;
}
@Override
public int read() throws IOException {
final int b = getInputStream().read();
update(b == -1, 1);
update((b == -1) ? -1 : 1);
return b;
}
@Override
public int read(byte b[], int off, int len) throws IOException {
return update(getInputStream().read(b, off, len));
}
/**
* Seek to the given offset from the start of the file.
* The next read() will be from that location. Can't
* seek past the end of the file.
*/
@Override
public void seek(long pos) throws IOException {
if (pos != currentPos) {
startPos = pos;
currentPos = pos;
status = StreamStatus.SEEK;
if (status != StreamStatus.CLOSED) {
status = StreamStatus.SEEK;
}
}
}
/**
* Return the current offset from the start of the file
*/
@Override
public long getPos() throws IOException {
return currentPos;
}
@ -156,7 +186,17 @@ public abstract class ByteRangeInputStream extends FSInputStream {
* Seeks a different copy of the data. Returns true if
* found a new source, false otherwise.
*/
@Override
public boolean seekToNewSource(long targetPos) throws IOException {
return false;
}
}
@Override
public void close() throws IOException {
if (in != null) {
in.close();
in = null;
}
status = StreamStatus.CLOSED;
}
}

View File

@ -78,8 +78,6 @@ import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsServerDefaults;
@ -91,6 +89,8 @@ import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@ -996,7 +996,7 @@ public class DFSClient implements java.io.Closeable {
* Call {@link #create(String, FsPermission, EnumSet, boolean, short,
* long, Progressable, int)} with <code>createParent</code> set to true.
*/
public OutputStream create(String src,
public DFSOutputStream create(String src,
FsPermission permission,
EnumSet<CreateFlag> flag,
short replication,
@ -1029,7 +1029,7 @@ public class DFSClient implements java.io.Closeable {
* @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable,
* boolean, short, long) for detailed description of exceptions thrown
*/
public OutputStream create(String src,
public DFSOutputStream create(String src,
FsPermission permission,
EnumSet<CreateFlag> flag,
boolean createParent,
@ -1046,9 +1046,9 @@ public class DFSClient implements java.io.Closeable {
if(LOG.isDebugEnabled()) {
LOG.debug(src + ": masked=" + masked);
}
final DFSOutputStream result = new DFSOutputStream(this, src, masked, flag,
createParent, replication, blockSize, progress, buffersize,
dfsClientConf.createChecksum());
final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
src, masked, flag, createParent, replication, blockSize, progress,
buffersize, dfsClientConf.createChecksum());
leaserenewer.put(src, result, this);
return result;
}
@ -1078,7 +1078,7 @@ public class DFSClient implements java.io.Closeable {
* Progressable, int)} except that the permission
* is absolute (ie has already been masked with umask.
*/
public OutputStream primitiveCreate(String src,
public DFSOutputStream primitiveCreate(String src,
FsPermission absPermission,
EnumSet<CreateFlag> flag,
boolean createParent,
@ -1095,7 +1095,7 @@ public class DFSClient implements java.io.Closeable {
DataChecksum checksum = DataChecksum.newDataChecksum(
dfsClientConf.checksumType,
bytesPerChecksum);
result = new DFSOutputStream(this, src, absPermission,
result = DFSOutputStream.newStreamForCreate(this, src, absPermission,
flag, createParent, replication, blockSize, progress, buffersize,
checksum);
}
@ -1154,7 +1154,7 @@ public class DFSClient implements java.io.Closeable {
UnsupportedOperationException.class,
UnresolvedPathException.class);
}
return new DFSOutputStream(this, src, buffersize, progress,
return DFSOutputStream.newStreamForAppend(this, src, buffersize, progress,
lastBlock, stat, dfsClientConf.createChecksum());
}
@ -1169,11 +1169,11 @@ public class DFSClient implements java.io.Closeable {
*
* @see ClientProtocol#append(String, String)
*/
public FSDataOutputStream append(final String src, final int buffersize,
public HdfsDataOutputStream append(final String src, final int buffersize,
final Progressable progress, final FileSystem.Statistics statistics
) throws IOException {
final DFSOutputStream out = append(src, buffersize, progress);
return new FSDataOutputStream(out, statistics, out.getInitialLen());
return new HdfsDataOutputStream(out, statistics, out.getInitialLen());
}
private DFSOutputStream append(String src, int buffersize, Progressable progress)
@ -1809,41 +1809,13 @@ public class DFSClient implements java.io.Closeable {
}
/**
* The Hdfs implementation of {@link FSDataInputStream}
* @deprecated use {@link HdfsDataInputStream} instead.
*/
@InterfaceAudience.Private
public static class DFSDataInputStream extends FSDataInputStream {
public DFSDataInputStream(DFSInputStream in)
throws IOException {
super(in);
}
/**
* Returns the datanode from which the stream is currently reading.
*/
public DatanodeInfo getCurrentDatanode() {
return ((DFSInputStream)in).getCurrentDatanode();
}
/**
* Returns the block containing the target position.
*/
public ExtendedBlock getCurrentBlock() {
return ((DFSInputStream)in).getCurrentBlock();
}
@Deprecated
public static class DFSDataInputStream extends HdfsDataInputStream {
/**
* Return collection of blocks that has already been located.
*/
synchronized List<LocatedBlock> getAllBlocks() throws IOException {
return ((DFSInputStream)in).getAllBlocks();
}
/**
* @return The visible length of the file.
*/
public long getVisibleLength() throws IOException {
return ((DFSInputStream)in).getFileLength();
public DFSDataInputStream(DFSInputStream in) throws IOException {
super(in);
}
}

View File

@ -118,6 +118,39 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
* Grab the open-file info from namenode
*/
synchronized void openInfo() throws IOException, UnresolvedLinkException {
lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
int retriesForLastBlockLength = 3;
while (retriesForLastBlockLength > 0) {
// Getting last block length as -1 is a special case. When cluster
// restarts, DNs may not report immediately. At this time partial block
// locations will not be available with NN for getting the length. Lets
// retry for 3 times to get the length.
if (lastBlockBeingWrittenLength == -1) {
DFSClient.LOG.warn("Last block locations not available. "
+ "Datanodes might not have reported blocks completely."
+ " Will retry for " + retriesForLastBlockLength + " times");
waitFor(4000);
lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
} else {
break;
}
retriesForLastBlockLength--;
}
if (retriesForLastBlockLength == 0) {
throw new IOException("Could not obtain the last block locations.");
}
}
private void waitFor(int waitTime) throws IOException {
try {
Thread.sleep(waitTime);
} catch (InterruptedException e) {
throw new IOException(
"Interrupted while getting the last block length.");
}
}
private long fetchLocatedBlocksAndGetLastBlockLength() throws IOException {
LocatedBlocks newInfo = DFSClient.callGetBlockLocations(dfsClient.namenode, src, 0, prefetchSize);
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("newInfo = " + newInfo);
@ -136,10 +169,13 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
}
}
locatedBlocks = newInfo;
lastBlockBeingWrittenLength = 0;
long lastBlockBeingWrittenLength = 0;
if (!locatedBlocks.isLastBlockComplete()) {
final LocatedBlock last = locatedBlocks.getLastLocatedBlock();
if (last != null) {
if (last.getLocations().length == 0) {
return -1;
}
final long len = readBlockLength(last);
last.getBlock().setNumBytes(len);
lastBlockBeingWrittenLength = len;
@ -147,13 +183,12 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
}
currentNode = null;
return lastBlockBeingWrittenLength;
}
/** Read the block length from one of the datanodes. */
private long readBlockLength(LocatedBlock locatedblock) throws IOException {
if (locatedblock == null || locatedblock.getLocations().length == 0) {
return 0;
}
assert locatedblock != null : "LocatedBlock cannot be null";
int replicaNotFoundCount = locatedblock.getLocations().length;
for(DatanodeInfo datanode : locatedblock.getLocations()) {
@ -224,7 +259,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
/**
* Return collection of blocks that has already been located.
*/
synchronized List<LocatedBlock> getAllBlocks() throws IOException {
public synchronized List<LocatedBlock> getAllBlocks() throws IOException {
return getBlockRange(0, getFileLength());
}

View File

@ -44,7 +44,7 @@ import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -99,7 +99,7 @@ import org.apache.hadoop.util.Progressable;
* starts sending packets from the dataQueue.
****************************************************************/
@InterfaceAudience.Private
class DFSOutputStream extends FSOutputSummer implements Syncable {
public class DFSOutputStream extends FSOutputSummer implements Syncable {
private final DFSClient dfsClient;
private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
private Socket s;
@ -1233,14 +1233,11 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
this.checksum = checksum;
}
/**
* Create a new output stream to the given DataNode.
* @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable, boolean, short, long)
*/
DFSOutputStream(DFSClient dfsClient, String src, FsPermission masked, EnumSet<CreateFlag> flag,
boolean createParent, short replication, long blockSize, Progressable progress,
int buffersize, DataChecksum checksum)
throws IOException {
/** Construct a new output stream for creating a file. */
private DFSOutputStream(DFSClient dfsClient, String src, FsPermission masked,
EnumSet<CreateFlag> flag, boolean createParent, short replication,
long blockSize, Progressable progress, int buffersize,
DataChecksum checksum) throws IOException {
this(dfsClient, src, blockSize, progress, checksum, replication);
computePacketChunkSize(dfsClient.getConf().writePacketSize,
@ -1260,14 +1257,21 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
UnresolvedPathException.class);
}
streamer = new DataStreamer();
streamer.start();
}
/**
* Create a new output stream to the given DataNode.
* @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
*/
DFSOutputStream(DFSClient dfsClient, String src, int buffersize, Progressable progress,
static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
short replication, long blockSize, Progressable progress, int buffersize,
DataChecksum checksum) throws IOException {
final DFSOutputStream out = new DFSOutputStream(dfsClient, src, masked,
flag, createParent, replication, blockSize, progress, buffersize,
checksum);
out.streamer.start();
return out;
}
/** Construct a new output stream for append. */
private DFSOutputStream(DFSClient dfsClient, String src, int buffersize, Progressable progress,
LocatedBlock lastBlock, HdfsFileStatus stat,
DataChecksum checksum) throws IOException {
this(dfsClient, src, stat.getBlockSize(), progress, checksum, stat.getReplication());
@ -1285,7 +1289,15 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
checksum.getBytesPerChecksum());
streamer = new DataStreamer();
}
streamer.start();
}
static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
int buffersize, Progressable progress, LocatedBlock lastBlock,
HdfsFileStatus stat, DataChecksum checksum) throws IOException {
final DFSOutputStream out = new DFSOutputStream(dfsClient, src, buffersize,
progress, lastBlock, stat, checksum);
out.streamer.start();
return out;
}
private void computePacketChunkSize(int psize, int csize) {
@ -1530,14 +1542,20 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
}
/**
* Returns the number of replicas of current block. This can be different
* from the designated replication factor of the file because the NameNode
* does not replicate the block to which a client is currently writing to.
* The client continues to write to a block even if a few datanodes in the
* write pipeline have failed.
* @deprecated use {@link HdfsDataOutputStream#getCurrentBlockReplication()}.
*/
@Deprecated
public synchronized int getNumCurrentReplicas() throws IOException {
return getCurrentBlockReplication();
}
/**
* Note that this is not a public API;
* use {@link HdfsDataOutputStream#getCurrentBlockReplication()} instead.
*
* @return the number of valid replicas of the current block
*/
public synchronized int getNumCurrentReplicas() throws IOException {
public synchronized int getCurrentBlockReplication() throws IOException {
dfsClient.checkOpen();
isClosed();
if (streamer == null) {

View File

@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.security.UserGroupInformation;
@ -139,37 +138,6 @@ public class DFSUtil {
return true;
}
/**
* Utility class to facilitate junit test error simulation.
*/
@InterfaceAudience.Private
public static class ErrorSimulator {
private static boolean[] simulation = null; // error simulation events
public static void initializeErrorSimulationEvent(int numberOfEvents) {
simulation = new boolean[numberOfEvents];
for (int i = 0; i < numberOfEvents; i++) {
simulation[i] = false;
}
}
public static boolean getErrorSimulation(int index) {
if(simulation == null)
return false;
assert(index < simulation.length);
return simulation[index];
}
public static void setErrorSimulation(int index) {
assert(index < simulation.length);
simulation[index] = true;
}
public static void clearErrorSimulation(int index) {
assert(index < simulation.length);
simulation[index] = false;
}
}
/**
* Converts a byte array to a string using UTF8 encoding.
*/
@ -1010,7 +978,7 @@ public class DFSUtil {
public static void addPBProtocol(Configuration conf, Class<?> protocol,
BlockingService service, RPC.Server server) throws IOException {
RPC.setProtocolEngine(conf, protocol, ProtobufRpcEngine.class);
server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
}
/**

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsServerDefaults;
@ -45,7 +44,8 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -88,6 +88,17 @@ public class DistributedFileSystem extends FileSystem {
public DistributedFileSystem() {
}
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>hdfs</code>
*/
@Override
public String getScheme() {
return "hdfs";
}
@Deprecated
public DistributedFileSystem(InetSocketAddress namenode,
Configuration conf) throws IOException {
@ -194,8 +205,9 @@ public class DistributedFileSystem extends FileSystem {
return dfs.recoverLease(getPathName(f));
}
@SuppressWarnings("deprecation")
@Override
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
public HdfsDataInputStream open(Path f, int bufferSize) throws IOException {
statistics.incrementReadOps(1);
return new DFSClient.DFSDataInputStream(
dfs.open(getPathName(f), bufferSize, verifyChecksum));
@ -203,31 +215,33 @@ public class DistributedFileSystem extends FileSystem {
/** This optional operation is not yet supported. */
@Override
public FSDataOutputStream append(Path f, int bufferSize,
public HdfsDataOutputStream append(Path f, int bufferSize,
Progressable progress) throws IOException {
statistics.incrementWriteOps(1);
return dfs.append(getPathName(f), bufferSize, progress, statistics);
}
@Override
public FSDataOutputStream create(Path f, FsPermission permission,
public HdfsDataOutputStream create(Path f, FsPermission permission,
boolean overwrite, int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException {
statistics.incrementWriteOps(1);
return new FSDataOutputStream(dfs.create(getPathName(f), permission,
overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
: EnumSet.of(CreateFlag.CREATE), replication, blockSize, progress,
bufferSize), statistics);
final EnumSet<CreateFlag> cflags = overwrite?
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
: EnumSet.of(CreateFlag.CREATE);
final DFSOutputStream out = dfs.create(getPathName(f), permission, cflags,
replication, blockSize, progress, bufferSize);
return new HdfsDataOutputStream(out, statistics);
}
@SuppressWarnings("deprecation")
@Override
protected FSDataOutputStream primitiveCreate(Path f,
protected HdfsDataOutputStream primitiveCreate(Path f,
FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
short replication, long blockSize, Progressable progress,
int bytesPerChecksum) throws IOException {
statistics.incrementReadOps(1);
return new FSDataOutputStream(dfs.primitiveCreate(getPathName(f),
return new HdfsDataOutputStream(dfs.primitiveCreate(getPathName(f),
absolutePermission, flag, true, replication, blockSize,
progress, bufferSize, bytesPerChecksum),statistics);
}
@ -235,14 +249,14 @@ public class DistributedFileSystem extends FileSystem {
/**
* Same as create(), except fails if parent directory doesn't already exist.
*/
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
public HdfsDataOutputStream createNonRecursive(Path f, FsPermission permission,
EnumSet<CreateFlag> flag, int bufferSize, short replication,
long blockSize, Progressable progress) throws IOException {
statistics.incrementWriteOps(1);
if (flag.contains(CreateFlag.OVERWRITE)) {
flag.add(CreateFlag.CREATE);
}
return new FSDataOutputStream(dfs.create(getPathName(f), permission, flag,
return new HdfsDataOutputStream(dfs.create(getPathName(f), permission, flag,
false, replication, blockSize, progress, bufferSize), statistics);
}
@ -627,14 +641,14 @@ public class DistributedFileSystem extends FileSystem {
FSDataInputStream in, long inPos,
FSDataInputStream sums, long sumsPos) {
if(!(in instanceof DFSDataInputStream && sums instanceof DFSDataInputStream))
throw new IllegalArgumentException("Input streams must be types " +
"of DFSDataInputStream");
if(!(in instanceof HdfsDataInputStream && sums instanceof HdfsDataInputStream))
throw new IllegalArgumentException(
"Input streams must be types of HdfsDataInputStream");
LocatedBlock lblocks[] = new LocatedBlock[2];
// Find block in data stream.
DFSClient.DFSDataInputStream dfsIn = (DFSClient.DFSDataInputStream) in;
HdfsDataInputStream dfsIn = (HdfsDataInputStream) in;
ExtendedBlock dataBlock = dfsIn.getCurrentBlock();
if (dataBlock == null) {
LOG.error("Error: Current block in data stream is null! ");
@ -647,7 +661,7 @@ public class DistributedFileSystem extends FileSystem {
+ dataNode[0]);
// Find block in checksum stream
DFSClient.DFSDataInputStream dfsSums = (DFSClient.DFSDataInputStream) sums;
HdfsDataInputStream dfsSums = (HdfsDataInputStream) sums;
ExtendedBlock sumsBlock = dfsSums.getCurrentBlock();
if (sumsBlock == null) {
LOG.error("Error: Current block in checksum stream is null! ");

View File

@ -94,8 +94,8 @@ public class HftpFileSystem extends FileSystem
protected UserGroupInformation ugi;
private URI hftpURI;
protected InetSocketAddress nnAddr;
protected InetSocketAddress nnSecureAddr;
protected URI nnUri;
protected URI nnSecureUri;
public static final String HFTP_TIMEZONE = "UTC";
public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
@ -139,11 +139,30 @@ public class HftpFileSystem extends FileSystem
return NetUtils.createSocketAddrForHost(uri.getHost(), getDefaultSecurePort());
}
protected URI getNamenodeUri(URI uri) {
return DFSUtil.createUri("http", getNamenodeAddr(uri));
}
protected URI getNamenodeSecureUri(URI uri) {
return DFSUtil.createUri("https", getNamenodeSecureAddr(uri));
}
@Override
public String getCanonicalServiceName() {
// unlike other filesystems, hftp's service is the secure port, not the
// actual port in the uri
return SecurityUtil.buildTokenService(nnSecureAddr).toString();
return SecurityUtil.buildTokenService(nnSecureUri).toString();
}
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>hftp</code>
*/
@Override
public String getScheme() {
return "hftp";
}
@Override
@ -152,8 +171,8 @@ public class HftpFileSystem extends FileSystem
super.initialize(name, conf);
setConf(conf);
this.ugi = UserGroupInformation.getCurrentUser();
this.nnAddr = getNamenodeAddr(name);
this.nnSecureAddr = getNamenodeSecureAddr(name);
this.nnUri = getNamenodeUri(name);
this.nnSecureUri = getNamenodeSecureUri(name);
try {
this.hftpURI = new URI(name.getScheme(), name.getAuthority(),
null, null, null);
@ -168,7 +187,7 @@ public class HftpFileSystem extends FileSystem
protected void initDelegationToken() throws IOException {
// look for hftp token, then try hdfs
Token<?> token = selectDelegationToken();
Token<?> token = selectDelegationToken(ugi);
// if we don't already have a token, go get one over https
boolean createdToken = false;
@ -189,8 +208,9 @@ public class HftpFileSystem extends FileSystem
}
}
protected Token<DelegationTokenIdentifier> selectDelegationToken() {
return hftpTokenSelector.selectToken(getUri(), ugi.getTokens(), getConf());
protected Token<DelegationTokenIdentifier> selectDelegationToken(
UserGroupInformation ugi) {
return hftpTokenSelector.selectToken(nnSecureUri, ugi.getTokens(), getConf());
}
@ -221,7 +241,7 @@ public class HftpFileSystem extends FileSystem
ugi.reloginFromKeytab();
return ugi.doAs(new PrivilegedExceptionAction<Token<?>>() {
public Token<?> run() throws IOException {
final String nnHttpUrl = DFSUtil.createUri("https", nnSecureAddr).toString();
final String nnHttpUrl = nnSecureUri.toString();
Credentials c;
try {
c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer);
@ -263,8 +283,8 @@ public class HftpFileSystem extends FileSystem
* @throws IOException on error constructing the URL
*/
protected URL getNamenodeURL(String path, String query) throws IOException {
final URL url = new URL("http", nnAddr.getHostName(),
nnAddr.getPort(), path + '?' + query);
final URL url = new URL("http", nnUri.getHost(),
nnUri.getPort(), path + '?' + query);
if (LOG.isTraceEnabled()) {
LOG.trace("url=" + url);
}

View File

@ -58,6 +58,17 @@ public class HsftpFileSystem extends HftpFileSystem {
private static final long MM_SECONDS_PER_DAY = 1000 * 60 * 60 * 24;
private volatile int ExpWarnDays = 0;
/**
* Return the protocol scheme for the FileSystem.
* <p/>
*
* @return <code>hsftp</code>
*/
@Override
public String getScheme() {
return "hsftp";
}
@Override
public void initialize(URI name, Configuration conf) throws IOException {
super.initialize(name, conf);
@ -132,12 +143,17 @@ public class HsftpFileSystem extends HftpFileSystem {
return getNamenodeAddr(uri);
}
@Override
protected URI getNamenodeUri(URI uri) {
return getNamenodeSecureUri(uri);
}
@Override
protected HttpURLConnection openConnection(String path, String query)
throws IOException {
query = addDelegationTokenParam(query);
final URL url = new URL("https", nnAddr.getHostName(),
nnAddr.getPort(), path + '?' + query);
final URL url = new URL("https", nnUri.getHost(),
nnUri.getPort(), path + '?' + query);
HttpsURLConnection conn = (HttpsURLConnection)URLUtils.openConnection(url);
// bypass hostname verification
conn.setHostnameVerifier(new DummyHostnameVerifier());

View File

@ -0,0 +1,71 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.client;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.hdfs.DFSInputStream;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
/**
* The Hdfs implementation of {@link FSDataInputStream}.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class HdfsDataInputStream extends FSDataInputStream {
public HdfsDataInputStream(DFSInputStream in) throws IOException {
super(in);
}
/**
* Get the datanode from which the stream is currently reading.
*/
public DatanodeInfo getCurrentDatanode() {
return ((DFSInputStream) in).getCurrentDatanode();
}
/**
* Get the block containing the target position.
*/
public ExtendedBlock getCurrentBlock() {
return ((DFSInputStream) in).getCurrentBlock();
}
/**
* Get the collection of blocks that has already been located.
*/
public synchronized List<LocatedBlock> getAllBlocks() throws IOException {
return ((DFSInputStream) in).getAllBlocks();
}
/**
* Get the visible length of the file. It will include the length of the last
* block even if that is in UnderConstruction state.
*
* @return The visible length of the file.
*/
public long getVisibleLength() throws IOException {
return ((DFSInputStream) in).getFileLength();
}
}

View File

@ -0,0 +1,59 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.client;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.DFSOutputStream;
/**
* The Hdfs implementation of {@link FSDataOutputStream}.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class HdfsDataOutputStream extends FSDataOutputStream {
public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats,
long startPosition) throws IOException {
super(out, stats, startPosition);
}
public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats
) throws IOException {
this(out, stats, 0L);
}
/**
* Get the actual number of replicas of the current block.
*
* This can be different from the designated replication factor of the file
* because the namenode does not maintain replication for the blocks which are
* currently being written to. Depending on the configuration, the client may
* continue to write to a block even if a few datanodes in the write pipeline
* have failed, or the client may add a new datanodes once a datanode has
* failed.
*
* @return the number of valid replicas of the current block
*/
public synchronized int getCurrentBlockReplication() throws IOException {
return ((DFSOutputStream)getWrappedStream()).getCurrentBlockReplication();
}
}

View File

@ -46,7 +46,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
@ -193,7 +192,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
@Override
public boolean isMethodSupported(String methodName) throws IOException {
return RpcClientUtil.isMethodSupported(rpcProxy,
ClientDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
ClientDatanodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), methodName);
}

View File

@ -109,7 +109,6 @@ import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.token.Token;
@ -812,7 +811,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean isMethodSupported(String methodName) throws IOException {
return RpcClientUtil.isMethodSupported(rpcProxy,
ClientNamenodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
}

View File

@ -69,7 +69,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
@ -308,7 +307,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
public boolean isMethodSupported(String methodName)
throws IOException {
return RpcClientUtil.isMethodSupported(rpcProxy, DatanodeProtocolPB.class,
RpcKind.RPC_PROTOCOL_BUFFER,
RPC.RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(DatanodeProtocolPB.class), methodName);
}
}

View File

@ -26,7 +26,6 @@ import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.tools.GetUserMappingsProtocol;
import com.google.protobuf.RpcController;
@ -65,7 +64,7 @@ public class GetUserMappingsProtocolClientSideTranslatorPB implements
@Override
public boolean isMethodSupported(String methodName) throws IOException {
return RpcClientUtil.isMethodSupported(rpcProxy,
GetUserMappingsProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
GetUserMappingsProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(GetUserMappingsProtocolPB.class), methodName);
}
}

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.security.UserGroupInformation;
import com.google.protobuf.RpcController;
@ -119,7 +118,7 @@ public class InterDatanodeProtocolTranslatorPB implements
@Override
public boolean isMethodSupported(String methodName) throws IOException {
return RpcClientUtil.isMethodSupported(rpcProxy,
InterDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
InterDatanodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(InterDatanodeProtocolPB.class), methodName);
}
}

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@ -109,7 +108,7 @@ public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
@Override
public boolean isMethodSupported(String methodName) throws IOException {
return RpcClientUtil.isMethodSupported(rpcProxy, JournalProtocolPB.class,
RpcKind.RPC_PROTOCOL_BUFFER,
RPC.RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(JournalProtocolPB.class), methodName);
}
}

Some files were not shown because too many files have changed in this diff Show More