diff --git a/BUILDING.txt b/BUILDING.txt
index 06bef1fc33048..6e38ad374a174 100644
--- a/BUILDING.txt
+++ b/BUILDING.txt
@@ -4,7 +4,7 @@ Build instructions for Hadoop
Requirements:
* Unix System
-* JDK 1.6+
+* JDK 1.7+
* Maven 3.0 or later
* Findbugs 1.3.9 (if running findbugs)
* ProtocolBuffer 2.5.0
@@ -204,15 +204,19 @@ Building on Windows
Requirements:
* Windows System
-* JDK 1.6+
+* JDK 1.7+
* Maven 3.0 or later
* Findbugs 1.3.9 (if running findbugs)
* ProtocolBuffer 2.5.0
* CMake 2.6 or newer
* Windows SDK or Visual Studio 2010 Professional
-* Unix command-line tools from GnuWin32 or Cygwin: sh, mkdir, rm, cp, tar, gzip
* zlib headers (if building native code bindings for zlib)
* Internet connection for first build (to fetch all Maven and Hadoop dependencies)
+* Unix command-line tools from GnuWin32: sh, mkdir, rm, cp, tar, gzip. These
+ tools must be present on your PATH.
+
+Unix command-line tools are also included with the Windows Git package which
+can be downloaded from http://git-scm.com/download/win.
If using Visual Studio, it must be Visual Studio 2010 Professional (not 2012).
Do not use Visual Studio Express. It does not support compiling for 64-bit,
@@ -221,6 +225,8 @@ download here:
http://www.microsoft.com/en-us/download/details.aspx?id=8279
+Cygwin is neither required nor supported.
+
----------------------------------------------------------------------------------
Building:
diff --git a/dev-support/determine-flaky-tests-hadoop.py b/dev-support/determine-flaky-tests-hadoop.py
new file mode 100755
index 0000000000000..ce152bacaefd5
--- /dev/null
+++ b/dev-support/determine-flaky-tests-hadoop.py
@@ -0,0 +1,204 @@
+#!/usr/bin/env python
+#
+# 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.
+#
+# Given a jenkins test job, this script examines all runs of the job done
+# within specified period of time (number of days prior to the execution
+# time of this script), and reports all failed tests.
+#
+# The output of this script includes a section for each run that has failed
+# tests, with each failed test name listed.
+#
+# More importantly, at the end, it outputs a summary section to list all failed
+# tests within all examined runs, and indicate how many runs a same test
+# failed, and sorted all failed tests by how many runs each test failed.
+#
+# This way, when we see failed tests in PreCommit build, we can quickly tell
+# whether a failed test is a new failure, or it failed before and how often it
+# failed, so to have idea whether it may just be a flaky test.
+#
+# Of course, to be 100% sure about the reason of a test failure, closer look
+# at the failed test for the specific run is necessary.
+#
+import sys
+import platform
+sysversion = sys.hexversion
+onward30 = False
+if sysversion < 0x020600F0:
+ sys.exit("Minimum supported python version is 2.6, the current version is " +
+ "Python" + platform.python_version())
+
+if sysversion == 0x030000F0:
+ sys.exit("There is a known bug with Python" + platform.python_version() +
+ ", please try a different version");
+
+if sysversion < 0x03000000:
+ import urllib2
+else:
+ onward30 = True
+ import urllib.request
+
+import datetime
+import json as simplejson
+import logging
+from optparse import OptionParser
+import time
+
+# Configuration
+DEFAULT_JENKINS_URL = "https://builds.apache.org"
+DEFAULT_JOB_NAME = "Hadoop-Common-trunk"
+DEFAULT_NUM_PREVIOUS_DAYS = 14
+
+SECONDS_PER_DAY = 86400
+
+# total number of runs to examine
+numRunsToExamine = 0
+
+""" Parse arguments """
+def parse_args():
+ parser = OptionParser()
+ parser.add_option("-J", "--jenkins-url", type="string",
+ dest="jenkins_url", help="Jenkins URL",
+ default=DEFAULT_JENKINS_URL)
+ parser.add_option("-j", "--job-name", type="string",
+ dest="job_name", help="Job name to look at",
+ default=DEFAULT_JOB_NAME)
+ parser.add_option("-n", "--num-days", type="int",
+ dest="num_prev_days", help="Number of days to examine",
+ default=DEFAULT_NUM_PREVIOUS_DAYS)
+
+ (options, args) = parser.parse_args()
+ if args:
+ parser.error("unexpected arguments: " + repr(args))
+ return options
+
+""" Load data from specified url """
+def load_url_data(url):
+ if onward30:
+ ourl = urllib.request.urlopen(url)
+ codec = ourl.info().get_param('charset')
+ content = ourl.read().decode(codec)
+ data = simplejson.loads(content, strict=False)
+ else:
+ ourl = urllib2.urlopen(url)
+ data = simplejson.load(ourl, strict=False)
+ return data
+
+""" List all builds of the target project. """
+def list_builds(jenkins_url, job_name):
+ url = "%(jenkins)s/job/%(job_name)s/api/json?tree=builds[url,result,timestamp]" % dict(
+ jenkins=jenkins_url,
+ job_name=job_name)
+
+ try:
+ data = load_url_data(url)
+
+ except:
+ logging.error("Could not fetch: %s" % url)
+ raise
+ return data['builds']
+
+""" Find the names of any tests which failed in the given build output URL. """
+def find_failing_tests(testReportApiJson, jobConsoleOutput):
+ ret = set()
+ try:
+ data = load_url_data(testReportApiJson)
+
+ except:
+ logging.error(" Could not open testReport, check " +
+ jobConsoleOutput + " for why it was reported failed")
+ return ret
+
+ for suite in data['suites']:
+ for cs in suite['cases']:
+ status = cs['status']
+ errDetails = cs['errorDetails']
+ if (status == 'REGRESSION' or status == 'FAILED' or (errDetails is not None)):
+ ret.add(cs['className'] + "." + cs['name'])
+
+ if len(ret) == 0:
+ logging.info(" No failed tests in testReport, check " +
+ jobConsoleOutput + " for why it was reported failed.")
+ return ret
+
+""" Iterate runs of specfied job within num_prev_days and collect results """
+def find_flaky_tests(jenkins_url, job_name, num_prev_days):
+ global numRunsToExamine
+ all_failing = dict()
+ # First list all builds
+ builds = list_builds(jenkins_url, job_name)
+
+ # Select only those in the last N days
+ min_time = int(time.time()) - SECONDS_PER_DAY * num_prev_days
+ builds = [b for b in builds if (int(b['timestamp']) / 1000) > min_time]
+
+ # Filter out only those that failed
+ failing_build_urls = [(b['url'] , b['timestamp']) for b in builds
+ if (b['result'] in ('UNSTABLE', 'FAILURE'))]
+
+ tnum = len(builds)
+ num = len(failing_build_urls)
+ numRunsToExamine = tnum
+ logging.info(" THERE ARE " + str(num) + " builds (out of " + str(tnum)
+ + ") that have failed tests in the past " + str(num_prev_days) + " days"
+ + ((".", ", as listed below:\n")[num > 0]))
+
+ for failed_build_with_time in failing_build_urls:
+ failed_build = failed_build_with_time[0];
+ jobConsoleOutput = failed_build + "Console";
+ testReport = failed_build + "testReport";
+ testReportApiJson = testReport + "/api/json";
+
+ ts = float(failed_build_with_time[1]) / 1000.
+ st = datetime.datetime.fromtimestamp(ts).strftime('%Y-%m-%d %H:%M:%S')
+ logging.info("===>%s" % str(testReport) + " (" + st + ")")
+ failing = find_failing_tests(testReportApiJson, jobConsoleOutput)
+ if failing:
+ for ftest in failing:
+ logging.info(" Failed test: %s" % ftest)
+ all_failing[ftest] = all_failing.get(ftest,0)+1
+
+ return all_failing
+
+def main():
+ global numRunsToExamine
+ logging.basicConfig(format='%(levelname)s:%(message)s', level=logging.INFO)
+
+ # set up logger to write to stdout
+ soh = logging.StreamHandler(sys.stdout)
+ soh.setLevel(logging.INFO)
+ logger = logging.getLogger()
+ logger.removeHandler(logger.handlers[0])
+ logger.addHandler(soh)
+
+ opts = parse_args()
+ logging.info("****Recently FAILED builds in url: " + opts.jenkins_url
+ + "/job/" + opts.job_name + "")
+
+ all_failing = find_flaky_tests(opts.jenkins_url, opts.job_name,
+ opts.num_prev_days)
+ if len(all_failing) == 0:
+ raise SystemExit(0)
+ logging.info("\nAmong " + str(numRunsToExamine) + " runs examined, all failed "
+ + "tests <#failedRuns: testName>:")
+
+ # print summary section: all failed tests sorted by how many times they failed
+ for tn in sorted(all_failing, key=all_failing.get, reverse=True):
+ logging.info(" " + str(all_failing[tn])+ ": " + tn)
+
+if __name__ == "__main__":
+ main()
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index e6512abf81809..b0fbb80a8b03c 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -857,74 +857,6 @@ findModules () {
rm $TMP_MODULES
echo $CHANGED_MODULES
}
-###############################################################################
-### Run the test-contrib target
-runContribTests () {
- echo ""
- echo ""
- echo "======================================================================"
- echo "======================================================================"
- echo " Running contrib tests."
- echo "======================================================================"
- echo "======================================================================"
- echo ""
- echo ""
-
- if [[ `$GREP -c 'test-contrib' build.xml` == 0 ]] ; then
- echo "No contrib tests in this project."
- return 0
- fi
-
- ### Kill any rogue build processes from the last attempt
- $PS auxwww | $GREP ${PROJECT_NAME}PatchProcess | $AWK '{print $2}' | /usr/bin/xargs -t -I {} /bin/kill -9 {} > /dev/null
-
- #echo "$ANT_HOME/bin/ant -Dversion="${VERSION}" $ECLIPSE_PROPERTY -DHadoopPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=no test-contrib"
- #$ANT_HOME/bin/ant -Dversion="${VERSION}" $ECLIPSE_PROPERTY -DHadoopPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=no test-contrib
- echo "NOP"
- if [[ $? != 0 ]] ; then
- JIRA_COMMENT="$JIRA_COMMENT
-
- {color:red}-1 contrib tests{color}. The patch failed contrib unit tests."
- return 1
- fi
- JIRA_COMMENT="$JIRA_COMMENT
-
- {color:green}+1 contrib tests{color}. The patch passed contrib unit tests."
- return 0
-}
-
-###############################################################################
-### Run the inject-system-faults target
-checkInjectSystemFaults () {
- echo ""
- echo ""
- echo "======================================================================"
- echo "======================================================================"
- echo " Checking the integrity of system test framework code."
- echo "======================================================================"
- echo "======================================================================"
- echo ""
- echo ""
-
- ### Kill any rogue build processes from the last attempt
- $PS auxwww | $GREP ${PROJECT_NAME}PatchProcess | $AWK '{print $2}' | /usr/bin/xargs -t -I {} /bin/kill -9 {} > /dev/null
-
- #echo "$ANT_HOME/bin/ant -Dversion="${VERSION}" -DHadoopPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=no -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME inject-system-faults"
- #$ANT_HOME/bin/ant -Dversion="${VERSION}" -DHadoopPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=no -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME inject-system-faults
- echo "NOP"
- return 0
- if [[ $? != 0 ]] ; then
- JIRA_COMMENT="$JIRA_COMMENT
-
- {color:red}-1 system test framework{color}. The patch failed system test framework compile."
- return 1
- fi
- JIRA_COMMENT="$JIRA_COMMENT
-
- {color:green}+1 system test framework{color}. The patch passed system test framework compile."
- return 0
-}
-
###############################################################################
### Submit a comment to the defect's Jira
submitJiraComment () {
@@ -1059,11 +991,7 @@ checkReleaseAuditWarnings
if [[ $JENKINS == "true" || $RUN_TESTS == "true" ]] ; then
runTests
(( RESULT = RESULT + $? ))
- runContribTests
- (( RESULT = RESULT + $? ))
fi
-checkInjectSystemFaults
-(( RESULT = RESULT + $? ))
JIRA_COMMENT_FOOTER="Test results: $BUILD_URL/testReport/
$JIRA_COMMENT_FOOTER"
diff --git a/hadoop-assemblies/pom.xml b/hadoop-assemblies/pom.xml
index 66b6bdb16bd14..b53baccaad516 100644
--- a/hadoop-assemblies/pom.xml
+++ b/hadoop-assemblies/pom.xml
@@ -45,10 +45,10 @@
- [3.0.0,)
+ ${enforced.maven.version}
- 1.6
+ ${enforced.java.version}
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml
index f01953532077d..1a5d7d00cef13 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml
@@ -48,6 +48,11 @@
0755
+
+ ${basedir}/src/main/shellprofile.d
+ /libexec/shellprofile.d
+ 0755
+
${basedir}/src/main/bin
/sbin
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml
index 749e16af05581..247b09c6fa768 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml
@@ -41,6 +41,11 @@
0755
+
+ shellprofile.d
+ libexec/shellprofile.d
+ 0755
+
bin
sbin
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
index a15e1243b2426..6d386f1d1cea2 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
@@ -46,6 +46,11 @@
0755
+
+ hadoop-yarn/shellprofile.d
+ libexec/shellprofile.d
+ 0755
+
hadoop-yarn/bin
sbin
diff --git a/hadoop-client/pom.xml b/hadoop-client/pom.xml
index 2b66790f62a5b..f8b6d97ca2efe 100644
--- a/hadoop-client/pom.xml
+++ b/hadoop-client/pom.xml
@@ -13,7 +13,7 @@
limitations under the License. See accompanying LICENSE file.
-->
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
4.0.0
org.apache.hadoop
diff --git a/hadoop-common-project/hadoop-annotations/pom.xml b/hadoop-common-project/hadoop-annotations/pom.xml
index 84a106e665325..c011b4581ea30 100644
--- a/hadoop-common-project/hadoop-annotations/pom.xml
+++ b/hadoop-common-project/hadoop-annotations/pom.xml
@@ -39,23 +39,6 @@
-
- os.linux
-
-
- !Mac
-
-
-
-
- jdk.tools
- jdk.tools
- 1.6
- system
- ${java.home}/../lib/tools.jar
-
-
-
jdk1.7
diff --git a/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/WhoClient.java b/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/WhoClient.java
index 2299ae1fd8089..f5cff2b529a5f 100644
--- a/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/WhoClient.java
+++ b/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/WhoClient.java
@@ -19,6 +19,7 @@
import java.io.InputStreamReader;
import java.net.HttpURLConnection;
import java.net.URL;
+import java.nio.charset.Charset;
/**
* Example that uses AuthenticatedURL.
@@ -39,7 +40,9 @@ public static void main(String[] args) {
System.out.println("Status code: " + conn.getResponseCode() + " " + conn.getResponseMessage());
System.out.println();
if (conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
- BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
+ BufferedReader reader = new BufferedReader(
+ new InputStreamReader(
+ conn.getInputStream(), Charset.forName("UTF-8")));
String line = reader.readLine();
while (line != null) {
System.out.println(line);
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
index 61c3c6d5f53c3..c50a5164a5780 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
@@ -24,19 +24,18 @@
/**
* The {@link AuthenticatedURL} class enables the use of the JDK {@link URL} class
* against HTTP endpoints protected with the {@link AuthenticationFilter}.
- *
+ *
* The authentication mechanisms supported by default are Hadoop Simple authentication
* (also known as pseudo authentication) and Kerberos SPNEGO authentication.
- *
+ *
* Additional authentication mechanisms can be supported via {@link Authenticator} implementations.
- *
+ *
* The default {@link Authenticator} is the {@link KerberosAuthenticator} class which supports
* automatic fallback from Kerberos SPNEGO to Hadoop Simple authentication.
- *
+ *
* AuthenticatedURL instances are not thread-safe.
- *
+ *
* The usage pattern of the {@link AuthenticatedURL} is:
- *
*
*
* // establishing an initial connection
@@ -240,7 +239,7 @@ public static void injectToken(HttpURLConnection conn, Token token) {
/**
* Helper method that extracts an authentication token received from a connection.
- *
+ *
* This method is used by {@link Authenticator} implementations.
*
* @param conn connection to extract the authentication token from.
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/Authenticator.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/Authenticator.java
index e7bae4a891593..6828970fdbb59 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/Authenticator.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/Authenticator.java
@@ -19,7 +19,7 @@
/**
* Interface for client authentication mechanisms.
- *
+ *
* Implementations are use-once instances, they don't need to be thread safe.
*/
public interface Authenticator {
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
index 928866c532cce..323b019eb827b 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
@@ -43,9 +43,9 @@
/**
* The {@link KerberosAuthenticator} implements the Kerberos SPNEGO authentication sequence.
- *
+ *
* It uses the default principal for the Kerberos cache (normally set via kinit).
- *
+ *
* It falls back to the {@link PseudoAuthenticator} if the HTTP endpoint does not trigger an SPNEGO authentication
* sequence.
*/
@@ -162,9 +162,9 @@ public void setConnectionConfigurator(ConnectionConfigurator configurator) {
/**
* Performs SPNEGO authentication against the specified URL.
- *
+ *
* If a token is given it does a NOP and returns the given token.
- *
+ *
* If no token is given, it will perform the SPNEGO authentication sequence using an
* HTTP OPTIONS request.
*
@@ -211,7 +211,7 @@ public void authenticate(URL url, AuthenticatedURL.Token token)
/**
* If the specified URL does not support SPNEGO authentication, a fallback {@link Authenticator} will be used.
- *
+ *
* This implementation returns a {@link PseudoAuthenticator}.
*
* @return the fallback {@link Authenticator}.
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java
index f534be9b20bc0..46d94b88dec67 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java
@@ -20,7 +20,7 @@
/**
* The {@link PseudoAuthenticator} implementation provides an authentication equivalent to Hadoop's
* Simple authentication, it trusts the value of the 'user.name' Java System property.
- *
+ *
* The 'user.name' value is propagated using an additional query string parameter {@link #USER_NAME} ('user.name').
*/
public class PseudoAuthenticator implements Authenticator {
@@ -47,13 +47,13 @@ public void setConnectionConfigurator(ConnectionConfigurator configurator) {
/**
* Performs simple authentication against the specified URL.
- *
+ *
* If a token is given it does a NOP and returns the given token.
- *
+ *
* If no token is given, it will perform an HTTP OPTIONS request injecting an additional
* parameter {@link #USER_NAME} in the query string with the value returned by the {@link #getUserName()}
* method.
- *
+ *
* If the response is successful it will update the authentication token.
*
* @param url the URl to authenticate against.
@@ -79,7 +79,7 @@ public void authenticate(URL url, AuthenticatedURL.Token token) throws IOExcepti
/**
* Returns the current user name.
- *
+ *
* This implementation returns the value of the Java system property 'user.name'
*
* @return the current user name.
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AltKerberosAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AltKerberosAuthenticationHandler.java
index e786e37df8ed7..987330fa0e5be 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AltKerberosAuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AltKerberosAuthenticationHandler.java
@@ -28,7 +28,6 @@
* to allow a developer to implement their own custom authentication for browser
* access. The alternateAuthenticate method will be called whenever a request
* comes from a browser.
- *
*/
public abstract class AltKerberosAuthenticationHandler
extends KerberosAuthenticationHandler {
@@ -52,7 +51,6 @@ public abstract class AltKerberosAuthenticationHandler
/**
* Returns the authentication type of the authentication handler,
* 'alt-kerberos'.
- *
*
* @return the authentication type of the authentication handler,
* 'alt-kerberos'.
@@ -80,7 +78,6 @@ public void init(Properties config) throws ServletException {
* completed successfully (in the case of Java access) and only after the
* custom authentication implemented by the subclass in alternateAuthenticate
* has completed successfully (in the case of browser access).
- *
*
* @param request the HTTP client request.
* @param response the HTTP client response.
@@ -109,7 +106,7 @@ public AuthenticationToken authenticate(HttpServletRequest request,
* refers to a browser. If its not a browser, then Kerberos authentication
* will be used; if it is a browser, alternateAuthenticate from the subclass
* will be used.
- *
+ *
* A User-Agent String is considered to be a browser if it does not contain
* any of the values from alt-kerberos.non-browser.user-agents; the default
* behavior is to consider everything a browser unless it contains one of:
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
index 0ac352ba2d116..e891ed2623dd5 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
@@ -44,18 +44,20 @@
import java.util.*;
/**
- * The {@link AuthenticationFilter} enables protecting web application resources with different (pluggable)
+ *
The {@link AuthenticationFilter} enables protecting web application
+ * resources with different (pluggable)
* authentication mechanisms and signer secret providers.
- *
+ *
+ *
* Out of the box it provides 2 authentication mechanisms: Pseudo and Kerberos SPNEGO.
- *
+ *
* Additional authentication mechanisms are supported via the {@link AuthenticationHandler} interface.
- *
+ *
* This filter delegates to the configured authentication handler for authentication and once it obtains an
* {@link AuthenticationToken} from it, sets a signed HTTP cookie with the token. For client requests
* that provide the signed HTTP cookie, it verifies the validity of the cookie, extracts the user information
* and lets the request proceed to the target resource.
- *
+ *
* The supported configuration properties are:
*
* - config.prefix: indicates the prefix to be used by all other configuration properties, the default value
@@ -73,18 +75,19 @@
*
- [#PREFIX#.]cookie.domain: domain to use for the HTTP cookie that stores the authentication token.
* - [#PREFIX#.]cookie.path: path to use for the HTTP cookie that stores the authentication token.
*
- *
+ *
* The rest of the configuration properties are specific to the {@link AuthenticationHandler} implementation and the
* {@link AuthenticationFilter} will take all the properties that start with the prefix #PREFIX#, it will remove
* the prefix from it and it will pass them to the the authentication handler for initialization. Properties that do
* not start with the prefix will not be passed to the authentication handler initialization.
- *
+ *
+ *
* Out of the box it provides 3 signer secret provider implementations:
* "string", "random", and "zookeeper"
- *
+ *
* Additional signer secret providers are supported via the
* {@link SignerSecretProvider} class.
- *
+ *
* For the HTTP cookies mentioned above, the SignerSecretProvider is used to
* determine the secret to use for signing the cookies. Different
* implementations can have different behaviors. The "string" implementation
@@ -94,7 +97,7 @@
* [#PREFIX#.]token.validity mentioned above. The "zookeeper" implementation
* is like the "random" one, except that it synchronizes the random secret
* and rollovers between multiple servers; it's meant for HA services.
- *
+ *
* The relevant configuration properties are:
*
* - signer.secret.provider: indicates the name of the SignerSecretProvider
@@ -108,10 +111,10 @@
* implementations are specified, this value is used as the rollover
* interval.
*
- *
+ *
* The "zookeeper" implementation has additional configuration properties that
* must be specified; see {@link ZKSignerSecretProvider} for details.
- *
+ *
* For subclasses of AuthenticationFilter that want additional control over the
* SignerSecretProvider, they can use the following attribute set in the
* ServletContext:
@@ -190,10 +193,9 @@ public class AuthenticationFilter implements Filter {
private String cookiePath;
/**
- * Initializes the authentication filter and signer secret provider.
- *
- * It instantiates and initializes the specified {@link AuthenticationHandler}.
- *
+ * Initializes the authentication filter and signer secret provider.
+ * It instantiates and initializes the specified {@link
+ * AuthenticationHandler}.
*
* @param filterConfig filter configuration.
*
@@ -375,7 +377,7 @@ protected String getCookiePath() {
/**
* Destroys the filter.
- *
+ *
* It invokes the {@link AuthenticationHandler#destroy()} method to release any resources it may hold.
*/
@Override
@@ -393,7 +395,7 @@ public void destroy() {
* Returns the filtered configuration (only properties starting with the specified prefix). The property keys
* are also trimmed from the prefix. The returned {@link Properties} object is used to initialized the
* {@link AuthenticationHandler}.
- *
+ *
* This method can be overriden by subclasses to obtain the configuration from other configuration source than
* the web.xml file.
*
@@ -419,7 +421,7 @@ protected Properties getConfiguration(String configPrefix, FilterConfig filterCo
/**
* Returns the full URL of the request including the query string.
- *
+ *
* Used as a convenience method for logging purposes.
*
* @param request the request object.
@@ -436,11 +438,11 @@ protected String getRequestURL(HttpServletRequest request) {
/**
* Returns the {@link AuthenticationToken} for the request.
- *
+ *
* It looks at the received HTTP cookies and extracts the value of the {@link AuthenticatedURL#AUTH_COOKIE}
* if present. It verifies the signature and if correct it creates the {@link AuthenticationToken} and returns
* it.
- *
+ *
* If this method returns null the filter will invoke the configured {@link AuthenticationHandler}
* to perform user authentication.
*
@@ -597,7 +599,7 @@ protected void doFilter(FilterChain filterChain, HttpServletRequest request,
*
* @param token authentication token for the cookie.
* @param expires UNIX timestamp that indicates the expire date of the
- * cookie. It has no effect if its value < 0.
+ * cookie. It has no effect if its value < 0.
*
* XXX the following code duplicate some logic in Jetty / Servlet API,
* because of the fact that Hadoop is stuck at servlet 2.5 and jetty 6
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java
index 04984be5a7d58..797e95a689dd5 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java
@@ -24,9 +24,7 @@
/**
* Interface for server authentication mechanisms.
- *
* The {@link AuthenticationFilter} manages the lifecycle of the authentication handler.
- *
* Implementations must be thread-safe as one instance is initialized and used for all requests.
*/
public interface AuthenticationHandler {
@@ -35,7 +33,6 @@ public interface AuthenticationHandler {
/**
* Returns the authentication type of the authentication handler.
- *
* This should be a name that uniquely identifies the authentication type.
* For example 'simple' or 'kerberos'.
*
@@ -45,7 +42,7 @@ public interface AuthenticationHandler {
/**
* Initializes the authentication handler instance.
- *
+ *
* This method is invoked by the {@link AuthenticationFilter#init} method.
*
* @param config configuration properties to initialize the handler.
@@ -56,21 +53,21 @@ public interface AuthenticationHandler {
/**
* Destroys the authentication handler instance.
- *
+ *
* This method is invoked by the {@link AuthenticationFilter#destroy} method.
*/
public void destroy();
/**
* Performs an authentication management operation.
- *
+ *
* This is useful for handling operations like get/renew/cancel
* delegation tokens which are being handled as operations of the
* service end-point.
- *
+ *
* If the method returns TRUE the request will continue normal
* processing, this means the method has not produced any HTTP response.
- *
+ *
* If the method returns FALSE the request will end, this means
* the method has produced the corresponding HTTP response.
*
@@ -91,17 +88,17 @@ public boolean managementOperation(AuthenticationToken token,
/**
* Performs an authentication step for the given HTTP client request.
- *
+ *
* This method is invoked by the {@link AuthenticationFilter} only if the HTTP client request is
* not yet authenticated.
- *
+ *
* Depending upon the authentication mechanism being implemented, a particular HTTP client may
* end up making a sequence of invocations before authentication is successfully established (this is
* the case of Kerberos SPNEGO).
- *
+ *
* This method must return an {@link AuthenticationToken} only if the the HTTP client request has
* been successfully and fully authenticated.
- *
+ *
* If the HTTP client request has not been completely authenticated, this method must take over
* the corresponding HTTP response and it must return null.
*
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationToken.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationToken.java
index ff68847c8a0bc..bb3e71da61c32 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationToken.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationToken.java
@@ -29,7 +29,7 @@
* The {@link AuthenticationToken} contains information about an authenticated
* HTTP client and doubles as the {@link Principal} to be returned by
* authenticated {@link HttpServletRequest}s
- *
+ *
* The token can be serialized/deserialized to and from a string as it is sent
* and received in HTTP client responses and requests as a HTTP cookie (this is
* done by the {@link AuthenticationFilter}).
@@ -170,7 +170,7 @@ public boolean isExpired() {
/**
* Returns the string representation of the token.
- *
+ *
* This string representation is parseable by the {@link #parse} method.
*
* @return the string representation of the token.
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
index 92bc57c413b45..846541b162b5f 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
@@ -51,7 +51,7 @@
/**
* The {@link KerberosAuthenticationHandler} implements the Kerberos SPNEGO authentication mechanism for HTTP.
- *
+ *
* The supported configuration properties are:
*
* - kerberos.principal: the Kerberos principal to used by the server. As stated by the Kerberos SPNEGO
@@ -168,9 +168,9 @@ public KerberosAuthenticationHandler(String type) {
/**
* Initializes the authentication handler instance.
- *
+ *
* It creates a Kerberos context using the principal and keytab specified in the configuration.
- *
+ *
* This method is invoked by the {@link AuthenticationFilter#init} method.
*
* @param config configuration properties to initialize the handler.
@@ -243,7 +243,7 @@ public GSSManager run() throws Exception {
/**
* Releases any resources initialized by the authentication handler.
- *
+ *
* It destroys the Kerberos context.
*/
@Override
@@ -262,7 +262,7 @@ public void destroy() {
/**
* Returns the authentication type of the authentication handler, 'kerberos'.
- *
+ *
*
* @return the authentication type of the authentication handler, 'kerberos'.
*/
@@ -313,7 +313,6 @@ public boolean managementOperation(AuthenticationToken token,
/**
* It enforces the the Kerberos SPNEGO authentication sequence returning an {@link AuthenticationToken} only
* after the Kerberos SPNEGO sequence has completed successfully.
- *
*
* @param request the HTTP client request.
* @param response the HTTP client response.
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java
index 2c7db8898fa56..50f0cf11fe2f5 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java
@@ -30,12 +30,12 @@
/**
* The PseudoAuthenticationHandler provides a pseudo authentication mechanism that accepts
* the user name specified as a query string parameter.
- *
+ *
* This mimics the model of Hadoop Simple authentication which trust the 'user.name' property provided in
* the configuration object.
- *
+ *
* This handler can be configured to support anonymous users.
- *
+ *
* The only supported configuration property is:
*
* - simple.anonymous.allowed:
true|false, default value is false
@@ -80,7 +80,7 @@ public PseudoAuthenticationHandler(String type) {
/**
* Initializes the authentication handler instance.
- *
+ *
* This method is invoked by the {@link AuthenticationFilter#init} method.
*
* @param config configuration properties to initialize the handler.
@@ -103,7 +103,7 @@ protected boolean getAcceptAnonymous() {
/**
* Releases any resources initialized by the authentication handler.
- *
+ *
* This implementation does a NOP.
*/
@Override
@@ -112,7 +112,6 @@ public void destroy() {
/**
* Returns the authentication type of the authentication handler, 'simple'.
- *
*
* @return the authentication type of the authentication handler, 'simple'.
*/
@@ -156,14 +155,14 @@ private String getUserName(HttpServletRequest request) {
/**
* Authenticates an HTTP client request.
- *
+ *
* It extracts the {@link PseudoAuthenticator#USER_NAME} parameter from the query string and creates
* an {@link AuthenticationToken} with it.
- *
+ *
* If the HTTP client request does not contain the {@link PseudoAuthenticator#USER_NAME} parameter and
* the handler is configured to allow anonymous users it returns the {@link AuthenticationToken#ANONYMOUS}
* token.
- *
+ *
* If the HTTP client request does not contain the {@link PseudoAuthenticator#USER_NAME} parameter and
* the handler is configured to disallow anonymous users it throws an {@link AuthenticationException}.
*
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
index 62bb00acab424..7ae8ab2672e9d 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
@@ -92,7 +92,7 @@ public class KerberosName {
/**
* Create a name from the full Kerberos principal name.
- * @param name
+ * @param name full Kerberos principal name.
*/
public KerberosName(String name) {
Matcher match = nameParser.matcher(name);
@@ -367,7 +367,7 @@ public static class NoMatchingRule extends IOException {
* Get the translation of the principal name into an operating system
* user name.
* @return the short name
- * @throws IOException
+ * @throws IOException throws if something is wrong with the rules
*/
public String getShortName() throws IOException {
String[] params;
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java
index ca0fce2251ea0..0e8d8db8ea520 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java
@@ -135,12 +135,10 @@ static final String[] getPrincipalNames(String keytabFileName) throws IOExceptio
/**
* Get all the unique principals from keytabfile which matches a pattern.
*
- * @param keytab
- * Name of the keytab file to be read.
- * @param pattern
- * pattern to be matched.
+ * @param keytab Name of the keytab file to be read.
+ * @param pattern pattern to be matched.
* @return list of unique principals which matches the pattern.
- * @throws IOException
+ * @throws IOException if cannot get the principal name
*/
public static final String[] getPrincipalNames(String keytab,
Pattern pattern) throws IOException {
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RandomSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RandomSignerSecretProvider.java
index 29e5661cb0bd5..41059a7e00900 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RandomSignerSecretProvider.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RandomSignerSecretProvider.java
@@ -14,6 +14,8 @@
package org.apache.hadoop.security.authentication.util;
import com.google.common.annotations.VisibleForTesting;
+
+import java.nio.charset.Charset;
import java.util.Random;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -46,6 +48,6 @@ public RandomSignerSecretProvider(long seed) {
@Override
protected byte[] generateNewSecret() {
- return Long.toString(rand.nextLong()).getBytes();
+ return Long.toString(rand.nextLong()).getBytes(Charset.forName("UTF-8"));
}
}
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/Signer.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/Signer.java
index e29301bc4ba05..aa63e403c6364 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/Signer.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/Signer.java
@@ -15,6 +15,7 @@
import org.apache.commons.codec.binary.Base64;
+import java.nio.charset.Charset;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
@@ -41,8 +42,6 @@ public Signer(SignerSecretProvider secretProvider) {
/**
* Returns a signed string.
- *
- * The signature '&s=SIGNATURE' is appended at the end of the string.
*
* @param str string to sign.
*
@@ -88,7 +87,7 @@ public String verifyAndExtract(String signedStr) throws SignerException {
protected String computeSignature(byte[] secret, String str) {
try {
MessageDigest md = MessageDigest.getInstance("SHA");
- md.update(str.getBytes());
+ md.update(str.getBytes(Charset.forName("UTF-8")));
md.update(secret);
byte[] digest = md.digest();
return new Base64(0).encodeToString(digest);
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java
index 7aaccd2914c22..57ddd372fe4b1 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java
@@ -13,6 +13,7 @@
*/
package org.apache.hadoop.security.authentication.util;
+import java.nio.charset.Charset;
import java.util.Properties;
import javax.servlet.ServletContext;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -36,7 +37,7 @@ public void init(Properties config, ServletContext servletContext,
long tokenValidity) throws Exception {
String signatureSecret = config.getProperty(
AuthenticationFilter.SIGNATURE_SECRET, null);
- secret = signatureSecret.getBytes();
+ secret = signatureSecret.getBytes(Charset.forName("UTF-8"));
secrets = new byte[][]{secret};
}
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
index 6c0fbbb0a26ee..11bfccd05c6e9 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
@@ -15,6 +15,7 @@
import com.google.common.annotations.VisibleForTesting;
import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@@ -44,7 +45,7 @@
/**
* A SignerSecretProvider that synchronizes a rolling random secret between
* multiple servers using ZooKeeper.
- *
+ *
* It works by storing the secrets and next rollover time in a ZooKeeper znode.
* All ZKSignerSecretProviders looking at that znode will use those
* secrets and next rollover time to ensure they are synchronized. There is no
@@ -55,7 +56,7 @@
* your own Curator client, you can pass it to ZKSignerSecretProvider; see
* {@link org.apache.hadoop.security.authentication.server.AuthenticationFilter}
* for more details.
- *
+ *
* The supported configuration properties are:
*
* - signer.secret.provider.zookeeper.connection.string: indicates the
@@ -77,11 +78,13 @@
*
*
* The following attribute in the ServletContext can also be set if desired:
+ *
* - signer.secret.provider.zookeeper.curator.client: A CuratorFramework
* client object can be passed here. If given, the "zookeeper" implementation
* will use this Curator client instead of creating its own, which is useful if
* you already have a Curator client or want more control over its
* configuration.
+ *
*/
@InterfaceStability.Unstable
@InterfaceAudience.Private
@@ -367,14 +370,14 @@ private synchronized void pullFromZK(boolean isInit) {
}
private byte[] generateRandomSecret() {
- return Long.toString(rand.nextLong()).getBytes();
+ return Long.toString(rand.nextLong()).getBytes(Charset.forName("UTF-8"));
}
/**
* This method creates the Curator client and connects to ZooKeeper.
* @param config configuration properties
* @return A Curator client
- * @throws java.lang.Exception
+ * @throws Exception
*/
protected CuratorFramework createCuratorClient(Properties config)
throws Exception {
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
new file mode 100644
index 0000000000000..9728f977bba70
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -0,0 +1,10 @@
+ BREAKDOWN OF HADOOP-11264 SUBTASKS AND RELATED JIRAS (Common part of HDFS-7285)
+
+ HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding
+ (Kai Zheng via umamahesh)
+
+ HADOOP-11534. Minor improvements for raw erasure coders
+ ( Kai Zheng via vinayakumarb )
+
+ HADOOP-11541. Raw XOR coder
+ ( Kai Zheng )
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index a6263887f0487..b02e6954a9430 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -11,18 +11,19 @@ Trunk (Unreleased)
HADOOP-9902. Shell script rewrite (aw)
+ HADOOP-10950. rework heap management vars (John Smith via aw)
+
NEW FEATURES
- HADOOP-9629. Support Windows Azure Storage - Blob as a file system in Hadoop.
- (Dexter Bradshaw, Mostafa Elhemali, Xi Fang, Johannes Klein, David Lao,
- Mike Liddell, Chuan Liu, Lengning Liu, Ivan Mitic, Michael Rys,
- Alexander Stojanovic, Brian Swan, and Min Wei via cnauroth)
+ HADOOP-6590. Add a username check for hadoop sub-commands (John Smith via aw)
+
+ HADOOP-11353. Add support for .hadooprc (aw)
+
+ HADOOP-9044. add FindClass main class to provide classpath checking
+ of installations (Steve Loughran via aw)
+
+ HADOOP-11485. Pluggable shell integration (aw)
- HADOOP-10728. Metrics system for Windows Azure Storage Filesystem.
- (Dexter Bradshaw, Mostafa Elhemali, Xi Fang, Johannes Klein, David Lao,
- Mike Liddell, Chuan Liu, Lengning Liu, Ivan Mitic, Michael Rys,
- Alexander Stojanovich, Brian Swan, and Min Wei via cnauroth)
-
IMPROVEMENTS
HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution
@@ -127,12 +128,6 @@ Trunk (Unreleased)
HADOOP-11092. hadoop shell commands should print usage if not given a
a class (aw)
- HADOOP-10809. hadoop-azure: page blob support. (Dexter Bradshaw,
- Mostafa Elhemali, Eric Hanson, and Mike Liddell via cnauroth)
-
- HADOOP-11188. hadoop-azure: automatically expand page blobs when they become
- full. (Eric Hanson via cnauroth)
-
HADOOP-11231. Remove dead code in ServletUtil. (Li Lu via wheat9)
HADOOP-11025. hadoop-daemons.sh should just call hdfs directly (Masatake
@@ -148,8 +143,33 @@ Trunk (Unreleased)
HADOOP-11081. Document hadoop properties expected to be set by the shell
code in *-env.sh (aw)
+ HADOOP-11352 Clean up test-patch.sh to disable "+1 contrib tests"
+ (Akira AJISAKA via stevel)
+
+ HADOOP-10788. Rewrite kms to use new shell framework (John Smith via aw)
+
+ HADOOP-11058. Missing HADOOP_CONF_DIR generates strange results
+ (Masatake Iwasaki via aw)
+
+ HADOOP-11460. Deprecate shell vars (John Smith via aw)
+
+ HADOOP-11346. Rewrite sls/rumen to use new shell framework (John Smith
+ via aw)
+
+ HADOOP-10976. moving the source code of hadoop-tools docs to the
+ directory under hadoop-tools (Masatake Iwasaki via aw)
+
+ HADOOP-7713. dfs -count -q should label output column (Jonathan Allen
+ via aw)
+
+ HADOOP-6964. Allow compact property description in xml (Kengo Seki
+ via aw)
+
BUG FIXES
+ HADOOP-11473. test-patch says "-1 overall" even when all checks are +1
+ (Jason Lowe via raviprak)
+
HADOOP-9451. Fault single-layer config if node group topology is enabled.
(Junping Du via llu)
@@ -319,20 +339,10 @@ Trunk (Unreleased)
HADOOP-10625. Trim configuration names when putting/getting them
to properties. (Wangda Tan via xgong)
- HADOOP-10689. InputStream is not closed in
- AzureNativeFileSystemStore#retrieve(). (Chen He via cnauroth)
-
- HADOOP-10690. Lack of synchronization on access to InputStream in
- NativeAzureFileSystem#NativeAzureFsInputStream#close().
- (Chen He via cnauroth)
-
HADOOP-10831. UserProvider is not thread safe. (Benoy Antony via umamahesh)
HADOOP-10834. Typo in CredentialShell usage. (Benoy Antony via umamahesh)
- HADOOP-10840. Fix OutOfMemoryError caused by metrics system in Azure File
- System. (Shanyu Zhao via cnauroth)
-
HADOOP-11002. shell escapes are incompatible with previous releases (aw)
HADOOP-10996. Stop violence in the *_HOME (aw)
@@ -347,9 +357,6 @@ Trunk (Unreleased)
HADOOP-11022. User replaced functions get lost 2-3 levels deep (e.g.,
sbin) (aw)
- HADOOP-11248. Add hadoop configuration to disable Azure Filesystem metrics
- collection. (Shanyu Zhao via cnauroth)
-
HADOOP-11284. Fix variable name mismatches in hadoop-functions.sh (Masatake
Iwasaki via aw)
@@ -358,6 +365,12 @@ Trunk (Unreleased)
HADOOP-11296. hadoop-daemons.sh throws 'host1: bash: host3:
command not found...' (vinayakumarb)
+ HADOOP-11380. Restore Rack Awareness documenation (aw)
+
+ HADOOP-11397. Can't override HADOOP_IDENT_STRING (Kengo Seki via aw)
+
+ HADOOP-10908. Common needs updates for shell rewrite (aw)
+
OPTIMIZATIONS
HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -368,6 +381,8 @@ Release 2.7.0 - UNRELEASED
INCOMPATIBLE CHANGES
+ HADOOP-10530 Make hadoop build on Java7+ only (stevel)
+
NEW FEATURES
HADOOP-10987. Provide an iterator-based listing API for FileSystem (kihwal)
@@ -375,10 +390,28 @@ Release 2.7.0 - UNRELEASED
HADOOP-7984. Add hadoop --loglevel option to change log level.
(Akira AJISAKA via cnauroth)
+ HADOOP-9629. Support Windows Azure Storage - Blob as a file system in Hadoop.
+ (Dexter Bradshaw, Mostafa Elhemali, Xi Fang, Johannes Klein, David Lao,
+ Mike Liddell, Chuan Liu, Lengning Liu, Ivan Mitic, Michael Rys,
+ Alexander Stojanovic, Brian Swan, and Min Wei via cnauroth)
+
+ HADOOP-10728. Metrics system for Windows Azure Storage Filesystem.
+ (Dexter Bradshaw, Mostafa Elhemali, Xi Fang, Johannes Klein, David Lao,
+ Mike Liddell, Chuan Liu, Lengning Liu, Ivan Mitic, Michael Rys,
+ Alexander Stojanovich, Brian Swan, and Min Wei via cnauroth)
+
HADOOP-8989. hadoop fs -find feature (Jonathan Allen via aw)
+ HADOOP-11490. Expose truncate API via FileSystem and shell command.
+ (Milan Desai via shv)
+
+ HADOOP-11045. Introducing a tool to detect flaky tests of hadoop jenkins testing
+ job. (Yongjun Zhang and Todd Lipcon via ozawa)
+
IMPROVEMENTS
+ HADOOP-11483. HardLink.java should use the jdk7 createLink method (aajisaka)
+
HADOOP-11156. DelegateToFileSystem should implement
getFsStatus(final Path f). (Zhihai Xu via wang)
@@ -416,13 +449,151 @@ Release 2.7.0 - UNRELEASED
HADOOP-11313. Adding a document about NativeLibraryChecker.
(Tsuyoshi OZAWA via cnauroth)
+ HADOOP-11287. Simplify UGI#reloginFromKeytab for Java 7+.
+ (Li Lu via wheat9)
+
+ HADOOP-10476) Bumping the findbugs version to 3.0.0. (wheat9)
+
+ HADOOP-11410. Make the rpath of libhadoop.so configurable (cmccabe)
+
+ HADOOP-11416. Move ChunkedArrayList into hadoop-common (cmccabe)
+
+ HADOOP-10840. Fix OutOfMemoryError caused by metrics system in Azure File
+ System. (Shanyu Zhao via cnauroth)
+
+ HADOOP-11248. Add hadoop configuration to disable Azure Filesystem metrics
+ collection. (Shanyu Zhao via cnauroth)
+
+ HADOOP-11421. Add IOUtils#listDirectory (cmccabe)
+
+ HADOOP-11427. ChunkedArrayList: fix removal via iterator and implement get
+ (cmccabe)
+
+ HADOOP-11430. Add GenericTestUtils#disableLog, GenericTestUtils#setLogLevel
+ (cmccabe)
+
+ HADOOP-11422. Check CryptoCodec is AES-CTR for Crypto input/output stream
+ (Yi Liu via Colin P. McCabe)
+
+ HADOOP-11213. Typos in html pages: SecureMode and EncryptedShuffle.
+ (Wei Yan via kasha)
+
+ HADOOP-11395. Add site documentation for Azure Storage FileSystem
+ integration. (Chris Nauroth via Arpit Agarwal)
+
+ HDFS-7555. Remove the support of unmanaged connectors in HttpServer2.
+ (wheat9)
+
+ HADOOP-11399. Java Configuration file and .xml files should be
+ automatically cross-compared (rchiang via rkanter)
+
+ HADOOP-11455. KMS and Credential CLI should request confirmation for
+ deletion by default. (Charles Lamb via yliu)
+
+ HADOOP-11390 Metrics 2 ganglia provider to include hostname in
+ unresolved address problems. (Varun Saxena via stevel)
+
+ HADOOP-11032. Replace use of Guava's Stopwatch with Hadoop's StopWatch
+ (ozawa)
+
+ HADOOP-11464. Reinstate support for launching Hadoop processes on Windows
+ using Cygwin. (cnauroth)
+
+ HADOOP-9992. Modify the NN loadGenerator to optionally run as a MapReduce job
+ (Akshay Radia via brandonli)
+
+ HADOOP-11465. Fix findbugs warnings in hadoop-gridmix. (Varun Saxena via
+ Arpit Agarwal)
+
+ HADOOP-11481. ClassCastException while using a key created by keytool to
+ create encryption zone. (Charles Lamb via Colin P. Mccabe)
+
+ HADOOP-8757. Metrics should disallow names with invalid characters
+ (rchiang via rkanter)
+
+ HADOOP-11261 Set custom endpoint for S3A. (Thomas Demoor via stevel)
+
+ HADOOP-11171 Enable using a proxy server to connect to S3a.
+ (Thomas Demoor via stevel)
+
+ HADOOP-11489 Dropping dependency on io.netty from hadoop-nfs' pom.xml
+ (Ted Yu via ozawa)
+
+ HADOOP-11419 Improve hadoop-maven-plugins. (Herve Boutemy via stevel)
+
+ HADOOP-11450. Cleanup DistCpV1 not to use deprecated methods and fix
+ javadocs. (Varun Saxena via ozawa)
+
+ HADOOP-4297. Enable Java assertions when running tests.
+ (Tsz Wo Nicholas Sze via wheat9)
+
+ HADOOP-10626. Limit Returning Attributes for LDAP search. (Jason Hubbard
+ via atm)
+
+ HADOOP-11317. Increment SLF4J version to 1.7.10. (Tim Robertson via ozawa)
+
+ HADOOP-10525. Remove DRFA.MaxBackupIndex config from log4j.properties
+ (aajisaka)
+
+ HADOOP-10574. Bump the maven plugin versions too -moving the numbers into
+ properties. (aajisaka)
+
+ HADOOP-11441. Hadoop-azure: Change few methods scope to public.
+ (Shashank Khandelwal via cnauroth)
+
+ HADOOP-9137. Support connection limiting in IPC server (kihwal)
+
+ HADOOP-11498. Bump the version of HTrace to 3.1.0-incubating (Masatake
+ Iwasaki via Colin P. McCabe)
+
+ HADOOP-11442. hadoop-azure: Create test jar.
+ (Shashank Khandelwal via cnauroth)
+
+ HADOOP-11544. Remove unused configuration keys for tracing. (Masatake
+ Iwasaki via aajisaka)
+
+ HADOOP-11492. Bump up curator version to 2.7.1. (Arun Suresh and
+ Karthik Kambatla via kasha)
+
+ HADOOP-11463 Replace method-local TransferManager object with
+ S3AFileSystem#transfers. (Ted Yu via stevel)
+
+ HADOOP-11543. Improve help message for hadoop/yarn command.
+ (Brahma Reddy Battula via ozawa)
+
+ HADOOP-11520. Clean incomplete multi-part uploads in S3A tests.
+ (Thomas Demoor via stevel)
+
OPTIMIZATIONS
HADOOP-11323. WritableComparator#compare keeps reference to byte array.
(Wilfred Spiegelenburg via wang)
+ HADOOP-11238. Update the NameNode's Group Cache in the background when
+ possible (Chris Li via Colin P. McCabe)
+
+ HADOOP-10809. hadoop-azure: page blob support. (Dexter Bradshaw,
+ Mostafa Elhemali, Eric Hanson, and Mike Liddell via cnauroth)
+
+ HADOOP-11188. hadoop-azure: automatically expand page blobs when they become
+ full. (Eric Hanson via cnauroth)
+
+ HADOOP-11506. Configuration variable expansion regex expensive for long
+ values. (Gera Shegalov via gera)
+
BUG FIXES
+ HADOOP-11488. Difference in default connection timeout for S3A FS
+ (Daisuke Kobayashi via harsh)
+
+ HADOOP-11256. Some site docs have inconsistent appearance (Masatake
+ Iwasaki via aw)
+
+ HADOOP-11318. Update the document for hadoop fs -stat (aajisaka)
+
+ HADOOP 11400. GraphiteSink does not reconnect to Graphite after 'broken pipe'
+ (Kamil Gorlo via raviprak)
+
HADOOP-11236. NFS: Fix javadoc warning in RpcProgram.java (Abhiraj Butala via harsh)
HADOOP-11166. Remove ulimit from test-patch.sh. (wang)
@@ -514,6 +685,218 @@ Release 2.7.0 - UNRELEASED
HADOOP-11343. Overflow is not properly handled in caclulating final iv for
AES CTR. (Jerry Chen via wang)
+ HADOOP-11354. ThrottledInputStream doesn't perform effective throttling.
+ (Ted Yu via jing9)
+
+ HADOOP-11329. Add JAVA_LIBRARY_PATH to KMS startup options. (Arun Suresh via wang)
+
+ HADOOP-11363 Hadoop maven surefire-plugin uses must set heap size. (stevel)
+
+ HADOOP-10134 [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in doc
+ comments. (apurtell via stevel)
+
+ HADOOP-11367. Fix warnings from findbugs 3.0 in hadoop-streaming. (Li Lu via wheat9)
+
+ HADOOP-11369. Fix new findbugs warnings in hadoop-mapreduce-client,
+ non-core directories. (Li Lu via wheat9)
+
+ HADOOP-11368. Fix SSLFactory truststore reloader thread leak in
+ KMSClientProvider. (Arun Suresh via wang)
+
+ HADOOP-11372. Fix new findbugs warnings in mapreduce-examples.
+ (Li Lu via wheat9)
+
+ HADOOP-11273. TestMiniKdc failure: login options not compatible with IBM
+ JDK. (Gao Zhong Liang via wheat9)
+
+ HADOOP-11379. Fix new findbugs warnings in hadoop-auth*. (Li Lu via wheat9)
+
+ HADOOP-11378. Fix new findbugs warnings in hadoop-kms. (Li Lu via wheat9)
+
+ HADOOP-11349. RawLocalFileSystem leaks file descriptor while creating a
+ file if creat succeeds but chmod fails. (Varun Saxena via Colin P. McCabe)
+
+ HADOOP-11381. Fix findbugs warnings in hadoop-distcp, hadoop-aws,
+ hadoop-azure, and hadoop-openstack. (Li Lu via wheat9)
+
+ HADOOP-10482. Fix various findbugs warnings in hadoop-common. (wheat9)
+
+ HADOOP-11388. Remove deprecated o.a.h.metrics.file.FileContext.
+ (Li Lu via wheat9)
+
+ HADOOP-11386. Replace \n by %n in format hadoop-common format strings.
+ (Li Lu via wheat9)
+
+ HADOOP-11211. mapreduce.job.classloader.system.classes semantics should be
+ be order-independent. (Yitong Zhou via gera)
+
+ HADOOP-11389. Clean up byte to string encoding issues in hadoop-common.
+ (wheat9)
+
+ HADOOP-11394. hadoop-aws documentation missing. (cnauroth)
+
+ HADOOP-11396. Provide navigation in the site documentation linking to the
+ Hadoop Compatible File Systems. (cnauroth)
+
+ HADOOP-11412 POMs mention "The Apache Software License" rather than
+ "Apache License". (Herve Boutemy via stevel)
+
+ HADOOP-11321. copyToLocal cannot save a file to an SMB share unless the user
+ has Full Control permissions. (cnauroth)
+
+ HADOOP-11420. Use latest maven-site-plugin and replace link to svn with
+ link to git. (Herve Boutemy via wheat9)
+
+ HADOOP-10689. InputStream is not closed in
+ AzureNativeFileSystemStore#retrieve(). (Chen He via cnauroth)
+
+ HADOOP-10690. Lack of synchronization on access to InputStream in
+ NativeAzureFileSystem#NativeAzureFsInputStream#close().
+ (Chen He via cnauroth)
+
+ HADOOP-11358. Tests for encryption/decryption with IV calculation
+ overflow. (yliu)
+
+ HADOOP-11125. Remove redundant tests in TestOsSecureRandom.
+ (Masanori Oyama via wheat9)
+
+ HADOOP-11385. Prevent cross site scripting attack on JMXJSONServlet.
+ (wheat9)
+
+ HADOOP-11409. FileContext.getFileContext can stack overflow if default fs
+ misconfigured (Gera Shegalov via jlowe)
+
+ HADOOP-11428. Remove obsolete reference to Cygwin in BUILDING.txt.
+ (Arpit Agarwal via wheat9)
+
+ HADOOP-11431. clean up redundant maven-site-plugin configuration.
+ (Herve Boutemy via wheat9)
+
+ HADOOP-11429. Findbugs warnings in hadoop extras.
+ (Varun Saxena via wheat9)
+
+ HADOOP-11414. FileBasedIPList#readLines() can leak file descriptors.
+ (ozawa)
+
+ HADOOP-11283. SequenceFile.Writer can leak file descriptors in
+ DistCpV1#setup(). (Varun Saxena via ozawa)
+
+ HADOOP-11448. Fix findbugs warnings in FileBasedIPList. (ozawa)
+
+ HADOOP-11039. ByteBufferReadable API doc is inconsistent with the
+ implementations. (Yi Liu via Colin P. McCabe)
+
+ HADOOP-11446. S3AOutputStream should use shared thread pool to
+ avoid OutOfMemoryError. (Ted Yu via stevel)
+
+ HADOOP-11459. Fix recent findbugs in ActiveStandbyElector, NetUtils
+ and ShellBasedIdMapping (vinayakumarb)
+
+ HADOOP-11445. Bzip2Codec: Data block is skipped when position of newly
+ created stream is equal to start of split (Ankit Kamboj via jlowe)
+
+ HADOOP-11462. TestSocketIOWithTimeout needs change for PowerPC platform.
+ (Ayappan via cnauroth)
+
+ HADOOP-11350. The size of header buffer of HttpServer is too small when
+ HTTPS is enabled. (Benoy Antony via wheat9)
+
+ HADOOP-10542 Potential null pointer dereference in Jets3tFileSystemStore
+ retrieveBlock(). (Ted Yu via stevel)
+
+ HADOOP-10668. TestZKFailoverControllerStress#testExpireBackAndForth
+ occasionally fails. (Ming Ma via cnauroth)
+
+ HADOOP-11327. BloomFilter#not() omits the last bit, resulting in an
+ incorrect filter (Eric Payne via jlowe)
+
+ HADOOP-11209. Configuration#updatingResource/finalParameters are not
+ thread-safe. (Varun Saxena via ozawa)
+
+ HADOOP-11500. InputStream is left unclosed in ApplicationClassLoader.
+ (Ted Yu via ozawa)
+
+ HADOOP-11008. Remove duplicated description about proxy-user in site
+ documents (Masatake Iwasaki via aw)
+
+ HADOOP-11493. Fix some typos in kms-acls.xml description.
+ (Charles Lamb via aajisaka)
+
+ HADOOP-11507 Hadoop RPC Authentication problem with different user locale.
+ (Talat UYARER via stevel)
+
+ HADOOP-11482. Use correct UGI when KMSClientProvider is called by a proxy
+ user. Contributed by Arun Suresh.
+
+ HADOOP-11499. Check of executorThreadsStarted in
+ ValueQueue#submitRefillTask() evades lock acquisition (Ted Yu via jlowe)
+
+ HADOOP-6221 RPC Client operations cannot be interrupted. (stevel)
+
+ HADOOP-11509. Change parsing sequence in GenericOptionsParser to parse -D
+ parameters before -files. (xgong)
+
+ HADOOP-11469. KMS should skip default.key.acl and whitelist.key.acl when
+ loading key acl. (Dian Fu via yliu)
+
+ HADOOP-11316. "mvn package -Pdist,docs -DskipTests -Dtar" fails because
+ of non-ascii characters. (ozawa)
+
+ HADOOP-9907. Webapp http://hostname:port/metrics link is not working.
+ (aajisaka)
+
+ HADOOP-11403. Avoid using sys_errlist on Solaris, which lacks support for it
+ (Malcolm Kavalsky via Colin P. McCabe)
+
+ HADOOP-11523. StorageException complaining " no lease ID" when updating
+ FolderLastModifiedTime in WASB. (Duo Xu via cnauroth)
+
+ HADOOP-11432. Fix SymlinkBaseTest#testCreateLinkUsingPartQualPath2.
+ (Liang Xie via gera)
+
+ HADOOP-10181. GangliaContext does not work with multicast ganglia setup.
+ (Andrew Johnson via cnauroth)
+
+ HADOOP-11529. Fix findbugs warnings in hadoop-archives.
+ (Masatake Iwasaki via wheat9)
+
+ HADOOP-11546. Checkstyle failing: Unable to instantiate
+ DoubleCheckedLockingCheck. (ozawa)
+
+ HADOOP-11548. checknative should display a nicer error message when openssl
+ support is not compiled in. (Anu Engineer via cnauroth)
+
+ HADOOP-11547. hadoop-common native compilation fails on Windows due to
+ missing support for __attribute__ declaration. (cnauroth)
+
+ HADOOP-11549. flaky test detection tool failed to handle special control
+ characters in test result. (Yongjun Zhang via aajisaka)
+
+ HADOOP-10062. race condition in MetricsSystemImpl#publishMetricsNow that
+ causes incorrect results. (Sangjin Lee via junping_du)
+
+ HADOOP-11526. Memory leak in Bzip2Compressor and Bzip2Decompressor.
+ (Anu Engineer via cnauroth)
+
+ HADOOP-11535 TableMapping related tests failed due to 'successful'
+ resolving of invalid test hostname. (Kai Zheng via stevel)
+
+Release 2.6.1 - UNRELEASED
+
+ INCOMPATIBLE CHANGES
+
+ NEW FEATURES
+
+ IMPROVEMENTS
+
+ OPTIMIZATIONS
+
+ BUG FIXES
+
+ HADOOP-11466. FastByteComparisons: do not use UNSAFE_COMPARER on the SPARC
+ architecture because it is slower there (Suman Somasundar via Colin P.
+ McCabe)
+
Release 2.6.0 - 2014-11-18
INCOMPATIBLE CHANGES
diff --git a/hadoop-common-project/hadoop-common/dev-support/checkstyle.xml b/hadoop-common-project/hadoop-common/dev-support/checkstyle.xml
index ad70bee1764a6..4caa03f05dfd2 100644
--- a/hadoop-common-project/hadoop-common/dev-support/checkstyle.xml
+++ b/hadoop-common-project/hadoop-common/dev-support/checkstyle.xml
@@ -150,7 +150,6 @@
-
diff --git a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
index 8de3c378bd6f7..ab8673b860e96 100644
--- a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
@@ -241,6 +241,16 @@
+
+
+
+
+
+
+
+
+
+
/** See core-default.xml */
public static final String KMS_CLIENT_ENC_KEY_CACHE_SIZE =
"hadoop.security.kms.client.encrypted.key.cache.size";
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
index 5d01637a0fb30..6276dda2addbd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
@@ -97,28 +97,35 @@ public void readFields(DataInput in) throws IOException {
this.spaceConsumed = in.readLong();
this.spaceQuota = in.readLong();
}
-
- /**
+
+ /**
* Output format:
* <----12----> <----12----> <-------18------->
- * DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME
+ * DIR_COUNT FILE_COUNT CONTENT_SIZE
*/
- private static final String STRING_FORMAT = "%12s %12s %18s ";
- /**
+ private static final String SUMMARY_FORMAT = "%12s %12s %18s ";
+ /**
* Output format:
- * <----12----> <----15----> <----15----> <----15----> <----12----> <----12----> <-------18------->
- * QUOTA REMAINING_QUATA SPACE_QUOTA SPACE_QUOTA_REM DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME
+ * <----12----> <------15-----> <------15-----> <------15----->
+ * QUOTA REM_QUOTA SPACE_QUOTA REM_SPACE_QUOTA
+ * <----12----> <----12----> <-------18------->
+ * DIR_COUNT FILE_COUNT CONTENT_SIZE
*/
- private static final String QUOTA_STRING_FORMAT = "%12s %15s ";
- private static final String SPACE_QUOTA_STRING_FORMAT = "%15s %15s ";
-
+ private static final String QUOTA_SUMMARY_FORMAT = "%12s %15s ";
+ private static final String SPACE_QUOTA_SUMMARY_FORMAT = "%15s %15s ";
+
+ private static final String[] HEADER_FIELDS = new String[] { "DIR_COUNT",
+ "FILE_COUNT", "CONTENT_SIZE"};
+ private static final String[] QUOTA_HEADER_FIELDS = new String[] { "QUOTA",
+ "REM_QUOTA", "SPACE_QUOTA", "REM_SPACE_QUOTA" };
+
/** The header string */
private static final String HEADER = String.format(
- STRING_FORMAT.replace('d', 's'), "directories", "files", "bytes");
+ SUMMARY_FORMAT, (Object[]) HEADER_FIELDS);
private static final String QUOTA_HEADER = String.format(
- QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT,
- "name quota", "rem name quota", "space quota", "rem space quota") +
+ QUOTA_SUMMARY_FORMAT + SPACE_QUOTA_SUMMARY_FORMAT,
+ (Object[]) QUOTA_HEADER_FIELDS) +
HEADER;
/** Return the header of the output.
@@ -131,7 +138,25 @@ public void readFields(DataInput in) throws IOException {
public static String getHeader(boolean qOption) {
return qOption ? QUOTA_HEADER : HEADER;
}
-
+
+ /**
+ * Returns the names of the fields from the summary header.
+ *
+ * @return names of fields as displayed in the header
+ */
+ public static String[] getHeaderFields() {
+ return HEADER_FIELDS;
+ }
+
+ /**
+ * Returns the names of the fields used in the quota summary.
+ *
+ * @return names of quota fields as displayed in the header
+ */
+ public static String[] getQuotaHeaderFields() {
+ return QUOTA_HEADER_FIELDS;
+ }
+
@Override
public String toString() {
return toString(true);
@@ -175,11 +200,11 @@ public String toString(boolean qOption, boolean hOption) {
spaceQuotaRem = formatSize(spaceQuota - spaceConsumed, hOption);
}
- prefix = String.format(QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT,
+ prefix = String.format(QUOTA_SUMMARY_FORMAT + SPACE_QUOTA_SUMMARY_FORMAT,
quotaStr, quotaRem, spaceQuotaStr, spaceQuotaRem);
}
- return prefix + String.format(STRING_FORMAT,
+ return prefix + String.format(SUMMARY_FORMAT,
formatSize(directoryCount, hOption),
formatSize(fileCount, hOption),
formatSize(length, hOption));
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
index c5d23b48e82e8..e008ecc59fd3f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
@@ -47,6 +47,10 @@
* - SYNC_BLOCK - to force closed blocks to the disk device.
* In addition {@link Syncable#hsync()} should be called after each write,
* if true synchronous behavior is required.
+ * - LAZY_PERSIST - Create the block on transient storage (RAM) if
+ * available.
+ * - APPEND_NEWBLOCK - Append data to a new block instead of end of the last
+ * partial block.
*
*
* Following combination is not valid and will result in
@@ -93,7 +97,13 @@ public enum CreateFlag {
* This flag must only be used for intermediate data whose loss can be
* tolerated by the application.
*/
- LAZY_PERSIST((short) 0x10);
+ LAZY_PERSIST((short) 0x10),
+
+ /**
+ * Append data to a new block instead of the end of the last partial block.
+ * This is only useful for APPEND.
+ */
+ NEW_BLOCK((short) 0x20);
private final short mode;
@@ -149,4 +159,16 @@ public static void validate(Object path, boolean pathExists,
+ ". Create option is not specified in " + flag);
}
}
+
+ /**
+ * Validate the CreateFlag for the append operation. The flag must contain
+ * APPEND, and cannot contain OVERWRITE.
+ */
+ public static void validateForAppend(EnumSet flag) {
+ validate(flag);
+ if (!flag.contains(APPEND)) {
+ throw new HadoopIllegalArgumentException(flag
+ + " does not contain APPEND");
+ }
+ }
}
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
index c8609d450f27c..6d39d1e0b89f6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
@@ -58,7 +58,7 @@ public FSDataInputStream(InputStream in) {
* @param desired offset to seek to
*/
@Override
- public synchronized void seek(long desired) throws IOException {
+ public void seek(long desired) throws IOException {
((Seekable)in).seek(desired);
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
index 934421a1884dc..13a5e26423b90 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
@@ -165,7 +165,7 @@ protected synchronized int flushBuffer(boolean keep,
count = partialLen;
System.arraycopy(buf, bufLen - count, buf, 0, count);
} else {
- count = 0;
+ count = 0;
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index 85f8136c0ac4f..e710ec0261216 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -457,9 +457,15 @@ public static FileContext getFileContext(final URI defaultFsUri,
*/
public static FileContext getFileContext(final Configuration aConf)
throws UnsupportedFileSystemException {
- return getFileContext(
- URI.create(aConf.get(FS_DEFAULT_NAME_KEY, FS_DEFAULT_NAME_DEFAULT)),
- aConf);
+ final URI defaultFsUri = URI.create(aConf.get(FS_DEFAULT_NAME_KEY,
+ FS_DEFAULT_NAME_DEFAULT));
+ if ( defaultFsUri.getScheme() != null
+ && !defaultFsUri.getScheme().trim().isEmpty()) {
+ return getFileContext(defaultFsUri, aConf);
+ }
+ throw new UnsupportedFileSystemException(String.format(
+ "%s: URI configured via %s carries no scheme",
+ defaultFsUri, FS_DEFAULT_NAME_KEY));
}
/**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 9edc54bac5922..cfa519861aea9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -1317,6 +1317,29 @@ protected void rename(final Path src, final Path dst,
throw new IOException("rename from " + src + " to " + dst + " failed.");
}
}
+
+ /**
+ * Truncate the file in the indicated path to the indicated size.
+ *
+ * - Fails if path is a directory.
+ *
- Fails if path does not exist.
+ *
- Fails if path is not closed.
+ *
- Fails if new size is greater than current size.
+ *
+ * @param f The path to the file to be truncated
+ * @param newLength The size the file is to be truncated to
+ *
+ * @return true if the file has been truncated to the desired
+ * newLength and is immediately available to be reused for
+ * write operations such as append, or
+ * false if a background process of adjusting the length of
+ * the last block has been started, and clients should wait for it to
+ * complete before proceeding with further file updates.
+ */
+ public boolean truncate(Path f, long newLength) throws IOException {
+ throw new UnsupportedOperationException("Not implemented by the " +
+ getClass().getSimpleName() + " FileSystem implementation");
+ }
/**
* Delete a file
@@ -2618,9 +2641,6 @@ public static Class extends FileSystem> getFileSystemClass(String scheme,
private static FileSystem createFileSystem(URI uri, Configuration conf
) throws IOException {
Class> clazz = getFileSystemClass(uri.getScheme(), conf);
- if (clazz == null) {
- throw new IOException("No FileSystem for scheme: " + uri.getScheme());
- }
FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
fs.initialize(uri, conf);
return fs;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
index 3d5a753b0ece1..d4080adc74dcb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
@@ -225,6 +225,11 @@ public boolean setReplication(Path src, short replication) throws IOException {
public boolean rename(Path src, Path dst) throws IOException {
return fs.rename(src, dst);
}
+
+ @Override
+ public boolean truncate(Path f, final long newLength) throws IOException {
+ return fs.truncate(f, newLength);
+ }
/** Delete a file */
@Override
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
index 3ba6de1f9a335..e89bc4949b8b0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
@@ -220,12 +220,7 @@ private URI decodeHarURI(URI rawURI, Configuration conf) throws IOException {
return FileSystem.getDefaultUri(conf);
}
String authority = rawURI.getAuthority();
- if (authority == null) {
- throw new IOException("URI: " + rawURI
- + " is an invalid Har URI since authority==null."
- + " Expecting har://-/.");
- }
-
+
int i = authority.indexOf('-');
if (i < 0) {
throw new IOException("URI: " + rawURI
@@ -489,19 +484,12 @@ public static int getHarHash(Path p) {
}
static class Store {
- public Store() {
- begin = end = startHash = endHash = 0;
- }
- public Store(long begin, long end, int startHash, int endHash) {
+ public Store(long begin, long end) {
this.begin = begin;
this.end = end;
- this.startHash = startHash;
- this.endHash = endHash;
}
public long begin;
public long end;
- public int startHash;
- public int endHash;
}
/**
@@ -594,7 +582,7 @@ private class HarStatus {
public HarStatus(String harString) throws UnsupportedEncodingException {
String[] splits = harString.split(" ");
this.name = decodeFileName(splits[0]);
- this.isDir = "dir".equals(splits[1]) ? true: false;
+ this.isDir = "dir".equals(splits[1]);
// this is equal to "none" if its a directory
this.partName = splits[2];
this.startIndex = Long.parseLong(splits[3]);
@@ -769,6 +757,14 @@ public FSDataOutputStream append(Path f) throws IOException {
throw new IOException("Har: append not allowed");
}
+ /**
+ * Not implemented.
+ */
+ @Override
+ public boolean truncate(Path f, long newLength) throws IOException {
+ throw new IOException("Har: truncate not allowed");
+ }
+
/**
* Not implemented.
*/
@@ -1167,11 +1163,8 @@ private void parseMetaData() throws IOException {
int b = lin.readLine(line);
read += b;
readStr = line.toString().split(" ");
- int startHash = Integer.parseInt(readStr[0]);
- int endHash = Integer.parseInt(readStr[1]);
stores.add(new Store(Long.parseLong(readStr[2]),
- Long.parseLong(readStr[3]), startHash,
- endHash));
+ Long.parseLong(readStr[3])));
line.clear();
}
} catch (IOException ioe) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
index e48354dba790d..209ba6997ae87 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
@@ -23,13 +23,16 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.StringReader;
-import java.util.Arrays;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.Shell.ExitCodeException;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import com.google.common.annotations.VisibleForTesting;
+
+import static java.nio.file.Files.createLink;
+
/**
* Class for creating hardlinks.
* Supports Unix/Linux, Windows via winutils , and Mac OS X.
@@ -75,114 +78,30 @@ public HardLink() {
/**
* This abstract class bridges the OS-dependent implementations of the
- * needed functionality for creating hardlinks and querying link counts.
+ * needed functionality for querying link counts.
* The particular implementation class is chosen during
* static initialization phase of the HardLink class.
- * The "getter" methods construct shell command strings for various purposes.
+ * The "getter" methods construct shell command strings.
*/
private static abstract class HardLinkCommandGetter {
-
- /**
- * Get the command string needed to hardlink a bunch of files from
- * a single source directory into a target directory. The source directory
- * is not specified here, but the command will be executed using the source
- * directory as the "current working directory" of the shell invocation.
- *
- * @param fileBaseNames - array of path-less file names, relative
- * to the source directory
- * @param linkDir - target directory where the hardlinks will be put
- * @return - an array of Strings suitable for use as a single shell command
- * @throws IOException - if any of the file or path names misbehave
- */
- abstract String[] linkMult(String[] fileBaseNames, File linkDir)
- throws IOException;
-
- /**
- * Get the command string needed to hardlink a single file
- */
- abstract String[] linkOne(File file, File linkName) throws IOException;
-
/**
* Get the command string to query the hardlink count of a file
*/
abstract String[] linkCount(File file) throws IOException;
-
- /**
- * Calculate the total string length of the shell command
- * resulting from execution of linkMult, plus the length of the
- * source directory name (which will also be provided to the shell)
- *
- * @param fileDir - source directory, parent of fileBaseNames
- * @param fileBaseNames - array of path-less file names, relative
- * to the source directory
- * @param linkDir - target directory where the hardlinks will be put
- * @return - total data length (must not exceed maxAllowedCmdArgLength)
- * @throws IOException
- */
- abstract int getLinkMultArgLength(
- File fileDir, String[] fileBaseNames, File linkDir)
- throws IOException;
-
- /**
- * Get the maximum allowed string length of a shell command on this OS,
- * which is just the documented minimum guaranteed supported command
- * length - aprx. 32KB for Unix, and 8KB for Windows.
- */
- abstract int getMaxAllowedCmdArgLength();
}
/**
* Implementation of HardLinkCommandGetter class for Unix
*/
- static class HardLinkCGUnix extends HardLinkCommandGetter {
- private static String[] hardLinkCommand = {"ln", null, null};
- private static String[] hardLinkMultPrefix = {"ln"};
- private static String[] hardLinkMultSuffix = {null};
+ private static class HardLinkCGUnix extends HardLinkCommandGetter {
private static String[] getLinkCountCommand = {"stat","-c%h", null};
- //Unix guarantees at least 32K bytes cmd length.
- //Subtract another 64b to allow for Java 'exec' overhead
- private static final int maxAllowedCmdArgLength = 32*1024 - 65;
-
private static synchronized
void setLinkCountCmdTemplate(String[] template) {
//May update this for specific unix variants,
//after static initialization phase
getLinkCountCommand = template;
}
-
- /*
- * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkOne(java.io.File, java.io.File)
- */
- @Override
- String[] linkOne(File file, File linkName)
- throws IOException {
- String[] buf = new String[hardLinkCommand.length];
- System.arraycopy(hardLinkCommand, 0, buf, 0, hardLinkCommand.length);
- //unix wants argument order: "ln "
- buf[1] = FileUtil.makeShellPath(file, true);
- buf[2] = FileUtil.makeShellPath(linkName, true);
- return buf;
- }
-
- /*
- * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkMult(java.lang.String[], java.io.File)
- */
- @Override
- String[] linkMult(String[] fileBaseNames, File linkDir)
- throws IOException {
- String[] buf = new String[fileBaseNames.length
- + hardLinkMultPrefix.length
- + hardLinkMultSuffix.length];
- int mark=0;
- System.arraycopy(hardLinkMultPrefix, 0, buf, mark,
- hardLinkMultPrefix.length);
- mark += hardLinkMultPrefix.length;
- System.arraycopy(fileBaseNames, 0, buf, mark, fileBaseNames.length);
- mark += fileBaseNames.length;
- buf[mark] = FileUtil.makeShellPath(linkDir, true);
- return buf;
- }
-
+
/*
* @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkCount(java.io.File)
*/
@@ -195,169 +114,30 @@ String[] linkCount(File file)
buf[getLinkCountCommand.length - 1] = FileUtil.makeShellPath(file, true);
return buf;
}
-
- /*
- * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getLinkMultArgLength(java.io.File, java.lang.String[], java.io.File)
- */
- @Override
- int getLinkMultArgLength(File fileDir, String[] fileBaseNames, File linkDir)
- throws IOException{
- int sum = 0;
- for (String x : fileBaseNames) {
- // add 1 to account for terminal null or delimiter space
- sum += 1 + ((x == null) ? 0 : x.length());
- }
- sum += 2 + FileUtil.makeShellPath(fileDir, true).length()
- + FileUtil.makeShellPath(linkDir, true).length();
- //add the fixed overhead of the hardLinkMult prefix and suffix
- sum += 3; //length("ln") + 1
- return sum;
- }
-
- /*
- * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getMaxAllowedCmdArgLength()
- */
- @Override
- int getMaxAllowedCmdArgLength() {
- return maxAllowedCmdArgLength;
- }
}
-
/**
* Implementation of HardLinkCommandGetter class for Windows
*/
+ @VisibleForTesting
static class HardLinkCGWin extends HardLinkCommandGetter {
- //The Windows command getter impl class and its member fields are
- //package-private ("default") access instead of "private" to assist
- //unit testing (sort of) on non-Win servers
- static String CMD_EXE = "cmd.exe";
- static String[] hardLinkCommand = {
- Shell.WINUTILS,"hardlink","create", null, null};
- static String[] hardLinkMultPrefix = {
- CMD_EXE, "/q", "/c", "for", "%f", "in", "("};
- static String hardLinkMultDir = "\\%f";
- static String[] hardLinkMultSuffix = {
- ")", "do", Shell.WINUTILS, "hardlink", "create", null,
- "%f"};
static String[] getLinkCountCommand = {
Shell.WINUTILS, "hardlink", "stat", null};
- //Windows guarantees only 8K - 1 bytes cmd length.
- //Subtract another 64b to allow for Java 'exec' overhead
- static final int maxAllowedCmdArgLength = 8*1024 - 65;
- /*
- * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkOne(java.io.File, java.io.File)
- */
- @Override
- String[] linkOne(File file, File linkName)
- throws IOException {
- String[] buf = new String[hardLinkCommand.length];
- System.arraycopy(hardLinkCommand, 0, buf, 0, hardLinkCommand.length);
- //windows wants argument order: "create "
- buf[4] = file.getCanonicalPath();
- buf[3] = linkName.getCanonicalPath();
- return buf;
- }
-
- /*
- * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkMult(java.lang.String[], java.io.File)
- */
- @Override
- String[] linkMult(String[] fileBaseNames, File linkDir)
- throws IOException {
- String[] buf = new String[fileBaseNames.length
- + hardLinkMultPrefix.length
- + hardLinkMultSuffix.length];
- String td = linkDir.getCanonicalPath() + hardLinkMultDir;
- int mark=0;
- System.arraycopy(hardLinkMultPrefix, 0, buf, mark,
- hardLinkMultPrefix.length);
- mark += hardLinkMultPrefix.length;
- System.arraycopy(fileBaseNames, 0, buf, mark, fileBaseNames.length);
- mark += fileBaseNames.length;
- System.arraycopy(hardLinkMultSuffix, 0, buf, mark,
- hardLinkMultSuffix.length);
- mark += hardLinkMultSuffix.length;
- buf[mark - 2] = td;
- return buf;
- }
-
/*
* @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkCount(java.io.File)
*/
@Override
- String[] linkCount(File file)
- throws IOException {
+ String[] linkCount(File file) throws IOException {
String[] buf = new String[getLinkCountCommand.length];
System.arraycopy(getLinkCountCommand, 0, buf, 0,
getLinkCountCommand.length);
buf[getLinkCountCommand.length - 1] = file.getCanonicalPath();
return buf;
}
-
- /*
- * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getLinkMultArgLength(java.io.File, java.lang.String[], java.io.File)
- */
- @Override
- int getLinkMultArgLength(File fileDir, String[] fileBaseNames, File linkDir)
- throws IOException {
- int sum = 0;
- for (String x : fileBaseNames) {
- // add 1 to account for terminal null or delimiter space
- sum += 1 + ((x == null) ? 0 : x.length());
- }
- sum += 2 + fileDir.getCanonicalPath().length() +
- linkDir.getCanonicalPath().length();
- //add the fixed overhead of the hardLinkMult command
- //(prefix, suffix, and Dir suffix)
- sum += (CMD_EXE + " /q /c for %f in ( ) do "
- + Shell.WINUTILS + " hardlink create \\%f %f").length();
- return sum;
- }
-
- /*
- * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getMaxAllowedCmdArgLength()
- */
- @Override
- int getMaxAllowedCmdArgLength() {
- return maxAllowedCmdArgLength;
- }
}
-
-
- /**
- * Calculate the nominal length of all contributors to the total
- * commandstring length, including fixed overhead of the OS-dependent
- * command. It's protected rather than private, to assist unit testing,
- * but real clients are not expected to need it -- see the way
- * createHardLinkMult() uses it internally so the user doesn't need to worry
- * about it.
- *
- * @param fileDir - source directory, parent of fileBaseNames
- * @param fileBaseNames - array of path-less file names, relative
- * to the source directory
- * @param linkDir - target directory where the hardlinks will be put
- * @return - total data length (must not exceed maxAllowedCmdArgLength)
- * @throws IOException
- */
- protected static int getLinkMultArgLength(
- File fileDir, String[] fileBaseNames, File linkDir)
- throws IOException {
- return getHardLinkCommand.getLinkMultArgLength(fileDir,
- fileBaseNames, linkDir);
- }
-
- /**
- * Return this private value for use by unit tests.
- * Shell commands are not allowed to have a total string length
- * exceeding this size.
- */
- protected static int getMaxAllowedCmdArgLength() {
- return getHardLinkCommand.getMaxAllowedCmdArgLength();
- }
-
+
/*
* ****************************************************
* Complexity is above. User-visible functionality is below
@@ -370,7 +150,7 @@ protected static int getMaxAllowedCmdArgLength() {
* @param linkName - desired target link file
*/
public static void createHardLink(File file, File linkName)
- throws IOException {
+ throws IOException {
if (file == null) {
throw new IOException(
"invalid arguments to createHardLink: source file is null");
@@ -379,17 +159,7 @@ public static void createHardLink(File file, File linkName)
throw new IOException(
"invalid arguments to createHardLink: link name is null");
}
- // construct and execute shell command
- String[] hardLinkCommand = getHardLinkCommand.linkOne(file, linkName);
- ShellCommandExecutor shexec = new ShellCommandExecutor(hardLinkCommand);
- try {
- shexec.execute();
- } catch (ExitCodeException e) {
- throw new IOException("Failed to execute command " +
- Arrays.toString(hardLinkCommand) +
- "; command output: \"" + shexec.getOutput() + "\"" +
- "; WrappedException: \"" + e.getMessage() + "\"");
- }
+ createLink(linkName.toPath(), file.toPath());
}
/**
@@ -398,30 +168,10 @@ public static void createHardLink(File file, File linkName)
* @param parentDir - directory containing source files
* @param fileBaseNames - list of path-less file names, as returned by
* parentDir.list()
- * @param linkDir - where the hardlinks should be put. It must already exist.
- *
- * If the list of files is too long (overflows maxAllowedCmdArgLength),
- * we will automatically split it into multiple invocations of the
- * underlying method.
+ * @param linkDir - where the hardlinks should be put. It must already exist.
*/
- public static void createHardLinkMult(File parentDir, String[] fileBaseNames,
+ public static void createHardLinkMult(File parentDir, String[] fileBaseNames,
File linkDir) throws IOException {
- //This is the public method all non-test clients are expected to use.
- //Normal case - allow up to maxAllowedCmdArgLength characters in the cmd
- createHardLinkMult(parentDir, fileBaseNames, linkDir,
- getHardLinkCommand.getMaxAllowedCmdArgLength());
- }
-
- /*
- * Implements {@link createHardLinkMult} with added variable "maxLength",
- * to ease unit testing of the auto-splitting feature for long lists.
- * Likewise why it returns "callCount", the number of sub-arrays that
- * the file list had to be split into.
- * Non-test clients are expected to call the public method instead.
- */
- protected static int createHardLinkMult(File parentDir,
- String[] fileBaseNames, File linkDir, int maxLength)
- throws IOException {
if (parentDir == null) {
throw new IOException(
"invalid arguments to createHardLinkMult: parent directory is null");
@@ -435,40 +185,13 @@ protected static int createHardLinkMult(File parentDir,
"invalid arguments to createHardLinkMult: "
+ "filename list can be empty but not null");
}
- if (fileBaseNames.length == 0) {
- //the OS cmds can't handle empty list of filenames,
- //but it's legal, so just return.
- return 0;
- }
if (!linkDir.exists()) {
throw new FileNotFoundException(linkDir + " not found.");
}
-
- //if the list is too long, split into multiple invocations
- int callCount = 0;
- if (getLinkMultArgLength(parentDir, fileBaseNames, linkDir) > maxLength
- && fileBaseNames.length > 1) {
- String[] list1 = Arrays.copyOf(fileBaseNames, fileBaseNames.length/2);
- callCount += createHardLinkMult(parentDir, list1, linkDir, maxLength);
- String[] list2 = Arrays.copyOfRange(fileBaseNames, fileBaseNames.length/2,
- fileBaseNames.length);
- callCount += createHardLinkMult(parentDir, list2, linkDir, maxLength);
- return callCount;
- } else {
- callCount = 1;
- }
-
- // construct and execute shell command
- String[] hardLinkCommand = getHardLinkCommand.linkMult(fileBaseNames,
- linkDir);
- ShellCommandExecutor shexec = new ShellCommandExecutor(hardLinkCommand,
- parentDir, null, 0L);
- try {
- shexec.execute();
- } catch (ExitCodeException e) {
- throw new IOException(shexec.getOutput() + e.getMessage());
+ for (String name : fileBaseNames) {
+ createLink(linkDir.toPath().resolve(name),
+ parentDir.toPath().resolve(name));
}
- return callCount;
}
/**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
index 88b4d4e42c45c..8f011ce409f4f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
@@ -372,7 +372,7 @@ public synchronized Path getLocalPathForWrite(String pathStr, long size,
// Keep rolling the wheel till we get a valid path
Random r = new java.util.Random();
while (numDirsSearched < numDirs && returnPath == null) {
- long randomPosition = Math.abs(r.nextLong()) % totalAvailable;
+ long randomPosition = (r.nextLong() >>> 1) % totalAvailable;
int dir = 0;
while (randomPosition > availableOnDisk[dir]) {
randomPosition -= availableOnDisk[dir];
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java
index 591899567288b..21f56ed1d948c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java
@@ -143,13 +143,13 @@ public static MD5MD5CRC32FileChecksum valueOf(Attributes attrs
switch (finalCrcType) {
case CRC32:
return new MD5MD5CRC32GzipFileChecksum(
- Integer.valueOf(bytesPerCRC),
- Integer.valueOf(crcPerBlock),
+ Integer.parseInt(bytesPerCRC),
+ Integer.parseInt(crcPerBlock),
new MD5Hash(md5));
case CRC32C:
return new MD5MD5CRC32CastagnoliFileChecksum(
- Integer.valueOf(bytesPerCRC),
- Integer.valueOf(crcPerBlock),
+ Integer.parseInt(bytesPerCRC),
+ Integer.parseInt(crcPerBlock),
new MD5Hash(md5));
default:
// we should never get here since finalCrcType will
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
index 54ddedaff1dc4..caeb7a1b799fa 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
@@ -60,7 +60,6 @@ public class Path implements Comparable {
/**
* Pathnames with scheme and relative path are illegal.
- * @param path to be checked
*/
void checkNotSchemeWithRelative() {
if (toUri().isAbsolute() && !isUriPathAbsolute()) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index b6b6f5905491f..d7866b84aafd9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -41,7 +41,9 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIOException;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils;
@@ -207,8 +209,28 @@ public FSDataInputStream open(Path f, int bufferSize) throws IOException {
class LocalFSFileOutputStream extends OutputStream {
private FileOutputStream fos;
- private LocalFSFileOutputStream(Path f, boolean append) throws IOException {
- this.fos = new FileOutputStream(pathToFile(f), append);
+ private LocalFSFileOutputStream(Path f, boolean append,
+ FsPermission permission) throws IOException {
+ File file = pathToFile(f);
+ if (permission == null) {
+ this.fos = new FileOutputStream(file, append);
+ } else {
+ if (Shell.WINDOWS && NativeIO.isAvailable()) {
+ this.fos = NativeIO.Windows.createFileOutputStreamWithMode(file,
+ append, permission.toShort());
+ } else {
+ this.fos = new FileOutputStream(file, append);
+ boolean success = false;
+ try {
+ setPermission(f, permission);
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.cleanup(LOG, this.fos);
+ }
+ }
+ }
+ }
}
/*
@@ -247,19 +269,20 @@ public FSDataOutputStream append(Path f, int bufferSize,
throw new IOException("Cannot append to a diretory (=" + f + " )");
}
return new FSDataOutputStream(new BufferedOutputStream(
- new LocalFSFileOutputStream(f, true), bufferSize), statistics);
+ createOutputStreamWithMode(f, true, null), bufferSize), statistics);
}
@Override
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
short replication, long blockSize, Progressable progress)
throws IOException {
- return create(f, overwrite, true, bufferSize, replication, blockSize, progress);
+ return create(f, overwrite, true, bufferSize, replication, blockSize,
+ progress, null);
}
private FSDataOutputStream create(Path f, boolean overwrite,
boolean createParent, int bufferSize, short replication, long blockSize,
- Progressable progress) throws IOException {
+ Progressable progress, FsPermission permission) throws IOException {
if (exists(f) && !overwrite) {
throw new FileAlreadyExistsException("File already exists: " + f);
}
@@ -268,12 +291,18 @@ private FSDataOutputStream create(Path f, boolean overwrite,
throw new IOException("Mkdirs failed to create " + parent.toString());
}
return new FSDataOutputStream(new BufferedOutputStream(
- createOutputStream(f, false), bufferSize), statistics);
+ createOutputStreamWithMode(f, false, permission), bufferSize),
+ statistics);
}
protected OutputStream createOutputStream(Path f, boolean append)
throws IOException {
- return new LocalFSFileOutputStream(f, append);
+ return createOutputStreamWithMode(f, append, null);
+ }
+
+ protected OutputStream createOutputStreamWithMode(Path f, boolean append,
+ FsPermission permission) throws IOException {
+ return new LocalFSFileOutputStream(f, append, permission);
}
@Override
@@ -285,7 +314,8 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
throw new FileAlreadyExistsException("File already exists: " + f);
}
return new FSDataOutputStream(new BufferedOutputStream(
- new LocalFSFileOutputStream(f, false), bufferSize), statistics);
+ createOutputStreamWithMode(f, false, permission), bufferSize),
+ statistics);
}
@Override
@@ -293,9 +323,8 @@ public FSDataOutputStream create(Path f, FsPermission permission,
boolean overwrite, int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException {
- FSDataOutputStream out = create(f,
- overwrite, bufferSize, replication, blockSize, progress);
- setPermission(f, permission);
+ FSDataOutputStream out = create(f, overwrite, true, bufferSize, replication,
+ blockSize, progress, permission);
return out;
}
@@ -304,9 +333,8 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
boolean overwrite,
int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException {
- FSDataOutputStream out = create(f,
- overwrite, false, bufferSize, replication, blockSize, progress);
- setPermission(f, permission);
+ FSDataOutputStream out = create(f, overwrite, false, bufferSize, replication,
+ blockSize, progress, permission);
return out;
}
@@ -343,6 +371,31 @@ public boolean rename(Path src, Path dst) throws IOException {
}
return FileUtil.copy(this, src, this, dst, true, getConf());
}
+
+ @Override
+ public boolean truncate(Path f, final long newLength) throws IOException {
+ FileStatus status = getFileStatus(f);
+ if(status == null) {
+ throw new FileNotFoundException("File " + f + " not found");
+ }
+ if(status.isDirectory()) {
+ throw new IOException("Cannot truncate a directory (=" + f + ")");
+ }
+ long oldLength = status.getLen();
+ if(newLength > oldLength) {
+ throw new IllegalArgumentException(
+ "Cannot truncate to a larger file size. Current size: " + oldLength +
+ ", truncate size: " + newLength + ".");
+ }
+ try (FileOutputStream out = new FileOutputStream(pathToFile(f), true)) {
+ try {
+ out.getChannel().truncate(newLength);
+ } catch(IOException e) {
+ throw new FSError(e);
+ }
+ }
+ return true;
+ }
/**
* Delete the given path to a file or directory.
@@ -413,7 +466,34 @@ public FileStatus[] listStatus(Path f) throws IOException {
}
protected boolean mkOneDir(File p2f) throws IOException {
- return p2f.mkdir();
+ return mkOneDirWithMode(new Path(p2f.getAbsolutePath()), p2f, null);
+ }
+
+ protected boolean mkOneDirWithMode(Path p, File p2f, FsPermission permission)
+ throws IOException {
+ if (permission == null) {
+ return p2f.mkdir();
+ } else {
+ if (Shell.WINDOWS && NativeIO.isAvailable()) {
+ try {
+ NativeIO.Windows.createDirectoryWithMode(p2f, permission.toShort());
+ return true;
+ } catch (IOException e) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(String.format(
+ "NativeIO.createDirectoryWithMode error, path = %s, mode = %o",
+ p2f, permission.toShort()), e);
+ }
+ return false;
+ }
+ } else {
+ boolean b = p2f.mkdir();
+ if (b) {
+ setPermission(p, permission);
+ }
+ return b;
+ }
+ }
}
/**
@@ -422,6 +502,16 @@ protected boolean mkOneDir(File p2f) throws IOException {
*/
@Override
public boolean mkdirs(Path f) throws IOException {
+ return mkdirsWithOptionalPermission(f, null);
+ }
+
+ @Override
+ public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+ return mkdirsWithOptionalPermission(f, permission);
+ }
+
+ private boolean mkdirsWithOptionalPermission(Path f, FsPermission permission)
+ throws IOException {
if(f == null) {
throw new IllegalArgumentException("mkdirs path arg is null");
}
@@ -440,25 +530,7 @@ public boolean mkdirs(Path f) throws IOException {
" and is not a directory: " + p2f.getCanonicalPath());
}
return (parent == null || parent2f.exists() || mkdirs(parent)) &&
- (mkOneDir(p2f) || p2f.isDirectory());
- }
-
- @Override
- public boolean mkdirs(Path f, FsPermission permission) throws IOException {
- boolean b = mkdirs(f);
- if(b) {
- setPermission(f, permission);
- }
- return b;
- }
-
-
- @Override
- protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
- throws IOException {
- boolean b = mkdirs(f);
- setPermission(f, absolutePermission);
- return b;
+ (mkOneDirWithMode(f, p2f, permission) || p2f.isDirectory());
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
index 9d36bcff8b03c..47fb25c2570ff 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
@@ -23,6 +23,7 @@
import java.net.ConnectException;
import java.net.URI;
+import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.net.ftp.FTP;
@@ -101,17 +102,12 @@ public void initialize(URI uri, Configuration conf) throws IOException { // get
if (userAndPassword == null) {
userAndPassword = (conf.get("fs.ftp.user." + host, null) + ":" + conf
.get("fs.ftp.password." + host, null));
- if (userAndPassword == null) {
- throw new IOException("Invalid user/passsword specified");
- }
}
String[] userPasswdInfo = userAndPassword.split(":");
+ Preconditions.checkState(userPasswdInfo.length > 1,
+ "Invalid username / password");
conf.set(FS_FTP_USER_PREFIX + host, userPasswdInfo[0]);
- if (userPasswdInfo.length > 1) {
- conf.set(FS_FTP_PASSWORD_PREFIX + host, userPasswdInfo[1]);
- } else {
- conf.set(FS_FTP_PASSWORD_PREFIX + host, null);
- }
+ conf.set(FS_FTP_PASSWORD_PREFIX + host, userPasswdInfo[1]);
setConf(conf);
this.uri = uri;
}
@@ -293,7 +289,8 @@ public FSDataOutputStream append(Path f, int bufferSize,
*/
private boolean exists(FTPClient client, Path file) throws IOException {
try {
- return getFileStatus(client, file) != null;
+ getFileStatus(client, file);
+ return true;
} catch (FileNotFoundException fnfe) {
return false;
}
@@ -333,10 +330,8 @@ private boolean delete(FTPClient client, Path file, boolean recursive)
if (dirEntries != null && dirEntries.length > 0 && !(recursive)) {
throw new IOException("Directory: " + file + " is not empty.");
}
- if (dirEntries != null) {
- for (int i = 0; i < dirEntries.length; i++) {
- delete(client, new Path(absolute, dirEntries[i].getPath()), recursive);
- }
+ for (FileStatus dirEntry : dirEntries) {
+ delete(client, new Path(absolute, dirEntry.getPath()), recursive);
}
return client.removeDirectory(pathName);
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java
index b65b7a0b438b2..b9def6447a870 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java
@@ -146,7 +146,9 @@ public Builder setType(AclEntryType type) {
* @return Builder this builder, for call chaining
*/
public Builder setName(String name) {
- this.name = name;
+ if (name != null && !name.isEmpty()) {
+ this.name = name;
+ }
return this;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java
index 4a7258f0a2738..9d7500a697b1b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
/**
@@ -36,6 +37,7 @@ public class AclStatus {
private final String group;
private final boolean stickyBit;
private final List entries;
+ private final FsPermission permission;
/**
* Returns the file owner.
@@ -73,6 +75,14 @@ public List getEntries() {
return entries;
}
+ /**
+ * Returns the permission set for the path
+ * @return {@link FsPermission} for the path
+ */
+ public FsPermission getPermission() {
+ return permission;
+ }
+
@Override
public boolean equals(Object o) {
if (o == null) {
@@ -113,6 +123,7 @@ public static class Builder {
private String group;
private boolean stickyBit;
private List entries = Lists.newArrayList();
+ private FsPermission permission = null;
/**
* Sets the file owner.
@@ -172,13 +183,22 @@ public Builder stickyBit(boolean stickyBit) {
return this;
}
+ /**
+ * Sets the permission for the file.
+ * @param permission
+ */
+ public Builder setPermission(FsPermission permission) {
+ this.permission = permission;
+ return this;
+ }
+
/**
* Builds a new AclStatus populated with the set properties.
*
* @return AclStatus new AclStatus
*/
public AclStatus build() {
- return new AclStatus(owner, group, stickyBit, entries);
+ return new AclStatus(owner, group, stickyBit, entries, permission);
}
}
@@ -190,12 +210,67 @@ public AclStatus build() {
* @param group String file group
* @param stickyBit the sticky bit
* @param entries the ACL entries
+ * @param permission permission of the path
*/
private AclStatus(String owner, String group, boolean stickyBit,
- Iterable entries) {
+ Iterable entries, FsPermission permission) {
this.owner = owner;
this.group = group;
this.stickyBit = stickyBit;
this.entries = Lists.newArrayList(entries);
+ this.permission = permission;
+ }
+
+ /**
+ * Get the effective permission for the AclEntry
+ * @param entry AclEntry to get the effective action
+ */
+ public FsAction getEffectivePermission(AclEntry entry) {
+ return getEffectivePermission(entry, permission);
+ }
+
+ /**
+ * Get the effective permission for the AclEntry.
+ * Recommended to use this API ONLY if client communicates with the old
+ * NameNode, needs to pass the Permission for the path to get effective
+ * permission, else use {@link AclStatus#getEffectivePermission(AclEntry)}.
+ * @param entry AclEntry to get the effective action
+ * @param permArg Permission for the path. However if the client is NOT
+ * communicating with old namenode, then this argument will not have
+ * any preference.
+ * @return Returns the effective permission for the entry.
+ * @throws IllegalArgumentException If the client communicating with old
+ * namenode and permission is not passed as an argument.
+ */
+ public FsAction getEffectivePermission(AclEntry entry, FsPermission permArg)
+ throws IllegalArgumentException {
+ // At least one permission bits should be available.
+ Preconditions.checkArgument(this.permission != null || permArg != null,
+ "Permission bits are not available to calculate effective permission");
+ if (this.permission != null) {
+ // permission bits from server response will have the priority for
+ // accuracy.
+ permArg = this.permission;
+ }
+ if ((entry.getName() != null || entry.getType() == AclEntryType.GROUP)) {
+ if (entry.getScope() == AclEntryScope.ACCESS) {
+ FsAction entryPerm = entry.getPermission();
+ return entryPerm.and(permArg.getGroupAction());
+ } else {
+ Preconditions.checkArgument(this.entries.contains(entry)
+ && this.entries.size() >= 3,
+ "Passed default ACL entry not found in the list of ACLs");
+ // default mask entry for effective permission calculation will be the
+ // penultimate entry. This can be mask entry in case of extended ACLs.
+ // In case of minimal ACL, this is the owner group entry, and we end up
+ // intersecting group FsAction with itself, which is a no-op.
+ FsAction defaultMask = this.entries.get(this.entries.size() - 2)
+ .getPermission();
+ FsAction entryPerm = entry.getPermission();
+ return entryPerm.and(defaultMask);
+ }
+ } else {
+ return entry.getPermission();
+ }
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
index 206576cfa547c..d139ebadce3bc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
@@ -86,22 +86,26 @@ protected void processPath(PathData item) throws IOException {
(perm.getOtherAction().implies(FsAction.EXECUTE) ? "t" : "T"));
}
- List entries = perm.getAclBit() ?
- item.fs.getAclStatus(item.path).getEntries() :
- Collections.emptyList();
+ AclStatus aclStatus = item.fs.getAclStatus(item.path);
+ List entries = perm.getAclBit() ? aclStatus.getEntries()
+ : Collections. emptyList();
ScopedAclEntries scopedEntries = new ScopedAclEntries(
AclUtil.getAclFromPermAndEntries(perm, entries));
- printAclEntriesForSingleScope(scopedEntries.getAccessEntries());
- printAclEntriesForSingleScope(scopedEntries.getDefaultEntries());
+ printAclEntriesForSingleScope(aclStatus, perm,
+ scopedEntries.getAccessEntries());
+ printAclEntriesForSingleScope(aclStatus, perm,
+ scopedEntries.getDefaultEntries());
out.println();
}
/**
* Prints all the ACL entries in a single scope.
- *
+ * @param aclStatus AclStatus for the path
+ * @param fsPerm FsPermission for the path
* @param entries List containing ACL entries of file
*/
- private void printAclEntriesForSingleScope(List entries) {
+ private void printAclEntriesForSingleScope(AclStatus aclStatus,
+ FsPermission fsPerm, List entries) {
if (entries.isEmpty()) {
return;
}
@@ -110,10 +114,8 @@ private void printAclEntriesForSingleScope(List entries) {
out.println(entry);
}
} else {
- // ACL sort order guarantees mask is the second-to-last entry.
- FsAction maskPerm = entries.get(entries.size() - 2).getPermission();
for (AclEntry entry: entries) {
- printExtendedAclEntry(entry, maskPerm);
+ printExtendedAclEntry(aclStatus, fsPerm, entry);
}
}
}
@@ -123,14 +125,16 @@ private void printAclEntriesForSingleScope(List entries) {
* permissions of the entry, then also prints the restricted version as the
* effective permissions. The mask applies to all named entries and also
* the unnamed group entry.
- *
+ * @param aclStatus AclStatus for the path
+ * @param fsPerm FsPermission for the path
* @param entry AclEntry extended ACL entry to print
- * @param maskPerm FsAction permissions in the ACL's mask entry
*/
- private void printExtendedAclEntry(AclEntry entry, FsAction maskPerm) {
+ private void printExtendedAclEntry(AclStatus aclStatus,
+ FsPermission fsPerm, AclEntry entry) {
if (entry.getName() != null || entry.getType() == AclEntryType.GROUP) {
FsAction entryPerm = entry.getPermission();
- FsAction effectivePerm = entryPerm.and(maskPerm);
+ FsAction effectivePerm = aclStatus
+ .getEffectivePermission(entry, fsPerm);
if (entryPerm != effectivePerm) {
out.println(String.format("%s\t#effective:%s", entry,
effectivePerm.SYMBOL));
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
index ff7a10f2975f5..dd7d1686ca70f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
@@ -21,6 +21,7 @@
import java.util.Arrays;
import java.util.LinkedList;
+import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@@ -44,18 +45,26 @@ public static void registerCommands(CommandFactory factory) {
private static final String OPTION_QUOTA = "q";
private static final String OPTION_HUMAN = "h";
+ private static final String OPTION_HEADER = "v";
public static final String NAME = "count";
public static final String USAGE =
- "[-" + OPTION_QUOTA + "] [-" + OPTION_HUMAN + "] ...";
- public static final String DESCRIPTION =
+ "[-" + OPTION_QUOTA + "] [-" + OPTION_HUMAN + "] [-" + OPTION_HEADER
+ + "] ...";
+ public static final String DESCRIPTION =
"Count the number of directories, files and bytes under the paths\n" +
- "that match the specified file pattern. The output columns are:\n" +
- "DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME or\n" +
- "QUOTA REMAINING_QUOTA SPACE_QUOTA REMAINING_SPACE_QUOTA \n" +
- " DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME\n" +
- "The -h option shows file sizes in human readable format.";
-
+ "that match the specified file pattern. The output columns are:\n" +
+ StringUtils.join(ContentSummary.getHeaderFields(), ' ') +
+ " PATHNAME\n" +
+ "or, with the -" + OPTION_QUOTA + " option:\n" +
+ StringUtils.join(ContentSummary.getQuotaHeaderFields(), ' ') + "\n" +
+ " " +
+ StringUtils.join(ContentSummary.getHeaderFields(), ' ') +
+ " PATHNAME\n" +
+ "The -" + OPTION_HUMAN +
+ " option shows file sizes in human readable format.\n" +
+ "The -" + OPTION_HEADER + " option displays a header line.";
+
private boolean showQuotas;
private boolean humanReadable;
@@ -65,7 +74,7 @@ public Count() {}
/** Constructor
* @deprecated invoke via {@link FsShell}
* @param cmd the count command
- * @param pos the starting index of the arguments
+ * @param pos the starting index of the arguments
* @param conf configuration
*/
@Deprecated
@@ -77,13 +86,16 @@ public Count(String[] cmd, int pos, Configuration conf) {
@Override
protected void processOptions(LinkedList args) {
CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE,
- OPTION_QUOTA, OPTION_HUMAN);
+ OPTION_QUOTA, OPTION_HUMAN, OPTION_HEADER);
cf.parse(args);
if (args.isEmpty()) { // default path is the current working directory
args.add(".");
}
showQuotas = cf.getOpt(OPTION_QUOTA);
humanReadable = cf.getOpt(OPTION_HUMAN);
+ if (cf.getOpt(OPTION_HEADER)) {
+ out.println(ContentSummary.getHeader(showQuotas) + "PATHNAME");
+ }
}
@Override
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java
index d437a663f5557..f0d7b8de4453c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java
@@ -32,6 +32,7 @@
import org.apache.avro.io.DatumWriter;
import org.apache.avro.io.EncoderFactory;
import org.apache.avro.io.JsonEncoder;
+import org.apache.commons.io.Charsets;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@@ -195,11 +196,11 @@ protected void processPath(PathData item) throws IOException {
FileChecksum checksum = item.fs.getFileChecksum(item.path);
if (checksum == null) {
- out.printf("%s\tNONE\t\n", item.toString());
+ out.printf("%s\tNONE\t%n", item.toString());
} else {
String checksumString = StringUtils.byteToHexString(
checksum.getBytes(), 0, checksum.getLength());
- out.printf("%s\t%s\t%s\n",
+ out.printf("%s\t%s\t%s%n",
item.toString(), checksum.getAlgorithmName(),
checksumString);
}
@@ -234,10 +235,10 @@ public int read() throws IOException {
if (!r.next(key, val)) {
return -1;
}
- byte[] tmp = key.toString().getBytes();
+ byte[] tmp = key.toString().getBytes(Charsets.UTF_8);
outbuf.write(tmp, 0, tmp.length);
outbuf.write('\t');
- tmp = val.toString().getBytes();
+ tmp = val.toString().getBytes(Charsets.UTF_8);
outbuf.write(tmp, 0, tmp.length);
outbuf.write('\n');
inbuf.reset(outbuf.getData(), outbuf.getLength());
@@ -299,7 +300,8 @@ public int read() throws IOException {
encoder.flush();
if (!fileReader.hasNext()) {
// Write a new line after the last Avro record.
- output.write(System.getProperty("line.separator").getBytes());
+ output.write(System.getProperty("line.separator")
+ .getBytes(Charsets.UTF_8));
output.flush();
}
pos = 0;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
index cc8fbb4f2f1c3..9515fde4c4447 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
@@ -60,6 +60,7 @@ public static void registerCommands(CommandFactory factory) {
factory.registerCommands(Tail.class);
factory.registerCommands(Test.class);
factory.registerCommands(Touch.class);
+ factory.registerCommands(Truncate.class);
factory.registerCommands(SnapshotCommands.class);
factory.registerCommands(XAttrCommands.class);
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
index 6024d885afb05..c7e80b6b4af1f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
@@ -57,7 +57,7 @@ public static void registerCommands(CommandFactory factory) {
- protected static final SimpleDateFormat dateFormat =
+ protected final SimpleDateFormat dateFormat =
new SimpleDateFormat("yyyy-MM-dd HH:mm");
protected int maxRepl = 3, maxLen = 10, maxOwner = 0, maxGroup = 0;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
index ee56fe6492c93..458d3ee7852b5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
@@ -30,15 +30,17 @@
/**
* Print statistics about path in specified format.
- * Format sequences:
- * %b: Size of file in blocks
- * %g: Group name of owner
- * %n: Filename
- * %o: Block size
- * %r: replication
- * %u: User name of owner
- * %y: UTC date as "yyyy-MM-dd HH:mm:ss"
- * %Y: Milliseconds since January 1, 1970 UTC
+ * Format sequences:
+ * %b: Size of file in blocks
+ * %F: Type
+ * %g: Group name of owner
+ * %n: Filename
+ * %o: Block size
+ * %r: replication
+ * %u: User name of owner
+ * %y: UTC date as "yyyy-MM-dd HH:mm:ss"
+ * %Y: Milliseconds since January 1, 1970 UTC
+ * If the format is not specified, %y is used by default.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
@@ -48,15 +50,22 @@ public static void registerCommands(CommandFactory factory) {
factory.addClass(Stat.class, "-stat");
}
+ private static final String NEWLINE = System.getProperty("line.separator");
+
public static final String NAME = "stat";
public static final String USAGE = "[format] ...";
public static final String DESCRIPTION =
- "Print statistics about the file/directory at " +
- "in the specified format. Format accepts filesize in blocks (%b), group name of owner(%g), " +
- "filename (%n), block size (%o), replication (%r), user name of owner(%u), modification date (%y, %Y)\n";
+ "Print statistics about the file/directory at " + NEWLINE +
+ "in the specified format. Format accepts filesize in" + NEWLINE +
+ "blocks (%b), type (%F), group name of owner (%g)," + NEWLINE +
+ "name (%n), block size (%o), replication (%r), user name" + NEWLINE +
+ "of owner (%u), modification date (%y, %Y)." + NEWLINE +
+ "%y shows UTC date as \"yyyy-MM-dd HH:mm:ss\" and" + NEWLINE +
+ "%Y shows milliseconds since January 1, 1970 UTC." + NEWLINE +
+ "If the format is not specified, %y is used by default." + NEWLINE;
- protected static final SimpleDateFormat timeFmt;
- static {
+ protected final SimpleDateFormat timeFmt;
+ {
timeFmt = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
timeFmt.setTimeZone(TimeZone.getTimeZone("UTC"));
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java
index 4cfdb084d7eb7..9984cf26fa4d3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Test.java
@@ -83,6 +83,8 @@ protected void processPath(PathData item) throws IOException {
case 'z':
test = (item.stat.getLen() == 0);
break;
+ default:
+ break;
}
if (!test) exitCode = 1;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Truncate.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Truncate.java
new file mode 100644
index 0000000000000..99128634900a1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Truncate.java
@@ -0,0 +1,117 @@
+/**
+ * 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.shell;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.PathIsDirectoryException;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Truncates a file to a new size
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Truncate extends FsCommand {
+ public static void registerCommands(CommandFactory factory) {
+ factory.addClass(Truncate.class, "-truncate");
+ }
+
+ public static final String NAME = "truncate";
+ public static final String USAGE = "[-w] ...";
+ public static final String DESCRIPTION =
+ "Truncate all files that match the specified file pattern to the " +
+ "specified length.\n" +
+ "-w: Requests that the command wait for block recovery to complete, " +
+ "if necessary.";
+
+ protected long newLength = -1;
+ protected List waitList = new LinkedList<>();
+ protected boolean waitOpt = false;
+
+ @Override
+ protected void processOptions(LinkedList args) throws IOException {
+ CommandFormat cf = new CommandFormat(2, Integer.MAX_VALUE, "w");
+ cf.parse(args);
+ waitOpt = cf.getOpt("w");
+
+ try {
+ newLength = Long.parseLong(args.removeFirst());
+ } catch(NumberFormatException nfe) {
+ displayWarning("Illegal length, a non-negative integer expected");
+ throw nfe;
+ }
+ if(newLength < 0) {
+ throw new IllegalArgumentException("length must be >= 0");
+ }
+ }
+
+ @Override
+ protected void processArguments(LinkedList args)
+ throws IOException {
+ super.processArguments(args);
+ if (waitOpt) waitForRecovery();
+ }
+
+ @Override
+ protected void processPath(PathData item) throws IOException {
+ if(item.stat.isDirectory()) {
+ throw new PathIsDirectoryException(item.toString());
+ }
+ long oldLength = item.stat.getLen();
+ if(newLength > oldLength) {
+ throw new IllegalArgumentException(
+ "Cannot truncate to a larger file size. Current size: " + oldLength +
+ ", truncate size: " + newLength + ".");
+ }
+ if(item.fs.truncate(item.path, newLength)) {
+ out.println("Truncated " + item + " to length: " + newLength);
+ }
+ else if(waitOpt) {
+ waitList.add(item);
+ }
+ else {
+ out.println("Truncating " + item + " to length: " + newLength + ". " +
+ "Wait for block recovery to complete before further updating this " +
+ "file.");
+ }
+ }
+
+ /**
+ * Wait for all files in waitList to have length equal to newLength.
+ */
+ private void waitForRecovery() throws IOException {
+ for(PathData item : waitList) {
+ out.println("Waiting for " + item + " ...");
+ out.flush();
+
+ for(;;) {
+ item.refreshStatus();
+ if(item.stat.getLen() == newLength) break;
+ try {Thread.sleep(1000);} catch(InterruptedException ignored) {}
+ }
+
+ out.println("Truncated " + item + " to length: " + newLength);
+ out.flush();
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 963289f43739b..0f77f470d9723 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -446,6 +446,14 @@ public boolean rename(final Path src, final Path dst) throws IOException {
return resSrc.targetFileSystem.rename(resSrc.remainingPath,
resDst.remainingPath);
}
+
+ @Override
+ public boolean truncate(final Path f, final long newLength)
+ throws IOException {
+ InodeTree.ResolveResult res =
+ fsState.resolve(getUriPath(f), true);
+ return res.targetFileSystem.truncate(f, newLength);
+ }
@Override
public void setOwner(final Path f, final String username,
@@ -833,6 +841,11 @@ public boolean rename(Path src, Path dst) throws AccessControlException,
throw readOnlyMountTable("rename", src);
}
+ @Override
+ public boolean truncate(Path f, long newLength) throws IOException {
+ throw readOnlyMountTable("truncate", f);
+ }
+
@Override
public void setOwner(Path f, String username, String groupname)
throws AccessControlException, IOException {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
index 123f119f709c5..947baa93e1e8e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
@@ -1064,7 +1064,9 @@ private void setZooKeeperRef(ZooKeeper zk) {
public void process(WatchedEvent event) {
hasReceivedEvent.countDown();
try {
- hasSetZooKeeper.await(zkSessionTimeout, TimeUnit.MILLISECONDS);
+ if (!hasSetZooKeeper.await(zkSessionTimeout, TimeUnit.MILLISECONDS)) {
+ LOG.debug("Event received with stale zk");
+ }
ActiveStandbyElector.this.processWatchEvent(
zk, event);
} catch (Throwable t) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
index bd6366c76cc44..f72df7739e32f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
@@ -69,15 +69,15 @@ public abstract class HAAdmin extends Configured implements Tool {
protected final static Map USAGE =
ImmutableMap.builder()
.put("-transitionToActive",
- new UsageInfo(" [--"+FORCEACTIVE+"]", "Transitions the service into Active state"))
+ new UsageInfo("[--"+FORCEACTIVE+"] ", "Transitions the service into Active state"))
.put("-transitionToStandby",
new UsageInfo("", "Transitions the service into Standby state"))
.put("-failover",
new UsageInfo("[--"+FORCEFENCE+"] [--"+FORCEACTIVE+"] ",
"Failover from the first service to the second.\n" +
- "Unconditionally fence services if the "+FORCEFENCE+" option is used.\n" +
+ "Unconditionally fence services if the --"+FORCEFENCE+" option is used.\n" +
"Try to failover to the target service even if it is not ready if the " +
- FORCEACTIVE + " option is used."))
+ "--" + FORCEACTIVE + " option is used."))
.put("-getServiceState",
new UsageInfo("", "Returns the state of the service"))
.put("-checkHealth",
@@ -168,12 +168,6 @@ private int transitionToActive(final CommandLine cmd)
private boolean isOtherTargetNodeActive(String targetNodeToActivate, boolean forceActive)
throws IOException {
Collection targetIds = getTargetIds(targetNodeToActivate);
- if(targetIds == null) {
- errOut.println("transitionToActive: No target node in the "
- + "current configuration");
- printUsage(errOut, "-transitionToActive");
- return true;
- }
targetIds.remove(targetNodeToActivate);
for(String targetId : targetIds) {
HAServiceTarget target = resolveTarget(targetId);
@@ -234,7 +228,7 @@ private boolean checkManualStateManagementOK(HAServiceTarget target) {
"Refusing to manually manage HA state, since it may cause\n" +
"a split-brain scenario or other incorrect state.\n" +
"If you are very sure you know what you are doing, please \n" +
- "specify the " + FORCEMANUAL + " flag.");
+ "specify the --" + FORCEMANUAL + " flag.");
return false;
} else {
LOG.warn("Proceeding with manual HA state management even though\n" +
@@ -468,7 +462,7 @@ protected int runCmd(String[] argv) throws Exception {
private boolean confirmForceManual() throws IOException {
return ToolRunner.confirmPrompt(
- "You have specified the " + FORCEMANUAL + " flag. This flag is " +
+ "You have specified the --" + FORCEMANUAL + " flag. This flag is " +
"dangerous, as it can induce a split-brain scenario that WILL " +
"CORRUPT your HDFS namespace, possibly irrecoverably.\n" +
"\n" +
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java
index 0f5465194b928..90eb915e69ffb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java
@@ -310,6 +310,8 @@ public void log(int level, String message) {
case com.jcraft.jsch.Logger.FATAL:
LOG.fatal(message);
break;
+ default:
+ break;
}
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/StreamPumper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/StreamPumper.java
index 8bc16af2afa99..00c6401d88d9d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/StreamPumper.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/StreamPumper.java
@@ -22,6 +22,7 @@
import java.io.InputStream;
import java.io.InputStreamReader;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
/**
@@ -76,7 +77,8 @@ void start() {
}
protected void pump() throws IOException {
- InputStreamReader inputStreamReader = new InputStreamReader(stream);
+ InputStreamReader inputStreamReader = new InputStreamReader(
+ stream, Charsets.UTF_8);
BufferedReader br = new BufferedReader(inputStreamReader);
String line = null;
while ((line = br.readLine()) != null) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
index 46c485b1d9a79..f58c3f4de997c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
@@ -153,7 +153,9 @@ protected abstract void checkRpcAdminAccess()
public HAServiceTarget getLocalTarget() {
return localTarget;
}
-
+
+ HAServiceState getServiceState() { return serviceState; }
+
public int run(final String[] args) throws Exception {
if (!localTarget.isAutoFailoverEnabled()) {
LOG.fatal("Automatic failover is not enabled for " + localTarget + "." +
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java
index 99befeea6eb43..57acebd85f4a7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.http;
+import org.apache.commons.io.Charsets;
+
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.OutputStream;
@@ -25,11 +27,11 @@
* This class is responsible for quoting HTML characters.
*/
public class HtmlQuoting {
- private static final byte[] ampBytes = "&".getBytes();
- private static final byte[] aposBytes = "'".getBytes();
- private static final byte[] gtBytes = ">".getBytes();
- private static final byte[] ltBytes = "<".getBytes();
- private static final byte[] quotBytes = """.getBytes();
+ private static final byte[] ampBytes = "&".getBytes(Charsets.UTF_8);
+ private static final byte[] aposBytes = "'".getBytes(Charsets.UTF_8);
+ private static final byte[] gtBytes = ">".getBytes(Charsets.UTF_8);
+ private static final byte[] ltBytes = "<".getBytes(Charsets.UTF_8);
+ private static final byte[] quotBytes = """.getBytes(Charsets.UTF_8);
/**
* Does the given string need to be quoted?
@@ -63,7 +65,7 @@ public static boolean needsQuoting(String str) {
if (str == null) {
return false;
}
- byte[] bytes = str.getBytes();
+ byte[] bytes = str.getBytes(Charsets.UTF_8);
return needsQuoting(bytes, 0 , bytes.length);
}
@@ -98,15 +100,16 @@ public static String quoteHtmlChars(String item) {
if (item == null) {
return null;
}
- byte[] bytes = item.getBytes();
+ byte[] bytes = item.getBytes(Charsets.UTF_8);
if (needsQuoting(bytes, 0, bytes.length)) {
ByteArrayOutputStream buffer = new ByteArrayOutputStream();
try {
quoteHtmlChars(buffer, bytes, 0, bytes.length);
+ return buffer.toString("UTF-8");
} catch (IOException ioe) {
// Won't happen, since it is a bytearrayoutputstream
+ return null;
}
- return buffer.toString();
} else {
return item;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
index 45b641957077c..80831e976aa57 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
@@ -20,7 +20,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
-import java.io.PrintWriter;
+import java.io.PrintStream;
import java.net.BindException;
import java.net.InetSocketAddress;
import java.net.URI;
@@ -123,26 +123,13 @@ public final class HttpServer2 implements FilterContainer {
protected final Server webServer;
- private static class ListenerInfo {
- /**
- * Boolean flag to determine whether the HTTP server should clean up the
- * listener in stop().
- */
- private final boolean isManaged;
- private final Connector listener;
- private ListenerInfo(boolean isManaged, Connector listener) {
- this.isManaged = isManaged;
- this.listener = listener;
- }
- }
-
- private final List listeners = Lists.newArrayList();
+ private final List listeners = Lists.newArrayList();
protected final WebAppContext webAppContext;
protected final boolean findPort;
protected final Map defaultContexts =
- new HashMap();
- protected final List filterNames = new ArrayList();
+ new HashMap<>();
+ protected final List filterNames = new ArrayList<>();
static final String STATE_DESCRIPTION_ALIVE = " - alive";
static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
@@ -151,7 +138,6 @@ private ListenerInfo(boolean isManaged, Connector listener) {
*/
public static class Builder {
private ArrayList endpoints = Lists.newArrayList();
- private Connector connector;
private String name;
private Configuration conf;
private String[] pathSpecs;
@@ -243,11 +229,6 @@ public Builder setConf(Configuration conf) {
return this;
}
- public Builder setConnector(Connector connector) {
- this.connector = connector;
- return this;
- }
-
public Builder setPathSpec(String[] pathSpec) {
this.pathSpecs = pathSpec;
return this;
@@ -274,17 +255,11 @@ public Builder setKeytabConfKey(String keytabConfKey) {
}
public HttpServer2 build() throws IOException {
- if (this.name == null) {
- throw new HadoopIllegalArgumentException("name is not set");
- }
-
- if (endpoints.size() == 0 && connector == null) {
- throw new HadoopIllegalArgumentException("No endpoints specified");
- }
+ Preconditions.checkNotNull(name, "name is not set");
+ Preconditions.checkState(!endpoints.isEmpty(), "No endpoints specified");
if (hostName == null) {
- hostName = endpoints.size() == 0 ? connector.getHost() : endpoints.get(
- 0).getHost();
+ hostName = endpoints.get(0).getHost();
}
if (this.conf == null) {
@@ -297,17 +272,14 @@ public HttpServer2 build() throws IOException {
server.initSpnego(conf, hostName, usernameConfKey, keytabConfKey);
}
- if (connector != null) {
- server.addUnmanagedListener(connector);
- }
-
for (URI ep : endpoints) {
- Connector listener = null;
+ final Connector listener;
String scheme = ep.getScheme();
if ("http".equals(scheme)) {
listener = HttpServer2.createDefaultChannelConnector();
} else if ("https".equals(scheme)) {
SslSocketConnector c = new SslSocketConnectorSecure();
+ c.setHeaderBufferSize(1024*64);
c.setNeedClientAuth(needsClientAuth);
c.setKeyPassword(keyPassword);
@@ -330,7 +302,7 @@ public HttpServer2 build() throws IOException {
}
listener.setHost(ep.getHost());
listener.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
- server.addManagedListener(listener);
+ server.addListener(listener);
}
server.loadListeners();
return server;
@@ -348,7 +320,7 @@ private HttpServer2(final Builder b) throws IOException {
private void initializeWebServer(String name, String hostName,
Configuration conf, String[] pathSpecs)
- throws FileNotFoundException, IOException {
+ throws IOException {
Preconditions.checkNotNull(webAppContext);
@@ -406,12 +378,8 @@ private void initializeWebServer(String name, String hostName,
}
}
- private void addUnmanagedListener(Connector connector) {
- listeners.add(new ListenerInfo(false, connector));
- }
-
- private void addManagedListener(Connector connector) {
- listeners.add(new ListenerInfo(true, connector));
+ private void addListener(Connector connector) {
+ listeners.add(connector);
}
private static WebAppContext createWebAppContext(String name,
@@ -442,15 +410,6 @@ private static void addNoCacheFilter(WebAppContext ctxt) {
Collections. emptyMap(), new String[] { "/*" });
}
- /**
- * Create a required listener for the Jetty instance listening on the port
- * provided. This wrapper and all subclasses must create at least one
- * listener.
- */
- public Connector createBaseListener(Configuration conf) {
- return HttpServer2.createDefaultChannelConnector();
- }
-
@InterfaceAudience.Private
public static Connector createDefaultChannelConnector() {
SelectChannelConnector ret = new SelectChannelConnector();
@@ -546,23 +505,6 @@ public void addContext(Context ctxt, boolean isFiltered) {
defaultContexts.put(ctxt, isFiltered);
}
- /**
- * Add a context
- * @param pathSpec The path spec for the context
- * @param dir The directory containing the context
- * @param isFiltered if true, the servlet is added to the filter path mapping
- * @throws IOException
- */
- protected void addContext(String pathSpec, String dir, boolean isFiltered) throws IOException {
- if (0 == webServer.getHandlers().length) {
- throw new RuntimeException("Couldn't find handler");
- }
- WebAppContext webAppCtx = new WebAppContext();
- webAppCtx.setContextPath(pathSpec);
- webAppCtx.setWar(dir);
- addContext(webAppCtx, true);
- }
-
/**
* Set a value in the webapp context. These values are available to the jsp
* pages as "application.getAttribute(name)".
@@ -654,8 +596,8 @@ public void addFilter(String name, String classname,
final String[] USER_FACING_URLS = { "*.html", "*.jsp" };
defineFilter(webAppContext, name, classname, parameters, USER_FACING_URLS);
- LOG.info("Added filter " + name + " (class=" + classname
- + ") to context " + webAppContext.getDisplayName());
+ LOG.info(
+ "Added filter " + name + " (class=" + classname + ") to context " + webAppContext.getDisplayName());
final String[] ALL_URLS = { "/*" };
for (Map.Entry e : defaultContexts.entrySet()) {
if (e.getValue()) {
@@ -782,7 +724,7 @@ public void setThreads(int min, int max) {
private void initSpnego(Configuration conf, String hostName,
String usernameConfKey, String keytabConfKey) throws IOException {
- Map params = new HashMap();
+ Map params = new HashMap<>();
String principalInConf = conf.get(usernameConfKey);
if (principalInConf != null && !principalInConf.isEmpty()) {
params.put("kerberos.principal", SecurityUtil.getServerPrincipal(
@@ -815,8 +757,8 @@ public void start() throws IOException {
}
// Make sure there is no handler failures.
Handler[] handlers = webServer.getHandlers();
- for (int i = 0; i < handlers.length; i++) {
- if (handlers[i].isFailed()) {
+ for (Handler handler : handlers) {
+ if (handler.isFailed()) {
throw new IOException(
"Problem in starting http server. Server handlers failed");
}
@@ -841,8 +783,8 @@ public void start() throws IOException {
}
private void loadListeners() {
- for (ListenerInfo li : listeners) {
- webServer.addConnector(li.listener);
+ for (Connector c : listeners) {
+ webServer.addConnector(c);
}
}
@@ -851,9 +793,8 @@ private void loadListeners() {
* @throws Exception
*/
void openListeners() throws Exception {
- for (ListenerInfo li : listeners) {
- Connector listener = li.listener;
- if (!li.isManaged || li.listener.getLocalPort() != -1) {
+ for (Connector listener : listeners) {
+ if (listener.getLocalPort() != -1) {
// This listener is either started externally or has been bound
continue;
}
@@ -886,13 +827,9 @@ void openListeners() throws Exception {
*/
public void stop() throws Exception {
MultiException exception = null;
- for (ListenerInfo li : listeners) {
- if (!li.isManaged) {
- continue;
- }
-
+ for (Connector c : listeners) {
try {
- li.listener.close();
+ c.close();
} catch (Exception e) {
LOG.error(
"Error while stopping listener for webapp"
@@ -945,23 +882,17 @@ public boolean isAlive() {
return webServer != null && webServer.isStarted();
}
- /**
- * Return the host and port of the HttpServer, if live
- * @return the classname and any HTTP URL
- */
@Override
public String toString() {
- if (listeners.size() == 0) {
- return "Inactive HttpServer";
- } else {
- StringBuilder sb = new StringBuilder("HttpServer (")
- .append(isAlive() ? STATE_DESCRIPTION_ALIVE : STATE_DESCRIPTION_NOT_LIVE).append("), listening at:");
- for (ListenerInfo li : listeners) {
- Connector l = li.listener;
- sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
- }
- return sb.toString();
+ Preconditions.checkState(!listeners.isEmpty());
+ StringBuilder sb = new StringBuilder("HttpServer (")
+ .append(isAlive() ? STATE_DESCRIPTION_ALIVE
+ : STATE_DESCRIPTION_NOT_LIVE)
+ .append("), listening at:");
+ for (Connector l : listeners) {
+ sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
}
+ return sb.toString();
}
/**
@@ -999,8 +930,6 @@ public static boolean isInstrumentationAccessAllowed(
* Does the user sending the HttpServletRequest has the administrator ACLs? If
* it isn't the case, response will be modified to send an error to the user.
*
- * @param servletContext
- * @param request
* @param response used to send the error response if user does not have admin access.
* @return true if admin-authorized, false otherwise
* @throws IOException
@@ -1065,13 +994,14 @@ public static class StackServlet extends HttpServlet {
public void doGet(HttpServletRequest request, HttpServletResponse response)
throws ServletException, IOException {
if (!HttpServer2.isInstrumentationAccessAllowed(getServletContext(),
- request, response)) {
+ request, response)) {
return;
}
response.setContentType("text/plain; charset=UTF-8");
- PrintWriter out = response.getWriter();
- ReflectionUtils.printThreadInfo(out, "");
- out.close();
+ try (PrintStream out = new PrintStream(
+ response.getOutputStream(), false, "UTF-8")) {
+ ReflectionUtils.printThreadInfo(out, "");
+ }
ReflectionUtils.logThreadInfo(LOG, "jsp requested", 1);
}
}
@@ -1138,7 +1068,7 @@ public String[] getParameterValues(String name) {
@SuppressWarnings("unchecked")
@Override
public Map getParameterMap() {
- Map result = new HashMap();
+ Map result = new HashMap<>();
Map raw = rawRequest.getParameterMap();
for (Map.Entry item: raw.entrySet()) {
String[] rawValue = item.getValue();
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DefaultStringifier.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DefaultStringifier.java
index d32d58b6008be..3ba577fc4f4de 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DefaultStringifier.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DefaultStringifier.java
@@ -23,6 +23,7 @@
import java.util.ArrayList;
import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.io.Charsets;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@@ -90,7 +91,7 @@ public String toString(T obj) throws IOException {
serializer.serialize(obj);
byte[] buf = new byte[outBuf.getLength()];
System.arraycopy(outBuf.getData(), 0, buf, 0, buf.length);
- return new String(Base64.encodeBase64(buf));
+ return new String(Base64.encodeBase64(buf), Charsets.UTF_8);
}
@Override
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FastByteComparisons.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FastByteComparisons.java
index 3f5881b2dd6ee..a3fea3115cbf5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FastByteComparisons.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FastByteComparisons.java
@@ -24,6 +24,9 @@
import sun.misc.Unsafe;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
import com.google.common.primitives.Longs;
import com.google.common.primitives.UnsignedBytes;
@@ -33,6 +36,7 @@
* class to be able to compare arrays that start at non-zero offsets.
*/
abstract class FastByteComparisons {
+ static final Log LOG = LogFactory.getLog(FastByteComparisons.class);
/**
* Lexicographically compare two byte arrays.
@@ -71,6 +75,13 @@ private static class LexicographicalComparerHolder {
* implementation if unable to do so.
*/
static Comparer getBestComparer() {
+ if (System.getProperty("os.arch").equals("sparc")) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Lexicographical comparer selected for "
+ + "byte aligned system architecture");
+ }
+ return lexicographicalComparerJavaImpl();
+ }
try {
Class> theClass = Class.forName(UNSAFE_COMPARER_NAME);
@@ -78,8 +89,16 @@ static Comparer getBestComparer() {
@SuppressWarnings("unchecked")
Comparer comparer =
(Comparer) theClass.getEnumConstants()[0];
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Unsafe comparer selected for "
+ + "byte unaligned system architecture");
+ }
return comparer;
} catch (Throwable t) { // ensure we really catch *everything*
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(t.getMessage());
+ LOG.trace("Lexicographical comparer selected");
+ }
return lexicographicalComparerJavaImpl();
}
}
@@ -234,4 +253,4 @@ public int compareTo(byte[] buffer1, int offset1, int length1,
}
}
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
index 6be7446c99450..e6c00c940bb24 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
@@ -23,6 +23,12 @@
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.channels.WritableByteChannel;
+import java.nio.file.DirectoryStream;
+import java.nio.file.DirectoryIteratorException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -30,6 +36,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ChunkedArrayList;
/**
* An utility class for I/O related functionality.
@@ -313,4 +320,33 @@ public static void writeFully(FileChannel fc, ByteBuffer buf,
offset += fc.write(buf, offset);
} while (buf.remaining() > 0);
}
+
+ /**
+ * Return the complete list of files in a directory as strings.
+ *
+ * This is better than File#listDir because it does not ignore IOExceptions.
+ *
+ * @param dir The directory to list.
+ * @param filter If non-null, the filter to use when listing
+ * this directory.
+ * @return The list of files in the directory.
+ *
+ * @throws IOException On I/O error
+ */
+ public static List listDirectory(File dir, FilenameFilter filter)
+ throws IOException {
+ ArrayList list = new ArrayList ();
+ try (DirectoryStream stream =
+ Files.newDirectoryStream(dir.toPath())) {
+ for (Path entry: stream) {
+ String fileName = entry.getFileName().toString();
+ if ((filter == null) || filter.accept(dir, fileName)) {
+ list.add(fileName);
+ }
+ }
+ } catch (DirectoryIteratorException e) {
+ throw e.getCause();
+ }
+ return list;
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/LongWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/LongWritable.java
index 6dec4aa618a4f..b77ca6781a639 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/LongWritable.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/LongWritable.java
@@ -99,11 +99,11 @@ public static class DecreasingComparator extends Comparator {
@Override
public int compare(WritableComparable a, WritableComparable b) {
- return -super.compare(a, b);
+ return super.compare(b, a);
}
@Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
- return -super.compare(b1, s1, l1, b2, s2, l2);
+ return super.compare(b2, s2, l2, b1, s1, l1);
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
index 4cda107748208..7a59149ff0596 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
@@ -22,6 +22,8 @@
import java.util.*;
import java.rmi.server.UID;
import java.security.MessageDigest;
+
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.*;
import org.apache.hadoop.util.Options;
import org.apache.hadoop.fs.*;
@@ -849,7 +851,7 @@ public static class Writer implements java.io.Closeable, Syncable {
try {
MessageDigest digester = MessageDigest.getInstance("MD5");
long time = Time.now();
- digester.update((new UID()+"@"+time).getBytes());
+ digester.update((new UID()+"@"+time).getBytes(Charsets.UTF_8));
sync = digester.digest();
} catch (Exception e) {
throw new RuntimeException(e);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
index 3dc507687f516..0bcaee3ea79ac 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
@@ -584,6 +584,8 @@ public static void validateUTF8(byte[] utf8, int start, int len)
state = TRAIL_BYTE;
}
break;
+ default:
+ break;
} // switch (state)
count++;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
index 37b97f2a641a9..2c5a7bec852ba 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
@@ -23,6 +23,7 @@
import java.io.InputStream;
import java.io.OutputStream;
+import org.apache.commons.io.Charsets;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
@@ -224,7 +225,7 @@ public SplitCompressionInputStream createInputStream(InputStream seekableIn,
// ........................................^^[We align at wrong position!]
// ...........................................................^^[While this pos is correct]
- if (in.getPos() <= start) {
+ if (in.getPos() < start) {
((Seekable)seekableIn).seek(start);
in = new BZip2CompressionInputStream(seekableIn, start, end, readMode);
}
@@ -281,7 +282,7 @@ private void writeStreamHeader() throws IOException {
// The compressed bzip2 stream should start with the
// identifying characters BZ. Caller of CBZip2OutputStream
// i.e. this class must write these characters.
- out.write(HEADER.getBytes());
+ out.write(HEADER.getBytes(Charsets.UTF_8));
}
}
@@ -415,7 +416,7 @@ private BufferedInputStream readStreamHeader() throws IOException {
byte[] headerBytes = new byte[HEADER_LEN];
int actualRead = bufferedIn.read(headerBytes, 0, HEADER_LEN);
if (actualRead != -1) {
- String header = new String(headerBytes);
+ String header = new String(headerBytes, Charsets.UTF_8);
if (header.compareTo(HEADER) != 0) {
bufferedIn.reset();
} else {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DecompressorStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DecompressorStream.java
index 16e0ad763ac57..6bee6b84e5318 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DecompressorStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DecompressorStream.java
@@ -40,7 +40,7 @@ public DecompressorStream(InputStream in, Decompressor decompressor,
throws IOException {
super(in);
- if (in == null || decompressor == null) {
+ if (decompressor == null) {
throw new NullPointerException();
} else if (bufferSize <= 0) {
throw new IllegalArgumentException("Illegal bufferSize");
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
new file mode 100644
index 0000000000000..01e8f3580e505
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
@@ -0,0 +1,93 @@
+/**
+ * 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.io.erasurecode;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A wrapper for ByteBuffer or bytes array for an erasure code chunk.
+ */
+public class ECChunk {
+
+ private ByteBuffer chunkBuffer;
+
+ /**
+ * Wrapping a ByteBuffer
+ * @param buffer
+ */
+ public ECChunk(ByteBuffer buffer) {
+ this.chunkBuffer = buffer;
+ }
+
+ /**
+ * Wrapping a bytes array
+ * @param buffer
+ */
+ public ECChunk(byte[] buffer) {
+ this.chunkBuffer = ByteBuffer.wrap(buffer);
+ }
+
+ /**
+ * Convert to ByteBuffer
+ * @return ByteBuffer
+ */
+ public ByteBuffer getBuffer() {
+ return chunkBuffer;
+ }
+
+ /**
+ * Convert an array of this chunks to an array of ByteBuffers
+ * @param chunks
+ * @return an array of ByteBuffers
+ */
+ public static ByteBuffer[] toBuffers(ECChunk[] chunks) {
+ ByteBuffer[] buffers = new ByteBuffer[chunks.length];
+
+ for (int i = 0; i < chunks.length; i++) {
+ buffers[i] = chunks[i].getBuffer();
+ }
+
+ return buffers;
+ }
+
+ /**
+ * Convert an array of this chunks to an array of byte array.
+ * Note the chunk buffers are not affected.
+ * @param chunks
+ * @return an array of byte array
+ */
+ public static byte[][] toArray(ECChunk[] chunks) {
+ byte[][] bytesArr = new byte[chunks.length][];
+
+ ByteBuffer buffer;
+ for (int i = 0; i < chunks.length; i++) {
+ buffer = chunks[i].getBuffer();
+ if (buffer.hasArray()) {
+ bytesArr[i] = buffer.array();
+ } else {
+ bytesArr[i] = new byte[buffer.remaining()];
+ // Avoid affecting the original one
+ buffer.mark();
+ buffer.get(bytesArr[i]);
+ buffer.reset();
+ }
+ }
+
+ return bytesArr;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
new file mode 100644
index 0000000000000..74d2ab6fc1ecf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
@@ -0,0 +1,63 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+/**
+ * A common class of basic facilities to be shared by encoder and decoder
+ *
+ * It implements the {@link RawErasureCoder} interface.
+ */
+public abstract class AbstractRawErasureCoder implements RawErasureCoder {
+
+ private int numDataUnits;
+ private int numParityUnits;
+ private int chunkSize;
+
+ @Override
+ public void initialize(int numDataUnits, int numParityUnits,
+ int chunkSize) {
+ this.numDataUnits = numDataUnits;
+ this.numParityUnits = numParityUnits;
+ this.chunkSize = chunkSize;
+ }
+
+ @Override
+ public int getNumDataUnits() {
+ return numDataUnits;
+ }
+
+ @Override
+ public int getNumParityUnits() {
+ return numParityUnits;
+ }
+
+ @Override
+ public int getChunkSize() {
+ return chunkSize;
+ }
+
+ @Override
+ public boolean preferNativeBuffer() {
+ return false;
+ }
+
+ @Override
+ public void release() {
+ // Nothing to do by default
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
new file mode 100644
index 0000000000000..4613b25eb13f3
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
@@ -0,0 +1,93 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+
+import java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure decoder that's to be inherited by new decoders.
+ *
+ * It implements the {@link RawErasureDecoder} interface.
+ */
+public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
+ implements RawErasureDecoder {
+
+ @Override
+ public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+ ByteBuffer[] outputs) {
+ if (erasedIndexes.length == 0) {
+ return;
+ }
+
+ doDecode(inputs, erasedIndexes, outputs);
+ }
+
+ /**
+ * Perform the real decoding using ByteBuffer
+ * @param inputs
+ * @param erasedIndexes
+ * @param outputs
+ */
+ protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+ ByteBuffer[] outputs);
+
+ @Override
+ public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) {
+ if (erasedIndexes.length == 0) {
+ return;
+ }
+
+ doDecode(inputs, erasedIndexes, outputs);
+ }
+
+ /**
+ * Perform the real decoding using bytes array
+ * @param inputs
+ * @param erasedIndexes
+ * @param outputs
+ */
+ protected abstract void doDecode(byte[][] inputs, int[] erasedIndexes,
+ byte[][] outputs);
+
+ @Override
+ public void decode(ECChunk[] inputs, int[] erasedIndexes,
+ ECChunk[] outputs) {
+ doDecode(inputs, erasedIndexes, outputs);
+ }
+
+ /**
+ * Perform the real decoding using chunks
+ * @param inputs
+ * @param erasedIndexes
+ * @param outputs
+ */
+ protected void doDecode(ECChunk[] inputs, int[] erasedIndexes,
+ ECChunk[] outputs) {
+ if (inputs[0].getBuffer().hasArray()) {
+ byte[][] inputBytesArr = ECChunk.toArray(inputs);
+ byte[][] outputBytesArr = ECChunk.toArray(outputs);
+ doDecode(inputBytesArr, erasedIndexes, outputBytesArr);
+ } else {
+ ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
+ ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
+ doDecode(inputBuffers, erasedIndexes, outputBuffers);
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
new file mode 100644
index 0000000000000..4feaf39f28302
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
@@ -0,0 +1,93 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+
+import java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure encoder that's to be inherited by new encoders.
+ *
+ * It implements the {@link RawErasureEncoder} interface.
+ */
+public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
+ implements RawErasureEncoder {
+
+ @Override
+ public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+ assert (inputs.length == getNumDataUnits());
+ assert (outputs.length == getNumParityUnits());
+
+ doEncode(inputs, outputs);
+ }
+
+ /**
+ * Perform the real encoding work using ByteBuffer
+ * @param inputs
+ * @param outputs
+ */
+ protected abstract void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs);
+
+ @Override
+ public void encode(byte[][] inputs, byte[][] outputs) {
+ assert (inputs.length == getNumDataUnits());
+ assert (outputs.length == getNumParityUnits());
+
+ doEncode(inputs, outputs);
+ }
+
+ /**
+ * Perform the real encoding work using bytes array
+ * @param inputs
+ * @param outputs
+ */
+ protected abstract void doEncode(byte[][] inputs, byte[][] outputs);
+
+ @Override
+ public void encode(ECChunk[] inputs, ECChunk[] outputs) {
+ assert (inputs.length == getNumDataUnits());
+ assert (outputs.length == getNumParityUnits());
+
+ doEncode(inputs, outputs);
+ }
+
+ /**
+ * Perform the real encoding work using chunks.
+ * @param inputs
+ * @param outputs
+ */
+ protected void doEncode(ECChunk[] inputs, ECChunk[] outputs) {
+ /**
+ * Note callers may pass byte array, or ByteBuffer via ECChunk according
+ * to how ECChunk is created. Some implementations of coder use byte array
+ * (ex: pure Java), some use native ByteBuffer (ex: ISA-L), all for the
+ * better performance.
+ */
+ if (inputs[0].getBuffer().hasArray()) {
+ byte[][] inputBytesArr = ECChunk.toArray(inputs);
+ byte[][] outputBytesArr = ECChunk.toArray(outputs);
+ doEncode(inputBytesArr, outputBytesArr);
+ } else {
+ ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
+ ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
+ doEncode(inputBuffers, outputBuffers);
+ }
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
new file mode 100644
index 0000000000000..91a9abfe2aad4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -0,0 +1,78 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+/**
+ * RawErasureCoder is a common interface for {@link RawErasureEncoder} and
+ * {@link RawErasureDecoder} as both encoder and decoder share some properties.
+ *
+ * RawErasureCoder is part of ErasureCodec framework, where ErasureCoder is
+ * used to encode/decode a group of blocks (BlockGroup) according to the codec
+ * specific BlockGroup layout and logic. An ErasureCoder extracts chunks of
+ * data from the blocks and can employ various low level RawErasureCoders to
+ * perform encoding/decoding against the chunks.
+ *
+ * To distinguish from ErasureCoder, here RawErasureCoder is used to mean the
+ * low level constructs, since it only takes care of the math calculation with
+ * a group of byte buffers.
+ */
+public interface RawErasureCoder {
+
+ /**
+ * Initialize with the important parameters for the code.
+ * @param numDataUnits how many data inputs for the coding
+ * @param numParityUnits how many parity outputs the coding generates
+ * @param chunkSize the size of the input/output buffer
+ */
+ public void initialize(int numDataUnits, int numParityUnits, int chunkSize);
+
+ /**
+ * The number of data input units for the coding. A unit can be a byte,
+ * chunk or buffer or even a block.
+ * @return count of data input units
+ */
+ public int getNumDataUnits();
+
+ /**
+ * The number of parity output units for the coding. A unit can be a byte,
+ * chunk, buffer or even a block.
+ * @return count of parity output units
+ */
+ public int getNumParityUnits();
+
+ /**
+ * Chunk buffer size for the input/output
+ * @return chunk buffer size
+ */
+ public int getChunkSize();
+
+ /**
+ * Tell if native or off-heap buffer is preferred or not. It's for callers to
+ * decide how to allocate coding chunk buffers, either on heap or off heap.
+ * It will return false by default.
+ * @return true if native buffer is preferred for performance consideration,
+ * otherwise false.
+ */
+ public boolean preferNativeBuffer();
+
+ /**
+ * Should be called when release this coder. Good chance to release encoding
+ * or decoding buffers
+ */
+ public void release();
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
new file mode 100644
index 0000000000000..1358b7d0bc534
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+
+import java.nio.ByteBuffer;
+
+/**
+ * RawErasureDecoder performs decoding given chunks of input data and generates
+ * missing data that corresponds to an erasure code scheme, like XOR and
+ * Reed-Solomon.
+ *
+ * It extends the {@link RawErasureCoder} interface.
+ */
+public interface RawErasureDecoder extends RawErasureCoder {
+
+ /**
+ * Decode with inputs and erasedIndexes, generates outputs
+ * @param inputs
+ * @param outputs
+ */
+ public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+ ByteBuffer[] outputs);
+
+ /**
+ * Decode with inputs and erasedIndexes, generates outputs
+ * @param inputs
+ * @param outputs
+ */
+ public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs);
+
+ /**
+ * Decode with inputs and erasedIndexes, generates outputs
+ * @param inputs
+ * @param outputs
+ */
+ public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs);
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java
new file mode 100644
index 0000000000000..974f86ca6ca62
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java
@@ -0,0 +1,54 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+
+import java.nio.ByteBuffer;
+
+/**
+ * RawErasureEncoder performs encoding given chunks of input data and generates
+ * parity outputs that corresponds to an erasure code scheme, like XOR and
+ * Reed-Solomon.
+ *
+ * It extends the {@link RawErasureCoder} interface.
+ */
+public interface RawErasureEncoder extends RawErasureCoder {
+
+ /**
+ * Encode with inputs and generates outputs
+ * @param inputs
+ * @param outputs
+ */
+ public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs);
+
+ /**
+ * Encode with inputs and generates outputs
+ * @param inputs
+ * @param outputs
+ */
+ public void encode(byte[][] inputs, byte[][] outputs);
+
+ /**
+ * Encode with inputs and generates outputs
+ * @param inputs
+ * @param outputs
+ */
+ public void encode(ECChunk[] inputs, ECChunk[] outputs);
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java
new file mode 100644
index 0000000000000..98307a7b3c2f6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java
@@ -0,0 +1,81 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ */
+public class XorRawDecoder extends AbstractRawErasureDecoder {
+
+ @Override
+ protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+ ByteBuffer[] outputs) {
+ assert(erasedIndexes.length == outputs.length);
+ assert(erasedIndexes.length <= 1);
+
+ int bufSize = inputs[0].remaining();
+ int erasedIdx = erasedIndexes[0];
+
+ // Set the output to zeros.
+ for (int j = 0; j < bufSize; j++) {
+ outputs[0].put(j, (byte) 0);
+ }
+
+ // Process the inputs.
+ for (int i = 0; i < inputs.length; i++) {
+ // Skip the erased location.
+ if (i == erasedIdx) {
+ continue;
+ }
+
+ for (int j = 0; j < bufSize; j++) {
+ outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
+ }
+ }
+ }
+
+ @Override
+ protected void doDecode(byte[][] inputs, int[] erasedIndexes,
+ byte[][] outputs) {
+ assert(erasedIndexes.length == outputs.length);
+ assert(erasedIndexes.length <= 1);
+
+ int bufSize = inputs[0].length;
+ int erasedIdx = erasedIndexes[0];
+
+ // Set the output to zeros.
+ for (int j = 0; j < bufSize; j++) {
+ outputs[0][j] = 0;
+ }
+
+ // Process the inputs.
+ for (int i = 0; i < inputs.length; i++) {
+ // Skip the erased location.
+ if (i == erasedIdx) {
+ continue;
+ }
+
+ for (int j = 0; j < bufSize; j++) {
+ outputs[0][j] ^= inputs[i][j];
+ }
+ }
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java
new file mode 100644
index 0000000000000..99b20b92e7ad4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java
@@ -0,0 +1,61 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ */
+public class XorRawEncoder extends AbstractRawErasureEncoder {
+
+ @Override
+ protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+ int bufSize = inputs[0].remaining();
+
+ // Get the first buffer's data.
+ for (int j = 0; j < bufSize; j++) {
+ outputs[0].put(j, inputs[0].get(j));
+ }
+
+ // XOR with everything else.
+ for (int i = 1; i < inputs.length; i++) {
+ for (int j = 0; j < bufSize; j++) {
+ outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
+ }
+ }
+ }
+
+ @Override
+ protected void doEncode(byte[][] inputs, byte[][] outputs) {
+ int bufSize = inputs[0].length;
+
+ // Get the first buffer's data.
+ for (int j = 0; j < bufSize; j++) {
+ outputs[0][j] = inputs[0][j];
+ }
+
+ // XOR with everything else.
+ for (int i = 1; i < inputs.length; i++) {
+ for (int j = 0; j < bufSize; j++) {
+ outputs[0][j] ^= inputs[i][j];
+ }
+ }
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java
index c11678dd5d0dc..f17be1ab90d3c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java
@@ -2341,7 +2341,7 @@ public void write(DataOutput out) throws IOException {
* A list of TFile paths.
*/
public static void main(String[] args) {
- System.out.printf("TFile Dumper (TFile %s, BCFile %s)\n", TFile.API_VERSION
+ System.out.printf("TFile Dumper (TFile %s, BCFile %s)%n", TFile.API_VERSION
.toString(), BCFile.API_VERSION.toString());
if (args.length == 0) {
System.out
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFileDumper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFileDumper.java
index 829a1c68c45e9..aabdf57a26661 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFileDumper.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFileDumper.java
@@ -24,6 +24,7 @@
import java.util.Map;
import java.util.Set;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -176,7 +177,7 @@ static public void dumpInfo(String file, PrintStream out, Configuration conf)
for (Iterator> it = entrySet.iterator(); it
.hasNext();) {
Map.Entry e = it.next();
- out.printf("%s : %s\n", Align.format(e.getKey(), maxKeyLength,
+ out.printf("%s : %s%n", Align.format(e.getKey(), maxKeyLength,
Align.LEFT), e.getValue());
}
out.println();
@@ -200,7 +201,7 @@ static public void dumpInfo(String file, PrintStream out, Configuration conf)
String endKey = "End-Key";
int endKeyWidth = Math.max(endKey.length(), maxKeySampleLen * 2 + 5);
- out.printf("%s %s %s %s %s %s\n", Align.format(blkID, blkIDWidth,
+ out.printf("%s %s %s %s %s %s%n", Align.format(blkID, blkIDWidth,
Align.CENTER), Align.format(offset, offsetWidth, Align.CENTER),
Align.format(blkLen, blkLenWidth, Align.CENTER), Align.format(
rawSize, rawSizeWidth, Align.CENTER), Align.format(records,
@@ -233,7 +234,7 @@ static public void dumpInfo(String file, PrintStream out, Configuration conf)
out.printf("%X", b);
}
} else {
- out.print(new String(key, 0, sampleLen));
+ out.print(new String(key, 0, sampleLen, Charsets.UTF_8));
}
if (sampleLen < key.length) {
out.print("...");
@@ -267,7 +268,7 @@ static public void dumpInfo(String file, PrintStream out, Configuration conf)
* 10);
String compression = "Compression";
int compressionWidth = compression.length();
- out.printf("%s %s %s %s %s\n", Align.format(name, nameWidth,
+ out.printf("%s %s %s %s %s%n", Align.format(name, nameWidth,
Align.CENTER), Align.format(offset, offsetWidth, Align.CENTER),
Align.format(blkLen, blkLenWidth, Align.CENTER), Align.format(
rawSize, rawSizeWidth, Align.CENTER), Align.format(compression,
@@ -280,7 +281,7 @@ static public void dumpInfo(String file, PrintStream out, Configuration conf)
BlockRegion region = e.getValue().getRegion();
String blkCompression =
e.getValue().getCompressionAlgorithm().getName();
- out.printf("%s %s %s %s %s\n", Align.format(blkName, nameWidth,
+ out.printf("%s %s %s %s %s%n", Align.format(blkName, nameWidth,
Align.LEFT), Align.format(region.getOffset(), offsetWidth,
Align.LEFT), Align.format(region.getCompressedSize(),
blkLenWidth, Align.LEFT), Align.format(region.getRawSize(),
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
index 4a1ae7a6b7078..bc6e62ae40275 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
@@ -508,11 +508,63 @@ public static class Windows {
public static final long FILE_ATTRIBUTE_NORMAL = 0x00000080L;
+ /**
+ * Create a directory with permissions set to the specified mode. By setting
+ * permissions at creation time, we avoid issues related to the user lacking
+ * WRITE_DAC rights on subsequent chmod calls. One example where this can
+ * occur is writing to an SMB share where the user does not have Full Control
+ * rights, and therefore WRITE_DAC is denied.
+ *
+ * @param path directory to create
+ * @param mode permissions of new directory
+ * @throws IOException if there is an I/O error
+ */
+ public static void createDirectoryWithMode(File path, int mode)
+ throws IOException {
+ createDirectoryWithMode0(path.getAbsolutePath(), mode);
+ }
+
+ /** Wrapper around CreateDirectory() on Windows */
+ private static native void createDirectoryWithMode0(String path, int mode)
+ throws NativeIOException;
+
/** Wrapper around CreateFile() on Windows */
public static native FileDescriptor createFile(String path,
long desiredAccess, long shareMode, long creationDisposition)
throws IOException;
+ /**
+ * Create a file for write with permissions set to the specified mode. By
+ * setting permissions at creation time, we avoid issues related to the user
+ * lacking WRITE_DAC rights on subsequent chmod calls. One example where
+ * this can occur is writing to an SMB share where the user does not have
+ * Full Control rights, and therefore WRITE_DAC is denied.
+ *
+ * This method mimics the semantics implemented by the JDK in
+ * {@link java.io.FileOutputStream}. The file is opened for truncate or
+ * append, the sharing mode allows other readers and writers, and paths
+ * longer than MAX_PATH are supported. (See io_util_md.c in the JDK.)
+ *
+ * @param path file to create
+ * @param append if true, then open file for append
+ * @param mode permissions of new directory
+ * @return FileOutputStream of opened file
+ * @throws IOException if there is an I/O error
+ */
+ public static FileOutputStream createFileOutputStreamWithMode(File path,
+ boolean append, int mode) throws IOException {
+ long desiredAccess = GENERIC_WRITE;
+ long shareMode = FILE_SHARE_READ | FILE_SHARE_WRITE;
+ long creationDisposition = append ? OPEN_ALWAYS : CREATE_ALWAYS;
+ return new FileOutputStream(createFileWithMode0(path.getAbsolutePath(),
+ desiredAccess, shareMode, creationDisposition, mode));
+ }
+
+ /** Wrapper around CreateFile() with security descriptor on Windows */
+ private static native FileDescriptor createFileWithMode0(String path,
+ long desiredAccess, long shareMode, long creationDisposition, int mode)
+ throws NativeIOException;
+
/** Wrapper around SetFilePointer() on Windows */
public static native long setFilePointer(FileDescriptor fd,
long distanceToMove, long moveMethod) throws IOException;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 45a46603fac06..bdcb96c74a67a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -88,7 +88,7 @@
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
-import org.htrace.Trace;
+import org.apache.htrace.Trace;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@@ -849,6 +849,12 @@ private void handleConnectionFailure(int curRetries, IOException ioe
throw ioe;
}
+ // Throw the exception if the thread is interrupted
+ if (Thread.currentThread().isInterrupted()) {
+ LOG.warn("Interrupted while trying for connection");
+ throw ioe;
+ }
+
try {
Thread.sleep(action.delayMillis);
} catch (InterruptedException e) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index ddda55cb404cc..e75de1581fe7c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -49,9 +49,8 @@
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.ProtoUtil;
import org.apache.hadoop.util.Time;
-import org.htrace.Sampler;
-import org.htrace.Trace;
-import org.htrace.TraceScope;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingService;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
index 40f6515e4a04d..8ada0fff9801f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
@@ -412,11 +412,18 @@ public static ProtocolProxy waitForProtocolProxy(Class protocol,
throw ioe;
}
+ if (Thread.currentThread().isInterrupted()) {
+ // interrupted during some IO; this may not have been caught
+ throw new InterruptedIOException("Interrupted waiting for the proxy");
+ }
+
// wait for retry
try {
Thread.sleep(1000);
} catch (InterruptedException ie) {
- // IGNORE
+ Thread.currentThread().interrupt();
+ throw (IOException) new InterruptedIOException(
+ "Interrupted waiting for the proxy").initCause(ioe);
}
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcConstants.java
index c457500e902b5..d5e795b92f17c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcConstants.java
@@ -19,6 +19,7 @@
import java.nio.ByteBuffer;
+import org.apache.commons.io.Charsets;
import org.apache.hadoop.classification.InterfaceAudience;
@InterfaceAudience.Private
@@ -53,7 +54,8 @@ private RpcConstants() {
/**
* The first four bytes of Hadoop RPC connections
*/
- public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
+ public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes
+ (Charsets.UTF_8));
public static final int HEADER_LEN_AFTER_HRPC_PART = 3; // 3 bytes that follow
// 1 : Introduce ping and server does not throw away RPCs
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index 021e03537b4e5..475fb1101dc88 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -69,6 +69,7 @@
import javax.security.sasl.SaslException;
import javax.security.sasl.SaslServer;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -116,10 +117,10 @@
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
-import org.htrace.Span;
-import org.htrace.Trace;
-import org.htrace.TraceInfo;
-import org.htrace.TraceScope;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceInfo;
+import org.apache.htrace.TraceScope;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ByteString;
@@ -182,7 +183,7 @@ boolean isTerse(Class> t) {
* and send back a nicer response.
*/
private static final ByteBuffer HTTP_GET_BYTES = ByteBuffer.wrap(
- "GET ".getBytes());
+ "GET ".getBytes(Charsets.UTF_8));
/**
* An HTTP response to send back if we detect an HTTP request to our IPC
@@ -665,7 +666,7 @@ void shutdown() {
assert !running;
readSelector.wakeup();
try {
- join();
+ super.join();
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
}
@@ -748,6 +749,13 @@ void doAccept(SelectionKey key) throws InterruptedException, IOException, OutOf
Reader reader = getReader();
Connection c = connectionManager.register(channel);
+ // If the connectionManager can't take it, close the connection.
+ if (c == null) {
+ if (channel.isOpen()) {
+ IOUtils.cleanup(null, channel);
+ }
+ continue;
+ }
key.attach(c); // so closeCurrentConnection can get the object
reader.addConnection(c);
}
@@ -1119,7 +1127,8 @@ public class Connection {
private ByteBuffer data;
private ByteBuffer dataLengthBuffer;
private LinkedList responseQueue;
- private volatile int rpcCount = 0; // number of outstanding rpcs
+ // number of outstanding rpcs
+ private AtomicInteger rpcCount = new AtomicInteger();
private long lastContact;
private int dataLength;
private Socket socket;
@@ -1207,17 +1216,17 @@ public long getLastContact() {
/* Return true if the connection has no outstanding rpc */
private boolean isIdle() {
- return rpcCount == 0;
+ return rpcCount.get() == 0;
}
/* Decrement the outstanding RPC count */
private void decRpcCount() {
- rpcCount--;
+ rpcCount.decrementAndGet();
}
/* Increment the outstanding RPC count */
private void incRpcCount() {
- rpcCount++;
+ rpcCount.incrementAndGet();
}
private UserGroupInformation getAuthorizedUgi(String authorizedId)
@@ -1708,7 +1717,7 @@ private void setupBadVersionResponse(int clientVersion) throws IOException {
private void setupHttpRequestOnIpcPortResponse() throws IOException {
Call fakeCall = new Call(0, RpcConstants.INVALID_RETRY_COUNT, null, this);
fakeCall.setResponse(ByteBuffer.wrap(
- RECEIVED_HTTP_REQ_RESPONSE.getBytes()));
+ RECEIVED_HTTP_REQ_RESPONSE.getBytes(Charsets.UTF_8)));
responder.doRespond(fakeCall);
}
@@ -2068,9 +2077,9 @@ private synchronized void close() {
LOG.debug("Ignoring socket shutdown exception", e);
}
if (channel.isOpen()) {
- try {channel.close();} catch(Exception e) {}
+ IOUtils.cleanup(null, channel);
}
- try {socket.close();} catch(Exception e) {}
+ IOUtils.cleanup(null, socket);
}
}
@@ -2729,6 +2738,7 @@ private class ConnectionManager {
final private int idleScanInterval;
final private int maxIdleTime;
final private int maxIdleToClose;
+ final private int maxConnections;
ConnectionManager() {
this.idleScanTimer = new Timer(
@@ -2745,6 +2755,9 @@ private class ConnectionManager {
this.maxIdleToClose = conf.getInt(
CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_KEY,
CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_DEFAULT);
+ this.maxConnections = conf.getInt(
+ CommonConfigurationKeysPublic.IPC_SERVER_MAX_CONNECTIONS_KEY,
+ CommonConfigurationKeysPublic.IPC_SERVER_MAX_CONNECTIONS_DEFAULT);
// create a set with concurrency -and- a thread-safe iterator, add 2
// for listener and idle closer threads
this.connections = Collections.newSetFromMap(
@@ -2772,11 +2785,19 @@ int size() {
return count.get();
}
+ boolean isFull() {
+ // The check is disabled when maxConnections <= 0.
+ return ((maxConnections > 0) && (size() >= maxConnections));
+ }
+
Connection[] toArray() {
return connections.toArray(new Connection[0]);
}
Connection register(SocketChannel channel) {
+ if (isFull()) {
+ return null;
+ }
Connection connection = new Connection(channel, Time.now());
add(connection);
if (LOG.isDebugEnabled()) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
index 94a6b1d593bf4..cea881c25da43 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
@@ -42,8 +42,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.*;
-import org.htrace.Trace;
-import org.htrace.TraceScope;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
/** An RpcEngine implementation for Writable data. */
@InterfaceStability.Evolving
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java
index f775dd71beb32..9ade62f27ab9c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java
@@ -17,12 +17,11 @@
package org.apache.hadoop.jmx;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.lang.management.ManagementFactory;
-import java.lang.reflect.Array;
-import java.util.Iterator;
-import java.util.Set;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.http.HttpServer2;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonGenerator;
import javax.management.AttributeNotFoundException;
import javax.management.InstanceNotFoundException;
@@ -43,12 +42,12 @@
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.http.HttpServer2;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.JsonGenerator;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Array;
+import java.util.Iterator;
+import java.util.Set;
/*
* This servlet is based off of the JMXProxyServlet from Tomcat 7.0.14. It has
@@ -114,16 +113,16 @@
*
* The bean's name and modelerType will be returned for all beans.
*
- * Optional paramater "callback" should be used to deliver JSONP response.
- *
*/
public class JMXJsonServlet extends HttpServlet {
private static final Log LOG = LogFactory.getLog(JMXJsonServlet.class);
+ static final String ACCESS_CONTROL_ALLOW_METHODS =
+ "Access-Control-Allow-Methods";
+ static final String ACCESS_CONTROL_ALLOW_ORIGIN =
+ "Access-Control-Allow-Origin";
private static final long serialVersionUID = 1L;
- private static final String CALLBACK_PARAM = "callback";
-
/**
* MBean server.
*/
@@ -164,19 +163,13 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) {
return;
}
JsonGenerator jg = null;
- String jsonpcb = null;
PrintWriter writer = null;
try {
writer = response.getWriter();
- // "callback" parameter implies JSONP outpout
- jsonpcb = request.getParameter(CALLBACK_PARAM);
- if (jsonpcb != null) {
- response.setContentType("application/javascript; charset=utf8");
- writer.write(jsonpcb + "(");
- } else {
- response.setContentType("application/json; charset=utf8");
- }
+ response.setContentType("application/json; charset=utf8");
+ response.setHeader(ACCESS_CONTROL_ALLOW_METHODS, "GET");
+ response.setHeader(ACCESS_CONTROL_ALLOW_ORIGIN, "*");
jg = jsonFactory.createJsonGenerator(writer);
jg.disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET);
@@ -209,9 +202,6 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) {
if (jg != null) {
jg.close();
}
- if (jsonpcb != null) {
- writer.write(");");
- }
if (writer != null) {
writer.close();
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java
index 77f74cc404911..4749ce19a65f8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java
@@ -24,6 +24,7 @@
import javax.servlet.*;
import javax.servlet.http.*;
+import com.google.common.base.Charsets;
import org.apache.commons.logging.*;
import org.apache.commons.logging.impl.*;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -66,7 +67,7 @@ private static void process(String urlstring) {
connection.connect();
BufferedReader in = new BufferedReader(new InputStreamReader(
- connection.getInputStream()));
+ connection.getInputStream(), Charsets.UTF_8));
for(String line; (line = in.readLine()) != null; )
if (line.startsWith(MARKER)) {
System.out.println(TAG.matcher(line).replaceAll(""));
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/file/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/file/FileContext.java
deleted file mode 100644
index fcbe7c4828750..0000000000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/file/FileContext.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * FileContext.java
- *
- * 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.metrics.file;
-
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext;
-import org.apache.hadoop.metrics.spi.OutputRecord;
-
-/**
- * Metrics context for writing metrics to a file.
- *
- * This class is configured by setting ContextFactory attributes which in turn
- * are usually configured through a properties file. All the attributes are
- * prefixed by the contextName. For example, the properties file might contain:
- *
- * myContextName.fileName=/tmp/metrics.log
- * myContextName.period=5
- *
- * @see org.apache.hadoop.metrics2.sink.FileSink for metrics 2.0.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-@Deprecated
-public class FileContext extends AbstractMetricsContext {
-
- /* Configuration attribute names */
- @InterfaceAudience.Private
- protected static final String FILE_NAME_PROPERTY = "fileName";
- @InterfaceAudience.Private
- protected static final String PERIOD_PROPERTY = "period";
-
- private File file = null; // file for metrics to be written to
- private PrintWriter writer = null;
-
- /** Creates a new instance of FileContext */
- @InterfaceAudience.Private
- public FileContext() {}
-
- @Override
- @InterfaceAudience.Private
- public void init(String contextName, ContextFactory factory) {
- super.init(contextName, factory);
-
- String fileName = getAttribute(FILE_NAME_PROPERTY);
- if (fileName != null) {
- file = new File(fileName);
- }
-
- parseAndSetPeriod(PERIOD_PROPERTY);
- }
-
- /**
- * Returns the configured file name, or null.
- */
- @InterfaceAudience.Private
- public String getFileName() {
- if (file == null) {
- return null;
- } else {
- return file.getName();
- }
- }
-
- /**
- * Starts or restarts monitoring, by opening in append-mode, the
- * file specified by the fileName attribute,
- * if specified. Otherwise the data will be written to standard
- * output.
- */
- @Override
- @InterfaceAudience.Private
- public void startMonitoring()
- throws IOException
- {
- if (file == null) {
- writer = new PrintWriter(new BufferedOutputStream(System.out));
- } else {
- writer = new PrintWriter(new FileWriter(file, true));
- }
- super.startMonitoring();
- }
-
- /**
- * Stops monitoring, closing the file.
- * @see #close()
- */
- @Override
- @InterfaceAudience.Private
- public void stopMonitoring() {
- super.stopMonitoring();
-
- if (writer != null) {
- writer.close();
- writer = null;
- }
- }
-
- /**
- * Emits a metrics record to a file.
- */
- @Override
- @InterfaceAudience.Private
- public void emitRecord(String contextName, String recordName, OutputRecord outRec) {
- writer.print(contextName);
- writer.print(".");
- writer.print(recordName);
- String separator = ": ";
- for (String tagName : outRec.getTagNames()) {
- writer.print(separator);
- separator = ", ";
- writer.print(tagName);
- writer.print("=");
- writer.print(outRec.getTag(tagName));
- }
- for (String metricName : outRec.getMetricNames()) {
- writer.print(separator);
- separator = ", ";
- writer.print(metricName);
- writer.print("=");
- writer.print(outRec.getMetric(metricName));
- }
- writer.println();
- }
-
- /**
- * Flushes the output writer, forcing updates to disk.
- */
- @Override
- @InterfaceAudience.Private
- public void flush() {
- writer.flush();
- }
-}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java
index 841874fc08e34..5ed2652b11c0f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java
@@ -21,14 +21,12 @@
package org.apache.hadoop.metrics.ganglia;
import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.DatagramSocket;
-import java.net.SocketAddress;
-import java.net.SocketException;
+import java.net.*;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -53,13 +51,16 @@ public class GangliaContext extends AbstractMetricsContext {
private static final String SLOPE_PROPERTY = "slope";
private static final String TMAX_PROPERTY = "tmax";
private static final String DMAX_PROPERTY = "dmax";
-
+ private static final String MULTICAST_PROPERTY = "multicast";
+ private static final String MULTICAST_TTL_PROPERTY = "multicast.ttl";
+
private static final String DEFAULT_UNITS = "";
private static final String DEFAULT_SLOPE = "both";
private static final int DEFAULT_TMAX = 60;
private static final int DEFAULT_DMAX = 0;
private static final int DEFAULT_PORT = 8649;
private static final int BUFFER_SIZE = 1500; // as per libgmond.c
+ private static final int DEFAULT_MULTICAST_TTL = 1;
private final Log LOG = LogFactory.getLog(this.getClass());
@@ -82,6 +83,8 @@ public class GangliaContext extends AbstractMetricsContext {
private Map slopeTable;
private Map tmaxTable;
private Map dmaxTable;
+ private boolean multicastEnabled;
+ private int multicastTtl;
protected DatagramSocket datagramSocket;
@@ -103,11 +106,26 @@ public void init(String contextName, ContextFactory factory) {
slopeTable = getAttributeTable(SLOPE_PROPERTY);
tmaxTable = getAttributeTable(TMAX_PROPERTY);
dmaxTable = getAttributeTable(DMAX_PROPERTY);
+ multicastEnabled = Boolean.parseBoolean(getAttribute(MULTICAST_PROPERTY));
+ String multicastTtlValue = getAttribute(MULTICAST_TTL_PROPERTY);
+ if (multicastEnabled) {
+ if (multicastTtlValue == null) {
+ multicastTtl = DEFAULT_MULTICAST_TTL;
+ } else {
+ multicastTtl = Integer.parseInt(multicastTtlValue);
+ }
+ }
try {
- datagramSocket = new DatagramSocket();
- } catch (SocketException se) {
- se.printStackTrace();
+ if (multicastEnabled) {
+ LOG.info("Enabling multicast for Ganglia with TTL " + multicastTtl);
+ datagramSocket = new MulticastSocket();
+ ((MulticastSocket) datagramSocket).setTimeToLive(multicastTtl);
+ } else {
+ datagramSocket = new DatagramSocket();
+ }
+ } catch (IOException e) {
+ LOG.error(e);
}
}
@@ -225,7 +243,7 @@ protected int getDmax(String metricName) {
* a multiple of 4.
*/
protected void xdr_string(String s) {
- byte[] bytes = s.getBytes();
+ byte[] bytes = s.getBytes(Charsets.UTF_8);
int len = bytes.length;
xdr_int(len);
System.arraycopy(bytes, 0, buffer, offset, len);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java
index 39509f0c89a81..f35ad1816f2b5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java
@@ -86,11 +86,6 @@ protected void emitMetric(String name, String type, String value)
value + " from hostname" + hostName);
String units = getUnits(name);
- if (units == null) {
- LOG.warn("Metric name " + name + ", value " + value
- + " had 'null' units");
- units = "";
- }
int slope = getSlope(name);
int tmax = getTmax(name);
int dmax = getDmax(name);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/package.html b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/package.html
index 87598e503325c..b9acfaea69976 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/package.html
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/package.html
@@ -54,6 +54,12 @@
- contextName.period
- The period in seconds on which the metric data is sent to the
server(s).
+
+ - contextName.multicast
+ - Enable multicast for Ganglia
+
+ - contextName.multicast.ttl
+ - TTL for multicast packets
- contextName.units.recordName.metricName
- The units for the specified metric in the specified record.
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/CompositeContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/CompositeContext.java
index 60f5fec44aa65..ad494040de7e6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/CompositeContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/CompositeContext.java
@@ -55,7 +55,7 @@ public void init(String contextName, ContextFactory factory) {
int nKids;
try {
String sKids = getAttribute(ARITY_LABEL);
- nKids = Integer.valueOf(sKids);
+ nKids = Integer.parseInt(sKids);
} catch (Exception e) {
LOG.error("Unable to initialize composite metric " + contextName +
": could not init arity", e);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsCollectorImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsCollectorImpl.java
index be442edb1b85b..5345c1baf88fd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsCollectorImpl.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsCollectorImpl.java
@@ -69,7 +69,8 @@ public Iterator iterator() {
return rbs.iterator();
}
- void clear() { rbs.clear(); }
+ @InterfaceAudience.Private
+ public void clear() { rbs.clear(); }
MetricsCollectorImpl setRecordFilter(MetricsFilter rf) {
recordFilter = rf;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java
index e4b5580536bb5..167205e93e3c2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java
@@ -269,14 +269,14 @@ public String toString() {
static String toString(Configuration c) {
ByteArrayOutputStream buffer = new ByteArrayOutputStream();
- PrintStream ps = new PrintStream(buffer);
- PropertiesConfiguration tmp = new PropertiesConfiguration();
- tmp.copy(c);
try {
+ PrintStream ps = new PrintStream(buffer, false, "UTF-8");
+ PropertiesConfiguration tmp = new PropertiesConfiguration();
+ tmp.copy(c);
tmp.save(ps);
+ return buffer.toString("UTF-8");
} catch (Exception e) {
throw new MetricsConfigException(e);
}
- return buffer.toString();
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
index baa75ab8cfdab..32b00f3fdfa37 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
@@ -384,7 +384,7 @@ synchronized void onTimerEvent() {
* Requests an immediate publish of all metrics from sources to sinks.
*/
@Override
- public void publishMetricsNow() {
+ public synchronized void publishMetricsNow() {
if (sinks.size() > 0) {
publishMetrics(sampleMetrics(), true);
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MetricsRegistry.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MetricsRegistry.java
index 1c0d30e234f84..4b561f2fb5649 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MetricsRegistry.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MetricsRegistry.java
@@ -363,6 +363,20 @@ Collection metrics() {
}
private void checkMetricName(String name) {
+ // Check for invalid characters in metric name
+ boolean foundWhitespace = false;
+ for (int i = 0; i < name.length(); i++) {
+ char c = name.charAt(i);
+ if (Character.isWhitespace(c)) {
+ foundWhitespace = true;
+ break;
+ }
+ }
+ if (foundWhitespace) {
+ throw new MetricsException("Metric name '"+ name +
+ "' contains illegal whitespace character");
+ }
+ // Check if name has already been registered
if (metricsMap.containsKey(name)) {
throw new MetricsException("Metric name "+ name +" already exists!");
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableCounterInt.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableCounterInt.java
index b0fb0d4709f0f..77139e2e96fda 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableCounterInt.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableCounterInt.java
@@ -23,23 +23,24 @@
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import java.util.concurrent.atomic.AtomicInteger;
+
/**
* A mutable int counter for implementing metrics sources
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class MutableCounterInt extends MutableCounter {
- private volatile int value;
+ private AtomicInteger value = new AtomicInteger();
MutableCounterInt(MetricsInfo info, int initValue) {
super(info);
- this.value = initValue;
+ this.value.set(initValue);
}
@Override
- public synchronized void incr() {
- ++value;
- setChanged();
+ public void incr() {
+ incr(1);
}
/**
@@ -47,18 +48,18 @@ public synchronized void incr() {
* @param delta of the increment
*/
public synchronized void incr(int delta) {
- value += delta;
+ value.addAndGet(delta);
setChanged();
}
public int value() {
- return value;
+ return value.get();
}
@Override
public void snapshot(MetricsRecordBuilder builder, boolean all) {
if (all || changed()) {
- builder.addCounter(info(), value);
+ builder.addCounter(info(), value());
clearChanged();
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableCounterLong.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableCounterLong.java
index 43ea4906bbafc..03a6043375f1c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableCounterLong.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableCounterLong.java
@@ -23,6 +23,8 @@
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import java.util.concurrent.atomic.AtomicLong;
+
/**
* A mutable long counter
*/
@@ -30,36 +32,35 @@
@InterfaceStability.Evolving
public class MutableCounterLong extends MutableCounter {
- private volatile long value;
+ private AtomicLong value = new AtomicLong();
MutableCounterLong(MetricsInfo info, long initValue) {
super(info);
- this.value = initValue;
+ this.value.set(initValue);
}
@Override
- public synchronized void incr() {
- ++value;
- setChanged();
+ public void incr() {
+ incr(1);
}
/**
* Increment the value by a delta
* @param delta of the increment
*/
- public synchronized void incr(long delta) {
- value += delta;
+ public void incr(long delta) {
+ value.addAndGet(delta);
setChanged();
}
public long value() {
- return value;
+ return value.get();
}
@Override
public void snapshot(MetricsRecordBuilder builder, boolean all) {
if (all || changed()) {
- builder.addCounter(info(), value);
+ builder.addCounter(info(), value());
clearChanged();
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableGaugeInt.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableGaugeInt.java
index 1a3a15cf6865c..cce45286ce675 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableGaugeInt.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableGaugeInt.java
@@ -23,6 +23,8 @@
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import java.util.concurrent.atomic.AtomicInteger;
+
/**
* A mutable int gauge
*/
@@ -30,44 +32,42 @@
@InterfaceStability.Evolving
public class MutableGaugeInt extends MutableGauge {
- private volatile int value;
+ private AtomicInteger value = new AtomicInteger();
MutableGaugeInt(MetricsInfo info, int initValue) {
super(info);
- this.value = initValue;
+ this.value.set(initValue);
}
public int value() {
- return value;
+ return value.get();
}
@Override
- public synchronized void incr() {
- ++value;
- setChanged();
+ public void incr() {
+ incr(1);
}
/**
* Increment by delta
* @param delta of the increment
*/
- public synchronized void incr(int delta) {
- value += delta;
+ public void incr(int delta) {
+ value.addAndGet(delta);
setChanged();
}
@Override
- public synchronized void decr() {
- --value;
- setChanged();
+ public void decr() {
+ decr(1);
}
/**
* decrement by delta
* @param delta of the decrement
*/
- public synchronized void decr(int delta) {
- value -= delta;
+ public void decr(int delta) {
+ value.addAndGet(-delta);
setChanged();
}
@@ -76,14 +76,14 @@ public synchronized void decr(int delta) {
* @param value to set
*/
public void set(int value) {
- this.value = value;
+ this.value.set(value);
setChanged();
}
@Override
public void snapshot(MetricsRecordBuilder builder, boolean all) {
if (all || changed()) {
- builder.addGauge(info(), value);
+ builder.addGauge(info(), value());
clearChanged();
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableGaugeLong.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableGaugeLong.java
index 69df4304e9b7b..a2a8632ad3c38 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableGaugeLong.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableGaugeLong.java
@@ -23,6 +23,8 @@
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import java.util.concurrent.atomic.AtomicLong;
+
/**
* A mutable long gauge
*/
@@ -30,44 +32,42 @@
@InterfaceStability.Evolving
public class MutableGaugeLong extends MutableGauge {
- private volatile long value;
+ private AtomicLong value = new AtomicLong();
MutableGaugeLong(MetricsInfo info, long initValue) {
super(info);
- this.value = initValue;
+ this.value.set(initValue);
}
public long value() {
- return value;
+ return value.get();
}
@Override
- public synchronized void incr() {
- ++value;
- setChanged();
+ public void incr() {
+ incr(1);
}
/**
* Increment by delta
* @param delta of the increment
*/
- public synchronized void incr(long delta) {
- value += delta;
+ public void incr(long delta) {
+ value.addAndGet(delta);
setChanged();
}
@Override
- public synchronized void decr() {
- --value;
- setChanged();
+ public void decr() {
+ decr(1);
}
/**
* decrement by delta
* @param delta of the decrement
*/
- public synchronized void decr(long delta) {
- value -= delta;
+ public void decr(long delta) {
+ value.addAndGet(-delta);
setChanged();
}
@@ -76,14 +76,14 @@ public synchronized void decr(long delta) {
* @param value to set
*/
public void set(long value) {
- this.value = value;
+ this.value.set(value);
setChanged();
}
@Override
public void snapshot(MetricsRecordBuilder builder, boolean all) {
if (all || changed()) {
- builder.addGauge(info(), value);
+ builder.addGauge(info(), value());
clearChanged();
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/FileSink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/FileSink.java
index d1364160e2d67..ab121bcf67f6b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/FileSink.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/FileSink.java
@@ -20,9 +20,9 @@
import java.io.Closeable;
import java.io.File;
-import java.io.FileWriter;
+import java.io.FileOutputStream;
import java.io.IOException;
-import java.io.PrintWriter;
+import java.io.PrintStream;
import org.apache.commons.configuration.SubsetConfiguration;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -40,15 +40,15 @@
@InterfaceStability.Evolving
public class FileSink implements MetricsSink, Closeable {
private static final String FILENAME_KEY = "filename";
- private PrintWriter writer;
+ private PrintStream writer;
@Override
public void init(SubsetConfiguration conf) {
String filename = conf.getString(FILENAME_KEY);
try {
- writer = filename == null
- ? new PrintWriter(System.out)
- : new PrintWriter(new FileWriter(new File(filename), true));
+ writer = filename == null ? System.out
+ : new PrintStream(new FileOutputStream(new File(filename)),
+ true, "UTF-8");
} catch (Exception e) {
throw new MetricsException("Error creating "+ filename, e);
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/GraphiteSink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/GraphiteSink.java
index 9bc3f15d97e91..e46a654e823e8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/GraphiteSink.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/GraphiteSink.java
@@ -18,24 +18,24 @@
package org.apache.hadoop.metrics2.sink;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-import java.io.Closeable;
-import java.net.Socket;
-
import org.apache.commons.configuration.SubsetConfiguration;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.metrics2.AbstractMetric;
import org.apache.hadoop.metrics2.MetricsException;
import org.apache.hadoop.metrics2.MetricsRecord;
import org.apache.hadoop.metrics2.MetricsSink;
import org.apache.hadoop.metrics2.MetricsTag;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.net.Socket;
+
/**
* A metrics sink that writes to a Graphite server
*/
@@ -46,29 +46,22 @@ public class GraphiteSink implements MetricsSink, Closeable {
private static final String SERVER_HOST_KEY = "server_host";
private static final String SERVER_PORT_KEY = "server_port";
private static final String METRICS_PREFIX = "metrics_prefix";
- private Writer writer = null;
private String metricsPrefix = null;
- private Socket socket = null;
+ private Graphite graphite = null;
@Override
public void init(SubsetConfiguration conf) {
// Get Graphite host configurations.
- String serverHost = conf.getString(SERVER_HOST_KEY);
- Integer serverPort = Integer.parseInt(conf.getString(SERVER_PORT_KEY));
+ final String serverHost = conf.getString(SERVER_HOST_KEY);
+ final int serverPort = Integer.parseInt(conf.getString(SERVER_PORT_KEY));
// Get Graphite metrics graph prefix.
metricsPrefix = conf.getString(METRICS_PREFIX);
if (metricsPrefix == null)
metricsPrefix = "";
- try {
- // Open an connection to Graphite server.
- socket = new Socket(serverHost, serverPort);
- writer = new OutputStreamWriter(socket.getOutputStream());
- } catch (Exception e) {
- throw new MetricsException("Error creating connection, "
- + serverHost + ":" + serverPort, e);
- }
+ graphite = new Graphite(serverHost, serverPort);
+ graphite.connect();
}
@Override
@@ -102,39 +95,111 @@ public void putMetrics(MetricsRecord record) {
}
try {
- if(writer != null){
- writer.write(lines.toString());
- } else {
- throw new MetricsException("Writer in GraphiteSink is null!");
- }
+ graphite.write(lines.toString());
} catch (Exception e) {
- throw new MetricsException("Error sending metrics", e);
+ LOG.warn("Error sending metrics to Graphite", e);
+ try {
+ graphite.close();
+ } catch (Exception e1) {
+ throw new MetricsException("Error closing connection to Graphite", e1);
+ }
}
}
@Override
public void flush() {
+ try {
+ graphite.flush();
+ } catch (Exception e) {
+ LOG.warn("Error flushing metrics to Graphite", e);
try {
- writer.flush();
- } catch (Exception e) {
- throw new MetricsException("Error flushing metrics", e);
+ graphite.close();
+ } catch (Exception e1) {
+ throw new MetricsException("Error closing connection to Graphite", e1);
}
+ }
}
@Override
public void close() throws IOException {
- try {
- IOUtils.closeStream(writer);
- writer = null;
- LOG.info("writer in GraphiteSink is closed!");
- } catch (Throwable e){
- throw new MetricsException("Error closing writer", e);
- } finally {
- if (socket != null && !socket.isClosed()) {
- socket.close();
+ graphite.close();
+ }
+
+ public static class Graphite {
+ private final static int MAX_CONNECTION_FAILURES = 5;
+
+ private String serverHost;
+ private int serverPort;
+ private Writer writer = null;
+ private Socket socket = null;
+ private int connectionFailures = 0;
+
+ public Graphite(String serverHost, int serverPort) {
+ this.serverHost = serverHost;
+ this.serverPort = serverPort;
+ }
+
+ public void connect() {
+ if (isConnected()) {
+ throw new MetricsException("Already connected to Graphite");
+ }
+ if (tooManyConnectionFailures()) {
+ // return silently (there was ERROR in logs when we reached limit for the first time)
+ return;
+ }
+ try {
+ // Open a connection to Graphite server.
+ socket = new Socket(serverHost, serverPort);
+ writer = new OutputStreamWriter(socket.getOutputStream(), Charsets.UTF_8);
+ } catch (Exception e) {
+ connectionFailures++;
+ if (tooManyConnectionFailures()) {
+ // first time when connection limit reached, report to logs
+ LOG.error("Too many connection failures, would not try to connect again.");
+ }
+ throw new MetricsException("Error creating connection, "
+ + serverHost + ":" + serverPort, e);
+ }
+ }
+
+ public void write(String msg) throws IOException {
+ if (!isConnected()) {
+ connect();
+ }
+ if (isConnected()) {
+ writer.write(msg);
+ }
+ }
+
+ public void flush() throws IOException {
+ if (isConnected()) {
+ writer.flush();
+ }
+ }
+
+ public boolean isConnected() {
+ return socket != null && socket.isConnected() && !socket.isClosed();
+ }
+
+ public void close() throws IOException {
+ try {
+ if (writer != null) {
+ writer.close();
+ }
+ } catch (IOException ex) {
+ if (socket != null) {
+ socket.close();
+ }
+ } finally {
socket = null;
- LOG.info("socket in GraphiteSink is closed!");
+ writer = null;
}
}
+
+ private boolean tooManyConnectionFailures() {
+ return connectionFailures > MAX_CONNECTION_FAILURES;
+ }
+
}
+
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
index b3581f9e8d9bd..c9df0ffcc3be8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
@@ -19,16 +19,13 @@
package org.apache.hadoop.metrics2.sink.ganglia;
import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.DatagramSocket;
-import java.net.SocketAddress;
-import java.net.SocketException;
-import java.net.UnknownHostException;
+import java.net.*;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.commons.configuration.SubsetConfiguration;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.metrics2.MetricsSink;
@@ -62,7 +59,11 @@ public abstract class AbstractGangliaSink implements MetricsSink {
public static final int DEFAULT_DMAX = 0;
public static final GangliaSlope DEFAULT_SLOPE = GangliaSlope.both;
public static final int DEFAULT_PORT = 8649;
+ public static final boolean DEFAULT_MULTICAST_ENABLED = false;
+ public static final int DEFAULT_MULTICAST_TTL = 1;
public static final String SERVERS_PROPERTY = "servers";
+ public static final String MULTICAST_ENABLED_PROPERTY = "multicast";
+ public static final String MULTICAST_TTL_PROPERTY = "multicast.ttl";
public static final int BUFFER_SIZE = 1500; // as per libgmond.c
public static final String SUPPORT_SPARSE_METRICS_PROPERTY = "supportsparse";
public static final boolean SUPPORT_SPARSE_METRICS_DEFAULT = false;
@@ -71,6 +72,8 @@ public abstract class AbstractGangliaSink implements MetricsSink {
private String hostName = "UNKNOWN.example.com";
private DatagramSocket datagramSocket;
private List extends SocketAddress> metricsServers;
+ private boolean multicastEnabled;
+ private int multicastTtl;
private byte[] buffer = new byte[BUFFER_SIZE];
private int offset;
private boolean supportSparseMetrics = SUPPORT_SPARSE_METRICS_DEFAULT;
@@ -132,6 +135,9 @@ public void init(SubsetConfiguration conf) {
// load the gannglia servers from properties
metricsServers = Servers.parse(conf.getString(SERVERS_PROPERTY),
DEFAULT_PORT);
+ multicastEnabled = conf.getBoolean(MULTICAST_ENABLED_PROPERTY,
+ DEFAULT_MULTICAST_ENABLED);
+ multicastTtl = conf.getInt(MULTICAST_TTL_PROPERTY, DEFAULT_MULTICAST_TTL);
// extract the Ganglia conf per metrics
gangliaConfMap = new HashMap();
@@ -141,9 +147,15 @@ public void init(SubsetConfiguration conf) {
loadGangliaConf(GangliaConfType.slope);
try {
- datagramSocket = new DatagramSocket();
- } catch (SocketException se) {
- LOG.error(se);
+ if (multicastEnabled) {
+ LOG.info("Enabling multicast for Ganglia with TTL " + multicastTtl);
+ datagramSocket = new MulticastSocket();
+ ((MulticastSocket) datagramSocket).setTimeToLive(multicastTtl);
+ } else {
+ datagramSocket = new DatagramSocket();
+ }
+ } catch (IOException e) {
+ LOG.error(e);
}
// see if sparseMetrics is supported. Default is false
@@ -223,7 +235,7 @@ protected String getHostName() {
* @param s the string to be written to buffer at offset location
*/
protected void xdr_string(String s) {
- byte[] bytes = s.getBytes();
+ byte[] bytes = s.getBytes(Charsets.UTF_8);
int len = bytes.length;
xdr_int(len);
System.arraycopy(bytes, 0, buffer, offset, len);
@@ -256,6 +268,12 @@ protected void xdr_int(int i) {
protected void emitToGangliaHosts() throws IOException {
try {
for (SocketAddress socketAddress : metricsServers) {
+ if (socketAddress == null || !(socketAddress instanceof InetSocketAddress))
+ throw new IllegalArgumentException("Unsupported Address type");
+ InetSocketAddress inetAddress = (InetSocketAddress)socketAddress;
+ if(inetAddress.isUnresolved()) {
+ throw new UnknownHostException("Unresolved host: " + inetAddress);
+ }
DatagramPacket packet =
new DatagramPacket(buffer, offset, socketAddress);
datagramSocket.send(packet);
@@ -287,4 +305,12 @@ protected boolean isSupportSparseMetrics() {
void setDatagramSocket(DatagramSocket datagramSocket) {
this.datagramSocket = datagramSocket;
}
+
+ /**
+ * Used only by unit tests
+ * @return the datagramSocket for this sink
+ */
+ DatagramSocket getDatagramSocket() {
+ return datagramSocket;
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
index b535ddaee2f7a..ef1092bb2f757 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
@@ -287,8 +287,8 @@ private static String canonicalizeHost(String host) {
if (fqHost == null) {
try {
fqHost = SecurityUtil.getByName(host).getHostName();
- // slight race condition, but won't hurt
- canonicalizedHostCache.put(host, fqHost);
+ // slight race condition, but won't hurt
+ canonicalizedHostCache.putIfAbsent(host, fqHost);
} catch (UnknownHostException e) {
fqHost = host;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMappingWithDependency.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMappingWithDependency.java
index 8a0a0033fd2d6..086650bd7d31a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMappingWithDependency.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMappingWithDependency.java
@@ -171,8 +171,7 @@ public List getDependency(String name) {
@Override
public String toString() {
- return super.toString() + ", " + dependencyScriptName != null ?
- ("dependency script " + dependencyScriptName) : NO_SCRIPT;
+ return "dependency script " + dependencyScriptName;
}
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketIOWithTimeout.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketIOWithTimeout.java
index ed12b3c6be027..b50f7e936ba9b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketIOWithTimeout.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketIOWithTimeout.java
@@ -338,6 +338,12 @@ int select(SelectableChannel channel, int ops, long timeout)
return ret;
}
+ if (Thread.currentThread().isInterrupted()) {
+ throw new InterruptedIOException("Interrupted while waiting for "
+ + "IO on channel " + channel + ". " + timeout
+ + " millis timeout left.");
+ }
+
/* Sometimes select() returns 0 much before timeout for
* unknown reasons. So select again if required.
*/
@@ -348,12 +354,6 @@ int select(SelectableChannel channel, int ops, long timeout)
}
}
- if (Thread.currentThread().isInterrupted()) {
- throw new InterruptedIOException("Interruped while waiting for " +
- "IO on channel " + channel +
- ". " + timeout +
- " millis timeout left.");
- }
}
} finally {
if (key != null) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/TableMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/TableMapping.java
index 2662108124d60..59c0ca96750a2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/TableMapping.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/TableMapping.java
@@ -20,13 +20,16 @@
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.NET_TOPOLOGY_TABLE_MAPPING_FILE_KEY;
import java.io.BufferedReader;
+import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
+import java.io.InputStreamReader;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import org.apache.commons.io.Charsets;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -96,9 +99,10 @@ private Map load() {
return null;
}
- BufferedReader reader = null;
- try {
- reader = new BufferedReader(new FileReader(filename));
+
+ try (BufferedReader reader =
+ new BufferedReader(new InputStreamReader(
+ new FileInputStream(filename), Charsets.UTF_8))) {
String line = reader.readLine();
while (line != null) {
line = line.trim();
@@ -115,15 +119,6 @@ private Map load() {
} catch (Exception e) {
LOG.warn(filename + " cannot be read.", e);
return null;
- } finally {
- if (reader != null) {
- try {
- reader.close();
- } catch (IOException e) {
- LOG.warn(filename + " cannot be read.", e);
- return null;
- }
- }
}
return loadMap;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java
index 4fb9e45614133..43d1b66d44f6d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.security;
+import com.google.common.base.Charsets;
import org.apache.hadoop.http.HttpServer2;
import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
import org.apache.hadoop.conf.Configuration;
@@ -24,8 +25,10 @@
import org.apache.hadoop.http.FilterInitializer;
import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
+import java.io.InputStreamReader;
import java.io.Reader;
import java.util.HashMap;
import java.util.Map;
@@ -78,10 +81,10 @@ public void initFilter(FilterContainer container, Configuration conf) {
if (signatureSecretFile == null) {
throw new RuntimeException("Undefined property: " + SIGNATURE_SECRET_FILE);
}
-
- try {
- StringBuilder secret = new StringBuilder();
- Reader reader = new FileReader(signatureSecretFile);
+
+ StringBuilder secret = new StringBuilder();
+ try (Reader reader = new InputStreamReader(
+ new FileInputStream(signatureSecretFile), Charsets.UTF_8)) {
int c = reader.read();
while (c > -1) {
secret.append((char)c);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
index b81e810f191c7..e6b8722c3558c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
@@ -32,6 +32,7 @@
import java.util.List;
import java.util.Map;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -217,7 +218,8 @@ public void readTokenStorageStream(DataInputStream in) throws IOException {
readFields(in);
}
- private static final byte[] TOKEN_STORAGE_MAGIC = "HDTS".getBytes();
+ private static final byte[] TOKEN_STORAGE_MAGIC =
+ "HDTS".getBytes(Charsets.UTF_8);
private static final byte TOKEN_STORAGE_VERSION = 0;
public void writeTokenStorageToStream(DataOutputStream os)
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
index c5004197e5638..9fd39b09abac0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
@@ -23,8 +23,16 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Ticker;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -52,21 +60,21 @@ public class Groups {
private static final Log LOG = LogFactory.getLog(Groups.class);
private final GroupMappingServiceProvider impl;
-
- private final Map userToGroupsMap =
- new ConcurrentHashMap();
- private final Map> staticUserToGroupsMap =
+
+ private final LoadingCache> cache;
+ private final Map> staticUserToGroupsMap =
new HashMap>();
private final long cacheTimeout;
private final long negativeCacheTimeout;
private final long warningDeltaMs;
private final Timer timer;
+ private Set negativeCache;
public Groups(Configuration conf) {
this(conf, new Timer());
}
- public Groups(Configuration conf, Timer timer) {
+ public Groups(Configuration conf, final Timer timer) {
impl =
ReflectionUtils.newInstance(
conf.getClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
@@ -86,12 +94,30 @@ public Groups(Configuration conf, Timer timer) {
parseStaticMapping(conf);
this.timer = timer;
+ this.cache = CacheBuilder.newBuilder()
+ .refreshAfterWrite(cacheTimeout, TimeUnit.MILLISECONDS)
+ .ticker(new TimerToTickerAdapter(timer))
+ .expireAfterWrite(10 * cacheTimeout, TimeUnit.MILLISECONDS)
+ .build(new GroupCacheLoader());
+
+ if(negativeCacheTimeout > 0) {
+ Cache tempMap = CacheBuilder.newBuilder()
+ .expireAfterWrite(negativeCacheTimeout, TimeUnit.MILLISECONDS)
+ .ticker(new TimerToTickerAdapter(timer))
+ .build();
+ negativeCache = Collections.newSetFromMap(tempMap.asMap());
+ }
if(LOG.isDebugEnabled())
LOG.debug("Group mapping impl=" + impl.getClass().getName() +
"; cacheTimeout=" + cacheTimeout + "; warningDeltaMs=" +
warningDeltaMs);
}
+
+ @VisibleForTesting
+ Set getNegativeCache() {
+ return negativeCache;
+ }
/*
* Parse the hadoop.user.group.static.mapping.overrides configuration to
@@ -123,78 +149,106 @@ private void parseStaticMapping(Configuration conf) {
}
}
- /**
- * Determine whether the CachedGroups is expired.
- * @param groups cached groups for one user.
- * @return true if groups is expired from useToGroupsMap.
- */
- private boolean hasExpired(CachedGroups groups, long startMs) {
- if (groups == null) {
- return true;
- }
- long timeout = cacheTimeout;
- if (isNegativeCacheEnabled() && groups.getGroups().isEmpty()) {
- // This CachedGroups is in the negative cache, thus it should expire
- // sooner.
- timeout = negativeCacheTimeout;
- }
- return groups.getTimestamp() + timeout <= startMs;
- }
-
private boolean isNegativeCacheEnabled() {
return negativeCacheTimeout > 0;
}
+ private IOException noGroupsForUser(String user) {
+ return new IOException("No groups found for user " + user);
+ }
+
/**
* Get the group memberships of a given user.
+ * If the user's group is not cached, this method may block.
* @param user User's name
* @return the group memberships of the user
- * @throws IOException
+ * @throws IOException if user does not exist
*/
- public List getGroups(String user) throws IOException {
+ public List getGroups(final String user) throws IOException {
// No need to lookup for groups of static users
List staticMapping = staticUserToGroupsMap.get(user);
if (staticMapping != null) {
return staticMapping;
}
- // Return cached value if available
- CachedGroups groups = userToGroupsMap.get(user);
- long startMs = timer.monotonicNow();
- if (!hasExpired(groups, startMs)) {
- if(LOG.isDebugEnabled()) {
- LOG.debug("Returning cached groups for '" + user + "'");
- }
- if (groups.getGroups().isEmpty()) {
- // Even with enabling negative cache, getGroups() has the same behavior
- // that throws IOException if the groups for the user is empty.
- throw new IOException("No groups found for user " + user);
+
+ // Check the negative cache first
+ if (isNegativeCacheEnabled()) {
+ if (negativeCache.contains(user)) {
+ throw noGroupsForUser(user);
}
- return groups.getGroups();
}
- // Create and cache user's groups
- List groupList = impl.getGroups(user);
- long endMs = timer.monotonicNow();
- long deltaMs = endMs - startMs ;
- UserGroupInformation.metrics.addGetGroups(deltaMs);
- if (deltaMs > warningDeltaMs) {
- LOG.warn("Potential performance problem: getGroups(user=" + user +") " +
- "took " + deltaMs + " milliseconds.");
+ try {
+ return cache.get(user);
+ } catch (ExecutionException e) {
+ throw (IOException)e.getCause();
}
- groups = new CachedGroups(groupList, endMs);
- if (groups.getGroups().isEmpty()) {
- if (isNegativeCacheEnabled()) {
- userToGroupsMap.put(user, groups);
+ }
+
+ /**
+ * Convert millisecond times from hadoop's timer to guava's nanosecond ticker.
+ */
+ private static class TimerToTickerAdapter extends Ticker {
+ private Timer timer;
+
+ public TimerToTickerAdapter(Timer timer) {
+ this.timer = timer;
+ }
+
+ @Override
+ public long read() {
+ final long NANOSECONDS_PER_MS = 1000000;
+ return timer.monotonicNow() * NANOSECONDS_PER_MS;
+ }
+ }
+
+ /**
+ * Deals with loading data into the cache.
+ */
+ private class GroupCacheLoader extends CacheLoader> {
+ /**
+ * This method will block if a cache entry doesn't exist, and
+ * any subsequent requests for the same user will wait on this
+ * request to return. If a user already exists in the cache,
+ * this will be run in the background.
+ * @param user key of cache
+ * @return List of groups belonging to user
+ * @throws IOException to prevent caching negative entries
+ */
+ @Override
+ public List load(String user) throws Exception {
+ List groups = fetchGroupList(user);
+
+ if (groups.isEmpty()) {
+ if (isNegativeCacheEnabled()) {
+ negativeCache.add(user);
+ }
+
+ // We throw here to prevent Cache from retaining an empty group
+ throw noGroupsForUser(user);
}
- throw new IOException("No groups found for user " + user);
+
+ return groups;
}
- userToGroupsMap.put(user, groups);
- if(LOG.isDebugEnabled()) {
- LOG.debug("Returning fetched groups for '" + user + "'");
+
+ /**
+ * Queries impl for groups belonging to the user. This could involve I/O and take awhile.
+ */
+ private List fetchGroupList(String user) throws IOException {
+ long startMs = timer.monotonicNow();
+ List groupList = impl.getGroups(user);
+ long endMs = timer.monotonicNow();
+ long deltaMs = endMs - startMs ;
+ UserGroupInformation.metrics.addGetGroups(deltaMs);
+ if (deltaMs > warningDeltaMs) {
+ LOG.warn("Potential performance problem: getGroups(user=" + user +") " +
+ "took " + deltaMs + " milliseconds.");
+ }
+
+ return groupList;
}
- return groups.getGroups();
}
-
+
/**
* Refresh all user-to-groups mappings.
*/
@@ -205,7 +259,10 @@ public void refresh() {
} catch (IOException e) {
LOG.warn("Error refreshing groups cache", e);
}
- userToGroupsMap.clear();
+ cache.invalidateAll();
+ if(isNegativeCacheEnabled()) {
+ negativeCache.clear();
+ }
}
/**
@@ -221,40 +278,6 @@ public void cacheGroupsAdd(List groups) {
}
}
- /**
- * Class to hold the cached groups
- */
- private static class CachedGroups {
- final long timestamp;
- final List groups;
-
- /**
- * Create and initialize group cache
- */
- CachedGroups(List groups, long timestamp) {
- this.groups = groups;
- this.timestamp = timestamp;
- }
-
- /**
- * Returns time of last cache update
- *
- * @return time of last cache update
- */
- public long getTimestamp() {
- return timestamp;
- }
-
- /**
- * Get list of cached groups
- *
- * @return cached groups
- */
- public List getGroups() {
- return groups;
- }
- }
-
private static Groups GROUPS = null;
/**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
index 76f53804b33ff..d463ac7a2be3f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
@@ -17,8 +17,10 @@
*/
package org.apache.hadoop.security;
+import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
+import java.io.InputStreamReader;
import java.io.Reader;
import java.util.ArrayList;
import java.util.Hashtable;
@@ -34,6 +36,7 @@
import javax.naming.directory.SearchControls;
import javax.naming.directory.SearchResult;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -338,6 +341,8 @@ public synchronized void setConf(Configuration conf) {
int dirSearchTimeout = conf.getInt(DIRECTORY_SEARCH_TIMEOUT, DIRECTORY_SEARCH_TIMEOUT_DEFAULT);
SEARCH_CONTROLS.setTimeLimit(dirSearchTimeout);
+ // Limit the attributes returned to only those required to speed up the search. See HADOOP-10626 for more details.
+ SEARCH_CONTROLS.setReturningAttributes(new String[] {groupNameAttr});
this.conf = conf;
}
@@ -366,11 +371,10 @@ String extractPassword(String pwFile) {
// an anonymous bind
return "";
}
-
- Reader reader = null;
- try {
- StringBuilder password = new StringBuilder();
- reader = new FileReader(pwFile);
+
+ StringBuilder password = new StringBuilder();
+ try (Reader reader = new InputStreamReader(
+ new FileInputStream(pwFile), Charsets.UTF_8)) {
int c = reader.read();
while (c > -1) {
password.append((char)c);
@@ -379,8 +383,6 @@ String extractPassword(String pwFile) {
return password.toString().trim();
} catch (IOException ioe) {
throw new RuntimeException("Could not read password file: " + pwFile, ioe);
- } finally {
- IOUtils.cleanup(LOG, reader);
}
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NetgroupCache.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NetgroupCache.java
index bd9c448da7f62..4495a66c4322f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NetgroupCache.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NetgroupCache.java
@@ -17,11 +17,11 @@
*/
package org.apache.hadoop.security;
+import java.util.Collections;
+import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
-import java.util.Map;
import java.util.Set;
-import java.util.HashSet;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -36,14 +36,9 @@
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Unstable
public class NetgroupCache {
- private static boolean netgroupToUsersMapUpdated = true;
- private static Map> netgroupToUsersMap =
- new ConcurrentHashMap>();
-
- private static Map> userToNetgroupsMap =
+ private static ConcurrentHashMap> userToNetgroupsMap =
new ConcurrentHashMap>();
-
/**
* Get netgroups for a given user
*
@@ -52,21 +47,11 @@ public class NetgroupCache {
*/
public static void getNetgroups(final String user,
List groups) {
- if(netgroupToUsersMapUpdated) {
- netgroupToUsersMapUpdated = false; // at the beginning to avoid race
- //update userToNetgroupsMap
- for(String netgroup : netgroupToUsersMap.keySet()) {
- for(String netuser : netgroupToUsersMap.get(netgroup)) {
- // add to userToNetgroupsMap
- if(!userToNetgroupsMap.containsKey(netuser)) {
- userToNetgroupsMap.put(netuser, new HashSet());
- }
- userToNetgroupsMap.get(netuser).add(netgroup);
- }
- }
- }
- if(userToNetgroupsMap.containsKey(user)) {
- groups.addAll(userToNetgroupsMap.get(user));
+ Set userGroups = userToNetgroupsMap.get(user);
+ //ConcurrentHashMap does not allow null values;
+ //So null value check can be used to check if the key exists
+ if (userGroups != null) {
+ groups.addAll(userGroups);
}
}
@@ -76,7 +61,15 @@ public static void getNetgroups(final String user,
* @return list of cached groups
*/
public static List getNetgroupNames() {
- return new LinkedList(netgroupToUsersMap.keySet());
+ return new LinkedList(getGroups());
+ }
+
+ private static Set getGroups() {
+ Set allGroups = new HashSet ();
+ for (Set userGroups : userToNetgroupsMap.values()) {
+ allGroups.addAll(userGroups);
+ }
+ return allGroups;
}
/**
@@ -86,14 +79,13 @@ public static List getNetgroupNames() {
* @return true if group is cached, false otherwise
*/
public static boolean isCached(String group) {
- return netgroupToUsersMap.containsKey(group);
+ return getGroups().contains(group);
}
/**
* Clear the cache
*/
public static void clear() {
- netgroupToUsersMap.clear();
userToNetgroupsMap.clear();
}
@@ -104,7 +96,20 @@ public static void clear() {
* @param users list of users for a given group
*/
public static void add(String group, List users) {
- netgroupToUsersMap.put(group, new HashSet(users));
- netgroupToUsersMapUpdated = true; // at the end to avoid race
+ for (String user : users) {
+ Set userGroups = userToNetgroupsMap.get(user);
+ // ConcurrentHashMap does not allow null values;
+ // So null value check can be used to check if the key exists
+ if (userGroups == null) {
+ //Generate a ConcurrentHashSet (backed by the keyset of the ConcurrentHashMap)
+ userGroups =
+ Collections.newSetFromMap(new ConcurrentHashMap());
+ Set currentSet = userToNetgroupsMap.putIfAbsent(user, userGroups);
+ if (currentSet != null) {
+ userGroups = currentSet;
+ }
+ }
+ userGroups.add(group);
+ }
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java
index c4fc965a95664..0b49cfbc4462b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.security;
import java.net.InetAddress;
+import java.util.Locale;
import java.util.Map;
import java.util.TreeMap;
@@ -65,7 +66,7 @@ public void setConf(Configuration conf) {
CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION,
QualityOfProtection.AUTHENTICATION.toString());
for (int i=0; i < qop.length; i++) {
- qop[i] = QualityOfProtection.valueOf(qop[i].toUpperCase()).getSaslQop();
+ qop[i] = QualityOfProtection.valueOf(qop[i].toUpperCase(Locale.ENGLISH)).getSaslQop();
}
properties.put(Sasl.QOP, StringUtils.join(",", qop));
properties.put(Sasl.SERVER_AUTH, "true");
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
index dfb0898a44923..4a1a397ed6f45 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
@@ -573,17 +573,15 @@ public int read(byte b[]) throws IOException {
}
@Override
- public int read(byte[] buf, int off, int len) throws IOException {
- synchronized(unwrappedRpcBuffer) {
- // fill the buffer with the next RPC message
- if (unwrappedRpcBuffer.remaining() == 0) {
- readNextRpcPacket();
- }
- // satisfy as much of the request as possible
- int readLen = Math.min(len, unwrappedRpcBuffer.remaining());
- unwrappedRpcBuffer.get(buf, off, readLen);
- return readLen;
+ public synchronized int read(byte[] buf, int off, int len) throws IOException {
+ // fill the buffer with the next RPC message
+ if (unwrappedRpcBuffer.remaining() == 0) {
+ readNextRpcPacket();
}
+ // satisfy as much of the request as possible
+ int readLen = Math.min(len, unwrappedRpcBuffer.remaining());
+ unwrappedRpcBuffer.get(buf, off, readLen);
+ return readLen;
}
// all messages must be RPC SASL wrapped, else an exception is thrown
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java
index 83f46efd6e456..f2b21e851bbc2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java
@@ -44,6 +44,7 @@
import javax.security.sasl.SaslServerFactory;
import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -184,11 +185,11 @@ public static void init(Configuration conf) {
}
static String encodeIdentifier(byte[] identifier) {
- return new String(Base64.encodeBase64(identifier));
+ return new String(Base64.encodeBase64(identifier), Charsets.UTF_8);
}
static byte[] decodeIdentifier(String identifier) {
- return Base64.decodeBase64(identifier.getBytes());
+ return Base64.decodeBase64(identifier.getBytes(Charsets.UTF_8));
}
public static T getIdentifier(String id,
@@ -206,7 +207,8 @@ public static T getIdentifier(String id,
}
static char[] encodePassword(byte[] password) {
- return new String(Base64.encodeBase64(password)).toCharArray();
+ return new String(Base64.encodeBase64(password),
+ Charsets.UTF_8).toCharArray();
}
/** Splitting fully qualified Kerberos name into parts */
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
index 768294d707b18..fd362d038d3d8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
@@ -22,11 +22,13 @@
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStreamReader;
+import java.nio.charset.Charset;
import java.util.HashMap;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -73,6 +75,10 @@ public class ShellBasedIdMapping implements IdMappingServiceProvider {
private final File staticMappingFile;
private StaticMapping staticMapping = null;
+ // Last time the static map was modified, measured time difference in
+ // milliseconds since midnight, January 1, 1970 UTC
+ private long lastModificationTimeStaticMap = 0;
+
private boolean constructFullMapAtInit = false;
// Used for parsing the static mapping file.
@@ -86,7 +92,6 @@ public class ShellBasedIdMapping implements IdMappingServiceProvider {
// Maps for id to name map. Guarded by this object monitor lock
private BiMap uidNameMap = HashBiMap.create();
private BiMap gidNameMap = HashBiMap.create();
-
private long lastUpdateTime = 0; // Last time maps were updated
/*
@@ -116,7 +121,7 @@ public ShellBasedIdMapping(Configuration conf,
conf.get(IdMappingConstant.STATIC_ID_MAPPING_FILE_KEY,
IdMappingConstant.STATIC_ID_MAPPING_FILE_DEFAULT);
staticMappingFile = new File(staticFilePath);
-
+ updateStaticMapping();
updateMaps();
}
@@ -181,7 +186,7 @@ private static void reportDuplicateEntry(final String header,
final Integer key, final String value,
final Integer ekey, final String evalue) {
LOG.warn("\n" + header + String.format(
- "new entry (%d, %s), existing entry: (%d, %s).\n%s\n%s",
+ "new entry (%d, %s), existing entry: (%d, %s).%n%s%n%s",
key, value, ekey, evalue,
"The new entry is to be ignored for the following reason.",
DUPLICATE_NAME_ID_DEBUG_INFO));
@@ -217,7 +222,9 @@ public static boolean updateMapInternal(BiMap map,
try {
Process process = Runtime.getRuntime().exec(
new String[] { "bash", "-c", command });
- br = new BufferedReader(new InputStreamReader(process.getInputStream()));
+ br = new BufferedReader(
+ new InputStreamReader(process.getInputStream(),
+ Charset.defaultCharset()));
String line = null;
while ((line = br.readLine()) != null) {
String[] nameId = line.split(regex);
@@ -286,20 +293,42 @@ private static boolean isInteger(final String s) {
return true;
}
- private void initStaticMapping() throws IOException {
- staticMapping = new StaticMapping(
- new HashMap(), new HashMap());
+ private synchronized void updateStaticMapping() throws IOException {
+ final boolean init = (staticMapping == null);
+ //
+ // if the static mapping file
+ // - was modified after last update, load the map again;
+ // - did not exist but was added since last update, load the map;
+ // - existed before but deleted since last update, clear the map
+ //
if (staticMappingFile.exists()) {
- LOG.info("Using '" + staticMappingFile + "' for static UID/GID mapping...");
- staticMapping = parseStaticMap(staticMappingFile);
+ // check modification time, reload the file if the last modification
+ // time changed since prior load.
+ long lmTime = staticMappingFile.lastModified();
+ if (lmTime != lastModificationTimeStaticMap) {
+ LOG.info(init? "Using " : "Reloading " + "'" + staticMappingFile
+ + "' for static UID/GID mapping...");
+ lastModificationTimeStaticMap = lmTime;
+ staticMapping = parseStaticMap(staticMappingFile);
+ }
} else {
- LOG.info("Not doing static UID/GID mapping because '" + staticMappingFile
- + "' does not exist.");
+ if (init) {
+ staticMapping = new StaticMapping(new HashMap(),
+ new HashMap());
+ }
+ if (lastModificationTimeStaticMap != 0 || init) {
+ // print the following log at initialization or when the static
+ // mapping file was deleted after prior load
+ LOG.info("Not doing static UID/GID mapping because '"
+ + staticMappingFile + "' does not exist.");
+ }
+ lastModificationTimeStaticMap = 0;
+ staticMapping.clear();
}
- }
+ }
/*
- * Reset the maps to empty.
+ * Refresh static map, and reset the other maps to empty.
* For testing code, a full map may be re-constructed here when the object
* was created with constructFullMapAtInit being set to true.
*/
@@ -310,15 +339,16 @@ synchronized public void updateMaps() throws IOException {
if (constructFullMapAtInit) {
loadFullMaps();
+ // set constructFullMapAtInit to false to allow testing code to
+ // do incremental update to maps after initial construction
+ constructFullMapAtInit = false;
} else {
+ updateStaticMapping();
clearNameMaps();
}
}
synchronized private void loadFullUserMap() throws IOException {
- if (staticMapping == null) {
- initStaticMapping();
- }
BiMap uMap = HashBiMap.create();
if (OS.startsWith("Mac")) {
updateMapInternal(uMap, "user", MAC_GET_ALL_USERS_CMD, "\\s+",
@@ -332,9 +362,6 @@ synchronized private void loadFullUserMap() throws IOException {
}
synchronized private void loadFullGroupMap() throws IOException {
- if (staticMapping == null) {
- initStaticMapping();
- }
BiMap gMap = HashBiMap.create();
if (OS.startsWith("Mac")) {
@@ -349,7 +376,6 @@ synchronized private void loadFullGroupMap() throws IOException {
}
synchronized private void loadFullMaps() throws IOException {
- initStaticMapping();
loadFullUserMap();
loadFullGroupMap();
}
@@ -439,9 +465,7 @@ synchronized private void updateMapIncr(final String name,
}
boolean updated = false;
- if (staticMapping == null) {
- initStaticMapping();
- }
+ updateStaticMapping();
if (OS.startsWith("Linux")) {
if (isGrp) {
@@ -477,9 +501,7 @@ synchronized private void updateMapIncr(final int id,
}
boolean updated = false;
- if (staticMapping == null) {
- initStaticMapping();
- }
+ updateStaticMapping();
if (OS.startsWith("Linux")) {
if (isGrp) {
@@ -543,6 +565,15 @@ public StaticMapping(Map uidMapping,
this.uidMapping = new PassThroughMap(uidMapping);
this.gidMapping = new PassThroughMap(gidMapping);
}
+
+ public void clear() {
+ uidMapping.clear();
+ gidMapping.clear();
+ }
+
+ public boolean isNonEmpty() {
+ return uidMapping.size() > 0 || gidMapping.size() > 0;
+ }
}
static StaticMapping parseStaticMap(File staticMapFile)
@@ -552,7 +583,7 @@ static StaticMapping parseStaticMap(File staticMapFile)
Map gidMapping = new HashMap();
BufferedReader in = new BufferedReader(new InputStreamReader(
- new FileInputStream(staticMapFile)));
+ new FileInputStream(staticMapFile), Charsets.UTF_8));
try {
String line = null;
@@ -574,8 +605,8 @@ static StaticMapping parseStaticMap(File staticMapFile)
// We know the line is fine to parse without error checking like this
// since it matched the regex above.
String firstComponent = lineMatcher.group(1);
- int remoteId = Integer.parseInt(lineMatcher.group(2));
- int localId = Integer.parseInt(lineMatcher.group(3));
+ int remoteId = parseId(lineMatcher.group(2));
+ int localId = parseId(lineMatcher.group(3));
if (firstComponent.equals("uid")) {
uidMapping.put(localId, remoteId);
} else {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index 0541f9d9cd00d..4b0b5f305fec1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -44,9 +44,9 @@
import javax.security.auth.Subject;
import javax.security.auth.callback.CallbackHandler;
-import javax.security.auth.kerberos.KerberosKey;
import javax.security.auth.kerberos.KerberosPrincipal;
import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.kerberos.KeyTab;
import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
import javax.security.auth.login.LoginContext;
@@ -610,20 +610,6 @@ private void setLogin(LoginContext login) {
user.setLogin(login);
}
- private static Class> KEY_TAB_CLASS = KerberosKey.class;
- static {
- try {
- // We use KEY_TAB_CLASS to determine if the UGI is logged in from
- // keytab. In JDK6 and JDK7, if useKeyTab and storeKey are specified
- // in the Krb5LoginModule, then some number of KerberosKey objects
- // are added to the Subject's private credentials. However, in JDK8,
- // a KeyTab object is added instead. More details in HADOOP-10786.
- KEY_TAB_CLASS = Class.forName("javax.security.auth.kerberos.KeyTab");
- } catch (ClassNotFoundException cnfe) {
- // Ignore. javax.security.auth.kerberos.KeyTab does not exist in JDK6.
- }
- }
-
/**
* Create a UserGroupInformation for the given subject.
* This does not change the subject or acquire new credentials.
@@ -632,7 +618,7 @@ private void setLogin(LoginContext login) {
UserGroupInformation(Subject subject) {
this.subject = subject;
this.user = subject.getPrincipals(User.class).iterator().next();
- this.isKeytab = !subject.getPrivateCredentials(KEY_TAB_CLASS).isEmpty();
+ this.isKeytab = !subject.getPrivateCredentials(KeyTab.class).isEmpty();
this.isKrbTkt = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty();
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
index 6d9c6af26310c..f39740309ea4e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
@@ -44,7 +44,7 @@ public class CredentialShell extends Configured implements Tool {
" [" + DeleteCommand.USAGE + "]\n" +
" [" + ListCommand.USAGE + "]\n";
- private boolean interactive = false;
+ private boolean interactive = true;
private Command command = null;
/** allows stdout to be captured if necessary */
@@ -116,8 +116,8 @@ protected int init(String[] args) throws IOException {
userSuppliedProvider = true;
getConf().set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
args[++i]);
- } else if (args[i].equals("-i") || (args[i].equals("-interactive"))) {
- interactive = true;
+ } else if (args[i].equals("-f") || (args[i].equals("-force"))) {
+ interactive = false;
} else if (args[i].equals("-v") || (args[i].equals("-value"))) {
value = args[++i];
} else if (args[i].equals("-help")) {
@@ -236,11 +236,13 @@ public String getUsage() {
}
private class DeleteCommand extends Command {
- public static final String USAGE = "delete [-provider] [-help]";
+ public static final String USAGE =
+ "delete [-provider] [-f] [-help]";
public static final String DESC =
- "The delete subcommand deletes the credenital\n" +
+ "The delete subcommand deletes the credential\n" +
"specified as the argument from within the provider\n" +
- "indicated through the -provider argument";
+ "indicated through the -provider argument. The command asks for\n" +
+ "confirmation unless the -f option is specified.";
String alias = null;
boolean cont = true;
@@ -267,9 +269,9 @@ public boolean validate() {
if (interactive) {
try {
cont = ToolRunner
- .confirmPrompt("You are about to DELETE the credential: " +
+ .confirmPrompt("You are about to DELETE the credential " +
alias + " from CredentialProvider " + provider.toString() +
- ". Continue?:");
+ ". Continue? ");
if (!cont) {
out.println("Nothing has been deleted.");
}
@@ -293,7 +295,7 @@ public void execute() throws IOException {
provider.flush();
printProviderWritten();
} catch (IOException e) {
- out.println(alias + "has NOT been deleted.");
+ out.println(alias + " has NOT been deleted.");
throw e;
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java
index 5dc2abfd13ff6..05958a058a3c1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.security.alias;
+import org.apache.commons.io.Charsets;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@@ -165,7 +166,7 @@ public CredentialEntry getCredentialEntry(String alias) throws IOException {
}
public static char[] bytesToChars(byte[] bytes) {
- String pass = new String(bytes);
+ String pass = new String(bytes, Charsets.UTF_8);
return pass.toCharArray();
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java
index 262cbadd71ae7..127ccf005d8ca 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java
@@ -23,6 +23,7 @@
import java.util.ArrayList;
import java.util.List;
+import org.apache.commons.io.Charsets;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
@@ -56,7 +57,8 @@ public synchronized CredentialEntry getCredentialEntry(String alias) {
if (bytes == null) {
return null;
}
- return new CredentialEntry(alias, new String(bytes).toCharArray());
+ return new CredentialEntry(
+ alias, new String(bytes, Charsets.UTF_8).toCharArray());
}
@Override
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
index 272538a90fd57..5d2951686006e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
@@ -33,6 +33,7 @@
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.MachineList;
import com.google.common.annotations.VisibleForTesting;
@@ -44,6 +45,7 @@
@InterfaceStability.Evolving
public class ServiceAuthorizationManager {
static final String BLOCKED = ".blocked";
+ static final String HOSTS = ".hosts";
private static final String HADOOP_POLICY_FILE = "hadoop-policy.xml";
@@ -51,6 +53,10 @@ public class ServiceAuthorizationManager {
// and second ACL specifies blocked entries.
private volatile Map, AccessControlList[]> protocolToAcls =
new IdentityHashMap, AccessControlList[]>();
+ // For each class, first MachineList in the array specifies the allowed entries
+ // and second MachineList specifies blocked entries.
+ private volatile Map, MachineList[]> protocolToMachineLists =
+ new IdentityHashMap, MachineList[]>();
/**
* Configuration key for controlling service-level authorization for Hadoop.
@@ -85,7 +91,8 @@ public void authorize(UserGroupInformation user,
InetAddress addr
) throws AuthorizationException {
AccessControlList[] acls = protocolToAcls.get(protocol);
- if (acls == null) {
+ MachineList[] hosts = protocolToMachineLists.get(protocol);
+ if (acls == null || hosts == null) {
throw new AuthorizationException("Protocol " + protocol +
" is not known.");
}
@@ -115,6 +122,16 @@ public void authorize(UserGroupInformation user,
" is not authorized for protocol " + protocol +
", expected client Kerberos principal is " + clientPrincipal);
}
+ if (addr != null) {
+ String hostAddress = addr.getHostAddress();
+ if (hosts.length != 2 || !hosts[0].includes(hostAddress) ||
+ hosts[1].includes(hostAddress)) {
+ AUDITLOG.warn(AUTHZ_FAILED_FOR + " for protocol=" + protocol
+ + " from host = " + hostAddress);
+ throw new AuthorizationException("Host " + hostAddress +
+ " is not authorized for protocol " + protocol) ;
+ }
+ }
AUDITLOG.info(AUTHZ_SUCCESSFUL_FOR + user + " for protocol="+protocol);
}
@@ -135,6 +152,8 @@ public void refreshWithLoadedConfiguration(Configuration conf,
PolicyProvider provider) {
final Map, AccessControlList[]> newAcls =
new IdentityHashMap, AccessControlList[]>();
+ final Map, MachineList[]> newMachineLists =
+ new IdentityHashMap, MachineList[]>();
String defaultAcl = conf.get(
CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_DEFAULT_ACL,
@@ -143,6 +162,13 @@ public void refreshWithLoadedConfiguration(Configuration conf,
String defaultBlockedAcl = conf.get(
CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_DEFAULT_BLOCKED_ACL, "");
+ String defaultServiceHostsKey = getHostKey(
+ CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_DEFAULT_ACL);
+ String defaultMachineList = conf.get(defaultServiceHostsKey,
+ MachineList.WILDCARD_VALUE);
+ String defaultBlockedMachineList= conf.get(
+ defaultServiceHostsKey+ BLOCKED, "");
+
// Parse the config file
Service[] services = provider.getServices();
if (services != null) {
@@ -157,11 +183,26 @@ public void refreshWithLoadedConfiguration(Configuration conf,
conf.get(service.getServiceKey() + BLOCKED,
defaultBlockedAcl));
newAcls.put(service.getProtocol(), new AccessControlList[] {acl, blockedAcl});
+ String serviceHostsKey = getHostKey(service.getServiceKey());
+ MachineList machineList = new MachineList (conf.get(serviceHostsKey, defaultMachineList));
+ MachineList blockedMachineList = new MachineList(
+ conf.get(serviceHostsKey + BLOCKED, defaultBlockedMachineList));
+ newMachineLists.put(service.getProtocol(),
+ new MachineList[] {machineList, blockedMachineList});
}
}
// Flip to the newly parsed permissions
protocolToAcls = newAcls;
+ protocolToMachineLists = newMachineLists;
+ }
+
+ private String getHostKey(String serviceKey) {
+ int endIndex = serviceKey.lastIndexOf(".");
+ if (endIndex != -1) {
+ return serviceKey.substring(0, endIndex)+ HOSTS;
+ }
+ return serviceKey;
}
@VisibleForTesting
@@ -178,4 +219,19 @@ public AccessControlList getProtocolsAcls(Class> className) {
public AccessControlList getProtocolsBlockedAcls(Class> className) {
return protocolToAcls.get(className)[1];
}
+
+ @VisibleForTesting
+ public Set> getProtocolsWithMachineLists() {
+ return protocolToMachineLists.keySet();
+ }
+
+ @VisibleForTesting
+ public MachineList getProtocolsMachineList(Class> className) {
+ return protocolToMachineLists.get(className)[0];
+ }
+
+ @VisibleForTesting
+ public MachineList getProtocolsBlockedMachineList(Class> className) {
+ return protocolToMachineLists.get(className)[1];
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java
index d6bc99535a986..ec522dcff89bf 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java
@@ -45,6 +45,7 @@
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
import org.apache.curator.framework.recipes.shared.SharedCount;
+import org.apache.curator.framework.recipes.shared.VersionedValue;
import org.apache.curator.retry.RetryNTimes;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -58,7 +59,6 @@
import org.apache.zookeeper.client.ZooKeeperSaslClient;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -109,10 +109,10 @@ public abstract class ZKDelegationTokenSecretManager versionedValue = sharedCount.getVersionedValue();
+ if (sharedCount.trySetCount(versionedValue, versionedValue.getValue() + 1)) {
+ break;
+ }
+ }
+ }
+
@Override
protected int incrementDelegationTokenSeqNum() {
try {
- while (!delTokSeqCounter.trySetCount(delTokSeqCounter.getCount() + 1)) {
- }
+ incrSharedCount(delTokSeqCounter);
} catch (InterruptedException e) {
// The ExpirationThread is just finishing.. so dont do anything..
LOG.debug("Thread interrupted while performing token counter increment", e);
@@ -537,8 +546,7 @@ protected int getCurrentKeyId() {
@Override
protected int incrementCurrentKeyId() {
try {
- while (!keyIdSeqCounter.trySetCount(keyIdSeqCounter.getCount() + 1)) {
- }
+ incrSharedCount(keyIdSeqCounter);
} catch (InterruptedException e) {
// The ExpirationThread is just finishing.. so dont do anything..
LOG.debug("Thread interrupted while performing keyId increment", e);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
index 81993e9af2b81..01ba76d849a15 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
@@ -25,23 +25,24 @@
import java.io.IOException;
import java.io.InputStreamReader;
import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.UUID;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
+import org.apache.hadoop.tracing.TraceUtils;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.ShutdownHookManager;
-import org.htrace.HTraceConfiguration;
-import org.htrace.SpanReceiver;
-import org.htrace.Trace;
+import org.apache.htrace.SpanReceiver;
+import org.apache.htrace.SpanReceiverBuilder;
+import org.apache.htrace.Trace;
/**
* This class provides functions for reading the names of SpanReceivers from
@@ -100,7 +101,8 @@ private static String getUniqueLocalTraceFileName() {
// out of /proc/self/stat. (There isn't any portable way to get the
// process ID from Java.)
reader = new BufferedReader(
- new InputStreamReader(new FileInputStream("/proc/self/stat")));
+ new InputStreamReader(new FileInputStream("/proc/self/stat"),
+ Charsets.UTF_8));
String line = reader.readLine();
if (line == null) {
throw new EOFException();
@@ -154,60 +156,13 @@ public synchronized void loadSpanReceivers(Configuration conf) {
private synchronized SpanReceiver loadInstance(String className,
List extraConfig) throws IOException {
- className = className.trim();
- if (!className.contains(".")) {
- className = "org.htrace.impl." + className;
- }
- Class> implClass = null;
- SpanReceiver impl;
- try {
- implClass = Class.forName(className);
- Object o = ReflectionUtils.newInstance(implClass, config);
- impl = (SpanReceiver)o;
- impl.configure(wrapHadoopConf(config, extraConfig));
- } catch (ClassCastException e) {
- throw new IOException("Class " + className +
- " does not implement SpanReceiver.");
- } catch (ClassNotFoundException e) {
- throw new IOException("Class " + className + " cannot be found.");
- } catch (SecurityException e) {
- throw new IOException("Got SecurityException while loading " +
- "SpanReceiver " + className);
- } catch (IllegalArgumentException e) {
- throw new IOException("Got IllegalArgumentException while loading " +
- "SpanReceiver " + className, e);
- } catch (RuntimeException e) {
- throw new IOException("Got RuntimeException while loading " +
- "SpanReceiver " + className, e);
- }
- return impl;
- }
-
- private static HTraceConfiguration wrapHadoopConf(final Configuration conf,
- List extraConfig) {
- final HashMap extraMap = new HashMap();
- for (ConfigurationPair pair : extraConfig) {
- extraMap.put(pair.getKey(), pair.getValue());
+ SpanReceiverBuilder builder =
+ new SpanReceiverBuilder(TraceUtils.wrapHadoopConf(config, extraConfig));
+ SpanReceiver rcvr = builder.spanReceiverClass(className.trim()).build();
+ if (rcvr == null) {
+ throw new IOException("Failed to load SpanReceiver " + className);
}
- return new HTraceConfiguration() {
- public static final String HTRACE_CONF_PREFIX = "hadoop.htrace.";
-
- @Override
- public String get(String key) {
- if (extraMap.containsKey(key)) {
- return extraMap.get(key);
- }
- return conf.get(HTRACE_CONF_PREFIX + key);
- }
-
- @Override
- public String get(String key, String defaultValue) {
- if (extraMap.containsKey(key)) {
- return extraMap.get(key);
- }
- return conf.get(HTRACE_CONF_PREFIX + key, defaultValue);
- }
- };
+ return rcvr;
}
/**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdmin.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdmin.java
index 4ae5aedccfa6c..5fdfbfadd2d92 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdmin.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdmin.java
@@ -25,6 +25,7 @@
import java.util.LinkedList;
import java.util.List;
+import org.apache.commons.io.Charsets;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@@ -91,7 +92,7 @@ private int addSpanReceiver(List args) throws IOException {
return 1;
}
ByteArrayOutputStream configStream = new ByteArrayOutputStream();
- PrintStream configsOut = new PrintStream(configStream);
+ PrintStream configsOut = new PrintStream(configStream, false, "UTF-8");
SpanReceiverInfoBuilder factory = new SpanReceiverInfoBuilder(className);
String prefix = "";
for (int i = 0; i < args.size(); ++i) {
@@ -113,13 +114,15 @@ private int addSpanReceiver(List args) throws IOException {
configsOut.print(prefix + key + " = " + value);
prefix = ", ";
}
+
+ String configStreamStr = configStream.toString("UTF-8");
try {
long id = remote.addSpanReceiver(factory.build());
System.out.println("Added trace span receiver " + id +
- " with configuration " + configStream.toString());
+ " with configuration " + configStreamStr);
} catch (IOException e) {
System.out.println("addSpanReceiver error with configuration " +
- configStream.toString());
+ configStreamStr);
throw e;
}
return 0;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceSamplerFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceSamplerFactory.java
deleted file mode 100644
index 54bcb81b2cda5..0000000000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceSamplerFactory.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.tracing;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.htrace.Sampler;
-import org.htrace.impl.ProbabilitySampler;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@InterfaceAudience.Private
-public class TraceSamplerFactory {
- private static final Logger LOG =
- LoggerFactory.getLogger(TraceSamplerFactory.class);
-
- public static Sampler> createSampler(Configuration conf) {
- String samplerStr = conf.get(CommonConfigurationKeys.HADOOP_TRACE_SAMPLER,
- CommonConfigurationKeys.HADOOP_TRACE_SAMPLER_DEFAULT);
- if (samplerStr.equals("NeverSampler")) {
- LOG.debug("HTrace is OFF for all spans.");
- return Sampler.NEVER;
- } else if (samplerStr.equals("AlwaysSampler")) {
- LOG.info("HTrace is ON for all spans.");
- return Sampler.ALWAYS;
- } else if (samplerStr.equals("ProbabilitySampler")) {
- double percentage =
- conf.getDouble("htrace.probability.sampler.percentage", 0.01d);
- LOG.info("HTrace is ON for " + percentage + "% of top-level spans.");
- return new ProbabilitySampler(percentage / 100.0d);
- } else {
- throw new RuntimeException("Can't create sampler " + samplerStr +
- ". Available samplers are NeverSampler, AlwaysSampler, " +
- "and ProbabilitySampler.");
- }
- }
-}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
new file mode 100644
index 0000000000000..11797e69b4af5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
@@ -0,0 +1,65 @@
+/**
+ * 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.tracing;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
+import org.apache.htrace.HTraceConfiguration;
+
+/**
+ * This class provides utility functions for tracing.
+ */
+@InterfaceAudience.Private
+public class TraceUtils {
+ public static final String HTRACE_CONF_PREFIX = "hadoop.htrace.";
+ private static List EMPTY = Collections.emptyList();
+
+ public static HTraceConfiguration wrapHadoopConf(final Configuration conf) {
+ return wrapHadoopConf(conf, EMPTY);
+ }
+
+ public static HTraceConfiguration wrapHadoopConf(final Configuration conf,
+ List extraConfig) {
+ final HashMap extraMap = new HashMap();
+ for (ConfigurationPair pair : extraConfig) {
+ extraMap.put(pair.getKey(), pair.getValue());
+ }
+ return new HTraceConfiguration() {
+ @Override
+ public String get(String key) {
+ if (extraMap.containsKey(key)) {
+ return extraMap.get(key);
+ }
+ return conf.get(HTRACE_CONF_PREFIX + key, "");
+ }
+
+ @Override
+ public String get(String key, String defaultValue) {
+ if (extraMap.containsKey(key)) {
+ return extraMap.get(key);
+ }
+ return conf.get(HTRACE_CONF_PREFIX + key, defaultValue);
+ }
+ };
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
index d2ab015567d86..6d37c28774e69 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
@@ -67,10 +67,8 @@ public boolean accept(File dir, String name) {
};
static {
- InputStream is = null;
- try {
- is = ApplicationClassLoader.class.getClassLoader().
- getResourceAsStream(PROPERTIES_FILE);
+ try (InputStream is = ApplicationClassLoader.class.getClassLoader()
+ .getResourceAsStream(PROPERTIES_FILE);) {
if (is == null) {
throw new ExceptionInInitializerError("properties file " +
PROPERTIES_FILE + " is not found");
@@ -216,28 +214,43 @@ protected synchronized Class> loadClass(String name, boolean resolve)
return c;
}
+ /**
+ * Checks if a class should be included as a system class.
+ *
+ * A class is a system class if and only if it matches one of the positive
+ * patterns and none of the negative ones.
+ *
+ * @param name the class name to check
+ * @param systemClasses a list of system class configurations.
+ * @return true if the class is a system class
+ */
public static boolean isSystemClass(String name, List systemClasses) {
+ boolean result = false;
if (systemClasses != null) {
String canonicalName = name.replace('/', '.');
while (canonicalName.startsWith(".")) {
canonicalName=canonicalName.substring(1);
}
for (String c : systemClasses) {
- boolean result = true;
+ boolean shouldInclude = true;
if (c.startsWith("-")) {
c = c.substring(1);
- result = false;
+ shouldInclude = false;
}
if (canonicalName.startsWith(c)) {
if ( c.endsWith(".") // package
|| canonicalName.length() == c.length() // class
|| canonicalName.length() > c.length() // nested
&& canonicalName.charAt(c.length()) == '$' ) {
- return result;
+ if (shouldInclude) {
+ result = true;
+ } else {
+ return false;
+ }
}
}
}
}
- return false;
+ return result;
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ChunkedArrayList.java
similarity index 83%
rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java
rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ChunkedArrayList.java
index 89a0db6eb47ee..84ddc32f88c1f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ChunkedArrayList.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdfs.util;
+package org.apache.hadoop.util;
import java.util.AbstractList;
import java.util.Iterator;
@@ -110,11 +110,33 @@ public ChunkedArrayList(int initialChunkCapacity, int maxChunkSize) {
@Override
public Iterator iterator() {
- return Iterables.concat(chunks).iterator();
+ final Iterator it = Iterables.concat(chunks).iterator();
+
+ return new Iterator() {
+ @Override
+ public boolean hasNext() {
+ return it.hasNext();
+ }
+
+ @Override
+ public T next() {
+ return it.next();
+ }
+
+ @Override
+ public void remove() {
+ it.remove();
+ size--;
+ }
+ };
}
@Override
public boolean add(T e) {
+ if (size == Integer.MAX_VALUE) {
+ throw new RuntimeException("Can't add an additional element to the " +
+ "list; list already has INT_MAX elements.");
+ }
if (lastChunk == null) {
addChunk(initialChunkCapacity);
} else if (lastChunk.size() >= lastChunkCapacity) {
@@ -164,8 +186,20 @@ int getMaxChunkSize() {
}
@Override
- public T get(int arg0) {
- throw new UnsupportedOperationException(
- this.getClass().getName() + " does not support random access");
+ public T get(int idx) {
+ if (idx < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+ int base = 0;
+ Iterator> it = chunks.iterator();
+ while (it.hasNext()) {
+ List list = it.next();
+ int size = list.size();
+ if (idx < base + size) {
+ return list.get(idx - base);
+ }
+ base += size;
+ }
+ throw new IndexOutOfBoundsException();
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ComparableVersion.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ComparableVersion.java
index a57342fa889db..65d85f79f8dad 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ComparableVersion.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ComparableVersion.java
@@ -9,8 +9,6 @@
// to
// package org.apache.hadoop.util;
// 2. Removed author tags to clear hadoop author tag warning
-// author Kenney Westerhof
-// author Hervé Boutemy
//
package org.apache.hadoop.util;
@@ -195,6 +193,8 @@ public StringItem( String value, boolean followedByDigit )
case 'm':
value = "milestone";
break;
+ default:
+ break;
}
}
this.value = ALIASES.getProperty( value , value );
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java
index 8bfb5d93aef88..b0c12be731c8a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java
@@ -19,24 +19,28 @@
import java.io.BufferedReader;
import java.io.File;
-import java.io.FileReader;
+import java.io.FileInputStream;
import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
/**
- * FileBasedIPList loads a list of subnets in CIDR format and ip addresses from a file.
+ * FileBasedIPList loads a list of subnets in CIDR format and ip addresses from
+ * a file.
*
- * Given an ip address, isIn method returns true if ip belongs to one of the subnets.
+ * Given an ip address, isIn method returns true if ip belongs to one of the
+ * subnets.
*
* Thread safe.
*/
-
public class FileBasedIPList implements IPList {
private static final Log LOG = LogFactory.getLog(FileBasedIPList.class);
@@ -46,7 +50,12 @@ public class FileBasedIPList implements IPList {
public FileBasedIPList(String fileName) {
this.fileName = fileName;
- String[] lines = readLines(fileName);
+ String[] lines;
+ try {
+ lines = readLines(fileName);
+ } catch (IOException e) {
+ lines = null;
+ }
if (lines != null) {
addressList = new MachineList(new HashSet(Arrays.asList(lines)));
} else {
@@ -67,35 +76,39 @@ public boolean isIn(String ipAddress) {
}
/**
- * reads the lines in a file.
+ * Reads the lines in a file.
* @param fileName
* @return lines in a String array; null if the file does not exist or if the
* file name is null
* @throws IOException
*/
- private static String[] readLines(String fileName) {
+ private static String[] readLines(String fileName) throws IOException {
try {
if (fileName != null) {
File file = new File (fileName);
if (file.exists()) {
- FileReader fileReader = new FileReader(file);
- BufferedReader bufferedReader = new BufferedReader(fileReader);
- List lines = new ArrayList();
- String line = null;
- while ((line = bufferedReader.readLine()) != null) {
- lines.add(line);
+ try (
+ Reader fileReader = new InputStreamReader(
+ new FileInputStream(file), Charsets.UTF_8);
+ BufferedReader bufferedReader = new BufferedReader(fileReader)) {
+ List lines = new ArrayList();
+ String line = null;
+ while ((line = bufferedReader.readLine()) != null) {
+ lines.add(line);
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Loaded IP list of size = " + lines.size() +
+ " from file = " + fileName);
+ }
+ return (lines.toArray(new String[lines.size()]));
}
- bufferedReader.close();
- LOG.debug("Loaded IP list of size = " + lines.size() +" from file = " + fileName);
- return(lines.toArray(new String[lines.size()]));
- }
- else {
+ } else {
LOG.debug("Missing ip list file : "+ fileName);
}
}
- }
- catch (Throwable t) {
- LOG.error(t);
+ } catch (IOException ioe) {
+ LOG.error(ioe);
+ throw ioe;
}
return null;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FindClass.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FindClass.java
new file mode 100644
index 0000000000000..b7feb22d34d21
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FindClass.java
@@ -0,0 +1,388 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.net.URL;
+import java.security.CodeSource;
+
+/**
+ * This entry point exists for diagnosing classloader problems:
+ * is a class or resource present -and if so, where?
+ *
+ *
+ * Actions
+ *
+ *
+ * load
: load a class but do not attempt to create it
+ * create
: load and create a class, print its string value
+ * printresource
: load a resource then print it to stdout
+ * resource
: load a resource then print the URL of that
+ * resource
+ *
+ *
+ * It returns an error code if a class/resource cannot be loaded/found
+ * -and optionally a class may be requested as being loaded.
+ * The latter action will call the class's constructor -it must support an
+ * empty constructor); any side effects from the
+ * constructor or static initializers will take place.
+ *
+ * All error messages are printed to {@link System#out}; errors
+ * to {@link System#err}.
+ *
+ */
+@SuppressWarnings("UseOfSystemOutOrSystemErr")
+public final class FindClass extends Configured implements Tool {
+
+ /**
+ * create command: {@value}
+ */
+ public static final String A_CREATE = "create";
+
+ /**
+ * Load command: {@value}
+ */
+ public static final String A_LOAD = "load";
+
+ /**
+ * Command to locate a resource: {@value}
+ */
+ public static final String A_RESOURCE = "locate";
+
+ /**
+ * Command to locate and print a resource: {@value}
+ */
+ public static final String A_PRINTRESOURCE = "print";
+
+ /**
+ * Exit code when the operation succeeded: {@value}
+ */
+ public static final int SUCCESS = 0;
+
+ /**
+ * generic error {@value}
+ */
+ protected static final int E_GENERIC = 1;
+
+ /**
+ * usage error -bad arguments or similar {@value}
+ */
+ protected static final int E_USAGE = 2;
+
+ /**
+ * class or resource not found {@value}
+ */
+ protected static final int E_NOT_FOUND = 3;
+
+ /**
+ * class load failed {@value}
+ */
+ protected static final int E_LOAD_FAILED = 4;
+
+ /**
+ * class creation failed {@value}
+ */
+ protected static final int E_CREATE_FAILED = 5;
+
+ /**
+ * Output stream. Defaults to {@link System#out}
+ */
+ private static PrintStream stdout = System.out;
+
+ /**
+ * Error stream. Defaults to {@link System#err}
+ */
+ private static PrintStream stderr = System.err;
+
+ /**
+ * Empty constructor; passes a new Configuration
+ * object instance to its superclass's constructor
+ */
+ public FindClass() {
+ super(new Configuration());
+ }
+
+ /**
+ * Create a class with a specified configuration
+ * @param conf configuration
+ */
+ public FindClass(Configuration conf) {
+ super(conf);
+ }
+
+ /**
+ * Change the output streams to be something other than the
+ * System.out and System.err streams
+ * @param out new stdout stream
+ * @param err new stderr stream
+ */
+ @VisibleForTesting
+ public static void setOutputStreams(PrintStream out, PrintStream err) {
+ stdout = out;
+ stderr = err;
+ }
+
+ /**
+ * Get a class fromt the configuration
+ * @param name the class name
+ * @return the class
+ * @throws ClassNotFoundException if the class was not found
+ * @throws Error on other classloading problems
+ */
+ private Class getClass(String name) throws ClassNotFoundException {
+ return getConf().getClassByName(name);
+ }
+
+ /**
+ * Get the resource
+ * @param name resource name
+ * @return URL or null for not found
+ */
+ private URL getResource(String name) {
+ return getConf().getResource(name);
+ }
+
+ /**
+ * Load a resource
+ * @param name resource name
+ * @return the status code
+ */
+ private int loadResource(String name) {
+ URL url = getResource(name);
+ if (url == null) {
+ err("Resource not found: %s", name);
+ return E_NOT_FOUND;
+ }
+ out("%s: %s", name, url);
+ return SUCCESS;
+ }
+
+ /**
+ * Dump a resource to out
+ * @param name resource name
+ * @return the status code
+ */
+ @SuppressWarnings("NestedAssignment")
+ private int dumpResource(String name) {
+ URL url = getResource(name);
+ if (url == null) {
+ err("Resource not found:" + name);
+ return E_NOT_FOUND;
+ }
+ try {
+ //open the resource
+ InputStream instream = url.openStream();
+ //read it in and print
+ int data;
+ while (-1 != (data = instream.read())) {
+ stdout.print((char) data);
+ }
+ //end of file
+ stdout.print('\n');
+ return SUCCESS;
+ } catch (IOException e) {
+ printStack(e, "Failed to read resource %s at URL %s", name, url);
+ return E_LOAD_FAILED;
+ }
+ }
+
+ /**
+ * print something to stderr
+ * @param s string to print
+ */
+ private static void err(String s, Object... args) {
+ stderr.format(s, args);
+ stderr.print('\n');
+ }
+
+ /**
+ * print something to stdout
+ * @param s string to print
+ */
+ private static void out(String s, Object... args) {
+ stdout.format(s, args);
+ stdout.print('\n');
+ }
+
+ /**
+ * print a stack trace with text
+ * @param e the exception to print
+ * @param text text to print
+ */
+ private static void printStack(Throwable e, String text, Object... args) {
+ err(text, args);
+ e.printStackTrace(stderr);
+ }
+
+ /**
+ * Loads the class of the given name
+ * @param name classname
+ * @return outcome code
+ */
+ private int loadClass(String name) {
+ try {
+ Class clazz = getClass(name);
+ loadedClass(name, clazz);
+ return SUCCESS;
+ } catch (ClassNotFoundException e) {
+ printStack(e, "Class not found " + name);
+ return E_NOT_FOUND;
+ } catch (Exception e) {
+ printStack(e, "Exception while loading class " + name);
+ return E_LOAD_FAILED;
+ } catch (Error e) {
+ printStack(e, "Error while loading class " + name);
+ return E_LOAD_FAILED;
+ }
+ }
+
+ /**
+ * Log that a class has been loaded, and where from.
+ * @param name classname
+ * @param clazz class
+ */
+ private void loadedClass(String name, Class clazz) {
+ out("Loaded %s as %s", name, clazz);
+ CodeSource source = clazz.getProtectionDomain().getCodeSource();
+ URL url = source.getLocation();
+ out("%s: %s", name, url);
+ }
+
+ /**
+ * Create an instance of a class
+ * @param name classname
+ * @return the outcome
+ */
+ private int createClassInstance(String name) {
+ try {
+ Class clazz = getClass(name);
+ loadedClass(name, clazz);
+ Object instance = clazz.newInstance();
+ try {
+ //stringify
+ out("Created instance " + instance.toString());
+ } catch (Exception e) {
+ //catch those classes whose toString() method is brittle, but don't fail the probe
+ printStack(e,
+ "Created class instance but the toString() operator failed");
+ }
+ return SUCCESS;
+ } catch (ClassNotFoundException e) {
+ printStack(e, "Class not found " + name);
+ return E_NOT_FOUND;
+ } catch (Exception e) {
+ printStack(e, "Exception while creating class " + name);
+ return E_CREATE_FAILED;
+ } catch (Error e) {
+ printStack(e, "Exception while creating class " + name);
+ return E_CREATE_FAILED;
+ }
+ }
+
+ /**
+ * Run the class/resource find or load operation
+ * @param args command specific arguments.
+ * @return the outcome
+ * @throws Exception if something went very wrong
+ */
+ @Override
+ public int run(String[] args) throws Exception {
+ if (args.length != 2) {
+ return usage(args);
+ }
+ String action = args[0];
+ String name = args[1];
+ int result;
+ if (A_LOAD.equals(action)) {
+ result = loadClass(name);
+ } else if (A_CREATE.equals(action)) {
+ //first load to separate load errors from create
+ result = loadClass(name);
+ if (result == SUCCESS) {
+ //class loads, so instantiate it
+ result = createClassInstance(name);
+ }
+ } else if (A_RESOURCE.equals(action)) {
+ result = loadResource(name);
+ } else if (A_PRINTRESOURCE.equals(action)) {
+ result = dumpResource(name);
+ } else {
+ result = usage(args);
+ }
+ return result;
+ }
+
+ /**
+ * Print a usage message
+ * @param args the command line arguments
+ * @return an exit code
+ */
+ private int usage(String[] args) {
+ err(
+ "Usage : [load | create] ");
+ err(
+ " [locate | print] ]");
+ err("The return codes are:");
+ explainResult(SUCCESS,
+ "The operation was successful");
+ explainResult(E_GENERIC,
+ "Something went wrong");
+ explainResult(E_USAGE,
+ "This usage message was printed");
+ explainResult(E_NOT_FOUND,
+ "The class or resource was not found");
+ explainResult(E_LOAD_FAILED,
+ "The class was found but could not be loaded");
+ explainResult(E_CREATE_FAILED,
+ "The class was loaded, but an instance of it could not be created");
+ return E_USAGE;
+ }
+
+ /**
+ * Explain an error code as part of the usage
+ * @param errorcode error code returned
+ * @param text error text
+ */
+ private void explainResult(int errorcode, String text) {
+ err(" %2d -- %s ", errorcode , text);
+ }
+
+ /**
+ * Main entry point.
+ * Runs the class via the {@link ToolRunner}, then
+ * exits with an appropriate exit code.
+ * @param args argument list
+ */
+ public static void main(String[] args) {
+ try {
+ int result = ToolRunner.run(new FindClass(), args);
+ System.exit(result);
+ } catch (Exception e) {
+ printStack(e, "Running FindClass");
+ System.exit(E_GENERIC);
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
index d0e765529c7b5..0a46a7a81caf9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
@@ -284,6 +284,17 @@ private void processGeneralOptions(Configuration conf,
conf.addResource(new Path(value));
}
}
+
+ if (line.hasOption('D')) {
+ String[] property = line.getOptionValues('D');
+ for(String prop : property) {
+ String[] keyval = prop.split("=", 2);
+ if (keyval.length == 2) {
+ conf.set(keyval[0], keyval[1], "from command line");
+ }
+ }
+ }
+
if (line.hasOption("libjars")) {
conf.set("tmpjars",
validateFiles(line.getOptionValue("libjars"), conf),
@@ -307,15 +318,6 @@ private void processGeneralOptions(Configuration conf,
validateFiles(line.getOptionValue("archives"), conf),
"from -archives command line option");
}
- if (line.hasOption('D')) {
- String[] property = line.getOptionValues('D');
- for(String prop : property) {
- String[] keyval = prop.split("=", 2);
- if (keyval.length == 2) {
- conf.set(keyval[0], keyval[1], "from command line");
- }
- }
- }
conf.setBoolean("mapreduce.client.genericoptionsparser.used", true);
// tokensFile
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java
index b012add42c5fb..ae77e6c33335f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java
@@ -22,6 +22,7 @@
import java.util.Set;
import java.util.HashSet;
+import org.apache.commons.io.Charsets;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -72,7 +73,8 @@ public static void readFileToSetWithFileInputStream(String type,
throws IOException {
BufferedReader reader = null;
try {
- reader = new BufferedReader(new InputStreamReader(fileInputStream));
+ reader = new BufferedReader(
+ new InputStreamReader(fileInputStream, Charsets.UTF_8));
String line;
while ((line = reader.readLine()) != null) {
String[] nodes = line.split("[ \t\n\f\r]+");
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java
index e8af45e746234..1fe77964514d7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java
@@ -22,6 +22,7 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -30,7 +31,6 @@
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
-import com.google.common.base.Stopwatch;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
@@ -172,7 +172,7 @@ public String toString() {
private class Monitor implements Runnable {
@Override
public void run() {
- Stopwatch sw = new Stopwatch();
+ StopWatch sw = new StopWatch();
Map gcTimesBeforeSleep = getGcTimes();
while (shouldRun) {
sw.reset().start();
@@ -181,7 +181,7 @@ public void run() {
} catch (InterruptedException ie) {
return;
}
- long extraSleepTime = sw.elapsedMillis() - SLEEP_INTERVAL_MS;
+ long extraSleepTime = sw.now(TimeUnit.MILLISECONDS) - SLEEP_INTERVAL_MS;
Map gcTimesAfterSleep = getGcTimes();
if (extraSleepTime > warnThresholdMs) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LogAdapter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LogAdapter.java
new file mode 100644
index 0000000000000..6ef9093a28248
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LogAdapter.java
@@ -0,0 +1,74 @@
+/**
+ * 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.slf4j.Logger;
+
+class LogAdapter {
+ private Log LOG;
+ private Logger LOGGER;
+
+ private LogAdapter(Log LOG) {
+ this.LOG = LOG;
+ }
+
+ private LogAdapter(Logger LOGGER) {
+ this.LOGGER = LOGGER;
+ }
+
+ public static LogAdapter create(Log LOG) {
+ return new LogAdapter(LOG);
+ }
+
+ public static LogAdapter create(Logger LOGGER) {
+ return new LogAdapter(LOGGER);
+ }
+
+ public void info(String msg) {
+ if (LOG != null) {
+ LOG.info(msg);
+ } else if (LOGGER != null) {
+ LOGGER.info(msg);
+ }
+ }
+
+ public void warn(String msg, Throwable t) {
+ if (LOG != null) {
+ LOG.warn(msg, t);
+ } else if (LOGGER != null) {
+ LOGGER.warn(msg, t);
+ }
+ }
+
+ public void debug(Throwable t) {
+ if (LOG != null) {
+ LOG.debug(t);
+ } else if (LOGGER != null) {
+ LOGGER.debug("", t);
+ }
+ }
+
+ public void error(String msg) {
+ if (LOG != null) {
+ LOG.error(msg);
+ } else if (LOGGER != null) {
+ LOGGER.error(msg);
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java
index d1a0870f679c2..d60d08387e863 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java
@@ -45,6 +45,7 @@
public class MachineList {
public static final Log LOG = LogFactory.getLog(MachineList.class);
+ public static final String WILDCARD_VALUE = "*";
/**
* InetAddressFactory is used to obtain InetAddress from host.
@@ -91,7 +92,7 @@ public MachineList(Collection hostEntries) {
public MachineList(Collection hostEntries, InetAddressFactory addressFactory) {
this.addressFactory = addressFactory;
if (hostEntries != null) {
- if ((hostEntries.size() == 1) && (hostEntries.contains("*"))) {
+ if ((hostEntries.size() == 1) && (hostEntries.contains(WILDCARD_VALUE))) {
all = true;
ipAddresses = null;
hostNames = null;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java
index 641635542fa42..81448ab2d4dbb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java
@@ -108,14 +108,14 @@ public static void main(String[] args) {
}
System.out.println("Native library checking:");
- System.out.printf("hadoop: %b %s\n", nativeHadoopLoaded, hadoopLibraryName);
- System.out.printf("zlib: %b %s\n", zlibLoaded, zlibLibraryName);
- System.out.printf("snappy: %b %s\n", snappyLoaded, snappyLibraryName);
- System.out.printf("lz4: %b %s\n", lz4Loaded, lz4LibraryName);
- System.out.printf("bzip2: %b %s\n", bzip2Loaded, bzip2LibraryName);
- System.out.printf("openssl: %b %s\n", openSslLoaded, openSslDetail);
+ System.out.printf("hadoop: %b %s%n", nativeHadoopLoaded, hadoopLibraryName);
+ System.out.printf("zlib: %b %s%n", zlibLoaded, zlibLibraryName);
+ System.out.printf("snappy: %b %s%n", snappyLoaded, snappyLibraryName);
+ System.out.printf("lz4: %b %s%n", lz4Loaded, lz4LibraryName);
+ System.out.printf("bzip2: %b %s%n", bzip2Loaded, bzip2LibraryName);
+ System.out.printf("openssl: %b %s%n", openSslLoaded, openSslDetail);
if (Shell.WINDOWS) {
- System.out.printf("winutils: %b %s\n", winutilsExists, winutilsPath);
+ System.out.printf("winutils: %b %s%n", winutilsExists, winutilsPath);
}
if ((!nativeHadoopLoaded) || (Shell.WINDOWS && (!winutilsExists)) ||
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PrintJarMainClass.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PrintJarMainClass.java
index efa4de3ea2043..df571f35e2fcd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PrintJarMainClass.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PrintJarMainClass.java
@@ -34,16 +34,13 @@ public class PrintJarMainClass {
* @param args
*/
public static void main(String[] args) {
- try {
- JarFile jar_file = new JarFile(args[0]);
- if (jar_file != null) {
- Manifest manifest = jar_file.getManifest();
- if (manifest != null) {
- String value = manifest.getMainAttributes().getValue("Main-Class");
- if (value != null) {
- System.out.println(value.replaceAll("/", "."));
- return;
- }
+ try (JarFile jar_file = new JarFile(args[0])) {
+ Manifest manifest = jar_file.getManifest();
+ if (manifest != null) {
+ String value = manifest.getMainAttributes().getValue("Main-Class");
+ if (value != null) {
+ System.out.println(value.replaceAll("/", "."));
+ return;
}
}
} catch (Throwable e) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
index 36b5ff11bc8c2..4b3b7efbf75ed 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
@@ -27,8 +27,8 @@
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*;
import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
import org.apache.hadoop.security.UserGroupInformation;
-import org.htrace.Span;
-import org.htrace.Trace;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
import com.google.protobuf.ByteString;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
index 3977e60287a4a..d9a73263d8521 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
@@ -20,13 +20,16 @@
import java.io.ByteArrayOutputStream;
import java.io.IOException;
+import java.io.PrintStream;
import java.io.PrintWriter;
+import java.io.UnsupportedEncodingException;
import java.lang.management.ManagementFactory;
import java.lang.management.ThreadInfo;
import java.lang.management.ThreadMXBean;
import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
+import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@@ -154,7 +157,7 @@ private static String getTaskName(long id, String name) {
* @param stream the stream to
* @param title a string title for the stack trace
*/
- public synchronized static void printThreadInfo(PrintWriter stream,
+ public synchronized static void printThreadInfo(PrintStream stream,
String title) {
final int STACK_DEPTH = 20;
boolean contention = threadBean.isThreadContentionMonitoringEnabled();
@@ -215,9 +218,12 @@ public static void logThreadInfo(Log log,
}
}
if (dumpStack) {
- ByteArrayOutputStream buffer = new ByteArrayOutputStream();
- printThreadInfo(new PrintWriter(buffer), title);
- log.info(buffer.toString());
+ try {
+ ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+ printThreadInfo(new PrintStream(buffer, false, "UTF-8"), title);
+ log.info(buffer.toString(Charset.defaultCharset().name()));
+ } catch (UnsupportedEncodingException ignored) {
+ }
}
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java
index 6a8ca0f9938e5..2fd9b5589dc81 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java
@@ -70,14 +70,14 @@ public static long parseLongParam(ServletRequest request, String param)
throw new IOException("Invalid request has no " + param + " parameter");
}
- return Long.valueOf(paramStr);
+ return Long.parseLong(paramStr);
}
public static final String HTML_TAIL = "
\n"
- + "Hadoop, "
+ + "Hadoop, "
+ Calendar.getInstance().get(Calendar.YEAR) + ".\n"
+ "