();
+ }
+ }
+
@Override
protected void processNonexistentPath(PathData item) throws IOException {
if (!ignoreFNF) super.processNonexistentPath(item);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
index 0d73f76e666..a99494573fd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
@@ -48,7 +48,7 @@ class FsUsage extends FsCommand {
protected String formatSize(long size) {
return humanReadable
- ? StringUtils.humanReadableInt(size)
+ ? StringUtils.TraditionalBinaryPrefix.long2String(size, "", 1)
: String.valueOf(size);
}
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 289adea8d34..76192e73b4a 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
@@ -67,7 +67,7 @@ class Ls extends FsCommand {
protected boolean humanReadable = false;
protected String formatSize(long size) {
return humanReadable
- ? StringUtils.humanReadableInt(size)
+ ? StringUtils.TraditionalBinaryPrefix.long2String(size, "", 1)
: String.valueOf(size);
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java
index 60836ab26a3..c27449d3618 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java
@@ -32,9 +32,10 @@ import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Unstable
public class BoundedByteArrayOutputStream extends OutputStream {
- private final byte[] buffer;
+ private byte[] buffer;
+ private int startOffset;
private int limit;
- private int count;
+ private int currentPointer;
/**
* Create a BoundedByteArrayOutputStream with the specified
@@ -52,20 +53,30 @@ public class BoundedByteArrayOutputStream extends OutputStream {
* @param limit The maximum limit upto which data can be written
*/
public BoundedByteArrayOutputStream(int capacity, int limit) {
+ this(new byte[capacity], 0, limit);
+ }
+
+ protected BoundedByteArrayOutputStream(byte[] buf, int offset, int limit) {
+ resetBuffer(buf, offset, limit);
+ }
+
+ protected void resetBuffer(byte[] buf, int offset, int limit) {
+ int capacity = buf.length - offset;
if ((capacity < limit) || (capacity | limit) < 0) {
throw new IllegalArgumentException("Invalid capacity/limit");
}
- this.buffer = new byte[capacity];
- this.limit = limit;
- this.count = 0;
+ this.buffer = buf;
+ this.startOffset = offset;
+ this.currentPointer = offset;
+ this.limit = offset + limit;
}
-
+
@Override
public void write(int b) throws IOException {
- if (count >= limit) {
+ if (currentPointer >= limit) {
throw new EOFException("Reaching the limit of the buffer.");
}
- buffer[count++] = (byte) b;
+ buffer[currentPointer++] = (byte) b;
}
@Override
@@ -77,12 +88,12 @@ public class BoundedByteArrayOutputStream extends OutputStream {
return;
}
- if (count + len > limit) {
+ if (currentPointer + len > limit) {
throw new EOFException("Reach the limit of the buffer");
}
- System.arraycopy(b, off, buffer, count, len);
- count += len;
+ System.arraycopy(b, off, buffer, currentPointer, len);
+ currentPointer += len;
}
/**
@@ -90,17 +101,17 @@ public class BoundedByteArrayOutputStream extends OutputStream {
* @param newlim New Limit
*/
public void reset(int newlim) {
- if (newlim > buffer.length) {
+ if (newlim > (buffer.length - startOffset)) {
throw new IndexOutOfBoundsException("Limit exceeds buffer size");
}
this.limit = newlim;
- this.count = 0;
+ this.currentPointer = startOffset;
}
/** Reset the buffer */
public void reset() {
- this.limit = buffer.length;
- this.count = 0;
+ this.limit = buffer.length - startOffset;
+ this.currentPointer = startOffset;
}
/** Return the current limit */
@@ -119,6 +130,10 @@ public class BoundedByteArrayOutputStream extends OutputStream {
* currently in the buffer.
*/
public int size() {
- return count;
+ return currentPointer - startOffset;
+ }
+
+ public int available() {
+ return limit - currentPointer;
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SortedMapWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SortedMapWritable.java
index eee744ec6a2..c80af15c9e2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SortedMapWritable.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SortedMapWritable.java
@@ -203,4 +203,27 @@ public class SortedMapWritable extends AbstractMapWritable
e.getValue().write(out);
}
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+
+ if (obj instanceof SortedMapWritable) {
+ Map map = (Map) obj;
+ if (size() != map.size()) {
+ return false;
+ }
+
+ return entrySet().equals(map.entrySet());
+ }
+
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return instance.hashCode();
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
index 5c29a33312d..9d99c47b32f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
@@ -35,6 +35,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.net.ConnectTimeoutException;
/**
*
@@ -543,6 +544,7 @@ public class RetryPolicies {
e instanceof NoRouteToHostException ||
e instanceof UnknownHostException ||
e instanceof StandbyException ||
+ e instanceof ConnectTimeoutException ||
isWrappedStandbyException(e)) {
return new RetryAction(
RetryAction.RetryDecision.FAILOVER_AND_RETRY,
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 36ea776b777..24a84c35661 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
@@ -67,6 +67,7 @@ import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto.OperationProto;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
+import org.apache.hadoop.net.ConnectTimeoutException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.SaslRpcClient;
@@ -511,14 +512,14 @@ public class Client {
}
this.socket.setSoTimeout(pingInterval);
return;
- } catch (SocketTimeoutException toe) {
+ } catch (ConnectTimeoutException toe) {
/* Check for an address change and update the local reference.
* Reset the failure counter if the address was changed
*/
if (updateAddress()) {
timeoutFailures = ioFailures = 0;
}
- handleConnectionFailure(timeoutFailures++,
+ handleConnectionTimeout(timeoutFailures++,
maxRetriesOnSocketTimeouts, toe);
} catch (IOException ie) {
if (updateAddress()) {
@@ -680,7 +681,7 @@ public class Client {
socket = null;
}
- /* Handle connection failures
+ /* Handle connection failures due to timeout on connect
*
* If the current number of retries is equal to the max number of retries,
* stop retrying and throw the exception; Otherwise backoff 1 second and
@@ -694,7 +695,7 @@ public class Client {
* @param ioe failure reason
* @throws IOException if max number of retries is reached
*/
- private void handleConnectionFailure(
+ private void handleConnectionTimeout(
int curRetries, int maxRetries, IOException ioe) throws IOException {
closeConnection();
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ConnectTimeoutException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ConnectTimeoutException.java
new file mode 100644
index 00000000000..12d7d175906
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ConnectTimeoutException.java
@@ -0,0 +1,37 @@
+/**
+ * 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.net;
+
+import java.net.SocketTimeoutException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown by {@link NetUtils#connect(java.net.Socket, java.net.SocketAddress, int)}
+ * if it times out while connecting to the remote host.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class ConnectTimeoutException extends SocketTimeoutException {
+ private static final long serialVersionUID = 1L;
+
+ public ConnectTimeoutException(String msg) {
+ super(msg);
+ }
+}
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 29d8af9fd7f..7ff9030f94a 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.net;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
+import java.lang.reflect.Constructor;
import java.net.BindException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
@@ -517,11 +518,15 @@ public class NetUtils {
socket.bind(localAddr);
}
- if (ch == null) {
- // let the default implementation handle it.
- socket.connect(endpoint, timeout);
- } else {
- SocketIOWithTimeout.connect(ch, endpoint, timeout);
+ try {
+ if (ch == null) {
+ // let the default implementation handle it.
+ socket.connect(endpoint, timeout);
+ } else {
+ SocketIOWithTimeout.connect(ch, endpoint, timeout);
+ }
+ } catch (SocketTimeoutException ste) {
+ throw new ConnectTimeoutException(ste.getMessage());
}
// There is a very rare case allowed by the TCP specification, such that
@@ -719,7 +724,7 @@ public class NetUtils {
+ see("BindException"));
} else if (exception instanceof ConnectException) {
// connection refused; include the host:port in the error
- return (ConnectException) new ConnectException(
+ return wrapWithMessage(exception,
"Call From "
+ localHost
+ " to "
@@ -729,32 +734,28 @@ public class NetUtils {
+ " failed on connection exception: "
+ exception
+ ";"
- + see("ConnectionRefused"))
- .initCause(exception);
+ + see("ConnectionRefused"));
} else if (exception instanceof UnknownHostException) {
- return (UnknownHostException) new UnknownHostException(
+ return wrapWithMessage(exception,
"Invalid host name: "
+ getHostDetailsAsString(destHost, destPort, localHost)
+ exception
+ ";"
- + see("UnknownHost"))
- .initCause(exception);
+ + see("UnknownHost"));
} else if (exception instanceof SocketTimeoutException) {
- return (SocketTimeoutException) new SocketTimeoutException(
+ return wrapWithMessage(exception,
"Call From "
+ localHost + " to " + destHost + ":" + destPort
+ " failed on socket timeout exception: " + exception
+ ";"
- + see("SocketTimeout"))
- .initCause(exception);
+ + see("SocketTimeout"));
} else if (exception instanceof NoRouteToHostException) {
- return (NoRouteToHostException) new NoRouteToHostException(
+ return wrapWithMessage(exception,
"No Route to Host from "
+ localHost + " to " + destHost + ":" + destPort
+ " failed on socket timeout exception: " + exception
+ ";"
- + see("NoRouteToHost"))
- .initCause(exception);
+ + see("NoRouteToHost"));
}
else {
return (IOException) new IOException("Failed on local exception: "
@@ -769,6 +770,21 @@ public class NetUtils {
private static String see(final String entry) {
return FOR_MORE_DETAILS_SEE + HADOOP_WIKI + entry;
}
+
+ @SuppressWarnings("unchecked")
+ private static T wrapWithMessage(
+ T exception, String msg) {
+ Class extends Throwable> clazz = exception.getClass();
+ try {
+ Constructor extends Throwable> ctor = clazz.getConstructor(String.class);
+ Throwable t = ctor.newInstance(msg);
+ return (T)(t.initCause(exception));
+ } catch (Throwable e) {
+ LOG.warn("Unable to wrap exception of type " +
+ clazz + ": it has no (String) constructor", e);
+ return exception;
+ }
+ }
/**
* Get the host details as a string
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 b7f87e5ea89..7983013539c 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
@@ -301,17 +301,25 @@ public class UserGroupInformation {
private static String OS_LOGIN_MODULE_NAME;
private static Class extends Principal> OS_PRINCIPAL_CLASS;
+
private static final boolean windows =
System.getProperty("os.name").startsWith("Windows");
private static final boolean is64Bit =
System.getProperty("os.arch").contains("64");
+ private static final boolean ibmJava = System.getProperty("java.vendor").contains("IBM");
+ private static final boolean aix = System.getProperty("os.name").equals("AIX");
+
/* Return the OS login module class name */
private static String getOSLoginModuleName() {
- if (System.getProperty("java.vendor").contains("IBM")) {
- return windows ? (is64Bit
- ? "com.ibm.security.auth.module.Win64LoginModule"
- : "com.ibm.security.auth.module.NTLoginModule")
- : "com.ibm.security.auth.module.LinuxLoginModule";
+ if (ibmJava) {
+ if (windows) {
+ return is64Bit ? "com.ibm.security.auth.module.Win64LoginModule"
+ : "com.ibm.security.auth.module.NTLoginModule";
+ } else if (aix) {
+ return "com.ibm.security.auth.module.AIXLoginModule";
+ } else {
+ return "com.ibm.security.auth.module.LinuxLoginModule";
+ }
} else {
return windows ? "com.sun.security.auth.module.NTLoginModule"
: "com.sun.security.auth.module.UnixLoginModule";
@@ -323,11 +331,14 @@ public class UserGroupInformation {
private static Class extends Principal> getOsPrincipalClass() {
ClassLoader cl = ClassLoader.getSystemClassLoader();
try {
- if (System.getProperty("java.vendor").contains("IBM")) {
+ if (ibmJava) {
if (windows) {
return (Class extends Principal>) (is64Bit
? cl.loadClass("com.ibm.security.auth.UsernamePrincipal")
: cl.loadClass("com.ibm.security.auth.NTUserPrincipal"));
+ } else if (aix) {
+ return (Class extends Principal>)
+ cl.loadClass("com.ibm.security.auth.AIXPrincipal");
} else {
return (Class extends Principal>) (is64Bit
? cl.loadClass("com.ibm.security.auth.UsernamePrincipal")
@@ -418,12 +429,21 @@ public class UserGroupInformation {
private static final Map USER_KERBEROS_OPTIONS =
new HashMap();
static {
- USER_KERBEROS_OPTIONS.put("doNotPrompt", "true");
- USER_KERBEROS_OPTIONS.put("useTicketCache", "true");
- USER_KERBEROS_OPTIONS.put("renewTGT", "true");
+ if (ibmJava) {
+ USER_KERBEROS_OPTIONS.put("useDefaultCcache", "true");
+ } else {
+ USER_KERBEROS_OPTIONS.put("doNotPrompt", "true");
+ USER_KERBEROS_OPTIONS.put("useTicketCache", "true");
+ USER_KERBEROS_OPTIONS.put("renewTGT", "true");
+ }
String ticketCache = System.getenv("KRB5CCNAME");
if (ticketCache != null) {
- USER_KERBEROS_OPTIONS.put("ticketCache", ticketCache);
+ if (ibmJava) {
+ // The first value searched when "useDefaultCcache" is used.
+ System.setProperty("KRB5CCNAME", ticketCache);
+ } else {
+ USER_KERBEROS_OPTIONS.put("ticketCache", ticketCache);
+ }
}
USER_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
}
@@ -434,10 +454,14 @@ public class UserGroupInformation {
private static final Map KEYTAB_KERBEROS_OPTIONS =
new HashMap();
static {
- KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
- KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
- KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
- KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
+ if (ibmJava) {
+ KEYTAB_KERBEROS_OPTIONS.put("credsType", "both");
+ } else {
+ KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
+ KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
+ KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
+ KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
+ }
KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
}
private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
@@ -462,7 +486,12 @@ public class UserGroupInformation {
} else if (USER_KERBEROS_CONFIG_NAME.equals(appName)) {
return USER_KERBEROS_CONF;
} else if (KEYTAB_KERBEROS_CONFIG_NAME.equals(appName)) {
- KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
+ if (ibmJava) {
+ KEYTAB_KERBEROS_OPTIONS.put("useKeytab",
+ prependFileAuthority(keytabFile));
+ } else {
+ KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
+ }
KEYTAB_KERBEROS_OPTIONS.put("principal", keytabPrincipal);
return KEYTAB_KERBEROS_CONF;
}
@@ -470,6 +499,11 @@ public class UserGroupInformation {
}
}
+ private static String prependFileAuthority(String keytabPath) {
+ return keytabPath.startsWith("file://") ? keytabPath
+ : "file://" + keytabPath;
+ }
+
/**
* Represents a javax.security configuration that is created at runtime.
*/
@@ -666,6 +700,7 @@ public class UserGroupInformation {
}
loginUser.spawnAutoRenewalThreadForUserCreds();
} catch (LoginException le) {
+ LOG.debug("failure to login", le);
throw new IOException("failure to login", le);
}
if (LOG.isDebugEnabled()) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
index 1681d6dfe70..682322dc640 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.util;
+import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
@@ -39,7 +40,7 @@ import org.apache.hadoop.io.Text;
*/
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
-public class LineReader {
+public class LineReader implements Closeable {
private static final int DEFAULT_BUFFER_SIZE = 64 * 1024;
private int bufferSize = DEFAULT_BUFFER_SIZE;
private InputStream in;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
index cf1d58b4e23..898901e5053 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
@@ -23,8 +23,6 @@ import java.io.StringWriter;
import java.net.URI;
import java.net.URISyntaxException;
import java.text.DateFormat;
-import java.text.DecimalFormat;
-import java.text.NumberFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -34,12 +32,13 @@ import java.util.List;
import java.util.Locale;
import java.util.StringTokenizer;
-import com.google.common.net.InetAddresses;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.net.NetUtils;
+import com.google.common.net.InetAddresses;
+
/**
* General string utils
*/
@@ -52,13 +51,6 @@ public class StringUtils {
*/
public static final int SHUTDOWN_HOOK_PRIORITY = 0;
- private static final DecimalFormat decimalFormat;
- static {
- NumberFormat numberFormat = NumberFormat.getNumberInstance(Locale.ENGLISH);
- decimalFormat = (DecimalFormat) numberFormat;
- decimalFormat.applyPattern("#.##");
- }
-
/**
* Make a string representation of the exception.
* @param e The exception to stringify
@@ -87,50 +79,33 @@ public class StringUtils {
}
return fullHostname;
}
-
- private static DecimalFormat oneDecimal = new DecimalFormat("0.0");
/**
* Given an integer, return a string that is in an approximate, but human
* readable format.
- * It uses the bases 'k', 'm', and 'g' for 1024, 1024**2, and 1024**3.
* @param number the number to format
* @return a human readable form of the integer
+ *
+ * @deprecated use {@link TraditionalBinaryPrefix#long2String(long, String, int)}.
*/
+ @Deprecated
public static String humanReadableInt(long number) {
- long absNumber = Math.abs(number);
- double result = number;
- String suffix = "";
- if (absNumber < 1024) {
- // since no division has occurred, don't format with a decimal point
- return String.valueOf(number);
- } else if (absNumber < 1024 * 1024) {
- result = number / 1024.0;
- suffix = "k";
- } else if (absNumber < 1024 * 1024 * 1024) {
- result = number / (1024.0 * 1024);
- suffix = "m";
- } else {
- result = number / (1024.0 * 1024 * 1024);
- suffix = "g";
- }
- return oneDecimal.format(result) + suffix;
+ return TraditionalBinaryPrefix.long2String(number, "", 1);
}
-
+
+ /** The same as String.format(Locale.ENGLISH, format, objects). */
+ public static String format(final String format, final Object... objects) {
+ return String.format(Locale.ENGLISH, format, objects);
+ }
+
/**
* Format a percentage for presentation to the user.
- * @param done the percentage to format (0.0 to 1.0)
- * @param digits the number of digits past the decimal point
+ * @param fraction the percentage as a fraction, e.g. 0.1 = 10%
+ * @param decimalPlaces the number of decimal places
* @return a string representation of the percentage
*/
- public static String formatPercent(double done, int digits) {
- DecimalFormat percentFormat = new DecimalFormat("0.00%");
- double scale = Math.pow(10.0, digits+2);
- double rounded = Math.floor(done * scale);
- percentFormat.setDecimalSeparatorAlwaysShown(false);
- percentFormat.setMinimumFractionDigits(digits);
- percentFormat.setMaximumFractionDigits(digits);
- return percentFormat.format(rounded / scale);
+ public static String formatPercent(double fraction, int decimalPlaces) {
+ return format("%." + decimalPlaces + "f%%", fraction*100);
}
/**
@@ -165,7 +140,7 @@ public class StringUtils {
}
StringBuilder s = new StringBuilder();
for(int i = start; i < end; i++) {
- s.append(String.format("%02x", bytes[i]));
+ s.append(format("%02x", bytes[i]));
}
return s.toString();
}
@@ -630,18 +605,22 @@ public class StringUtils {
* TraditionalBinaryPrefix symbol are case insensitive.
*/
public static enum TraditionalBinaryPrefix {
- KILO(1024),
- MEGA(KILO.value << 10),
- GIGA(MEGA.value << 10),
- TERA(GIGA.value << 10),
- PETA(TERA.value << 10),
- EXA(PETA.value << 10);
+ KILO(10),
+ MEGA(KILO.bitShift + 10),
+ GIGA(MEGA.bitShift + 10),
+ TERA(GIGA.bitShift + 10),
+ PETA(TERA.bitShift + 10),
+ EXA (PETA.bitShift + 10);
public final long value;
public final char symbol;
+ public final int bitShift;
+ public final long bitMask;
- TraditionalBinaryPrefix(long value) {
- this.value = value;
+ private TraditionalBinaryPrefix(int bitShift) {
+ this.bitShift = bitShift;
+ this.value = 1L << bitShift;
+ this.bitMask = this.value - 1L;
this.symbol = toString().charAt(0);
}
@@ -692,8 +671,58 @@ public class StringUtils {
return num * prefix;
}
}
+
+ /**
+ * Convert a long integer to a string with traditional binary prefix.
+ *
+ * @param n the value to be converted
+ * @param unit The unit, e.g. "B" for bytes.
+ * @param decimalPlaces The number of decimal places.
+ * @return a string with traditional binary prefix.
+ */
+ public static String long2String(long n, String unit, int decimalPlaces) {
+ if (unit == null) {
+ unit = "";
+ }
+ //take care a special case
+ if (n == Long.MIN_VALUE) {
+ return "-8 " + EXA.symbol + unit;
+ }
+
+ final StringBuilder b = new StringBuilder();
+ //take care negative numbers
+ if (n < 0) {
+ b.append('-');
+ n = -n;
+ }
+ if (n < KILO.value) {
+ //no prefix
+ b.append(n);
+ return (unit.isEmpty()? b: b.append(" ").append(unit)).toString();
+ } else {
+ //find traditional binary prefix
+ int i = 0;
+ for(; i < values().length && n >= values()[i].value; i++);
+ TraditionalBinaryPrefix prefix = values()[i - 1];
+
+ if ((n & prefix.bitMask) == 0) {
+ //exact division
+ b.append(n >> prefix.bitShift);
+ } else {
+ final String format = "%." + decimalPlaces + "f";
+ String s = format(format, n/(double)prefix.value);
+ //check a special rounding up case
+ if (s.startsWith("1024")) {
+ prefix = values()[i];
+ s = format(format, n/(double)prefix.value);
+ }
+ b.append(s);
+ }
+ return b.append(' ').append(prefix.symbol).append(unit).toString();
+ }
+ }
}
-
+
/**
* Escapes HTML Special characters present in the string.
* @param string
@@ -731,32 +760,16 @@ public class StringUtils {
}
/**
- * Return an abbreviated English-language desc of the byte length
+ * @return a byte description of the given long interger value.
*/
public static String byteDesc(long len) {
- double val = 0.0;
- String ending = "";
- if (len < 1024 * 1024) {
- val = (1.0 * len) / 1024;
- ending = " KB";
- } else if (len < 1024 * 1024 * 1024) {
- val = (1.0 * len) / (1024 * 1024);
- ending = " MB";
- } else if (len < 1024L * 1024 * 1024 * 1024) {
- val = (1.0 * len) / (1024 * 1024 * 1024);
- ending = " GB";
- } else if (len < 1024L * 1024 * 1024 * 1024 * 1024) {
- val = (1.0 * len) / (1024L * 1024 * 1024 * 1024);
- ending = " TB";
- } else {
- val = (1.0 * len) / (1024L * 1024 * 1024 * 1024 * 1024);
- ending = " PB";
- }
- return limitDecimalTo2(val) + ending;
+ return TraditionalBinaryPrefix.long2String(len, "B", 2);
}
- public static synchronized String limitDecimalTo2(double d) {
- return decimalFormat.format(d);
+ /** @deprecated use StringUtils.format("%.2f", d). */
+ @Deprecated
+ public static String limitDecimalTo2(double d) {
+ return format("%.2f", d);
}
/**
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
index 5f28d7c3072..da69b303b37 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
@@ -325,6 +325,13 @@ Hadoop MapReduce Next Generation - Cluster Setup
| | | How long to keep aggregation logs before deleting them. -1 disables. |
| | | Be careful, set this too small and you will spam the name node. |
*-------------------------+-------------------------+------------------------+
+| <<>> | | |
+| | <-1> | |
+| | | Time between checks for aggregated log retention. If set to 0 or a |
+| | | negative value then the value is computed as one-tenth of the |
+| | | aggregated log retention time. |
+| | | Be careful, set this too small and you will spam the name node. |
+*-------------------------+-------------------------+------------------------+
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
index c6812a19304..92980776637 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
@@ -303,6 +303,46 @@ public class TestFsShellReturnCode {
}
}
+ @Test
+ public void testRmWithNonexistentGlob() throws Exception {
+ Configuration conf = new Configuration();
+ FsShell shell = new FsShell();
+ shell.setConf(conf);
+ final ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+ final PrintStream err = new PrintStream(bytes);
+ final PrintStream oldErr = System.err;
+ System.setErr(err);
+ final String results;
+ try {
+ int exit = shell.run(new String[]{"-rm", "nomatch*"});
+ assertEquals(1, exit);
+ results = bytes.toString();
+ assertTrue(results.contains("rm: `nomatch*': No such file or directory"));
+ } finally {
+ IOUtils.closeStream(err);
+ System.setErr(oldErr);
+ }
+ }
+
+ @Test
+ public void testRmForceWithNonexistentGlob() throws Exception {
+ Configuration conf = new Configuration();
+ FsShell shell = new FsShell();
+ shell.setConf(conf);
+ final ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+ final PrintStream err = new PrintStream(bytes);
+ final PrintStream oldErr = System.err;
+ System.setErr(err);
+ try {
+ int exit = shell.run(new String[]{"-rm", "-f", "nomatch*"});
+ assertEquals(0, exit);
+ assertTrue(bytes.toString().isEmpty());
+ } finally {
+ IOUtils.closeStream(err);
+ System.setErr(oldErr);
+ }
+ }
+
@Test
public void testInvalidDefaultFS() throws Exception {
// if default fs doesn't exist or is invalid, but the path provided in
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
index c11afbadd27..de7c3fd1745 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
@@ -28,6 +28,7 @@ import java.net.URI;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -46,8 +47,18 @@ public class TestHarFileSystemBasics {
private static final String ROOT_PATH = System.getProperty("test.build.data",
"build/test/data");
- private static final Path rootPath = new Path(
- new File(ROOT_PATH).getAbsolutePath() + "/localfs");
+ private static final Path rootPath;
+ static {
+ String root = new Path(new File(ROOT_PATH).getAbsolutePath(), "localfs")
+ .toUri().getPath();
+ // Strip drive specifier on Windows, which would make the HAR URI invalid and
+ // cause tests to fail.
+ if (Shell.WINDOWS) {
+ root = root.substring(root.indexOf(':') + 1);
+ }
+ rootPath = new Path(root);
+ }
+
// NB: .har suffix is necessary
private static final Path harPath = new Path(rootPath, "path1/path2/my.har");
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java
index aae013fd775..df519c84e8c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java
@@ -45,19 +45,39 @@ public class TestListFiles {
final protected static Configuration conf = new Configuration();
protected static FileSystem fs;
- final protected static Path TEST_DIR = getTestDir();
+ protected static Path TEST_DIR;
final private static int FILE_LEN = 10;
- final private static Path FILE1 = new Path(TEST_DIR, "file1");
- final private static Path DIR1 = new Path(TEST_DIR, "dir1");
- final private static Path FILE2 = new Path(DIR1, "file2");
- final private static Path FILE3 = new Path(DIR1, "file3");
+ private static Path FILE1;
+ private static Path DIR1;
+ private static Path FILE2;
+ private static Path FILE3;
+
+ static {
+ setTestPaths(new Path(
+ System.getProperty("test.build.data", "build/test/data/work-dir/localfs"),
+ "main_"));
+ }
protected static Path getTestDir() {
- return new Path(
- System.getProperty("test.build.data","build/test/data/work-dir/localfs"),
- "main_");
+ return TEST_DIR;
}
-
+
+ /**
+ * Sets the root testing directory and reinitializes any additional test paths
+ * that are under the root. This method is intended to be called from a
+ * subclass's @BeforeClass method if there is a need to override the testing
+ * directory.
+ *
+ * @param testDir Path root testing directory
+ */
+ protected static void setTestPaths(Path testDir) {
+ TEST_DIR = testDir;
+ FILE1 = new Path(TEST_DIR, "file1");
+ DIR1 = new Path(TEST_DIR, "dir1");
+ FILE2 = new Path(DIR1, "file2");
+ FILE3 = new Path(DIR1, "file3");
+ }
+
@BeforeClass
public static void testSetUp() throws Exception {
fs = FileSystem.getLocal(conf);
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
index cf1d2df340c..b843e9cf13e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
@@ -28,6 +28,7 @@ import java.io.*;
import static org.junit.Assert.*;
import static org.junit.Assume.assumeTrue;
+import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -38,8 +39,9 @@ public class TestLocalFileSystem {
private static final String TEST_ROOT_DIR
= System.getProperty("test.build.data","build/test/data") + "/work-dir/localfs";
+ private final File base = new File(TEST_ROOT_DIR);
private Configuration conf;
- private FileSystem fileSys;
+ private LocalFileSystem fileSys;
private void cleanupFile(FileSystem fs, Path name) throws IOException {
assertTrue(fs.exists(name));
@@ -53,6 +55,13 @@ public class TestLocalFileSystem {
fileSys = FileSystem.getLocal(conf);
fileSys.delete(new Path(TEST_ROOT_DIR), true);
}
+
+ @After
+ public void after() throws IOException {
+ base.setWritable(true);
+ FileUtil.fullyDelete(base);
+ assertTrue(!base.exists());
+ }
/**
* Test the capability of setting the working directory.
@@ -269,10 +278,83 @@ public class TestLocalFileSystem {
LocalFileSystem fs = FileSystem.getLocal(conf);
File colonFile = new File(TEST_ROOT_DIR, "foo:bar");
colonFile.mkdirs();
- colonFile.createNewFile();
FileStatus[] stats = fs.listStatus(new Path(TEST_ROOT_DIR));
assertEquals("Unexpected number of stats", 1, stats.length);
assertEquals("Bad path from stat", colonFile.getAbsolutePath(),
stats[0].getPath().toUri().getPath());
}
+
+ @Test
+ public void testReportChecksumFailure() throws IOException {
+ base.mkdirs();
+ assertTrue(base.exists() && base.isDirectory());
+
+ final File dir1 = new File(base, "dir1");
+ final File dir2 = new File(dir1, "dir2");
+ dir2.mkdirs();
+ assertTrue(dir2.exists() && dir2.canWrite());
+
+ final String dataFileName = "corruptedData";
+ final Path dataPath = new Path(new File(dir2, dataFileName).toURI());
+ final Path checksumPath = fileSys.getChecksumFile(dataPath);
+ final FSDataOutputStream fsdos = fileSys.create(dataPath);
+ try {
+ fsdos.writeUTF("foo");
+ } finally {
+ fsdos.close();
+ }
+ assertTrue(fileSys.pathToFile(dataPath).exists());
+ final long dataFileLength = fileSys.getFileStatus(dataPath).getLen();
+ assertTrue(dataFileLength > 0);
+
+ // check the the checksum file is created and not empty:
+ assertTrue(fileSys.pathToFile(checksumPath).exists());
+ final long checksumFileLength = fileSys.getFileStatus(checksumPath).getLen();
+ assertTrue(checksumFileLength > 0);
+
+ // this is a hack to force the #reportChecksumFailure() method to stop
+ // climbing up at the 'base' directory and use 'dir1/bad_files' as the
+ // corrupted files storage:
+ base.setWritable(false);
+
+ FSDataInputStream dataFsdis = fileSys.open(dataPath);
+ FSDataInputStream checksumFsdis = fileSys.open(checksumPath);
+
+ boolean retryIsNecessary = fileSys.reportChecksumFailure(dataPath, dataFsdis, 0, checksumFsdis, 0);
+ assertTrue(!retryIsNecessary);
+
+ // the data file should be moved:
+ assertTrue(!fileSys.pathToFile(dataPath).exists());
+ // the checksum file should be moved:
+ assertTrue(!fileSys.pathToFile(checksumPath).exists());
+
+ // check that the files exist in the new location where they were moved:
+ File[] dir1files = dir1.listFiles(new FileFilter() {
+ @Override
+ public boolean accept(File pathname) {
+ return pathname != null && !pathname.getName().equals("dir2");
+ }
+ });
+ assertTrue(dir1files != null);
+ assertTrue(dir1files.length == 1);
+ File badFilesDir = dir1files[0];
+
+ File[] badFiles = badFilesDir.listFiles();
+ assertTrue(badFiles != null);
+ assertTrue(badFiles.length == 2);
+ boolean dataFileFound = false;
+ boolean checksumFileFound = false;
+ for (File badFile: badFiles) {
+ if (badFile.getName().startsWith(dataFileName)) {
+ assertTrue(dataFileLength == badFile.length());
+ dataFileFound = true;
+ } else if (badFile.getName().contains(dataFileName + ".crc")) {
+ assertTrue(checksumFileLength == badFile.length());
+ checksumFileFound = true;
+ }
+ }
+ assertTrue(dataFileFound);
+ assertTrue(checksumFileFound);
+ }
+
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
index a00d38bfa63..44215278ca6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
@@ -88,4 +88,61 @@ public class TestBoundedByteArrayOutputStream extends TestCase {
assertTrue("Writing beyond limit did not throw an exception",
caughtException);
}
+
+
+ static class ResettableBoundedByteArrayOutputStream
+ extends BoundedByteArrayOutputStream {
+
+ public ResettableBoundedByteArrayOutputStream(int capacity) {
+ super(capacity);
+ }
+
+ public void resetBuffer(byte[] buf, int offset, int length) {
+ super.resetBuffer(buf, offset, length);
+ }
+
+ }
+
+ public void testResetBuffer() throws IOException {
+
+ ResettableBoundedByteArrayOutputStream stream =
+ new ResettableBoundedByteArrayOutputStream(SIZE);
+
+ // Write to the stream, get the data back and check for contents
+ stream.write(INPUT, 0, SIZE);
+ assertTrue("Array Contents Mismatch",
+ Arrays.equals(INPUT, stream.getBuffer()));
+
+ // Try writing beyond end of buffer. Should throw an exception
+ boolean caughtException = false;
+
+ try {
+ stream.write(INPUT[0]);
+ } catch (Exception e) {
+ caughtException = true;
+ }
+
+ assertTrue("Writing beyond limit did not throw an exception",
+ caughtException);
+
+ //Reset the stream and try, should succeed
+ byte[] newBuf = new byte[SIZE];
+ stream.resetBuffer(newBuf, 0, newBuf.length);
+ assertTrue("Limit did not get reset correctly",
+ (stream.getLimit() == SIZE));
+ stream.write(INPUT, 0, SIZE);
+ assertTrue("Array Contents Mismatch",
+ Arrays.equals(INPUT, stream.getBuffer()));
+
+ // Try writing one more byte, should fail
+ caughtException = false;
+ try {
+ stream.write(INPUT[0]);
+ } catch (Exception e) {
+ caughtException = true;
+ }
+ assertTrue("Writing beyond limit did not throw an exception",
+ caughtException);
+ }
+
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSortedMapWritable.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSortedMapWritable.java
index 927bfc1f42d..1fbfcad7627 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSortedMapWritable.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSortedMapWritable.java
@@ -17,15 +17,20 @@
*/
package org.apache.hadoop.io;
-import java.util.Map;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
-import junit.framework.TestCase;
+import java.util.Map;
+import org.junit.Test;
/**
* Tests SortedMapWritable
*/
-public class TestSortedMapWritable extends TestCase {
+public class TestSortedMapWritable {
/** the test */
+ @Test
@SuppressWarnings("unchecked")
public void testSortedMapWritable() {
Text[] keys = {
@@ -90,6 +95,7 @@ public class TestSortedMapWritable extends TestCase {
/**
* Test that number of "unknown" classes is propagated across multiple copies.
*/
+ @Test
@SuppressWarnings("deprecation")
public void testForeignClass() {
SortedMapWritable inMap = new SortedMapWritable();
@@ -99,4 +105,63 @@ public class TestSortedMapWritable extends TestCase {
SortedMapWritable copyOfCopy = new SortedMapWritable(outMap);
assertEquals(1, copyOfCopy.getNewClasses());
}
+
+ /**
+ * Tests if equal and hashCode method still hold the contract.
+ */
+ @Test
+ public void testEqualsAndHashCode() {
+ String failureReason;
+ SortedMapWritable mapA = new SortedMapWritable();
+ SortedMapWritable mapB = new SortedMapWritable();
+
+ // Sanity checks
+ failureReason = "SortedMapWritable couldn't be initialized. Got null reference";
+ assertNotNull(failureReason, mapA);
+ assertNotNull(failureReason, mapB);
+
+ // Basic null check
+ assertFalse("equals method returns true when passed null", mapA.equals(null));
+
+ // When entry set is empty, they should be equal
+ assertTrue("Two empty SortedMapWritables are no longer equal", mapA.equals(mapB));
+
+ // Setup
+ Text[] keys = {
+ new Text("key1"),
+ new Text("key2")
+ };
+
+ BytesWritable[] values = {
+ new BytesWritable("value1".getBytes()),
+ new BytesWritable("value2".getBytes())
+ };
+
+ mapA.put(keys[0], values[0]);
+ mapB.put(keys[1], values[1]);
+
+ // entrySets are different
+ failureReason = "Two SortedMapWritables with different data are now equal";
+ assertTrue(failureReason, mapA.hashCode() != mapB.hashCode());
+ assertTrue(failureReason, !mapA.equals(mapB));
+ assertTrue(failureReason, !mapB.equals(mapA));
+
+ mapA.put(keys[1], values[1]);
+ mapB.put(keys[0], values[0]);
+
+ // entrySets are now same
+ failureReason = "Two SortedMapWritables with same entry sets formed in different order are now different";
+ assertEquals(failureReason, mapA.hashCode(), mapB.hashCode());
+ assertTrue(failureReason, mapA.equals(mapB));
+ assertTrue(failureReason, mapB.equals(mapA));
+
+ // Let's check if entry sets of same keys but different values
+ mapA.put(keys[0], values[1]);
+ mapA.put(keys[1], values[0]);
+
+ failureReason = "Two SortedMapWritables with different content are now equal";
+ assertTrue(failureReason, mapA.hashCode() != mapB.hashCode());
+ assertTrue(failureReason, !mapA.equals(mapB));
+ assertTrue(failureReason, !mapB.equals(mapA));
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
index 5762b56b9a0..3847bfd0814 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.net.ConnectTimeoutException;
import org.apache.hadoop.net.NetUtils;
import java.util.Random;
@@ -586,7 +587,7 @@ public class TestIPC {
private void assertRetriesOnSocketTimeouts(Configuration conf,
int maxTimeoutRetries) throws IOException, InterruptedException {
SocketFactory mockFactory = Mockito.mock(SocketFactory.class);
- doThrow(new SocketTimeoutException()).when(mockFactory).createSocket();
+ doThrow(new ConnectTimeoutException("fake")).when(mockFactory).createSocket();
Client client = new Client(IntWritable.class, conf, mockFactory);
InetSocketAddress address = new InetSocketAddress("127.0.0.1", 9090);
try {
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/GetGroupsTestBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/GetGroupsTestBase.java
index f223f1b18e9..a31700778dc 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/GetGroupsTestBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/GetGroupsTestBase.java
@@ -108,7 +108,7 @@ public abstract class GetGroupsTestBase {
for (String group : user.getGroupNames()) {
expectedOutput += " " + group;
}
- return expectedOutput + "\n";
+ return expectedOutput + System.getProperty("line.separator");
}
private String runTool(Configuration conf, String[] args, boolean success)
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
index 8d2fa155e1f..3dcf8dd3979 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
@@ -18,6 +18,8 @@
package org.apache.hadoop.util;
+import static org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix.long2String;
+import static org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix.string2long;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
@@ -26,6 +28,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.test.UnitTestcaseTimeLimit;
+import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.junit.Test;
public class TestStringUtils extends UnitTestcaseTimeLimit {
@@ -134,45 +137,34 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
@Test
public void testTraditionalBinaryPrefix() throws Exception {
+ //test string2long(..)
String[] symbol = {"k", "m", "g", "t", "p", "e"};
long m = 1024;
for(String s : symbol) {
- assertEquals(0, StringUtils.TraditionalBinaryPrefix.string2long(0 + s));
- assertEquals(m, StringUtils.TraditionalBinaryPrefix.string2long(1 + s));
+ assertEquals(0, string2long(0 + s));
+ assertEquals(m, string2long(1 + s));
m *= 1024;
}
- assertEquals(0L, StringUtils.TraditionalBinaryPrefix.string2long("0"));
- assertEquals(1024L, StringUtils.TraditionalBinaryPrefix.string2long("1k"));
- assertEquals(-1024L, StringUtils.TraditionalBinaryPrefix.string2long("-1k"));
- assertEquals(1259520L,
- StringUtils.TraditionalBinaryPrefix.string2long("1230K"));
- assertEquals(-1259520L,
- StringUtils.TraditionalBinaryPrefix.string2long("-1230K"));
- assertEquals(104857600L,
- StringUtils.TraditionalBinaryPrefix.string2long("100m"));
- assertEquals(-104857600L,
- StringUtils.TraditionalBinaryPrefix.string2long("-100M"));
- assertEquals(956703965184L,
- StringUtils.TraditionalBinaryPrefix.string2long("891g"));
- assertEquals(-956703965184L,
- StringUtils.TraditionalBinaryPrefix.string2long("-891G"));
- assertEquals(501377302265856L,
- StringUtils.TraditionalBinaryPrefix.string2long("456t"));
- assertEquals(-501377302265856L,
- StringUtils.TraditionalBinaryPrefix.string2long("-456T"));
- assertEquals(11258999068426240L,
- StringUtils.TraditionalBinaryPrefix.string2long("10p"));
- assertEquals(-11258999068426240L,
- StringUtils.TraditionalBinaryPrefix.string2long("-10P"));
- assertEquals(1152921504606846976L,
- StringUtils.TraditionalBinaryPrefix.string2long("1e"));
- assertEquals(-1152921504606846976L,
- StringUtils.TraditionalBinaryPrefix.string2long("-1E"));
+ assertEquals(0L, string2long("0"));
+ assertEquals(1024L, string2long("1k"));
+ assertEquals(-1024L, string2long("-1k"));
+ assertEquals(1259520L, string2long("1230K"));
+ assertEquals(-1259520L, string2long("-1230K"));
+ assertEquals(104857600L, string2long("100m"));
+ assertEquals(-104857600L, string2long("-100M"));
+ assertEquals(956703965184L, string2long("891g"));
+ assertEquals(-956703965184L, string2long("-891G"));
+ assertEquals(501377302265856L, string2long("456t"));
+ assertEquals(-501377302265856L, string2long("-456T"));
+ assertEquals(11258999068426240L, string2long("10p"));
+ assertEquals(-11258999068426240L, string2long("-10P"));
+ assertEquals(1152921504606846976L, string2long("1e"));
+ assertEquals(-1152921504606846976L, string2long("-1E"));
String tooLargeNumStr = "10e";
try {
- StringUtils.TraditionalBinaryPrefix.string2long(tooLargeNumStr);
+ string2long(tooLargeNumStr);
fail("Test passed for a number " + tooLargeNumStr + " too large");
} catch (IllegalArgumentException e) {
assertEquals(tooLargeNumStr + " does not fit in a Long", e.getMessage());
@@ -180,7 +172,7 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
String tooSmallNumStr = "-10e";
try {
- StringUtils.TraditionalBinaryPrefix.string2long(tooSmallNumStr);
+ string2long(tooSmallNumStr);
fail("Test passed for a number " + tooSmallNumStr + " too small");
} catch (IllegalArgumentException e) {
assertEquals(tooSmallNumStr + " does not fit in a Long", e.getMessage());
@@ -189,7 +181,7 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
String invalidFormatNumStr = "10kb";
char invalidPrefix = 'b';
try {
- StringUtils.TraditionalBinaryPrefix.string2long(invalidFormatNumStr);
+ string2long(invalidFormatNumStr);
fail("Test passed for a number " + invalidFormatNumStr
+ " has invalid format");
} catch (IllegalArgumentException e) {
@@ -199,6 +191,74 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
e.getMessage());
}
+ //test long2string(..)
+ assertEquals("0", long2String(0, null, 2));
+ for(int decimalPlace = 0; decimalPlace < 2; decimalPlace++) {
+ for(int n = 1; n < TraditionalBinaryPrefix.KILO.value; n++) {
+ assertEquals(n + "", long2String(n, null, decimalPlace));
+ assertEquals(-n + "", long2String(-n, null, decimalPlace));
+ }
+ assertEquals("1 K", long2String(1L << 10, null, decimalPlace));
+ assertEquals("-1 K", long2String(-1L << 10, null, decimalPlace));
+ }
+
+ assertEquals("8.00 E", long2String(Long.MAX_VALUE, null, 2));
+ assertEquals("8.00 E", long2String(Long.MAX_VALUE - 1, null, 2));
+ assertEquals("-8 E", long2String(Long.MIN_VALUE, null, 2));
+ assertEquals("-8.00 E", long2String(Long.MIN_VALUE + 1, null, 2));
+
+ final String[] zeros = {" ", ".0 ", ".00 "};
+ for(int decimalPlace = 0; decimalPlace < zeros.length; decimalPlace++) {
+ final String trailingZeros = zeros[decimalPlace];
+
+ for(int e = 11; e < Long.SIZE - 1; e++) {
+ final TraditionalBinaryPrefix p
+ = TraditionalBinaryPrefix.values()[e/10 - 1];
+
+ { // n = 2^e
+ final long n = 1L << e;
+ final String expected = (n/p.value) + " " + p.symbol;
+ assertEquals("n=" + n, expected, long2String(n, null, 2));
+ }
+
+ { // n = 2^e + 1
+ final long n = (1L << e) + 1;
+ final String expected = (n/p.value) + trailingZeros + p.symbol;
+ assertEquals("n=" + n, expected, long2String(n, null, decimalPlace));
+ }
+
+ { // n = 2^e - 1
+ final long n = (1L << e) - 1;
+ final String expected = ((n+1)/p.value) + trailingZeros + p.symbol;
+ assertEquals("n=" + n, expected, long2String(n, null, decimalPlace));
+ }
+ }
+ }
+
+ assertEquals("1.50 K", long2String(3L << 9, null, 2));
+ assertEquals("1.5 K", long2String(3L << 9, null, 1));
+ assertEquals("1.50 M", long2String(3L << 19, null, 2));
+ assertEquals("2 M", long2String(3L << 19, null, 0));
+ assertEquals("3 G", long2String(3L << 30, null, 2));
+
+ // test byteDesc(..)
+ assertEquals("0 B", StringUtils.byteDesc(0));
+ assertEquals("-100 B", StringUtils.byteDesc(-100));
+ assertEquals("1 KB", StringUtils.byteDesc(1024));
+ assertEquals("1.50 KB", StringUtils.byteDesc(3L << 9));
+ assertEquals("1.50 MB", StringUtils.byteDesc(3L << 19));
+ assertEquals("3 GB", StringUtils.byteDesc(3L << 30));
+
+ // test formatPercent(..)
+ assertEquals("10%", StringUtils.formatPercent(0.1, 0));
+ assertEquals("10.0%", StringUtils.formatPercent(0.1, 1));
+ assertEquals("10.00%", StringUtils.formatPercent(0.1, 2));
+
+ assertEquals("1%", StringUtils.formatPercent(0.00543, 0));
+ assertEquals("0.5%", StringUtils.formatPercent(0.00543, 1));
+ assertEquals("0.54%", StringUtils.formatPercent(0.00543, 2));
+ assertEquals("0.543%", StringUtils.formatPercent(0.00543, 3));
+ assertEquals("0.5430%", StringUtils.formatPercent(0.00543, 4));
}
@Test
@@ -314,10 +374,9 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
}
long et = System.nanoTime();
if (outer > 3) {
- System.out.println(
- (useOurs ? "StringUtils impl" : "Java impl") +
- " #" + outer + ":" +
- (et - st)/1000000 + "ms");
+ System.out.println( (useOurs ? "StringUtils impl" : "Java impl")
+ + " #" + outer + ":" + (et - st)/1000000 + "ms, components="
+ + components );
}
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
index c64cfd0879c..345b58a462b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.fs.http.client;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary;
@@ -86,6 +88,7 @@ public class HttpFSFileSystem extends FileSystem
public static final String PERMISSION_PARAM = "permission";
public static final String DESTINATION_PARAM = "destination";
public static final String RECURSIVE_PARAM = "recursive";
+ public static final String SOURCES_PARAM = "sources";
public static final String OWNER_PARAM = "owner";
public static final String GROUP_PARAM = "group";
public static final String MODIFICATION_TIME_PARAM = "modificationtime";
@@ -167,7 +170,7 @@ public class HttpFSFileSystem extends FileSystem
GETHOMEDIRECTORY(HTTP_GET), GETCONTENTSUMMARY(HTTP_GET),
GETFILECHECKSUM(HTTP_GET), GETFILEBLOCKLOCATIONS(HTTP_GET),
INSTRUMENTATION(HTTP_GET),
- APPEND(HTTP_POST),
+ APPEND(HTTP_POST), CONCAT(HTTP_POST),
CREATE(HTTP_PUT), MKDIRS(HTTP_PUT), RENAME(HTTP_PUT), SETOWNER(HTTP_PUT),
SETPERMISSION(HTTP_PUT), SETREPLICATION(HTTP_PUT), SETTIMES(HTTP_PUT),
DELETE(HTTP_DELETE);
@@ -528,6 +531,29 @@ public class HttpFSFileSystem extends FileSystem
HttpURLConnection.HTTP_OK);
}
+ /**
+ * Concat existing files together.
+ * @param f the path to the target destination.
+ * @param psrcs the paths to the sources to use for the concatenation.
+ *
+ * @throws IOException
+ */
+ @Override
+ public void concat(Path f, Path[] psrcs) throws IOException {
+ List strPaths = new ArrayList(psrcs.length);
+ for(Path psrc : psrcs) {
+ strPaths.add(psrc.toUri().getPath());
+ }
+ String srcs = StringUtils.join(",", strPaths);
+
+ Map params = new HashMap();
+ params.put(OP_PARAM, Operation.CONCAT.toString());
+ params.put(SOURCES_PARAM, srcs);
+ HttpURLConnection conn = getConnection(Operation.CONCAT.getMethod(),
+ params, f, true);
+ HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+ }
+
/**
* Renames Path src to Path dst. Can take place on local fs
* or remote DFS.
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
index f81e90e0642..8e41d04e443 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
@@ -198,6 +198,47 @@ public class FSOperations {
}
+ /**
+ * Executor that performs an append FileSystemAccess files system operation.
+ */
+ @InterfaceAudience.Private
+ public static class FSConcat implements FileSystemAccess.FileSystemExecutor {
+ private Path path;
+ private Path[] sources;
+
+ /**
+ * Creates a Concat executor.
+ *
+ * @param path target path to concat to.
+ * @param sources comma seperated absolute paths to use as sources.
+ */
+ public FSConcat(String path, String[] sources) {
+ this.sources = new Path[sources.length];
+
+ for(int i = 0; i < sources.length; i++) {
+ this.sources[i] = new Path(sources[i]);
+ }
+
+ this.path = new Path(path);
+ }
+
+ /**
+ * Executes the filesystem operation.
+ *
+ * @param fs filesystem instance to use.
+ *
+ * @return void.
+ *
+ * @throws IOException thrown if an IO error occured.
+ */
+ @Override
+ public Void execute(FileSystem fs) throws IOException {
+ fs.concat(path, sources);
+ return null;
+ }
+
+ }
+
/**
* Executor that performs a content-summary FileSystemAccess files system operation.
*/
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
index b2a28053da7..d217322b6ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
@@ -58,6 +58,7 @@ public class HttpFSParametersProvider extends ParametersProvider {
PARAMS_DEF.put(Operation.INSTRUMENTATION, new Class[]{DoAsParam.class});
PARAMS_DEF.put(Operation.APPEND,
new Class[]{DoAsParam.class, DataParam.class});
+ PARAMS_DEF.put(Operation.CONCAT, new Class[]{SourcesParam.class});
PARAMS_DEF.put(Operation.CREATE,
new Class[]{DoAsParam.class, PermissionParam.class, OverwriteParam.class,
ReplicationParam.class, BlockSizeParam.class, DataParam.class});
@@ -388,6 +389,25 @@ public class HttpFSParametersProvider extends ParametersProvider {
}
}
+ /**
+ * Class for concat sources parameter.
+ */
+ @InterfaceAudience.Private
+ public static class SourcesParam extends StringParam {
+
+ /**
+ * Parameter name.
+ */
+ public static final String NAME = HttpFSFileSystem.SOURCES_PARAM;
+
+ /**
+ * Constructor.
+ */
+ public SourcesParam() {
+ super(NAME, null);
+ }
+ }
+
/**
* Class for to-path parameter.
*/
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
index 0c3418f5c40..ca7edcc7acc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
@@ -22,22 +22,23 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
-import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OperationParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AccessTimeParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.BlockSizeParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DataParam;
-import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.RecursiveParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DestinationParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DoAsParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.FilterParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.GroupParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.LenParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ModifiedTimeParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OffsetParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OperationParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OverwriteParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OwnerParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.PermissionParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.RecursiveParam;
import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ReplicationParam;
-import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DestinationParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.SourcesParam;
import org.apache.hadoop.lib.service.FileSystemAccess;
import org.apache.hadoop.lib.service.FileSystemAccessException;
import org.apache.hadoop.lib.service.Groups;
@@ -403,9 +404,9 @@ public class HttpFSServer {
Response response;
path = makeAbsolute(path);
MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
- String doAs = params.get(DoAsParam.NAME, DoAsParam.class);
switch (op.value()) {
case APPEND: {
+ String doAs = params.get(DoAsParam.NAME, DoAsParam.class);
Boolean hasData = params.get(DataParam.NAME, DataParam.class);
if (!hasData) {
response = Response.temporaryRedirect(
@@ -420,6 +421,18 @@ public class HttpFSServer {
}
break;
}
+ case CONCAT: {
+ System.out.println("HTTPFS SERVER CONCAT");
+ String sources = params.get(SourcesParam.NAME, SourcesParam.class);
+
+ FSOperations.FSConcat command =
+ new FSOperations.FSConcat(path, sources.split(","));
+ fsExecute(user, null, command);
+ AUDIT_LOG.info("[{}]", path);
+ System.out.println("SENT RESPONSE");
+ response = Response.ok().build();
+ break;
+ }
default: {
throw new IOException(
MessageFormat.format("Invalid HTTP POST operation [{0}]",
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
index d44bcc44943..3d96fd83264 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.HFSTestCase;
import org.apache.hadoop.test.HadoopUsersConfTestHelper;
@@ -206,6 +208,30 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
}
}
+ private void testConcat() throws Exception {
+ Configuration config = getProxiedFSConf();
+ config.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);
+ if (!isLocalFS()) {
+ FileSystem fs = FileSystem.get(config);
+ fs.mkdirs(getProxiedFSTestDir());
+ Path path1 = new Path("/test/foo.txt");
+ Path path2 = new Path("/test/bar.txt");
+ Path path3 = new Path("/test/derp.txt");
+ DFSTestUtil.createFile(fs, path1, 1024, (short) 3, 0);
+ DFSTestUtil.createFile(fs, path2, 1024, (short) 3, 0);
+ DFSTestUtil.createFile(fs, path3, 1024, (short) 3, 0);
+ fs.close();
+ fs = getHttpFSFileSystem();
+ fs.concat(path1, new Path[]{path2, path3});
+ fs.close();
+ fs = FileSystem.get(config);
+ Assert.assertTrue(fs.exists(path1));
+ Assert.assertFalse(fs.exists(path2));
+ Assert.assertFalse(fs.exists(path3));
+ fs.close();
+ }
+ }
+
private void testRename() throws Exception {
FileSystem fs = FileSystem.get(getProxiedFSConf());
Path path = new Path(getProxiedFSTestDir(), "foo");
@@ -450,7 +476,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
}
protected enum Operation {
- GET, OPEN, CREATE, APPEND, RENAME, DELETE, LIST_STATUS, WORKING_DIRECTORY, MKDIRS,
+ GET, OPEN, CREATE, APPEND, CONCAT, RENAME, DELETE, LIST_STATUS, WORKING_DIRECTORY, MKDIRS,
SET_TIMES, SET_PERMISSION, SET_OWNER, SET_REPLICATION, CHECKSUM, CONTENT_SUMMARY
}
@@ -468,6 +494,8 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
case APPEND:
testAppend();
break;
+ case CONCAT:
+ testConcat();
case RENAME:
testRename();
break;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7008d74ce22..79872e1557f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -296,7 +296,28 @@ Trunk (Unreleased)
HDFS-4382. Fix typo MAX_NOT_CHANGED_INTERATIONS. (Ted Yu via suresh)
-Release 2.0.3-alpha - Unreleased
+ HDFS-4340. Update addBlock() to inculde inode id as additional argument.
+ (Brandon Li via suresh)
+
+Release 2.0.4-beta - UNRELEASED
+
+ INCOMPATIBLE CHANGES
+
+ NEW FEATURES
+
+ IMPROVEMENTS
+
+ OPTIMIZATIONS
+
+ BUG FIXES
+
+ HDFS-4470. Several HDFS tests attempt file operations on invalid HDFS
+ paths when running on Windows. (Chris Nauroth via suresh)
+
+ HDFS-4471. Namenode WebUI file browsing does not work with wildcard
+ addresses configured. (Andrew Wang via atm)
+
+Release 2.0.3-alpha - 2013-02-06
INCOMPATIBLE CHANGES
@@ -317,6 +338,9 @@ Release 2.0.3-alpha - Unreleased
HDFS-4451. hdfs balancer command returns exit code 1 on success instead
of 0. (Joshua Blatt via suresh)
+ HDFS-4350. Make enabling of stale marking on read and write paths
+ independent. (Andrew Wang via suresh)
+
NEW FEATURES
@@ -505,6 +529,8 @@ Release 2.0.3-alpha - Unreleased
HDFS-3598. WebHDFS support for file concat. (Plamen Jeliazkov via shv)
+ HDFS-4456. Add concat to HttpFS and WebHDFS REST API docs. (plamenj2003 via tucu)
+
OPTIMIZATIONS
HDFS-3429. DataNode reads checksums even if client does not need them (todd)
@@ -742,6 +768,28 @@ Release 2.0.3-alpha - Unreleased
HDFS-4428. FsDatasetImpl should disclose what the error is when a rename
fails. (Colin Patrick McCabe via atm)
+ HDFS-4452. getAdditionalBlock() can create multiple blocks if the client
+ times out and retries. (shv)
+
+ HDFS-4445. All BKJM ledgers are not checked while tailing, So failover will fail.
+ (Vinay via umamahesh)
+
+ HDFS-4462. 2NN will fail to checkpoint after an HDFS upgrade from a
+ pre-federation version of HDFS. (atm)
+
+ HDFS-4404. Create file failure when the machine of first attempted NameNode
+ is down. (Todd Lipcon via atm)
+
+ HDFS-4344. dfshealth.jsp throws NumberFormatException when
+ dfs.hosts/dfs.hosts.exclude includes port number. (Andy Isaacson via atm)
+
+ HDFS-4468. Use the new StringUtils methods added by HADOOP-9252 and fix
+ TestHDFSCLI and TestQuota. (szetszwo)
+
+ HDFS-4458. In DFSUtil.getNameServiceUris(..), convert default fs URI using
+ NetUtils.createSocketAddr(..) for being consistent with other addresses.
+ (Binglin Chang via szetszwo)
+
BREAKDOWN OF HDFS-3077 SUBTASKS
HDFS-3077. Quorum-based protocol for reading and writing edit logs.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
index 5d1814233f8..2baf4dc0747 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
@@ -503,7 +503,8 @@ public class BookKeeperJournalManager implements JournalManager {
@Override
public void selectInputStreams(Collection streams,
long fromTxId, boolean inProgressOk) throws IOException {
- List currentLedgerList = getLedgerList(inProgressOk);
+ List currentLedgerList = getLedgerList(fromTxId,
+ inProgressOk);
try {
BookKeeperEditLogInputStream elis = null;
for (EditLogLedgerMetadata l : currentLedgerList) {
@@ -511,6 +512,8 @@ public class BookKeeperJournalManager implements JournalManager {
if (l.isInProgress()) {
lastTxId = recoverLastTxId(l, false);
}
+ // Check once again, required in case of InProgress and is case of any
+ // gap.
if (fromTxId >= l.getFirstTxId() && fromTxId <= lastTxId) {
LedgerHandle h;
if (l.isInProgress()) { // we don't want to fence the current journal
@@ -523,6 +526,8 @@ public class BookKeeperJournalManager implements JournalManager {
elis = new BookKeeperEditLogInputStream(h, l);
elis.skipTo(fromTxId);
} else {
+ // If mismatches then there might be some gap, so we should not check
+ // further.
return;
}
streams.add(elis);
@@ -732,6 +737,11 @@ public class BookKeeperJournalManager implements JournalManager {
*/
List getLedgerList(boolean inProgressOk)
throws IOException {
+ return getLedgerList(-1, inProgressOk);
+ }
+
+ private List getLedgerList(long fromTxId,
+ boolean inProgressOk) throws IOException {
List ledgers
= new ArrayList();
try {
@@ -744,6 +754,12 @@ public class BookKeeperJournalManager implements JournalManager {
try {
EditLogLedgerMetadata editLogLedgerMetadata = EditLogLedgerMetadata
.read(zkc, legderMetadataPath);
+ if (editLogLedgerMetadata.getLastTxId() != HdfsConstants.INVALID_TXID
+ && editLogLedgerMetadata.getLastTxId() < fromTxId) {
+ // exclude already read closed edits, but include inprogress edits
+ // as this will be handled in caller
+ continue;
+ }
ledgers.add(editLogLedgerMetadata);
} catch (KeeperException.NoNodeException e) {
LOG.warn("ZNode: " + legderMetadataPath
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java
index ebbf80aa375..0a14e785758 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.*;
import org.junit.Test;
import org.junit.Before;
-import org.junit.After;
import org.junit.BeforeClass;
import org.junit.AfterClass;
@@ -34,11 +33,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogTestUtil;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.ipc.RemoteException;
@@ -352,4 +349,42 @@ public class TestBookKeeperAsHASharedDir {
}
}
}
+
+ /**
+ * NameNode should load the edits correctly if the applicable edits are
+ * present in the BKJM.
+ */
+ @Test
+ public void testNameNodeMultipleSwitchesUsingBKJM() throws Exception {
+ MiniDFSCluster cluster = null;
+ try {
+ Configuration conf = new Configuration();
+ conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+ conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
+ .createJournalURI("/correctEditLogSelection").toString());
+ BKJMUtil.addJournalManagerDefinition(conf);
+
+ cluster = new MiniDFSCluster.Builder(conf)
+ .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
+ .manageNameDfsSharedDirs(false).build();
+ NameNode nn1 = cluster.getNameNode(0);
+ NameNode nn2 = cluster.getNameNode(1);
+ cluster.waitActive();
+ cluster.transitionToActive(0);
+ nn1.getRpcServer().rollEditLog(); // Roll Edits from current Active.
+ // Transition to standby current active gracefully.
+ cluster.transitionToStandby(0);
+ // Make the other Active and Roll edits multiple times
+ cluster.transitionToActive(1);
+ nn2.getRpcServer().rollEditLog();
+ nn2.getRpcServer().rollEditLog();
+ // Now One more failover. So NN1 should be able to failover successfully.
+ cluster.transitionToStandby(1);
+ cluster.transitionToActive(0);
+ } finally {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index c62e9f7fc26..24cdba7e1bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -181,10 +181,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY = "dfs.datanode.socket.reuse.keepalive";
public static final int DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT = 1000;
- // Whether to enable datanode's stale state detection and usage
- public static final String DFS_NAMENODE_CHECK_STALE_DATANODE_KEY = "dfs.namenode.check.stale.datanode";
- public static final boolean DFS_NAMENODE_CHECK_STALE_DATANODE_DEFAULT = false;
- // Whether to enable datanode's stale state detection and usage
+ // Whether to enable datanode's stale state detection and usage for reads
+ public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
+ public static final boolean DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT = false;
+ // Whether to enable datanode's stale state detection and usage for writes
public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY = "dfs.namenode.avoid.write.stale.datanode";
public static final boolean DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_DEFAULT = false;
// The default value of the time interval for marking datanodes as stale
@@ -195,8 +195,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_STALE_DATANODE_MINIMUM_INTERVAL_KEY = "dfs.namenode.stale.datanode.minimum.interval";
public static final int DFS_NAMENODE_STALE_DATANODE_MINIMUM_INTERVAL_DEFAULT = 3; // i.e. min_interval is 3 * heartbeat_interval = 9s
- // When the number stale datanodes marked as stale reached this certian ratio,
- // stop avoiding writing to stale nodes so as to prevent causing hotspots.
+ // When the percentage of stale datanodes reaches this ratio,
+ // allow writing to stale nodes to prevent hotspots.
public static final String DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY = "dfs.namenode.write.stale.datanode.ratio";
public static final float DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_DEFAULT = 0.5f;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 7fc8428318c..660a09cbe68 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -116,6 +116,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
private volatile boolean closed = false;
private String src;
+ private final long fileId;
private final long blockSize;
private final DataChecksum checksum;
// both dataQueue and ackQueue are protected by dataQueue lock
@@ -1149,7 +1150,8 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
long localstart = Time.now();
while (true) {
try {
- return dfsClient.namenode.addBlock(src, dfsClient.clientName, block, excludedNodes);
+ return dfsClient.namenode.addBlock(src, dfsClient.clientName,
+ block, excludedNodes, fileId);
} catch (RemoteException e) {
IOException ue =
e.unwrapRemoteException(FileNotFoundException.class,
@@ -1262,20 +1264,21 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
return value;
}
- private DFSOutputStream(DFSClient dfsClient, String src, long blockSize, Progressable progress,
- DataChecksum checksum, short replication) throws IOException {
+ private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
+ HdfsFileStatus stat, DataChecksum checksum) throws IOException {
super(checksum, checksum.getBytesPerChecksum(), checksum.getChecksumSize());
- int bytesPerChecksum = checksum.getBytesPerChecksum();
this.dfsClient = dfsClient;
this.src = src;
- this.blockSize = blockSize;
- this.blockReplication = replication;
+ this.fileId = stat.getFileId();
+ this.blockSize = stat.getBlockSize();
+ this.blockReplication = stat.getReplication();
this.progress = progress;
if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug(
"Set non-null progress callback on DFSOutputStream " + src);
}
+ final int bytesPerChecksum = checksum.getBytesPerChecksum();
if ( bytesPerChecksum < 1 || blockSize % bytesPerChecksum != 0) {
throw new IOException("io.bytes.per.checksum(" + bytesPerChecksum +
") and blockSize(" + blockSize +
@@ -1287,19 +1290,27 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
}
/** Construct a new output stream for creating a file. */
- private DFSOutputStream(DFSClient dfsClient, String src, FsPermission masked,
- EnumSet flag, boolean createParent, short replication,
- long blockSize, Progressable progress, int buffersize,
+ private DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
+ EnumSet flag, Progressable progress,
DataChecksum checksum) throws IOException {
- this(dfsClient, src, blockSize, progress, checksum, replication);
+ this(dfsClient, src, progress, stat, checksum);
this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
computePacketChunkSize(dfsClient.getConf().writePacketSize,
checksum.getBytesPerChecksum());
+ streamer = new DataStreamer();
+ }
+
+ static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
+ FsPermission masked, EnumSet flag, boolean createParent,
+ short replication, long blockSize, Progressable progress, int buffersize,
+ DataChecksum checksum) throws IOException {
+ final HdfsFileStatus stat;
try {
- dfsClient.namenode.create(
- src, masked, dfsClient.clientName, new EnumSetWritable(flag), createParent, replication, blockSize);
+ stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
+ new EnumSetWritable(flag), createParent, replication,
+ blockSize);
} catch(RemoteException re) {
throw re.unwrapRemoteException(AccessControlException.class,
DSQuotaExceededException.class,
@@ -1311,30 +1322,20 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
UnresolvedPathException.class,
SnapshotAccessControlException.class);
}
- streamer = new DataStreamer();
- }
-
- static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
- FsPermission masked, EnumSet flag, boolean createParent,
- short replication, long blockSize, Progressable progress, int buffersize,
- DataChecksum checksum) throws IOException {
- final DFSOutputStream out = new DFSOutputStream(dfsClient, src, masked,
- flag, createParent, replication, blockSize, progress, buffersize,
- checksum);
- out.streamer.start();
+ final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
+ flag, progress, checksum);
+ out.start();
return out;
}
/** Construct a new output stream for append. */
- private DFSOutputStream(DFSClient dfsClient, String src, int buffersize, Progressable progress,
- LocatedBlock lastBlock, HdfsFileStatus stat,
+ private DFSOutputStream(DFSClient dfsClient, String src,
+ Progressable progress, LocatedBlock lastBlock, HdfsFileStatus stat,
DataChecksum checksum) throws IOException {
- this(dfsClient, src, stat.getBlockSize(), progress, checksum, stat.getReplication());
+ this(dfsClient, src, progress, stat, checksum);
initialFileSize = stat.getLen(); // length of file when opened
- //
// The last partial block of the file has to be filled.
- //
if (lastBlock != null) {
// indicate that we are appending to an existing block
bytesCurBlock = lastBlock.getBlockSize();
@@ -1349,9 +1350,9 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
int buffersize, Progressable progress, LocatedBlock lastBlock,
HdfsFileStatus stat, DataChecksum checksum) throws IOException {
- final DFSOutputStream out = new DFSOutputStream(dfsClient, src, buffersize,
+ final DFSOutputStream out = new DFSOutputStream(dfsClient, src,
progress, lastBlock, stat, checksum);
- out.streamer.start();
+ out.start();
return out;
}
@@ -1718,6 +1719,10 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
isClosed();
}
+ private synchronized void start() {
+ streamer.start();
+ }
+
/**
* Aborts this output stream and releases any system
* resources associated with this stream.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 5eac3bde40d..a6e938c8fc8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -134,7 +134,7 @@ public class DFSUtil {
/**
* Comparator for sorting DataNodeInfo[] based on decommissioned/stale states.
* Decommissioned/stale nodes are moved to the end of the array on sorting
- * with this compartor.
+ * with this comparator.
*/
@InterfaceAudience.Private
public static class DecomStaleComparator implements Comparator {
@@ -144,7 +144,7 @@ public class DFSUtil {
* Constructor of DecomStaleComparator
*
* @param interval
- * The time invertal for marking datanodes as stale is passed from
+ * The time interval for marking datanodes as stale is passed from
* outside, since the interval may be changed dynamically
*/
public DecomStaleComparator(long interval) {
@@ -780,6 +780,13 @@ public class DFSUtil {
// Add the default URI if it is an HDFS URI.
URI defaultUri = FileSystem.getDefaultUri(conf);
+ // checks if defaultUri is ip:port format
+ // and convert it to hostname:port format
+ if (defaultUri != null && (defaultUri.getPort() != -1)) {
+ defaultUri = createUri(defaultUri.getScheme(),
+ NetUtils.createSocketAddr(defaultUri.getHost(),
+ defaultUri.getPort()));
+ }
if (defaultUri != null &&
HdfsConstants.HDFS_URI_SCHEME.equals(defaultUri.getScheme()) &&
!nonPreferredUris.contains(defaultUri)) {
@@ -939,6 +946,11 @@ public class DFSUtil {
return capacity <= 0 ? 0 : (remaining * 100.0f)/capacity;
}
+ /** Convert percentage to a string. */
+ public static String percent2String(double percentage) {
+ return StringUtils.format("%.2f%%", percentage);
+ }
+
/**
* Round bytes to GiB (gibibyte)
* @param bytes number of bytes
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index e426fc2a89e..5e308e6acae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -151,6 +151,8 @@ public interface ClientProtocol {
* @param replication block replication factor.
* @param blockSize maximum block size.
*
+ * @return the status of the created file, it could be null if the server
+ * doesn't support returning the file status
* @throws AccessControlException If access is denied
* @throws AlreadyBeingCreatedException if the path does not exist.
* @throws DSQuotaExceededException If file creation violates disk space
@@ -170,13 +172,14 @@ public interface ClientProtocol {
* RuntimeExceptions:
* @throws InvalidPathException Path src
is invalid
*/
- public void create(String src, FsPermission masked, String clientName,
- EnumSetWritable flag, boolean createParent,
- short replication, long blockSize) throws AccessControlException,
- AlreadyBeingCreatedException, DSQuotaExceededException,
- FileAlreadyExistsException, FileNotFoundException,
- NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
- UnresolvedLinkException, SnapshotAccessControlException, IOException;
+ public HdfsFileStatus create(String src, FsPermission masked,
+ String clientName, EnumSetWritable flag,
+ boolean createParent, short replication, long blockSize)
+ throws AccessControlException, AlreadyBeingCreatedException,
+ DSQuotaExceededException, FileAlreadyExistsException,
+ FileNotFoundException, NSQuotaExceededException,
+ ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+ SnapshotAccessControlException, IOException;
/**
* Append to the end of the file.
@@ -302,6 +305,7 @@ public interface ClientProtocol {
* @param previous previous block
* @param excludeNodes a list of nodes that should not be
* allocated for the current block
+ * @param fileId the id uniquely identifying a file
*
* @return LocatedBlock allocated block information.
*
@@ -316,7 +320,7 @@ public interface ClientProtocol {
*/
@Idempotent
public LocatedBlock addBlock(String src, String clientName,
- ExtendedBlock previous, DatanodeInfo[] excludeNodes)
+ ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId)
throws AccessControlException, FileNotFoundException,
NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
IOException;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java
index c7b22f7ac24..481c1305e8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.protocol;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.util.StringUtils;
+import static org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix.long2String;
@InterfaceAudience.Private
@InterfaceStability.Evolving
@@ -41,9 +41,9 @@ public class DSQuotaExceededException extends QuotaExceededException {
public String getMessage() {
String msg = super.getMessage();
if (msg == null) {
- return "The DiskSpace quota" + (pathName==null?"":(" of " + pathName)) +
- " is exceeded: quota=" + StringUtils.humanReadableInt(quota) +
- " diskspace consumed=" + StringUtils.humanReadableInt(count);
+ return "The DiskSpace quota" + (pathName==null?"": " of " + pathName)
+ + " is exceeded: quota = " + quota + " B = " + long2String(quota, "B", 2)
+ + " but diskspace consumed = " + count + " B = " + long2String(count, "B", 2);
} else {
return msg;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
index cf7438a7ed5..df46cd6a0c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
@@ -17,10 +17,13 @@
*/
package org.apache.hadoop.hdfs.protocol;
+import static org.apache.hadoop.hdfs.DFSUtil.percent2String;
+
import java.util.Date;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NetworkTopology;
@@ -244,8 +247,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
buffer.append("DFS Used: "+u+" ("+StringUtils.byteDesc(u)+")"+"\n");
buffer.append("Non DFS Used: "+nonDFSUsed+" ("+StringUtils.byteDesc(nonDFSUsed)+")"+"\n");
buffer.append("DFS Remaining: " +r+ " ("+StringUtils.byteDesc(r)+")"+"\n");
- buffer.append("DFS Used%: "+StringUtils.limitDecimalTo2(usedPercent)+"%\n");
- buffer.append("DFS Remaining%: "+StringUtils.limitDecimalTo2(remainingPercent)+"%\n");
+ buffer.append("DFS Used%: "+percent2String(usedPercent) + "\n");
+ buffer.append("DFS Remaining%: "+percent2String(remainingPercent) + "\n");
buffer.append("Last contact: "+new Date(lastUpdate)+"\n");
return buffer.toString();
}
@@ -269,7 +272,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
}
buffer.append(" " + c + "(" + StringUtils.byteDesc(c)+")");
buffer.append(" " + u + "(" + StringUtils.byteDesc(u)+")");
- buffer.append(" " + StringUtils.limitDecimalTo2(((1.0*u)/c)*100)+"%");
+ buffer.append(" " + percent2String(u/(double)c));
buffer.append(" " + r + "(" + StringUtils.byteDesc(r)+")");
buffer.append(" " + new Date(lastUpdate));
return buffer.toString();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
index 26b6e984d53..7d1bd32ba82 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
@@ -40,6 +40,7 @@ public class HdfsFileStatus {
private FsPermission permission;
private String owner;
private String group;
+ private long fileId;
public static final byte[] EMPTY_NAME = new byte[0];
@@ -55,11 +56,12 @@ public class HdfsFileStatus {
* @param owner the owner of the path
* @param group the group of the path
* @param path the local name in java UTF8 encoding the same as that in-memory
+ * @param fileId the file id
*/
public HdfsFileStatus(long length, boolean isdir, int block_replication,
long blocksize, long modification_time, long access_time,
FsPermission permission, String owner, String group,
- byte[] symlink, byte[] path) {
+ byte[] symlink, byte[] path, long fileId) {
this.length = length;
this.isdir = isdir;
this.block_replication = (short)block_replication;
@@ -75,6 +77,7 @@ public class HdfsFileStatus {
this.group = (group == null) ? "" : group;
this.symlink = symlink;
this.path = path;
+ this.fileId = fileId;
}
/**
@@ -223,4 +226,8 @@ public class HdfsFileStatus {
final public byte[] getSymlinkInBytes() {
return symlink;
}
+
+ final public long getFileId() {
+ return fileId;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
index 167b66b1e16..987ba42091c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
@@ -44,19 +44,19 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
* @param group group
* @param symlink symbolic link
* @param path local path name in java UTF8 format
+ * @param fileId the file id
* @param locations block locations
*/
public HdfsLocatedFileStatus(long length, boolean isdir,
- int block_replication,
- long blocksize, long modification_time, long access_time,
- FsPermission permission, String owner, String group,
- byte[] symlink, byte[] path, LocatedBlocks locations) {
- super(length, isdir, block_replication, blocksize, modification_time,
- access_time, permission, owner, group, symlink, path);
+ int block_replication, long blocksize, long modification_time,
+ long access_time, FsPermission permission, String owner, String group,
+ byte[] symlink, byte[] path, long fileId, LocatedBlocks locations) {
+ super(length, isdir, block_replication, blocksize, modification_time,
+ access_time, permission, owner, group, symlink, path, fileId);
this.locations = locations;
- }
+ }
- public LocatedBlocks getBlockLocations() {
- return locations;
- }
+ public LocatedBlocks getBlockLocations() {
+ return locations;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
index b4f7f46b052..17710da8f60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
@@ -40,8 +40,9 @@ public class SnapshottableDirectoryStatus {
public SnapshottableDirectoryStatus(long modification_time, long access_time,
FsPermission permission, String owner, String group, byte[] localName,
int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
+//TODO: fix fileId
this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
- access_time, permission, owner, group, null, localName);
+ access_time, permission, owner, group, null, localName, 0L);
this.snapshotNumber = snapshotNumber;
this.snapshotQuota = snapshotQuota;
this.parentFullPath = parentFullPath;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index d780e711a80..201a5ceba2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -297,14 +297,19 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
public CreateResponseProto create(RpcController controller,
CreateRequestProto req) throws ServiceException {
try {
- server.create(req.getSrc(), PBHelper.convert(req.getMasked()),
- req.getClientName(), PBHelper.convert(req.getCreateFlag()),
- req.getCreateParent(), (short) req.getReplication(),
- req.getBlockSize());
+ HdfsFileStatus result = server.create(req.getSrc(),
+ PBHelper.convert(req.getMasked()), req.getClientName(),
+ PBHelper.convert(req.getCreateFlag()), req.getCreateParent(),
+ (short) req.getReplication(), req.getBlockSize());
+
+ if (result != null) {
+ return CreateResponseProto.newBuilder().setFs(PBHelper.convert(result))
+ .build();
+ }
+ return VOID_CREATE_RESPONSE;
} catch (IOException e) {
throw new ServiceException(e);
}
- return VOID_CREATE_RESPONSE;
}
@Override
@@ -377,13 +382,14 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
try {
List excl = req.getExcludeNodesList();
- LocatedBlock result = server.addBlock(req.getSrc(), req.getClientName(),
+ LocatedBlock result = server.addBlock(
+ req.getSrc(),
+ req.getClientName(),
req.hasPrevious() ? PBHelper.convert(req.getPrevious()) : null,
- (excl == null ||
- excl.size() == 0) ? null :
- PBHelper.convert(excl.toArray(new DatanodeInfoProto[excl.size()])));
- return AddBlockResponseProto.newBuilder().setBlock(
- PBHelper.convert(result)).build();
+ (excl == null || excl.size() == 0) ? null : PBHelper.convert(excl
+ .toArray(new DatanodeInfoProto[excl.size()])), req.getFileId());
+ return AddBlockResponseProto.newBuilder()
+ .setBlock(PBHelper.convert(result)).build();
} catch (IOException e) {
throw new ServiceException(e);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 6219a70d7d7..ffb19bea872 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Comple
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto;
@@ -110,6 +111,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.io.EnumSetWritable;
@@ -204,13 +206,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
- public void create(String src, FsPermission masked, String clientName,
- EnumSetWritable flag, boolean createParent,
- short replication, long blockSize) throws AccessControlException,
- AlreadyBeingCreatedException, DSQuotaExceededException,
- FileAlreadyExistsException, FileNotFoundException,
- NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
- UnresolvedLinkException, IOException {
+ public HdfsFileStatus create(String src, FsPermission masked,
+ String clientName, EnumSetWritable flag,
+ boolean createParent, short replication, long blockSize)
+ throws AccessControlException, AlreadyBeingCreatedException,
+ DSQuotaExceededException, FileAlreadyExistsException,
+ FileNotFoundException, NSQuotaExceededException,
+ ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+ IOException {
CreateRequestProto req = CreateRequestProto.newBuilder()
.setSrc(src)
.setMasked(PBHelper.convert(masked))
@@ -221,7 +224,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
.setBlockSize(blockSize)
.build();
try {
- rpcProxy.create(null, req);
+ CreateResponseProto res = rpcProxy.create(null, req);
+ return res.hasFs() ? PBHelper.convert(res.getFs()) : null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@@ -305,15 +309,15 @@ public class ClientNamenodeProtocolTranslatorPB implements
throw ProtobufHelper.getRemoteException(e);
}
}
-
+
@Override
public LocatedBlock addBlock(String src, String clientName,
- ExtendedBlock previous, DatanodeInfo[] excludeNodes)
+ ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId)
throws AccessControlException, FileNotFoundException,
NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
IOException {
- AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder().setSrc(src)
- .setClientName(clientName);
+ AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
+ .setSrc(src).setClientName(clientName).setFileId(fileId);
if (previous != null)
req.setPrevious(PBHelper.convert(previous));
if (excludeNodes != null)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 6329984da96..f85a2c49886 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -113,6 +113,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
@@ -1052,6 +1053,7 @@ public class PBHelper {
fs.getFileType().equals(FileType.IS_SYMLINK) ?
fs.getSymlink().toByteArray() : null,
fs.getPath().toByteArray(),
+ fs.hasFileId()? fs.getFileId(): INodeId.GRANDFATHER_INODE_ID,
fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null);
}
@@ -1091,6 +1093,7 @@ public class PBHelper {
setPermission(PBHelper.convert(fs.getPermission())).
setOwner(fs.getOwner()).
setGroup(fs.getGroup()).
+ setFileId(fs.getFileId()).
setPath(ByteString.copyFrom(fs.getLocalNameInBytes()));
if (fs.isSymlink()) {
builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index e0a81e0d407..e4b9ffb8dd8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -126,15 +126,26 @@ public class DatanodeManager {
private final long heartbeatExpireInterval;
/** Ask Datanode only up to this many blocks to delete. */
final int blockInvalidateLimit;
-
- /** Whether or not to check stale DataNodes for read/write */
- private final boolean checkForStaleDataNodes;
/** The interval for judging stale DataNodes for read/write */
private final long staleInterval;
- /** Whether or not to avoid using stale DataNodes for writing */
- private volatile boolean avoidStaleDataNodesForWrite;
+ /** Whether or not to avoid using stale DataNodes for reading */
+ private final boolean avoidStaleDataNodesForRead;
+
+ /**
+ * Whether or not to avoid using stale DataNodes for writing.
+ * Note that, even if this is configured, the policy may be
+ * temporarily disabled when a high percentage of the nodes
+ * are marked as stale.
+ */
+ private final boolean avoidStaleDataNodesForWrite;
+
+ /**
+ * When the ratio of stale datanodes reaches this number, stop avoiding
+ * writing to stale datanodes, i.e., continue using stale nodes for writing.
+ */
+ private final float ratioUseStaleDataNodesForWrite;
/** The number of stale DataNodes */
private volatile int numStaleNodes;
@@ -183,14 +194,23 @@ public class DatanodeManager {
DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY, blockInvalidateLimit);
LOG.info(DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY
+ "=" + this.blockInvalidateLimit);
-
- checkForStaleDataNodes = conf.getBoolean(
- DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY,
- DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_DEFAULT);
- staleInterval = getStaleIntervalFromConf(conf, heartbeatExpireInterval);
- avoidStaleDataNodesForWrite = getAvoidStaleForWriteFromConf(conf,
- checkForStaleDataNodes);
+ this.avoidStaleDataNodesForRead = conf.getBoolean(
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY,
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT);
+ this.avoidStaleDataNodesForWrite = conf.getBoolean(
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY,
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_DEFAULT);
+ this.staleInterval = getStaleIntervalFromConf(conf, heartbeatExpireInterval);
+ this.ratioUseStaleDataNodesForWrite = conf.getFloat(
+ DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY,
+ DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_DEFAULT);
+ Preconditions.checkArgument(
+ (ratioUseStaleDataNodesForWrite > 0 &&
+ ratioUseStaleDataNodesForWrite <= 1.0f),
+ DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY +
+ " = '" + ratioUseStaleDataNodesForWrite + "' is invalid. " +
+ "It should be a positive non-zero float value, not greater than 1.0f.");
}
private static long getStaleIntervalFromConf(Configuration conf,
@@ -230,22 +250,6 @@ public class DatanodeManager {
return staleInterval;
}
- static boolean getAvoidStaleForWriteFromConf(Configuration conf,
- boolean checkForStale) {
- boolean avoid = conf.getBoolean(
- DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY,
- DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_DEFAULT);
- boolean avoidStaleDataNodesForWrite = checkForStale && avoid;
- if (!checkForStale && avoid) {
- LOG.warn("Cannot set "
- + DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY
- + " as false while setting "
- + DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY
- + " as true.");
- }
- return avoidStaleDataNodesForWrite;
- }
-
void activate(final Configuration conf) {
final DecommissionManager dm = new DecommissionManager(namesystem, blockManager);
this.decommissionthread = new Daemon(dm.new Monitor(
@@ -299,7 +303,7 @@ public class DatanodeManager {
client = new NodeBase(rName + NodeBase.PATH_SEPARATOR_STR + targethost);
}
- Comparator comparator = checkForStaleDataNodes ?
+ Comparator comparator = avoidStaleDataNodesForRead ?
new DFSUtil.DecomStaleComparator(staleInterval) :
DFSUtil.DECOM_COMPARATOR;
@@ -825,32 +829,20 @@ public class DatanodeManager {
}
/* Getter and Setter for stale DataNodes related attributes */
-
- /**
- * @return whether or not to avoid writing to stale datanodes
- */
- public boolean isAvoidingStaleDataNodesForWrite() {
- return avoidStaleDataNodesForWrite;
- }
/**
- * Set the value of {@link DatanodeManager#avoidStaleDataNodesForWrite}.
- * The HeartbeatManager disable avoidStaleDataNodesForWrite when more than
- * half of the DataNodes are marked as stale.
+ * Whether stale datanodes should be avoided as targets on the write path.
+ * The result of this function may change if the number of stale datanodes
+ * eclipses a configurable threshold.
*
- * @param avoidStaleDataNodesForWrite
- * The value to set to
- * {@link DatanodeManager#avoidStaleDataNodesForWrite}
+ * @return whether stale datanodes should be avoided on the write path
*/
- void setAvoidStaleDataNodesForWrite(boolean avoidStaleDataNodesForWrite) {
- this.avoidStaleDataNodesForWrite = avoidStaleDataNodesForWrite;
- }
-
- /**
- * @return Whether or not to check stale DataNodes for R/W
- */
- boolean isCheckingForStaleDataNodes() {
- return checkForStaleDataNodes;
+ public boolean shouldAvoidStaleDataNodesForWrite() {
+ // If # stale exceeds maximum staleness ratio, disable stale
+ // datanode avoidance on the write path
+ return avoidStaleDataNodesForWrite &&
+ (numStaleNodes <= heartbeatManager.getLiveDatanodeCount()
+ * ratioUseStaleDataNodesForWrite);
}
/**
@@ -967,7 +959,7 @@ public class DatanodeManager {
port = DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT;
} else {
hostStr = hostLine.substring(0, idx);
- port = Integer.valueOf(hostLine.substring(idx));
+ port = Integer.valueOf(hostLine.substring(idx+1));
}
if (InetAddresses.isInetAddress(hostStr)) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
index 6ee65d38c79..a033da36fbc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
@@ -30,8 +30,6 @@ import org.apache.hadoop.hdfs.server.namenode.Namesystem;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
-import com.google.common.base.Preconditions;
-
/**
* Manage the heartbeats received from datanodes.
* The datanode list and statistics are synchronized
@@ -56,16 +54,7 @@ class HeartbeatManager implements DatanodeStatistics {
private final long heartbeatRecheckInterval;
/** Heartbeat monitor thread */
private final Daemon heartbeatThread = new Daemon(new Monitor());
- /**
- * The initial setting of end user which indicates whether or not to avoid
- * writing to stale datanodes.
- */
- private final boolean initialAvoidWriteStaleNodes;
- /**
- * When the ratio of stale datanodes reaches this number, stop avoiding
- * writing to stale datanodes, i.e., continue using stale nodes for writing.
- */
- private final float ratioUseStaleDataNodesForWrite;
+
final Namesystem namesystem;
final BlockManager blockManager;
@@ -74,30 +63,25 @@ class HeartbeatManager implements DatanodeStatistics {
final BlockManager blockManager, final Configuration conf) {
this.namesystem = namesystem;
this.blockManager = blockManager;
- boolean checkStaleNodes = conf.getBoolean(
- DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY,
- DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_DEFAULT);
+ boolean avoidStaleDataNodesForWrite = conf.getBoolean(
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY,
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_DEFAULT);
long recheckInterval = conf.getInt(
- DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
+ DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT); // 5 min
long staleInterval = conf.getLong(
- DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
+ DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);// 30s
- this.initialAvoidWriteStaleNodes = DatanodeManager
- .getAvoidStaleForWriteFromConf(conf, checkStaleNodes);
- this.ratioUseStaleDataNodesForWrite = conf.getFloat(
- DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY,
- DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_DEFAULT);
- Preconditions.checkArgument(
- (ratioUseStaleDataNodesForWrite > 0 &&
- ratioUseStaleDataNodesForWrite <= 1.0f),
- DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY +
- " = '" + ratioUseStaleDataNodesForWrite + "' is invalid. " +
- "It should be a positive non-zero float value, not greater than 1.0f.");
-
- this.heartbeatRecheckInterval = (checkStaleNodes
- && initialAvoidWriteStaleNodes
- && staleInterval < recheckInterval) ? staleInterval : recheckInterval;
+
+ if (avoidStaleDataNodesForWrite && staleInterval < recheckInterval) {
+ this.heartbeatRecheckInterval = staleInterval;
+ LOG.info("Setting heartbeat recheck interval to " + staleInterval
+ + " since " + DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY
+ + " is less than "
+ + DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY);
+ } else {
+ this.heartbeatRecheckInterval = recheckInterval;
+ }
}
void activate(Configuration conf) {
@@ -242,7 +226,6 @@ class HeartbeatManager implements DatanodeStatistics {
if (namesystem.isInSafeMode()) {
return;
}
- boolean checkStaleNodes = dm.isCheckingForStaleDataNodes();
boolean allAlive = false;
while (!allAlive) {
// locate the first dead node.
@@ -254,29 +237,14 @@ class HeartbeatManager implements DatanodeStatistics {
if (dead == null && dm.isDatanodeDead(d)) {
stats.incrExpiredHeartbeats();
dead = d;
- if (!checkStaleNodes) {
- break;
- }
}
- if (checkStaleNodes &&
- d.isStale(dm.getStaleInterval())) {
+ if (d.isStale(dm.getStaleInterval())) {
numOfStaleNodes++;
}
}
- // Change whether to avoid using stale datanodes for writing
- // based on proportion of stale datanodes
- if (checkStaleNodes) {
- dm.setNumStaleNodes(numOfStaleNodes);
- if (numOfStaleNodes >
- datanodes.size() * ratioUseStaleDataNodesForWrite) {
- dm.setAvoidStaleDataNodesForWrite(false);
- } else {
- if (this.initialAvoidWriteStaleNodes) {
- dm.setAvoidStaleDataNodesForWrite(true);
- }
- }
- }
+ // Set the number of stale nodes in the DatanodeManager
+ dm.setNumStaleNodes(numOfStaleNodes);
}
allAlive = dead == null;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
index fc69978fb22..acb4ce79588 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
@@ -905,7 +905,7 @@ public abstract class Storage extends StorageInfo {
props.setProperty("storageType", storageType.toString());
props.setProperty("namespaceID", String.valueOf(namespaceID));
// Set clusterID in version with federation support
- if (LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+ if (versionSupportsFederation()) {
props.setProperty("clusterID", clusterID);
}
props.setProperty("cTime", String.valueOf(cTime));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
index c3dd5d6209b..1dc83405303 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
@@ -18,6 +18,8 @@
package org.apache.hadoop.hdfs.server.common;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import com.google.common.base.Joiner;
@@ -77,6 +79,10 @@ public class StorageInfo {
namespaceID = from.namespaceID;
cTime = from.cTime;
}
+
+ public boolean versionSupportsFederation() {
+ return LayoutVersion.supports(Feature.FEDERATION, layoutVersion);
+ }
@Override
public String toString() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
index 8c4d79cc9bf..13d8f85cbfe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
@@ -123,6 +123,10 @@ public class CheckpointSignature extends StorageInfo
blockpoolID.equals(si.getBlockPoolID());
}
+ boolean namespaceIdMatches(FSImage si) {
+ return namespaceID == si.getStorage().namespaceID;
+ }
+
void validateStorageInfo(FSImage si) throws IOException {
if (!isSameCluster(si)
|| !storageVersionMatches(si.getStorage())) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
index 1b3db818d15..0f0a989f8c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
@@ -569,12 +569,10 @@ class ClusterJspHelper {
toXmlItemBlock(doc, "DFS Remaining", StringUtils.byteDesc(free));
// dfsUsedPercent
- toXmlItemBlock(doc, "DFS Used%",
- StringUtils.limitDecimalTo2(dfsUsedPercent)+ "%");
+ toXmlItemBlock(doc, "DFS Used%", DFSUtil.percent2String(dfsUsedPercent));
// dfsRemainingPercent
- toXmlItemBlock(doc, "DFS Remaining%",
- StringUtils.limitDecimalTo2(dfsRemainingPercent) + "%");
+ toXmlItemBlock(doc, "DFS Remaining%", DFSUtil.percent2String(dfsRemainingPercent));
doc.endTag(); // storage
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index c8d3f589012..93c7d74a46a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -2157,7 +2157,8 @@ public class FSDirectory implements Closeable {
node.getUserName(snapshot),
node.getGroupName(snapshot),
node.isSymlink() ? ((INodeSymlink)node).getSymlink() : null,
- path);
+ path,
+ node.getId());
}
/**
@@ -2194,6 +2195,7 @@ public class FSDirectory implements Closeable {
node.getGroupName(snapshot),
node.isSymlink() ? ((INodeSymlink)node).getSymlink() : null,
path,
+ node.getId(),
loc);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 0bba9bbaf39..1d4c6ac82a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1791,16 +1791,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* Create a new file entry in the namespace.
*
* For description of parameters and exceptions thrown see
- * {@link ClientProtocol#create()}
+ * {@link ClientProtocol#create()}, except it returns valid file status
+ * upon success
*/
- void startFile(String src, PermissionStatus permissions, String holder,
- String clientMachine, EnumSet flag, boolean createParent,
- short replication, long blockSize) throws AccessControlException,
- SafeModeException, FileAlreadyExistsException, UnresolvedLinkException,
+ HdfsFileStatus startFile(String src, PermissionStatus permissions,
+ String holder, String clientMachine, EnumSet flag,
+ boolean createParent, short replication, long blockSize)
+ throws AccessControlException, SafeModeException,
+ FileAlreadyExistsException, UnresolvedLinkException,
FileNotFoundException, ParentNotDirectoryException, IOException {
try {
- startFileInt(src, permissions, holder, clientMachine, flag, createParent,
- replication, blockSize);
+ return startFileInt(src, permissions, holder, clientMachine, flag,
+ createParent, replication, blockSize);
} catch (AccessControlException e) {
if (isAuditEnabled() && isExternalInvocation()) {
logAuditEvent(false, UserGroupInformation.getCurrentUser(),
@@ -1811,18 +1813,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
}
- private void startFileInt(String src, PermissionStatus permissions, String holder,
- String clientMachine, EnumSet flag, boolean createParent,
- short replication, long blockSize) throws AccessControlException,
- SafeModeException, FileAlreadyExistsException, UnresolvedLinkException,
+ private HdfsFileStatus startFileInt(String src, PermissionStatus permissions,
+ String holder, String clientMachine, EnumSet flag,
+ boolean createParent, short replication, long blockSize)
+ throws AccessControlException, SafeModeException,
+ FileAlreadyExistsException, UnresolvedLinkException,
FileNotFoundException, ParentNotDirectoryException, IOException {
boolean skipSync = false;
+ final HdfsFileStatus stat;
writeLock();
try {
checkOperation(OperationCategory.WRITE);
startFileInternal(src, permissions, holder, clientMachine, flag,
createParent, replication, blockSize);
+ stat = dir.getFileInfo(src, false);
} catch (StandbyException se) {
skipSync = true;
throw se;
@@ -1836,11 +1841,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
if (isAuditEnabled() && isExternalInvocation()) {
- final HdfsFileStatus stat = dir.getFileInfo(src, false);
logAuditEvent(UserGroupInformation.getCurrentUser(),
getRemoteIp(),
"create", src, null, stat);
}
+ return stat;
}
/**
@@ -2207,20 +2212,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* are replicated. Will return an empty 2-elt array if we want the
* client to "try again later".
*/
- LocatedBlock getAdditionalBlock(String src,
- String clientName,
- ExtendedBlock previous,
- HashMap excludedNodes
- )
+ LocatedBlock getAdditionalBlock(String src, long fileId, String clientName,
+ ExtendedBlock previous, HashMap excludedNodes)
throws LeaseExpiredException, NotReplicatedYetException,
QuotaExceededException, SafeModeException, UnresolvedLinkException,
IOException {
- checkBlock(previous);
- Block previousBlock = ExtendedBlock.getLocalBlock(previous);
- long fileLength, blockSize;
+ long blockSize;
int replication;
DatanodeDescriptor clientNode = null;
- Block newBlock = null;
if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug(
@@ -2228,118 +2227,61 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
+src+" for "+clientName);
}
- writeLock();
+ // Part I. Analyze the state of the file with respect to the input data.
+ readLock();
try {
- checkOperation(OperationCategory.WRITE);
+ LocatedBlock[] onRetryBlock = new LocatedBlock[1];
+ final INode[] inodes = analyzeFileState(
+ src, fileId, clientName, previous, onRetryBlock).getINodes();
+ final INodeFileUnderConstruction pendingFile =
+ (INodeFileUnderConstruction) inodes[inodes.length - 1];
- if (isInSafeMode()) {
- throw new SafeModeException("Cannot add block to " + src, safeMode);
+ if(onRetryBlock[0] != null) {
+ // This is a retry. Just return the last block.
+ return onRetryBlock[0];
}
- // have we exceeded the configured limit of fs objects.
- checkFsObjectLimit();
-
- INodeFileUnderConstruction pendingFile = checkLease(src, clientName);
- BlockInfo lastBlockInFile = pendingFile.getLastBlock();
- if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
- // The block that the client claims is the current last block
- // doesn't match up with what we think is the last block. There are
- // three possibilities:
- // 1) This is the first block allocation of an append() pipeline
- // which started appending exactly at a block boundary.
- // In this case, the client isn't passed the previous block,
- // so it makes the allocateBlock() call with previous=null.
- // We can distinguish this since the last block of the file
- // will be exactly a full block.
- // 2) This is a retry from a client that missed the response of a
- // prior getAdditionalBlock() call, perhaps because of a network
- // timeout, or because of an HA failover. In that case, we know
- // by the fact that the client is re-issuing the RPC that it
- // never began to write to the old block. Hence it is safe to
- // abandon it and allocate a new one.
- // 3) This is an entirely bogus request/bug -- we should error out
- // rather than potentially appending a new block with an empty
- // one in the middle, etc
-
- BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
- if (previous == null &&
- lastBlockInFile != null &&
- lastBlockInFile.getNumBytes() == pendingFile.getPreferredBlockSize() &&
- lastBlockInFile.isComplete()) {
- // Case 1
- if (NameNode.stateChangeLog.isDebugEnabled()) {
- NameNode.stateChangeLog.debug(
- "BLOCK* NameSystem.allocateBlock: handling block allocation" +
- " writing to a file with a complete previous block: src=" +
- src + " lastBlock=" + lastBlockInFile);
- }
- } else if (Block.matchingIdAndGenStamp(penultimateBlock, previousBlock)) {
- // Case 2
- if (lastBlockInFile.getNumBytes() != 0) {
- throw new IOException(
- "Request looked like a retry to allocate block " +
- lastBlockInFile + " but it already contains " +
- lastBlockInFile.getNumBytes() + " bytes");
- }
-
- // The retry case ("b" above) -- abandon the old block.
- NameNode.stateChangeLog.info("BLOCK* allocateBlock: " +
- "caught retry for allocation of a new block in " +
- src + ". Abandoning old block " + lastBlockInFile);
- dir.removeBlock(src, pendingFile, lastBlockInFile);
- dir.persistBlocks(src, pendingFile);
- } else {
-
- throw new IOException("Cannot allocate block in " + src + ": " +
- "passed 'previous' block " + previous + " does not match actual " +
- "last block in file " + lastBlockInFile);
- }
- }
-
- // commit the last block and complete it if it has minimum replicas
- commitOrCompleteLastBlock(pendingFile, previousBlock);
-
- //
- // If we fail this, bad things happen!
- //
- if (!checkFileProgress(pendingFile, false)) {
- throw new NotReplicatedYetException("Not replicated yet:" + src);
- }
- fileLength = pendingFile.computeContentSummary().getLength();
blockSize = pendingFile.getPreferredBlockSize();
clientNode = pendingFile.getClientNode();
replication = pendingFile.getFileReplication();
} finally {
- writeUnlock();
+ readUnlock();
}
// choose targets for the new block to be allocated.
- final DatanodeDescriptor targets[] = blockManager.chooseTarget(
+ final DatanodeDescriptor targets[] = getBlockManager().chooseTarget(
src, replication, clientNode, excludedNodes, blockSize);
- // Allocate a new block and record it in the INode.
+ // Part II.
+ // Allocate a new block, add it to the INode and the BlocksMap.
+ Block newBlock = null;
+ long offset;
writeLock();
try {
- checkOperation(OperationCategory.WRITE);
- if (isInSafeMode()) {
- throw new SafeModeException("Cannot add block to " + src, safeMode);
+ // Run the full analysis again, since things could have changed
+ // while chooseTarget() was executing.
+ LocatedBlock[] onRetryBlock = new LocatedBlock[1];
+ INodesInPath inodesInPath =
+ analyzeFileState(src, fileId, clientName, previous, onRetryBlock);
+ INode[] inodes = inodesInPath.getINodes();
+ final INodeFileUnderConstruction pendingFile =
+ (INodeFileUnderConstruction) inodes[inodes.length - 1];
+
+ if(onRetryBlock[0] != null) {
+ // This is a retry. Just return the last block.
+ return onRetryBlock[0];
}
- final INodesInPath iip = dir.getINodesInPath4Write(src);
- final INodeFileUnderConstruction pendingFile
- = checkLease(src, clientName, iip.getLastINode());
-
- if (!checkFileProgress(pendingFile, false)) {
- throw new NotReplicatedYetException("Not replicated yet:" + src);
- }
+ // commit the last block and complete it if it has minimum replicas
+ commitOrCompleteLastBlock(pendingFile,
+ ExtendedBlock.getLocalBlock(previous));
+
+ // allocate new block, record block locations in INode.
+ newBlock = createNewBlock();
+ saveAllocatedBlock(src, inodesInPath, newBlock, targets);
- // allocate new block record block locations in INode.
- newBlock = allocateBlock(src, iip, targets);
-
- for (DatanodeDescriptor dn : targets) {
- dn.incBlocksScheduled();
- }
dir.persistBlocks(src, pendingFile);
+ offset = pendingFile.computeFileSize(true);
} finally {
writeUnlock();
}
@@ -2347,10 +2289,114 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
getEditLog().logSync();
}
- // Create next block
- LocatedBlock b = new LocatedBlock(getExtendedBlock(newBlock), targets, fileLength);
- blockManager.setBlockToken(b, BlockTokenSecretManager.AccessMode.WRITE);
- return b;
+ // Return located block
+ return makeLocatedBlock(newBlock, targets, offset);
+ }
+
+ INodesInPath analyzeFileState(String src,
+ long fileId,
+ String clientName,
+ ExtendedBlock previous,
+ LocatedBlock[] onRetryBlock)
+ throws IOException {
+ assert hasReadOrWriteLock();
+
+ checkBlock(previous);
+ onRetryBlock[0] = null;
+ checkOperation(OperationCategory.WRITE);
+ if (isInSafeMode()) {
+ throw new SafeModeException("Cannot add block to " + src, safeMode);
+ }
+
+ // have we exceeded the configured limit of fs objects.
+ checkFsObjectLimit();
+
+ Block previousBlock = ExtendedBlock.getLocalBlock(previous);
+ final INodesInPath inodesInPath = dir.getINodesInPath4Write(src);
+ final INode[] inodes = inodesInPath.getINodes();
+ final INodeFileUnderConstruction pendingFile
+ = checkLease(src, fileId, clientName, inodes[inodes.length - 1]);
+ BlockInfo lastBlockInFile = pendingFile.getLastBlock();
+ if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
+ // The block that the client claims is the current last block
+ // doesn't match up with what we think is the last block. There are
+ // four possibilities:
+ // 1) This is the first block allocation of an append() pipeline
+ // which started appending exactly at a block boundary.
+ // In this case, the client isn't passed the previous block,
+ // so it makes the allocateBlock() call with previous=null.
+ // We can distinguish this since the last block of the file
+ // will be exactly a full block.
+ // 2) This is a retry from a client that missed the response of a
+ // prior getAdditionalBlock() call, perhaps because of a network
+ // timeout, or because of an HA failover. In that case, we know
+ // by the fact that the client is re-issuing the RPC that it
+ // never began to write to the old block. Hence it is safe to
+ // to return the existing block.
+ // 3) This is an entirely bogus request/bug -- we should error out
+ // rather than potentially appending a new block with an empty
+ // one in the middle, etc
+ // 4) This is a retry from a client that timed out while
+ // the prior getAdditionalBlock() is still being processed,
+ // currently working on chooseTarget().
+ // There are no means to distinguish between the first and
+ // the second attempts in Part I, because the first one hasn't
+ // changed the namesystem state yet.
+ // We run this analysis again in Part II where case 4 is impossible.
+
+ BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
+ if (previous == null &&
+ lastBlockInFile != null &&
+ lastBlockInFile.getNumBytes() == pendingFile.getPreferredBlockSize() &&
+ lastBlockInFile.isComplete()) {
+ // Case 1
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
+ NameNode.stateChangeLog.debug(
+ "BLOCK* NameSystem.allocateBlock: handling block allocation" +
+ " writing to a file with a complete previous block: src=" +
+ src + " lastBlock=" + lastBlockInFile);
+ }
+ } else if (Block.matchingIdAndGenStamp(penultimateBlock, previousBlock)) {
+ if (lastBlockInFile.getNumBytes() != 0) {
+ throw new IOException(
+ "Request looked like a retry to allocate block " +
+ lastBlockInFile + " but it already contains " +
+ lastBlockInFile.getNumBytes() + " bytes");
+ }
+
+ // Case 2
+ // Return the last block.
+ NameNode.stateChangeLog.info("BLOCK* allocateBlock: " +
+ "caught retry for allocation of a new block in " +
+ src + ". Returning previously allocated block " + lastBlockInFile);
+ long offset = pendingFile.computeFileSize(true);
+ onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
+ ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedLocations(),
+ offset);
+ return inodesInPath;
+ } else {
+ // Case 3
+ throw new IOException("Cannot allocate block in " + src + ": " +
+ "passed 'previous' block " + previous + " does not match actual " +
+ "last block in file " + lastBlockInFile);
+ }
+ }
+
+ // Check if the penultimate block is minimally replicated
+ if (!checkFileProgress(pendingFile, false)) {
+ throw new NotReplicatedYetException("Not replicated yet: " + src);
+ }
+ return inodesInPath;
+ }
+
+ LocatedBlock makeLocatedBlock(Block blk,
+ DatanodeInfo[] locs,
+ long offset) throws IOException {
+ LocatedBlock lBlk = new LocatedBlock(
+ getExtendedBlock(blk), locs, offset);
+ getBlockManager().setBlockToken(
+ lBlk, BlockTokenSecretManager.AccessMode.WRITE);
+ return lBlk;
}
/** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */
@@ -2438,13 +2484,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
/** make sure that we still have the lease on this file. */
- private INodeFileUnderConstruction checkLease(String src, String holder)
- throws LeaseExpiredException, UnresolvedLinkException {
- return checkLease(src, holder, dir.getINode(src));
+ private INodeFileUnderConstruction checkLease(String src, String holder)
+ throws LeaseExpiredException, UnresolvedLinkException,
+ FileNotFoundException {
+ return checkLease(src, INodeId.GRANDFATHER_INODE_ID, holder,
+ dir.getINode(src));
}
-
- private INodeFileUnderConstruction checkLease(String src, String holder,
- INode file) throws LeaseExpiredException {
+
+ private INodeFileUnderConstruction checkLease(String src, long fileId,
+ String holder, INode file) throws LeaseExpiredException,
+ FileNotFoundException {
assert hasReadOrWriteLock();
if (file == null || !(file instanceof INodeFile)) {
Lease lease = leaseManager.getLease(holder);
@@ -2465,6 +2514,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throw new LeaseExpiredException("Lease mismatch on " + src + " owned by "
+ pendingFile.getClientName() + " but is accessed by " + holder);
}
+ INodeId.checkId(fileId, pendingFile);
return pendingFile;
}
@@ -2506,7 +2556,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
final INodesInPath iip = dir.getLastINodeInPath(src);
final INodeFileUnderConstruction pendingFile;
try {
- pendingFile = checkLease(src, holder, iip.getINode(0));
+ pendingFile = checkLease(src, INodeId.GRANDFATHER_INODE_ID,
+ holder, iip.getINode(0));
} catch (LeaseExpiredException lee) {
final INode inode = dir.getINode(src);
if (inode != null && inode instanceof INodeFile && !inode.isUnderConstruction()) {
@@ -2543,22 +2594,33 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
/**
- * Allocate a block at the given pending filename
+ * Save allocated block at the given pending filename
*
* @param src path to the file
* @param inodesInPath representing each of the components of src.
* The last INode is the INode for the file.
* @throws QuotaExceededException If addition of block exceeds space quota
*/
- private Block allocateBlock(String src, INodesInPath inodesInPath,
- DatanodeDescriptor targets[]) throws IOException {
+ BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
+ Block newBlock, DatanodeDescriptor targets[]) throws IOException {
+ assert hasWriteLock();
+ BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets);
+ NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
+ + getBlockPoolId() + " " + b);
+ for (DatanodeDescriptor dn : targets) {
+ dn.incBlocksScheduled();
+ }
+ return b;
+ }
+
+ /**
+ * Create new block with a unique block id and a new generation stamp.
+ */
+ Block createNewBlock() throws IOException {
assert hasWriteLock();
Block b = new Block(getFSImage().getUniqueBlockId(), 0, 0);
// Increment the generation stamp for every new block.
b.setGenerationStamp(nextGenerationStamp());
- b = dir.addBlock(src, inodesInPath, b, targets);
- NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
- + blockPoolId + " " + b);
return b;
}
@@ -5623,7 +5685,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
@Override
public boolean isAvoidingStaleDataNodesForWrite() {
return this.blockManager.getDatanodeManager()
- .isAvoidingStaleDataNodesForWrite();
+ .shouldAvoidStaleDataNodesForWrite();
}
public SnapshotManager getSnapshotManager() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
index 11157b2eb3d..293afb82c3b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
@@ -17,18 +17,21 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
+import java.io.FileNotFoundException;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.util.SequentialNumber;
/**
- * An id which uniquely identifies an inode
+ * An id which uniquely identifies an inode. Id 1 to 1000 are reserved for
+ * potential future usage. The id won't be recycled and is not expected to wrap
+ * around in a very long time. Root inode id is always 1001. Id 0 is used for
+ * backward compatibility support.
*/
@InterfaceAudience.Private
-class INodeId extends SequentialNumber {
+public class INodeId extends SequentialNumber {
/**
- * The last reserved inode id. Reserve id 1 to 1000 for potential future
- * usage. The id won't be recycled and is not expected to wrap around in a
- * very long time. Root inode id will be 1001.
+ * The last reserved inode id.
*/
public static final long LAST_RESERVED_ID = 1000L;
@@ -38,6 +41,19 @@ class INodeId extends SequentialNumber {
*/
public static final long GRANDFATHER_INODE_ID = 0;
+ /**
+ * To check if the request id is the same as saved id. Don't check fileId
+ * with GRANDFATHER_INODE_ID for backward compatibility.
+ */
+ public static void checkId(long requestId, INode inode)
+ throws FileNotFoundException {
+ if (requestId != GRANDFATHER_INODE_ID && requestId != inode.getId()) {
+ throw new FileNotFoundException(
+ "ID mismatch. Request id and saved id: " + requestId + " , "
+ + inode.getId());
+ }
+ }
+
INodeId() {
super(LAST_RESERVED_ID);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index abc871fa9f1..a5a4167e294 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -587,7 +587,7 @@ public class NNStorage extends Storage implements Closeable,
}
// Set Block pool ID in version with federation support
- if (LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+ if (versionSupportsFederation()) {
String sbpid = props.getProperty("blockpoolID");
setBlockPoolID(sd.getRoot(), sbpid);
}
@@ -634,7 +634,7 @@ public class NNStorage extends Storage implements Closeable,
) throws IOException {
super.setPropertiesFromFields(props, sd);
// Set blockpoolID in version with federation support
- if (LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+ if (versionSupportsFederation()) {
props.setProperty("blockpoolID", blockpoolID);
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index a096e370b78..98f14e5b8d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -424,13 +424,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
}
@Override // ClientProtocol
- public void create(String src,
- FsPermission masked,
- String clientName,
- EnumSetWritable flag,
- boolean createParent,
- short replication,
- long blockSize) throws IOException {
+ public HdfsFileStatus create(String src, FsPermission masked,
+ String clientName, EnumSetWritable flag,
+ boolean createParent, short replication, long blockSize)
+ throws IOException {
String clientMachine = getClientMachine();
if (stateChangeLog.isDebugEnabled()) {
stateChangeLog.debug("*DIR* NameNode.create: file "
@@ -440,12 +437,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
throw new IOException("create: Pathname too long. Limit "
+ MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
}
- namesystem.startFile(src,
- new PermissionStatus(UserGroupInformation.getCurrentUser().getShortUserName(),
- null, masked),
- clientName, clientMachine, flag.get(), createParent, replication, blockSize);
+ HdfsFileStatus fileStatus = namesystem.startFile(src, new PermissionStatus(
+ UserGroupInformation.getCurrentUser().getShortUserName(), null, masked),
+ clientName, clientMachine, flag.get(), createParent, replication,
+ blockSize);
metrics.incrFilesCreated();
metrics.incrCreateFileOps();
+ return fileStatus;
}
@Override // ClientProtocol
@@ -484,26 +482,24 @@ class NameNodeRpcServer implements NamenodeProtocols {
throws IOException {
namesystem.setOwner(src, username, groupname);
}
-
- @Override // ClientProtocol
- public LocatedBlock addBlock(String src,
- String clientName,
- ExtendedBlock previous,
- DatanodeInfo[] excludedNodes)
+
+ @Override
+ public LocatedBlock addBlock(String src, String clientName,
+ ExtendedBlock previous, DatanodeInfo[] excludedNodes, long fileId)
throws IOException {
- if(stateChangeLog.isDebugEnabled()) {
- stateChangeLog.debug("*BLOCK* NameNode.addBlock: file "
- +src+" for "+clientName);
+ if (stateChangeLog.isDebugEnabled()) {
+ stateChangeLog.debug("*BLOCK* NameNode.addBlock: file " + src
+ + " fileId=" + fileId + " for " + clientName);
}
HashMap excludedNodesSet = null;
if (excludedNodes != null) {
excludedNodesSet = new HashMap(excludedNodes.length);
- for (Node node:excludedNodes) {
+ for (Node node : excludedNodes) {
excludedNodesSet.put(node, node);
}
}
- LocatedBlock locatedBlock =
- namesystem.getAdditionalBlock(src, clientName, previous, excludedNodesSet);
+ LocatedBlock locatedBlock = namesystem.getAdditionalBlock(src, fileId,
+ clientName, previous, excludedNodesSet);
if (locatedBlock != null)
metrics.incrAddBlockOps();
return locatedBlock;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
index 3679af4b432..aad1592be8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
@@ -17,12 +17,15 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
+import static org.apache.hadoop.hdfs.DFSUtil.percent2String;
+
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.lang.management.MemoryMXBean;
import java.lang.management.MemoryUsage;
import java.net.InetAddress;
import java.net.InetSocketAddress;
+import java.net.URI;
import java.net.URLEncoder;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
@@ -64,6 +67,14 @@ import org.znerd.xmlenc.XMLOutputter;
import com.google.common.base.Preconditions;
class NamenodeJspHelper {
+ static String fraction2String(double value) {
+ return StringUtils.format("%.2f", value);
+ }
+
+ static String fraction2String(long numerator, long denominator) {
+ return fraction2String(numerator/(double)denominator);
+ }
+
static String getSafeModeText(FSNamesystem fsn) {
if (!fsn.isInSafeMode())
return "";
@@ -361,20 +372,20 @@ class NamenodeJspHelper {
+ "DFS Remaining" + colTxt() + ":" + colTxt()
+ StringUtils.byteDesc(remaining) + rowTxt() + colTxt() + "DFS Used%"
+ colTxt() + ":" + colTxt()
- + StringUtils.limitDecimalTo2(percentUsed) + " %" + rowTxt()
+ + percent2String(percentUsed) + rowTxt()
+ colTxt() + "DFS Remaining%" + colTxt() + ":" + colTxt()
- + StringUtils.limitDecimalTo2(percentRemaining) + " %"
+ + percent2String(percentRemaining)
+ rowTxt() + colTxt() + "Block Pool Used" + colTxt() + ":" + colTxt()
+ StringUtils.byteDesc(bpUsed) + rowTxt()
+ colTxt() + "Block Pool Used%"+ colTxt() + ":" + colTxt()
- + StringUtils.limitDecimalTo2(percentBpUsed) + " %"
+ + percent2String(percentBpUsed)
+ rowTxt() + colTxt() + "DataNodes usages" + colTxt() + ":" + colTxt()
+ "Min %" + colTxt() + "Median %" + colTxt() + "Max %" + colTxt()
+ "stdev %" + rowTxt() + colTxt() + colTxt() + colTxt()
- + StringUtils.limitDecimalTo2(min) + " %"
- + colTxt() + StringUtils.limitDecimalTo2(median) + " %"
- + colTxt() + StringUtils.limitDecimalTo2(max) + " %"
- + colTxt() + StringUtils.limitDecimalTo2(dev) + " %"
+ + percent2String(min)
+ + colTxt() + percent2String(median)
+ + colTxt() + percent2String(max)
+ + colTxt() + percent2String(dev)
+ rowTxt() + colTxt()
+ "Live Nodes "
+ colTxt() + ":" + colTxt() + live.size()
@@ -443,7 +454,13 @@ class NamenodeJspHelper {
nodeToRedirect = nn.getHttpAddress().getHostName();
redirectPort = nn.getHttpAddress().getPort();
}
- String addr = nn.getNameNodeAddressHostPortString();
+
+ InetSocketAddress rpcAddr = nn.getNameNodeAddress();
+ String rpcHost = rpcAddr.getAddress().isAnyLocalAddress()
+ ? URI.create(request.getRequestURL().toString()).getHost()
+ : rpcAddr.getAddress().getHostAddress();
+ String addr = rpcHost + ":" + rpcAddr.getPort();
+
String fqdn = InetAddress.getByName(nodeToRedirect).getCanonicalHostName();
redirectLocation = HttpConfig.getSchemePrefix() + fqdn + ":" + redirectPort
+ "/browseDirectory.jsp?namenodeInfoPort="
@@ -562,9 +579,9 @@ class NamenodeJspHelper {
long u = d.getDfsUsed();
long nu = d.getNonDfsUsed();
long r = d.getRemaining();
- String percentUsed = StringUtils.limitDecimalTo2(d.getDfsUsedPercent());
- String percentRemaining = StringUtils.limitDecimalTo2(d
- .getRemainingPercent());
+ final double percentUsedValue = d.getDfsUsedPercent();
+ String percentUsed = fraction2String(percentUsedValue);
+ String percentRemaining = fraction2String(d.getRemainingPercent());
String adminState = d.getAdminState().toString();
@@ -572,32 +589,30 @@ class NamenodeJspHelper {
long currentTime = Time.now();
long bpUsed = d.getBlockPoolUsed();
- String percentBpUsed = StringUtils.limitDecimalTo2(d
- .getBlockPoolUsedPercent());
+ String percentBpUsed = fraction2String(d.getBlockPoolUsedPercent());
out.print(" "
+ ((currentTime - timestamp) / 1000)
+ " | "
+ adminState
+ " | "
- + StringUtils.limitDecimalTo2(c * 1.0 / diskBytes)
+ + fraction2String(c, diskBytes)
+ " | "
- + StringUtils.limitDecimalTo2(u * 1.0 / diskBytes)
+ + fraction2String(u, diskBytes)
+ " | "
- + StringUtils.limitDecimalTo2(nu * 1.0 / diskBytes)
+ + fraction2String(nu, diskBytes)
+ " | "
- + StringUtils.limitDecimalTo2(r * 1.0 / diskBytes)
+ + fraction2String(r, diskBytes)
+ " | "
+ percentUsed
+ " | "
- + ServletUtil.percentageGraph((int) Double.parseDouble(percentUsed),
- 100)
+ + ServletUtil.percentageGraph((int)percentUsedValue, 100)
+ " | "
+ percentRemaining
+ " | " + d.numBlocks()+"\n"
+ " | "
- + StringUtils.limitDecimalTo2(bpUsed * 1.0 / diskBytes)
+ + fraction2String(bpUsed, diskBytes)
+ " | "
+ percentBpUsed
+ " | "
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index c22d941010a..763c7089abd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -475,14 +475,20 @@ public class SecondaryNameNode implements Runnable {
// Returns a token that would be used to upload the merged image.
CheckpointSignature sig = namenode.rollEditLog();
- if ((checkpointImage.getNamespaceID() == 0) ||
- (sig.isSameCluster(checkpointImage) &&
+ boolean loadImage = false;
+ boolean isFreshCheckpointer = (checkpointImage.getNamespaceID() == 0);
+ boolean isSameCluster =
+ (dstStorage.versionSupportsFederation() && sig.isSameCluster(checkpointImage)) ||
+ (!dstStorage.versionSupportsFederation() && sig.namespaceIdMatches(checkpointImage));
+ if (isFreshCheckpointer ||
+ (isSameCluster &&
!sig.storageVersionMatches(checkpointImage.getStorage()))) {
// if we're a fresh 2NN, or if we're on the same cluster and our storage
// needs an upgrade, just take the storage info from the server.
dstStorage.setStorageInfo(sig);
dstStorage.setClusterID(sig.getClusterID());
dstStorage.setBlockPoolID(sig.getBlockpoolID());
+ loadImage = true;
}
sig.validateStorageInfo(checkpointImage);
@@ -492,7 +498,7 @@ public class SecondaryNameNode implements Runnable {
RemoteEditLogManifest manifest =
namenode.getEditLogManifest(sig.mostRecentCheckpointTxId + 1);
- boolean loadImage = downloadCheckpointFiles(
+ loadImage |= downloadCheckpointFiles(
fsName, checkpointImage, sig, manifest); // Fetch fsimage and edits
doMerge(sig, manifest, loadImage, checkpointImage, namesystem);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index be21b801e28..f7605ff9e98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -316,8 +316,7 @@ public class DFSAdmin extends FsShell {
System.out.println("DFS Used: " + used
+ " (" + StringUtils.byteDesc(used) + ")");
System.out.println("DFS Used%: "
- + StringUtils.limitDecimalTo2(((1.0 * used) / presentCapacity) * 100)
- + "%");
+ + StringUtils.formatPercent(used/(double)presentCapacity, 2));
/* These counts are not always upto date. They are updated after
* iteration of an internal list. Should be updated in a few seconds to
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index f251e34c13a..e04fb694bdb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -219,6 +219,7 @@ public class JsonUtil {
m.put("modificationTime", status.getModificationTime());
m.put("blockSize", status.getBlockSize());
m.put("replication", status.getReplication());
+ m.put("fileId", status.getFileId());
return includeType ? toJsonString(FileStatus.class, m): JSON.toString(m);
}
@@ -243,9 +244,10 @@ public class JsonUtil {
final long mTime = (Long) m.get("modificationTime");
final long blockSize = (Long) m.get("blockSize");
final short replication = (short) (long) (Long) m.get("replication");
+ final long fileId = (Long) m.get("fileId");
return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
blockSize, mTime, aTime, permission, owner, group,
- symlink, DFSUtil.string2Bytes(localName));
+ symlink, DFSUtil.string2Bytes(localName), fileId);
}
/** Convert an ExtendedBlock to a Json map. */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java
index c29f2329c48..e6afbe3e4ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hdfs.web.resources;
/** The concat source paths parameter. */
public class ConcatSourcesParam extends StringParam {
/** Parameter name. */
- public static final String NAME = "srcs";
+ public static final String NAME = "sources";
public static final String DEFAULT = NULL;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
index 12afc6dae74..ed5d4b8d0e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -67,7 +67,8 @@ message CreateRequestProto {
required uint64 blockSize = 7;
}
-message CreateResponseProto { // void response
+message CreateResponseProto {
+ optional HdfsFileStatusProto fs = 1;
}
message AppendRequestProto {
@@ -119,6 +120,7 @@ message AddBlockRequestProto {
required string clientName = 2;
optional ExtendedBlockProto previous = 3;
repeated DatanodeInfoProto excludeNodes = 4;
+ optional uint64 fileId = 5 [default = 0]; // default as a bogus id
}
message AddBlockResponseProto {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index aab028abfe4..1ac3cd66cd7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -170,6 +170,9 @@ message HdfsFileStatusProto {
optional uint32 block_replication = 10 [default = 0]; // only 16bits used
optional uint64 blocksize = 11 [default = 0];
optional LocatedBlocksProto locations = 12; // suppled only if asked by client
+
+ // Optional field for fileId
+ optional uint64 fileId = 13 [default = 0]; // default as an invalid id
}
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 34cd8465fd7..fa103e7f29a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -999,17 +999,14 @@
- dfs.namenode.check.stale.datanode
+ dfs.namenode.avoid.read.stale.datanode
false
- Indicate whether or not to check "stale" datanodes whose
+ Indicate whether or not to avoid reading from "stale" datanodes whose
heartbeat messages have not been received by the namenode
- for more than a specified time interval. If this configuration
- parameter is set as true, the system will keep track
- of the number of stale datanodes. The stale datanodes will be
+ for more than a specified time interval. Stale datanodes will be
moved to the end of the node list returned for reading. See
- dfs.namenode.avoid.write.stale.datanode for details on how this
- affects writes.
+ dfs.namenode.avoid.write.stale.datanode for a similar setting for writes.
@@ -1017,13 +1014,13 @@
dfs.namenode.avoid.write.stale.datanode
false
- Indicate whether or not to avoid writing to "stale" datanodes whose
+ Indicate whether or not to avoid writing to "stale" datanodes whose
heartbeat messages have not been received by the namenode
- for more than a specified time interval. If this configuration
- parameter and dfs.namenode.check.stale.datanode are both set as true,
- the writing will avoid using stale datanodes unless a high number
- of datanodes are marked as stale. See
- dfs.namenode.write.stale.datanode.ratio for details.
+ for more than a specified time interval. Writes will avoid using
+ stale datanodes unless more than a configured ratio
+ (dfs.namenode.write.stale.datanode.ratio) of datanodes are marked as
+ stale. See dfs.namenode.avoid.read.stale.datanode for a similar setting
+ for reads.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/WebHDFS.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/WebHDFS.apt.vm
index 38b8dc8ab09..90f8dabce71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/WebHDFS.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/WebHDFS.apt.vm
@@ -109,6 +109,9 @@ WebHDFS REST API
* {{{Append to a File}<<>>}}
(see {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.append)
+ * {{{Concat File(s)}<<>>}}
+ (see {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.concat)
+
* HTTP DELETE
* {{{Delete a File/Directory}<<>>}}
@@ -299,6 +302,32 @@ Content-Length: 0
{{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.append
+** {Concat File(s)}
+
+ * Submit a HTTP POST request.
+
++---------------------------------
+curl -i -X POST "http://:/webhdfs/v1/?op=CONCAT&sources="
++---------------------------------
+
+ The client receives a response with zero content length:
+
++---------------------------------
+HTTP/1.1 200 OK
+Content-Length: 0
++---------------------------------
+
+ []
+
+ This REST API call is available as of Hadoop version 2.0.3.
+ Please note that is a comma seperated list of absolute paths.
+ (Example: sources=/test/file1,/test/file2,/test/file3)
+
+ See also:
+ {{{Sources}<<>>}},
+ {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.concat
+
+
** {Open and Read a File}
* Submit a HTTP GET request with automatically following redirects.
@@ -1727,6 +1756,29 @@ var tokenProperties =
{{{Set Replication Factor}<<>>}}
+** {Sources}
+
+*----------------+-------------------------------------------------------------------+
+|| Name | <<>> |
+*----------------+-------------------------------------------------------------------+
+|| Description | The comma seperated absolute paths used for concatenation. |
+*----------------+-------------------------------------------------------------------+
+|| Type | String |
+*----------------+-------------------------------------------------------------------+
+|| Default Value | \ |
+*----------------+-------------------------------------------------------------------+
+|| Valid Values | A list of comma seperated absolute FileSystem paths without scheme and authority. |
+*----------------+-------------------------------------------------------------------+
+|| Syntax | See the note in {{Delegation}}. |
+*----------------+-------------------------------------------------------------------+
+
+ <> that sources are absolute FileSystem paths.
+
+
+ See also:
+ {{{Concat File(s)}<<>>}}
+
+
** {Token}
*----------------+-------------------------------------------------------------------+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index d5907d5a5b1..ff445ad9ba3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -33,6 +33,7 @@ import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
+import java.net.HttpURLConnection;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.URL;
@@ -639,6 +640,9 @@ public class DFSTestUtil {
*/
public static byte[] urlGetBytes(URL url) throws IOException {
URLConnection conn = url.openConnection();
+ HttpURLConnection hc = (HttpURLConnection)conn;
+
+ assertEquals(HttpURLConnection.HTTP_OK, hc.getResponseCode());
ByteArrayOutputStream out = new ByteArrayOutputStream();
IOUtils.copyBytes(conn.getInputStream(), out, 4096, true);
return out.toByteArray();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
index a88e8a74edf..86bcae3a0a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
@@ -23,22 +23,34 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
+import javax.net.SocketFactory;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.ConnectTimeoutException;
+import org.apache.hadoop.net.StandardSocketFactory;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
+import org.mockito.Mockito;
public class TestDFSClientFailover {
@@ -91,6 +103,63 @@ public class TestDFSClientFailover {
fs.close();
}
+ /**
+ * Test that even a non-idempotent method will properly fail-over if the
+ * first IPC attempt times out trying to connect. Regression test for
+ * HDFS-4404.
+ */
+ @Test
+ public void testFailoverOnConnectTimeout() throws Exception {
+ conf.setClass(CommonConfigurationKeysPublic.HADOOP_RPC_SOCKET_FACTORY_CLASS_DEFAULT_KEY,
+ InjectingSocketFactory.class, SocketFactory.class);
+ // Set up the InjectingSocketFactory to throw a ConnectTimeoutException
+ // when connecting to the first NN.
+ InjectingSocketFactory.portToInjectOn = cluster.getNameNodePort(0);
+
+ FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+
+ // Make the second NN the active one.
+ cluster.shutdownNameNode(0);
+ cluster.transitionToActive(1);
+
+ // Call a non-idempotent method, and ensure the failover of the call proceeds
+ // successfully.
+ IOUtils.closeStream(fs.create(TEST_FILE));
+ }
+
+ private static class InjectingSocketFactory extends StandardSocketFactory {
+
+ static SocketFactory defaultFactory = SocketFactory.getDefault();
+
+ static int portToInjectOn;
+
+ @Override
+ public Socket createSocket() throws IOException {
+ Socket spy = Mockito.spy(defaultFactory.createSocket());
+ // Simplify our spying job by not having to also spy on the channel
+ Mockito.doReturn(null).when(spy).getChannel();
+ // Throw a ConnectTimeoutException when connecting to our target "bad"
+ // host.
+ Mockito.doThrow(new ConnectTimeoutException("injected"))
+ .when(spy).connect(
+ Mockito.argThat(new MatchesPort()),
+ Mockito.anyInt());
+ return spy;
+ }
+
+ private class MatchesPort extends BaseMatcher {
+ @Override
+ public boolean matches(Object arg0) {
+ return ((InetSocketAddress)arg0).getPort() == portToInjectOn;
+ }
+
+ @Override
+ public void describeTo(Description desc) {
+ desc.appendText("matches port " + portToInjectOn);
+ }
+ }
+ }
+
/**
* Regression test for HDFS-2683.
*/
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index dc77b251946..a7292e2c8e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -23,7 +23,10 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.anyShort;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
@@ -49,13 +52,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.client.HdfsUtils;
import org.apache.hadoop.hdfs.protocol.Block;
@@ -64,12 +67,14 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Writable;
@@ -208,7 +213,7 @@ public class TestDFSClientRetries {
* Verify that client will correctly give up after the specified number
* of times trying to add a block
*/
- @SuppressWarnings("serial")
+ @SuppressWarnings({ "serial", "unchecked" })
@Test
public void testNotYetReplicatedErrors() throws IOException
{
@@ -235,7 +240,22 @@ public class TestDFSClientRetries {
when(mockNN.addBlock(anyString(),
anyString(),
any(ExtendedBlock.class),
- any(DatanodeInfo[].class))).thenAnswer(answer);
+ any(DatanodeInfo[].class),
+ anyLong())).thenAnswer(answer);
+
+ Mockito.doReturn(
+ new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
+ (short) 777), "owner", "group", new byte[0], new byte[0],
+ 1010)).when(mockNN).getFileInfo(anyString());
+
+ Mockito.doReturn(
+ new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
+ (short) 777), "owner", "group", new byte[0], new byte[0],
+ 1010))
+ .when(mockNN)
+ .create(anyString(), (FsPermission) anyObject(), anyString(),
+ (EnumSetWritable) anyObject(), anyBoolean(),
+ anyShort(), anyLong());
final DFSClient client = new DFSClient(null, mockNN, conf, null);
OutputStream os = client.create("testfile", true);
@@ -369,7 +389,8 @@ public class TestDFSClientRetries {
return ret2;
}
}).when(spyNN).addBlock(Mockito.anyString(), Mockito.anyString(),
- Mockito.any(), Mockito.any());
+ Mockito. any(), Mockito. any(),
+ Mockito.anyLong());
doAnswer(new Answer() {
@@ -410,7 +431,8 @@ public class TestDFSClientRetries {
// Make sure the mock was actually properly injected.
Mockito.verify(spyNN, Mockito.atLeastOnce()).addBlock(
Mockito.anyString(), Mockito.anyString(),
- Mockito.any(), Mockito.any());
+ Mockito. any(), Mockito. any(),
+ Mockito.anyLong());
Mockito.verify(spyNN, Mockito.atLeastOnce()).complete(
Mockito.anyString(), Mockito.anyString(),
Mockito.any());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index d59acfa6ec4..8482f81ddb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -619,6 +619,16 @@ public class TestDFSUtil {
assertEquals(1, uris.size());
assertTrue(uris.contains(new URI("hdfs://" + NN1_SRVC_ADDR)));
+
+ // Make sure when config FS_DEFAULT_NAME_KEY using IP address,
+ // it will automatically convert it to hostname
+ conf = new HdfsConfiguration();
+ conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://127.0.0.1:8020");
+ uris = DFSUtil.getNameServiceUris(conf);
+ assertEquals(1, uris.size());
+ for (URI uri : uris) {
+ assertFalse(uri.getHost().equals("127.0.0.1"));
+ }
}
@Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
index e6ce7c43f19..5a01ea57a8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.io.EnumSetWritable;
@@ -517,8 +518,8 @@ public class TestFileCreation {
+ "The file has " + locations.locatedBlockCount() + " blocks.");
// add one block to the file
- LocatedBlock location = client.getNamenode().addBlock(file1.toString(),
- client.clientName, null, null);
+ LocatedBlock location = client.getNamenode().addBlock(file1.toString(),
+ client.clientName, null, null, INodeId.GRANDFATHER_INODE_ID);
System.out.println("testFileCreationError2: "
+ "Added block " + location.getBlock());
@@ -568,8 +569,8 @@ public class TestFileCreation {
final Path f = new Path("/foo.txt");
createFile(dfs, f, 3);
try {
- cluster.getNameNodeRpc().addBlock(f.toString(),
- client.clientName, null, null);
+ cluster.getNameNodeRpc().addBlock(f.toString(), client.clientName,
+ null, null, INodeId.GRANDFATHER_INODE_ID);
fail();
} catch(IOException ioe) {
FileSystem.LOG.info("GOOD!", ioe);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java
index f63ba9a53a7..8bafee67f85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java
@@ -43,7 +43,7 @@ public class TestFileLengthOnClusterRestart {
.numDataNodes(2).build();
HdfsDataInputStream in = null;
try {
- Path path = new Path(MiniDFSCluster.getBaseDirectory(), "test");
+ Path path = new Path("/tmp/TestFileLengthOnClusterRestart", "test");
DistributedFileSystem dfs = (DistributedFileSystem) cluster
.getFileSystem();
FSDataOutputStream out = dfs.create(path);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
index 55d1aa7ee51..6eaa55fbe92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
@@ -88,7 +88,7 @@ public class TestGetBlocks {
@Test
public void testReadSelectNonStaleDatanode() throws Exception {
HdfsConfiguration conf = new HdfsConfiguration();
- conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY, true);
+ conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY, true);
long staleInterval = 30 * 1000 * 60;
conf.setLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
staleInterval);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java
index 9563361094c..64c5ef4c056 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java
@@ -183,8 +183,7 @@ public class TestLargeBlock {
try {
// create a new file in test data directory
- Path file1 = new Path(System.getProperty("test.build.data") + "/" +
- Long.toString(blockSize) + ".dat");
+ Path file1 = new Path("/tmp/TestLargeBlock", blockSize + ".dat");
FSDataOutputStream stm = createFile(fs, file1, 1, blockSize);
LOG.info("File " + file1 + " created with file size " +
fileSize +
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
index 1940b6dcd03..969e4a1fd41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
@@ -18,6 +18,10 @@
package org.apache.hadoop.hdfs;
import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.anyShort;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.spy;
@@ -29,14 +33,18 @@ import java.security.PrivilegedExceptionAction;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -256,6 +264,7 @@ public class TestLease {
}
}
+ @SuppressWarnings("unchecked")
@Test
public void testFactory() throws Exception {
final String[] groups = new String[]{"supergroup"};
@@ -264,6 +273,20 @@ public class TestLease {
ugi[i] = UserGroupInformation.createUserForTesting("user" + i, groups);
}
+ Mockito.doReturn(
+ new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
+ (short) 777), "owner", "group", new byte[0], new byte[0],
+ 1010)).when(mcp).getFileInfo(anyString());
+ Mockito
+ .doReturn(
+ new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
+ (short) 777), "owner", "group", new byte[0], new byte[0],
+ 1010))
+ .when(mcp)
+ .create(anyString(), (FsPermission) anyObject(), anyString(),
+ (EnumSetWritable) anyObject(), anyBoolean(),
+ anyShort(), anyLong());
+
final Configuration conf = new Configuration();
final DFSClient c1 = createDFSClientAs(ugi[0], conf);
FSDataOutputStream out1 = createFsOut(c1, "/out1");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestListFilesInDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestListFilesInDFS.java
index ec9e7e2e481..d68563dec87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestListFilesInDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestListFilesInDFS.java
@@ -38,6 +38,7 @@ public class TestListFilesInDFS extends TestListFiles {
@BeforeClass
public static void testSetUp() throws Exception {
+ setTestPaths(new Path("/tmp/TestListFilesInDFS"));
cluster = new MiniDFSCluster.Builder(conf).build();
fs = cluster.getFileSystem();
fs.delete(TEST_DIR, true);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
index 600829b1186..0f6d7ada666 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
@@ -70,8 +70,8 @@ public class TestQuota {
throw new DSQuotaExceededException(bytes, bytes);
} catch(DSQuotaExceededException e) {
- assertEquals("The DiskSpace quota is exceeded: quota=1.0k " +
- "diskspace consumed=1.0k", e.getMessage());
+ assertEquals("The DiskSpace quota is exceeded: quota = 1024 B = 1 KB"
+ + " but diskspace consumed = 1024 B = 1 KB", e.getMessage());
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
index c07fae4773b..0b7eaeeed9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
@@ -67,7 +67,7 @@ public class TestRBWBlockInvalidation {
try {
final FSNamesystem namesystem = cluster.getNamesystem();
FileSystem fs = cluster.getFileSystem();
- Path testPath = new Path(MiniDFSCluster.getBaseDirectory(), "foo1");
+ Path testPath = new Path("/tmp/TestRBWBlockInvalidation", "foo1");
out = fs.create(testPath, (short) 2);
out.writeBytes("HDFS-3157: " + testPath);
out.hsync();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 78ac007e35a..38a5f0df3bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -88,9 +88,11 @@ public class TestReplicationPolicy {
"test.build.data", "build/test/data"), "dfs/");
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
new File(baseDir, "name").getPath());
- // Enable the checking for stale datanodes in the beginning
- conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY, true);
+ conf.setBoolean(
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY, true);
+ conf.setBoolean(
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY, true);
DFSTestUtil.formatNameNode(conf);
namenode = new NameNode(conf);
@@ -100,6 +102,8 @@ public class TestReplicationPolicy {
// construct network topology
for (int i=0; i < NUM_OF_DATANODES; i++) {
cluster.add(dataNodes[i]);
+ bm.getDatanodeManager().getHeartbeatManager().addDatanode(
+ dataNodes[i]);
}
for (int i=0; i < NUM_OF_DATANODES; i++) {
dataNodes[i].updateHeartbeat(
@@ -393,11 +397,11 @@ public class TestReplicationPolicy {
throws Exception {
try {
namenode.getNamesystem().getBlockManager().getDatanodeManager()
- .setAvoidStaleDataNodesForWrite(true);
+ .setNumStaleNodes(NUM_OF_DATANODES);
testChooseTargetWithMoreThanAvailableNodes();
} finally {
namenode.getNamesystem().getBlockManager().getDatanodeManager()
- .setAvoidStaleDataNodesForWrite(false);
+ .setNumStaleNodes(0);
}
}
@@ -479,12 +483,12 @@ public class TestReplicationPolicy {
@Test
public void testChooseTargetWithStaleNodes() throws Exception {
- // Enable avoidng writing to stale datanodes
- namenode.getNamesystem().getBlockManager().getDatanodeManager()
- .setAvoidStaleDataNodesForWrite(true);
// Set dataNodes[0] as stale
dataNodes[0].setLastUpdate(Time.now() - staleInterval - 1);
-
+ namenode.getNamesystem().getBlockManager()
+ .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
+ assertTrue(namenode.getNamesystem().getBlockManager()
+ .getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
DatanodeDescriptor[] targets;
// We set the datanode[0] as stale, thus should choose datanode[1] since
// datanode[1] is on the same rack with datanode[0] (writer)
@@ -503,9 +507,9 @@ public class TestReplicationPolicy {
assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
// reset
- namenode.getNamesystem().getBlockManager().getDatanodeManager()
- .setAvoidStaleDataNodesForWrite(false);
dataNodes[0].setLastUpdate(Time.now());
+ namenode.getNamesystem().getBlockManager()
+ .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
}
/**
@@ -518,20 +522,20 @@ public class TestReplicationPolicy {
*/
@Test
public void testChooseTargetWithHalfStaleNodes() throws Exception {
- // Enable stale datanodes checking
- namenode.getNamesystem().getBlockManager().getDatanodeManager()
- .setAvoidStaleDataNodesForWrite(true);
// Set dataNodes[0], dataNodes[1], and dataNodes[2] as stale
for (int i = 0; i < 3; i++) {
dataNodes[i].setLastUpdate(Time.now() - staleInterval - 1);
}
+ namenode.getNamesystem().getBlockManager()
+ .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
DatanodeDescriptor[] targets;
targets = replicator.chooseTarget(filename, 0, dataNodes[0],
new ArrayList(), BLOCK_SIZE);
assertEquals(targets.length, 0);
- // We set the datanode[0] as stale, thus should choose datanode[1]
+ // Since we have 6 datanodes total, stale nodes should
+ // not be returned until we ask for more than 3 targets
targets = replicator.chooseTarget(filename, 1, dataNodes[0],
new ArrayList(), BLOCK_SIZE);
assertEquals(targets.length, 1);
@@ -557,18 +561,16 @@ public class TestReplicationPolicy {
assertTrue(containsWithinRange(dataNodes[4], targets, 0, 3));
assertTrue(containsWithinRange(dataNodes[5], targets, 0, 3));
- // reset
- namenode.getNamesystem().getBlockManager().getDatanodeManager()
- .setAvoidStaleDataNodesForWrite(false);
for (int i = 0; i < dataNodes.length; i++) {
dataNodes[i].setLastUpdate(Time.now());
}
+ namenode.getNamesystem().getBlockManager()
+ .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
}
@Test
public void testChooseTargetWithMoreThanHalfStaleNodes() throws Exception {
HdfsConfiguration conf = new HdfsConfiguration();
- conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY, true);
conf.setBoolean(
DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY, true);
String[] hosts = new String[]{"host1", "host2", "host3",
@@ -598,7 +600,7 @@ public class TestReplicationPolicy {
.getBlockManager().getDatanodeManager().getNumStaleNodes();
assertEquals(numStaleNodes, 2);
assertTrue(miniCluster.getNameNode().getNamesystem().getBlockManager()
- .getDatanodeManager().isAvoidingStaleDataNodesForWrite());
+ .getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
// Call chooseTarget
DatanodeDescriptor staleNodeInfo = miniCluster.getNameNode()
.getNamesystem().getBlockManager().getDatanodeManager()
@@ -627,7 +629,7 @@ public class TestReplicationPolicy {
// According to our strategy, stale datanodes will be included for writing
// to avoid hotspots
assertFalse(miniCluster.getNameNode().getNamesystem().getBlockManager()
- .getDatanodeManager().isAvoidingStaleDataNodesForWrite());
+ .getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
// Call chooseTarget
targets = replicator.chooseTarget(filename, 3,
staleNodeInfo, new ArrayList(), BLOCK_SIZE);
@@ -650,7 +652,7 @@ public class TestReplicationPolicy {
.getBlockManager().getDatanodeManager().getNumStaleNodes();
assertEquals(numStaleNodes, 2);
assertTrue(miniCluster.getNameNode().getNamesystem().getBlockManager()
- .getDatanodeManager().isAvoidingStaleDataNodesForWrite());
+ .getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
// Call chooseTarget
targets = replicator.chooseTarget(filename, 3,
staleNodeInfo, new ArrayList(), BLOCK_SIZE);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
index d75c17bf964..f4dc62bf45e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
@@ -506,7 +506,11 @@ public abstract class FSImageTestUtil {
props.load(fis);
IOUtils.closeStream(fis);
- props.setProperty(key, value);
+ if (value == null || value.isEmpty()) {
+ props.remove(key);
+ } else {
+ props.setProperty(key, value);
+ }
out = new FileOutputStream(versionFile);
props.store(out, null);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index a744b0b2f12..891c6699454 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -1058,7 +1058,8 @@ public class NNThroughputBenchmark {
throws IOException {
ExtendedBlock prevBlock = null;
for(int jdx = 0; jdx < blocksPerFile; jdx++) {
- LocatedBlock loc = nameNodeProto.addBlock(fileName, clientName, prevBlock, null);
+ LocatedBlock loc = nameNodeProto.addBlock(fileName, clientName,
+ prevBlock, null, INodeId.GRANDFATHER_INODE_ID);
prevBlock = loc.getBlock();
for(DatanodeInfo dnInfo : loc.getLocations()) {
int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getXferAddr());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
new file mode 100644
index 00000000000..793cec6e93a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.namenode;
+
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
+import java.lang.reflect.Field;
+import java.util.EnumSet;
+import java.util.HashMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.net.Node;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Race between two threads simultaneously calling
+ * FSNamesystem.getAdditionalBlock().
+ */
+public class TestAddBlockRetry {
+ public static final Log LOG = LogFactory.getLog(TestAddBlockRetry.class);
+
+ private static final short REPLICATION = 3;
+
+ private Configuration conf;
+ private MiniDFSCluster cluster;
+
+ private int count = 0;
+ private LocatedBlock lb1;
+ private LocatedBlock lb2;
+
+ @Before
+ public void setUp() throws Exception {
+ conf = new Configuration();
+ cluster = new MiniDFSCluster.Builder(conf)
+ .numDataNodes(REPLICATION)
+ .build();
+ cluster.waitActive();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
+
+ /**
+ * Retry addBlock() while another thread is in chooseTarget().
+ * See HDFS-4452.
+ */
+ @Test
+ public void testRetryAddBlockWhileInChooseTarget() throws Exception {
+ final String src = "/testRetryAddBlockWhileInChooseTarget";
+
+ FSNamesystem ns = cluster.getNamesystem();
+ BlockManager spyBM = spy(ns.getBlockManager());
+ final NamenodeProtocols nn = cluster.getNameNodeRpc();
+
+ // substitute mocked BlockManager into FSNamesystem
+ Class extends FSNamesystem> nsClass = ns.getClass();
+ Field bmField = nsClass.getDeclaredField("blockManager");
+ bmField.setAccessible(true);
+ bmField.set(ns, spyBM);
+
+ doAnswer(new Answer() {
+ @Override
+ public DatanodeDescriptor[] answer(InvocationOnMock invocation)
+ throws Throwable {
+ LOG.info("chooseTarget for " + src);
+ DatanodeDescriptor[] ret =
+ (DatanodeDescriptor[]) invocation.callRealMethod();
+ count++;
+ if(count == 1) { // run second addBlock()
+ LOG.info("Starting second addBlock for " + src);
+ nn.addBlock(src, "clientName", null, null,
+ INodeId.GRANDFATHER_INODE_ID);
+ LocatedBlocks lbs = nn.getBlockLocations(src, 0, Long.MAX_VALUE);
+ assertEquals("Must be one block", 1, lbs.getLocatedBlocks().size());
+ lb2 = lbs.get(0);
+ assertEquals("Wrong replication",
+ REPLICATION, lb2.getLocations().length);
+ }
+ return ret;
+ }
+ }).when(spyBM).chooseTarget(Mockito.anyString(), Mockito.anyInt(),
+ Mockito.any(), Mockito.>any(),
+ Mockito.anyLong());
+
+ // create file
+ nn.create(src, FsPermission.getFileDefault(),
+ "clientName",
+ new EnumSetWritable(EnumSet.of(CreateFlag.CREATE)),
+ true, (short)3, 1024);
+
+ // start first addBlock()
+ LOG.info("Starting first addBlock for " + src);
+ nn.addBlock(src, "clientName", null, null, INodeId.GRANDFATHER_INODE_ID);
+
+ // check locations
+ LocatedBlocks lbs = nn.getBlockLocations(src, 0, Long.MAX_VALUE);
+ assertEquals("Must be one block", 1, lbs.getLocatedBlocks().size());
+ lb1 = lbs.get(0);
+ assertEquals("Wrong replication", REPLICATION, lb1.getLocations().length);
+ assertEquals("Blocks are not equal", lb1.getBlock(), lb2.getBlock());
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
new file mode 100644
index 00000000000..2a0ab2003b8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.assertTrue;
+
+import java.net.InetSocketAddress;
+import java.net.URL;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.junit.Test;
+
+/**
+ * DFS_HOSTS and DFS_HOSTS_EXCLUDE tests
+ *
+ */
+public class TestHostsFiles {
+ private static final Log LOG =
+ LogFactory.getLog(TestHostsFiles.class.getName());
+
+ /*
+ * Return a configuration object with low timeouts for testing and
+ * a topology script set (which enables rack awareness).
+ */
+ private Configuration getConf() {
+ Configuration conf = new HdfsConfiguration();
+
+ // Lower the heart beat interval so the NN quickly learns of dead
+ // or decommissioned DNs and the NN issues replication and invalidation
+ // commands quickly (as replies to heartbeats)
+ conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+
+ // Have the NN ReplicationMonitor compute the replication and
+ // invalidation commands to send DNs every second.
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
+
+ // Have the NN check for pending replications every second so it
+ // quickly schedules additional replicas as they are identified.
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 1);
+
+ // The DNs report blocks every second.
+ conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
+
+ // Indicates we have multiple racks
+ conf.set(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, "xyz");
+ return conf;
+ }
+
+ @Test
+ public void testHostsExcludeDfshealthJsp() throws Exception {
+ Configuration conf = getConf();
+ short REPLICATION_FACTOR = 2;
+ final Path filePath = new Path("/testFile");
+
+ // Configure an excludes file
+ FileSystem localFileSys = FileSystem.getLocal(conf);
+ Path workingDir = localFileSys.getWorkingDirectory();
+ Path dir = new Path(workingDir, "build/test/data/temp/decommission");
+ Path excludeFile = new Path(dir, "exclude");
+ Path includeFile = new Path(dir, "include");
+ assertTrue(localFileSys.mkdirs(dir));
+ DFSTestUtil.writeFile(localFileSys, excludeFile, "");
+ DFSTestUtil.writeFile(localFileSys, includeFile, "");
+ conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
+ conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath());
+
+ // Two blocks and four racks
+ String racks[] = {"/rack1", "/rack1", "/rack2", "/rack2"};
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+ .numDataNodes(racks.length).racks(racks).build();
+ final FSNamesystem ns = cluster.getNameNode().getNamesystem();
+
+ try {
+ // Create a file with one block
+ final FileSystem fs = cluster.getFileSystem();
+ DFSTestUtil.createFile(fs, filePath, 1L, REPLICATION_FACTOR, 1L);
+ ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath);
+ DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0);
+
+ // Decommission one of the hosts with the block, this should cause
+ // the block to get replicated to another host on the same rack,
+ // otherwise the rack policy is violated.
+ BlockLocation locs[] = fs.getFileBlockLocations(
+ fs.getFileStatus(filePath), 0, Long.MAX_VALUE);
+ String name = locs[0].getNames()[0];
+ String names = name + "\n" + "localhost:42\n";
+ LOG.info("adding '" + names + "' to exclude file " + excludeFile.toUri().getPath());
+ DFSTestUtil.writeFile(localFileSys, excludeFile, name);
+ ns.getBlockManager().getDatanodeManager().refreshNodes(conf);
+ DFSTestUtil.waitForDecommission(fs, name);
+
+ // Check the block still has sufficient # replicas across racks
+ DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0);
+
+ InetSocketAddress nnHttpAddress = cluster.getNameNode().getHttpAddress();
+ LOG.info("nnaddr = '" + nnHttpAddress + "'");
+ URL nnjsp = new URL("http://" + nnHttpAddress.getHostName() + ":" + nnHttpAddress.getPort() + "/dfshealth.jsp");
+ LOG.info("fetching " + nnjsp);
+ String dfshealthPage = StringEscapeUtils.unescapeHtml(DFSTestUtil.urlGet(nnjsp));
+ LOG.info("got " + dfshealthPage);
+ assertTrue("dfshealth should contain localhost, got:" + dfshealthPage,
+ dfshealthPage.contains("localhost"));
+
+ } finally {
+ cluster.shutdown();
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index 3c5681df022..7c690e9fae4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -24,8 +24,10 @@ import static org.junit.Assert.fail;
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.util.EnumSet;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Options;
@@ -39,6 +41,8 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.io.EnumSetWritable;
import org.junit.Test;
public class TestINodeFile {
@@ -376,7 +380,7 @@ public class TestINodeFile {
* @throws IOException
*/
@Test
- public void TestInodeId() throws IOException {
+ public void testInodeId() throws IOException {
Configuration conf = new Configuration();
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
@@ -396,9 +400,14 @@ public class TestINodeFile {
assertTrue(fs.mkdirs(path));
assertTrue(fsn.getLastInodeId() == 1002);
- Path filePath = new Path("/test1/file");
- fs.create(filePath);
+ // Use namenode rpc to create a file
+ NamenodeProtocols nnrpc = cluster.getNameNodeRpc();
+ HdfsFileStatus fileStatus = nnrpc.create("/test1/file", new FsPermission(
+ (short) 0755), "client",
+ new EnumSetWritable(EnumSet.of(CreateFlag.CREATE)), true,
+ (short) 1, 128 * 1024 * 1024L);
assertTrue(fsn.getLastInodeId() == 1003);
+ assertTrue(fileStatus.getFileId() == 1003);
// Rename doesn't increase inode id
Path renamedPath = new Path("/test2");
@@ -412,4 +421,44 @@ public class TestINodeFile {
cluster.waitActive();
assertTrue(fsn.getLastInodeId() == 1003);
}
+
+ @Test
+ public void testWriteToRenamedFile() throws IOException {
+
+ Configuration conf = new Configuration();
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
+ .build();
+ cluster.waitActive();
+ FileSystem fs = cluster.getFileSystem();
+
+ Path path = new Path("/test1");
+ assertTrue(fs.mkdirs(path));
+
+ int size = conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
+ byte[] data = new byte[size];
+
+ // Create one file
+ Path filePath = new Path("/test1/file");
+ FSDataOutputStream fos = fs.create(filePath);
+
+ // Rename /test1 to test2, and recreate /test1/file
+ Path renamedPath = new Path("/test2");
+ fs.rename(path, renamedPath);
+ fs.create(filePath, (short) 1);
+
+ // Add new block should fail since /test1/file has a different fileId
+ try {
+ fos.write(data, 0, data.length);
+ // make sure addBlock() request gets to NN immediately
+ fos.hflush();
+
+ fail("Write should fail after rename");
+ } catch (Exception e) {
+ /* Ignore */
+ } finally {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryNameNodeUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryNameNodeUpgrade.java
index f3925c963cd..7f5110739ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryNameNodeUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryNameNodeUpgrade.java
@@ -17,9 +17,12 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
+import com.google.common.collect.ImmutableMap;
+
import java.io.File;
import java.io.IOException;
import java.util.List;
+import java.util.Map;
import org.junit.Test;
import org.junit.Before;
@@ -51,7 +54,7 @@ public class TestSecondaryNameNodeUpgrade {
}
}
- private void doIt(String param, String val) throws IOException {
+ private void doIt(Map paramsToCorrupt) throws IOException {
MiniDFSCluster cluster = null;
FileSystem fs = null;
SecondaryNameNode snn = null;
@@ -76,8 +79,12 @@ public class TestSecondaryNameNodeUpgrade {
snn.shutdown();
for (File versionFile : versionFiles) {
- System.out.println("Changing '" + param + "' to '" + val + "' in " + versionFile);
- FSImageTestUtil.corruptVersionFile(versionFile, param, val);
+ for (Map.Entry paramToCorrupt : paramsToCorrupt.entrySet()) {
+ String param = paramToCorrupt.getKey();
+ String val = paramToCorrupt.getValue();
+ System.out.println("Changing '" + param + "' to '" + val + "' in " + versionFile);
+ FSImageTestUtil.corruptVersionFile(versionFile, param, val);
+ }
}
snn = new SecondaryNameNode(conf);
@@ -94,13 +101,19 @@ public class TestSecondaryNameNodeUpgrade {
@Test
public void testUpgradeLayoutVersionSucceeds() throws IOException {
- doIt("layoutVersion", "-39");
+ doIt(ImmutableMap.of("layoutVersion", "-39"));
+ }
+
+ @Test
+ public void testUpgradePreFedSucceeds() throws IOException {
+ doIt(ImmutableMap.of("layoutVersion", "-19", "clusterID", "",
+ "blockpoolID", ""));
}
@Test
public void testChangeNsIDFails() throws IOException {
try {
- doIt("namespaceID", "2");
+ doIt(ImmutableMap.of("namespaceID", "2"));
Assert.fail("Should throw InconsistentFSStateException");
} catch(IOException e) {
GenericTestUtils.assertExceptionContains("Inconsistent checkpoint fields", e);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index d31f12ddfa3..56bf4a8ecae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -82,7 +82,7 @@ public class TestNameNodeMetrics {
CONF.set(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY,
"" + PERCENTILES_INTERVAL);
// Enable stale DataNodes checking
- CONF.setBoolean(DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY, true);
+ CONF.setBoolean(DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY, true);
((Log4JLogger)LogFactory.getLog(MetricsAsserts.class))
.getLogger().setLevel(Level.DEBUG);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
index aa2393d7c01..4e77c07fed5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
import org.apache.hadoop.util.Time;
import org.junit.Assert;
import org.junit.Test;
@@ -42,9 +43,10 @@ public class TestJsonUtil {
public void testHdfsFileStatus() {
final long now = Time.now();
final String parent = "/dir";
- final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L<<26,
- now, now + 10, new FsPermission((short)0644), "user", "group",
- DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"));
+ final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
+ now, now + 10, new FsPermission((short) 0644), "user", "group",
+ DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
+ INodeId.GRANDFATHER_INODE_ID);
final FileStatus fstatus = toFileStatus(status, parent);
System.out.println("status = " + status);
System.out.println("fstatus = " + fstatus);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
index 81f955a9be8..2fb10837fcd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -1182,7 +1182,7 @@
RegexpComparator
- ^1\.0k\s+hdfs:///dir0/data1k
+ ^1\.0 K\s+hdfs:///dir0/data1k
@@ -15590,7 +15590,7 @@
RegexpComparator
- put: The DiskSpace quota of /dir1 is exceeded: quota=1.0k diskspace consumed=[0-9.]+[kmg]*
+ put: The DiskSpace quota of /dir1 is exceeded: quota = 1024 B = 1 KB but diskspace consumed = [0-9]+ B = [0-9.]+ [KMG]B*
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index e405fde57dc..d2dcb9f44d4 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -152,7 +152,22 @@ Trunk (Unreleased)
MAPREDUCE-4884. Streaming tests fail to start MiniMRCluster due to missing
queue configuration. (Chris Nauroth via suresh)
-Release 2.0.3-alpha - Unreleased
+Release 2.0.4-beta - UNRELEASED
+
+ INCOMPATIBLE CHANGES
+
+ NEW FEATURES
+
+ IMPROVEMENTS
+
+ OPTIMIZATIONS
+
+ BUG FIXES
+
+ MAPREDUCE-4671. AM does not tell the RM about container requests which are
+ no longer needed. (Bikas Saha via sseth)
+
+Release 2.0.3-alpha - 2013-02-06
INCOMPATIBLE CHANGES
@@ -215,6 +230,12 @@ Release 2.0.3-alpha - Unreleased
MAPREDUCE-4838. Add additional fields like Locality, Avataar to the
JobHistory logs. (Zhijie Shen via sseth)
+ MAPREDUCE-4971. Minor extensibility enhancements to Counters &
+ FileOutputFormat. (Arun C Murthy via sseth)
+
+ MAPREDUCE-4977. Documentation for pluggable shuffle and pluggable sort.
+ (tucu)
+
OPTIMIZATIONS
MAPREDUCE-4893. Fixed MR ApplicationMaster to do optimal assignment of
@@ -284,6 +305,8 @@ Release 2.0.3-alpha - Unreleased
MAPREDUCE-4969. TestKeyValueTextInputFormat test fails with Open JDK 7.
(Arpit Agarwal via suresh)
+ MAPREDUCE-4953. HadoopPipes misuses fprintf. (Andy Isaacson via atm)
+
Release 2.0.2-alpha - 2012-09-07
INCOMPATIBLE CHANGES
@@ -668,11 +691,17 @@ Release 0.23.7 - UNRELEASED
IMPROVEMENTS
+ MAPREDUCE-4905. test org.apache.hadoop.mapred.pipes
+ (Aleksey Gorshkov via bobby)
+
OPTIMIZATIONS
MAPREDUCE-4946. Fix a performance problem for large jobs by reducing the
number of map completion event type conversions. (Jason Lowe via sseth)
+ MAPREDUCE-4822. Unnecessary conversions in History Events. (Chu Tong via
+ jlowe)
+
BUG FIXES
MAPREDUCE-4458. Warn if java.library.path is used for AM or Task
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
index 59c9795e2d8..5f3a7f5f960 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
@@ -72,7 +72,10 @@ public abstract class RMContainerRequestor extends RMCommunicator {
remoteRequestsTable =
new TreeMap>>();
- private final Set ask = new TreeSet();
+ // use custom comparator to make sure ResourceRequest objects differing only in
+ // numContainers dont end up as duplicates
+ private final Set ask = new TreeSet(
+ new org.apache.hadoop.yarn.util.BuilderUtils.ResourceRequestComparator());
private final Set release = new TreeSet();
private boolean nodeBlacklistingEnabled;
@@ -235,7 +238,7 @@ public abstract class RMContainerRequestor extends RMCommunicator {
ResourceRequest zeroedRequest = BuilderUtils.newResourceRequest(req);
zeroedRequest.setNumContainers(0);
// to be sent to RM on next heartbeat
- ask.add(zeroedRequest);
+ addResourceRequestToAsk(zeroedRequest);
}
}
// if all requests were still in ask queue
@@ -320,7 +323,7 @@ public abstract class RMContainerRequestor extends RMCommunicator {
remoteRequest.setNumContainers(remoteRequest.getNumContainers() + 1);
// Note this down for next interaction with ResourceManager
- ask.add(remoteRequest);
+ addResourceRequestToAsk(remoteRequest);
if (LOG.isDebugEnabled()) {
LOG.debug("addResourceRequest:" + " applicationId="
+ applicationId.getId() + " priority=" + priority.getPriority()
@@ -353,7 +356,12 @@ public abstract class RMContainerRequestor extends RMCommunicator {
+ remoteRequest.getNumContainers() + " #asks=" + ask.size());
}
- remoteRequest.setNumContainers(remoteRequest.getNumContainers() -1);
+ if(remoteRequest.getNumContainers() > 0) {
+ // based on blacklisting comments above we can end up decrementing more
+ // than requested. so guard for that.
+ remoteRequest.setNumContainers(remoteRequest.getNumContainers() -1);
+ }
+
if (remoteRequest.getNumContainers() == 0) {
reqMap.remove(capability);
if (reqMap.size() == 0) {
@@ -362,13 +370,12 @@ public abstract class RMContainerRequestor extends RMCommunicator {
if (remoteRequests.size() == 0) {
remoteRequestsTable.remove(priority);
}
- //remove from ask if it may have
- ask.remove(remoteRequest);
- } else {
- ask.add(remoteRequest);//this will override the request if ask doesn't
- //already have it.
}
+ // send the updated resource request to RM
+ // send 0 container count requests also to cancel previous requests
+ addResourceRequestToAsk(remoteRequest);
+
if (LOG.isDebugEnabled()) {
LOG.info("AFTER decResourceRequest:" + " applicationId="
+ applicationId.getId() + " priority=" + priority.getPriority()
@@ -376,6 +383,16 @@ public abstract class RMContainerRequestor extends RMCommunicator {
+ remoteRequest.getNumContainers() + " #asks=" + ask.size());
}
}
+
+ private void addResourceRequestToAsk(ResourceRequest remoteRequest) {
+ // because objects inside the resource map can be deleted ask can end up
+ // containing an object that matches new resource object but with different
+ // numContainers. So exisintg values must be replaced explicitly
+ if(ask.contains(remoteRequest)) {
+ ask.remove(remoteRequest);
+ }
+ ask.add(remoteRequest);
+ }
protected void release(ContainerId containerId) {
release.add(containerId);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
index bd00e1b1608..47845a0aa6f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
@@ -167,6 +167,7 @@ public class TestRMContainerAllocator {
List assigned = allocator.schedule();
dispatcher.await();
Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
+ Assert.assertEquals(4, rm.getMyFifoScheduler().lastAsk.size());
// send another request with different resource and priority
ContainerRequestEvent event3 = createReq(jobId, 3, 1024,
@@ -178,7 +179,8 @@ public class TestRMContainerAllocator {
assigned = allocator.schedule();
dispatcher.await();
Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
-
+ Assert.assertEquals(3, rm.getMyFifoScheduler().lastAsk.size());
+
// update resources in scheduler
nodeManager1.nodeHeartbeat(true); // Node heartbeat
nodeManager2.nodeHeartbeat(true); // Node heartbeat
@@ -187,8 +189,14 @@ public class TestRMContainerAllocator {
assigned = allocator.schedule();
dispatcher.await();
+ Assert.assertEquals(0, rm.getMyFifoScheduler().lastAsk.size());
checkAssignments(new ContainerRequestEvent[] { event1, event2, event3 },
assigned, false);
+
+ // check that the assigned container requests are cancelled
+ assigned = allocator.schedule();
+ dispatcher.await();
+ Assert.assertEquals(5, rm.getMyFifoScheduler().lastAsk.size());
}
@Test
@@ -422,7 +430,7 @@ public class TestRMContainerAllocator {
}
private static class MyResourceManager extends MockRM {
-
+
public MyResourceManager(Configuration conf) {
super(conf);
}
@@ -446,6 +454,10 @@ public class TestRMContainerAllocator {
protected ResourceScheduler createScheduler() {
return new MyFifoScheduler(this.getRMContext());
}
+
+ MyFifoScheduler getMyFifoScheduler() {
+ return (MyFifoScheduler) scheduler;
+ }
}
@Test
@@ -1194,7 +1206,9 @@ public class TestRMContainerAllocator {
assert (false);
}
}
-
+
+ List lastAsk = null;
+
// override this to copy the objects otherwise FifoScheduler updates the
// numContainers in same objects as kept by RMContainerAllocator
@Override
@@ -1208,6 +1222,7 @@ public class TestRMContainerAllocator {
.getNumContainers());
askCopy.add(reqCopy);
}
+ lastAsk = ask;
return super.allocate(applicationAttemptId, askCopy, release);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
index 656e49e6e70..c0cfc072284 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
@@ -60,7 +60,7 @@ import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.SystemClock;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.resourcetracker.InlineDispatcher;
+import org.apache.hadoop.yarn.event.InlineDispatcher;
import org.apache.hadoop.yarn.util.Records;
import org.junit.After;
import org.junit.Before;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Counters.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Counters.java
index da25b2aa137..7d648a49126 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Counters.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Counters.java
@@ -230,6 +230,10 @@ public class Counters
public static class Group implements CounterGroupBase {
private CounterGroupBase realGroup;
+ protected Group() {
+ realGroup = null;
+ }
+
Group(GenericGroup group) {
this.realGroup = group;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java
index a6190d2060d..496280a7371 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java
@@ -92,7 +92,7 @@ public class FileOutputCommitter extends OutputCommitter {
}
@Private
- Path getTaskAttemptPath(TaskAttemptContext context) throws IOException {
+ public Path getTaskAttemptPath(TaskAttemptContext context) throws IOException {
Path out = getOutputPath(context);
return out == null ? null : getTaskAttemptPath(context, out);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputFormat.java
index 892e6906b0c..9082de8ce50 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputFormat.java
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.text.NumberFormat;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -152,8 +153,8 @@ public abstract class FileOutputFormat implements OutputFormat {
* @param outputDir the {@link Path} of the output directory
* for the map-reduce job.
*/
-
- static void setWorkOutputPath(JobConf conf, Path outputDir) {
+ @Private
+ public static void setWorkOutputPath(JobConf conf, Path outputDir) {
outputDir = new Path(conf.getWorkingDirectory(), outputDir);
conf.set(JobContext.TASK_OUTPUT_DIR, outputDir.toString());
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
index 1304755e3c3..e76f62856da 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
@@ -28,6 +28,7 @@ import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -419,7 +420,8 @@ public class JobConf extends Configuration {
return credentials;
}
- void setCredentials(Credentials credentials) {
+ @Private
+ public void setCredentials(Credentials credentials) {
this.credentials = credentials;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java
index 57370872e30..8f4259e3ec1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.mapred.lib.LazyOutputFormat;
import org.apache.hadoop.mapred.lib.NullOutputFormat;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
+import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.Tool;
@@ -515,7 +516,7 @@ public class Submitter extends Configured implements Tool {
*/
public static void main(String[] args) throws Exception {
int exitCode = new Submitter().run(args);
- System.exit(exitCode);
+ ExitUtil.terminate(exitCode);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FileSystemCounterGroup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FileSystemCounterGroup.java
index 3f3729fb056..b1b67969cfd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FileSystemCounterGroup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FileSystemCounterGroup.java
@@ -35,6 +35,7 @@ import com.google.common.collect.Iterators;
import com.google.common.collect.Maps;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.FileSystemCounter;
@@ -72,6 +73,16 @@ public abstract class FileSystemCounterGroup
this.scheme = scheme;
key = ref;
}
+
+ @Private
+ public String getScheme() {
+ return scheme;
+ }
+
+ @Private
+ public FileSystemCounter getFileSystemCounter() {
+ return key;
+ }
@Override
public String getName() {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FrameworkCounterGroup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FrameworkCounterGroup.java
index 64e4cc81631..5ebed5a7319 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FrameworkCounterGroup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FrameworkCounterGroup.java
@@ -29,6 +29,7 @@ import java.util.Iterator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.util.ResourceBundles;
@@ -66,7 +67,17 @@ public abstract class FrameworkCounterGroup,
key = ref;
this.groupName = groupName;
}
+
+ @Private
+ public T getKey() {
+ return key;
+ }
+ @Private
+ public String getGroupName() {
+ return groupName;
+ }
+
@Override
public String getName() {
return key.name();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
index d4f9fa316b9..62df2aae929 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
@@ -178,7 +178,7 @@ public class MapAttemptFinishedEvent implements HistoryEvent {
/** Get the task type */
public TaskType getTaskType() {
- return TaskType.valueOf(taskType.toString());
+ return taskType;
}
/** Get the task status */
public String getTaskStatus() { return taskStatus.toString(); }
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
index 10b8c1f0139..a779fca6ec8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
@@ -176,11 +176,11 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent {
public TaskID getTaskId() { return attemptId.getTaskID(); }
/** Get the attempt id */
public TaskAttemptID getAttemptId() {
- return TaskAttemptID.forName(attemptId.toString());
+ return attemptId;
}
/** Get the task type */
public TaskType getTaskType() {
- return TaskType.valueOf(taskType.toString());
+ return taskType;
}
/** Get the task status */
public String getTaskStatus() { return taskStatus.toString(); }
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
index a62ca38e4a8..78b9ca914f6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
@@ -105,11 +105,11 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
public TaskID getTaskId() { return attemptId.getTaskID(); }
/** Get the task attempt id */
public TaskAttemptID getAttemptId() {
- return TaskAttemptID.forName(attemptId.toString());
+ return attemptId;
}
/** Get the task type */
public TaskType getTaskType() {
- return TaskType.valueOf(taskType.toString());
+ return taskType;
}
/** Get the task status */
public String getTaskStatus() { return taskStatus.toString(); }
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
index 55de80ca63f..edbf0099ff9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
@@ -95,14 +95,10 @@ public class TaskFinishedEvent implements HistoryEvent {
}
/** Get task id */
- public TaskID getTaskId() { return TaskID.forName(taskid.toString()); }
+ public TaskID getTaskId() { return taskid; }
/** Get successful task attempt id */
public TaskAttemptID getSuccessfulTaskAttemptId() {
- if(successfulAttemptId != null)
- {
- return TaskAttemptID.forName(successfulAttemptId.toString());
- }
- return null;
+ return successfulAttemptId;
}
/** Get the task finish time */
public long getFinishTime() { return finishTime; }
@@ -110,7 +106,7 @@ public class TaskFinishedEvent implements HistoryEvent {
public Counters getCounters() { return counters; }
/** Get task type */
public TaskType getTaskType() {
- return TaskType.valueOf(taskType.toString());
+ return taskType;
}
/** Get task status */
public String getTaskStatus() { return status.toString(); }
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/apt/PluggableShuffleAndPluggableSort.apt.vm b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/apt/PluggableShuffleAndPluggableSort.apt.vm
new file mode 100644
index 00000000000..8dd2f2eceff
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/apt/PluggableShuffleAndPluggableSort.apt.vm
@@ -0,0 +1,96 @@
+~~ Licensed under the Apache License, Version 2.0 (the "License");
+~~ you may not use this file except in compliance with the License.
+~~ You may obtain a copy of the License at
+~~
+~~ http://www.apache.org/licenses/LICENSE-2.0
+~~
+~~ Unless required by applicable law or agreed to in writing, software
+~~ distributed under the License is distributed on an "AS IS" BASIS,
+~~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+~~ See the License for the specific language governing permissions and
+~~ limitations under the License. See accompanying LICENSE file.
+
+ ---
+ Hadoop Map Reduce Next Generation-${project.version} - Pluggable Shuffle and Pluggable Sort
+ ---
+ ---
+ ${maven.build.timestamp}
+
+Hadoop MapReduce Next Generation - Pluggable Shuffle and Pluggable Sort
+
+ \[ {{{./index.html}Go Back}} \]
+
+* Introduction
+
+ The pluggable shuffle and pluggable sort capabilities allow replacing the
+ built in shuffle and sort logic with alternate implementations. Example use
+ cases for this are: using a different application protocol other than HTTP
+ such as RDMA for shuffling data from the Map nodes to the Reducer nodes; or
+ replacing the sort logic with custom algorithms that enable Hash aggregation
+ and Limit-N query.
+
+ <> The pluggable shuffle and pluggable sort capabilities are
+ experimental and unstable. This means the provided APIs may change and break
+ compatibility in future versions of Hadoop.
+
+* Implementing a Custom Shuffle and a Custom Sort
+
+ A custom shuffle implementation requires a
+ <<>>
+ implementation class running in the NodeManagers and a
+ <<>> implementation class
+ running in the Reducer tasks.
+
+ The default implementations provided by Hadoop can be used as references:
+
+ * <<>>
+
+ * <<>>
+
+ A custom sort implementation requires a <<>>
+ implementation class running in the Mapper tasks and (optionally, depending
+ on the sort implementation) a <<>>
+ implementation class running in the Reducer tasks.
+
+ The default implementations provided by Hadoop can be used as references:
+
+ * <<>>
+
+ * <<>>
+
+* Configuration
+
+ Except for the auxiliary service running in the NodeManagers serving the
+ shuffle (by default the <<>>), all the pluggable components
+ run in the job tasks. This means, they can be configured on per job basis.
+ The auxiliary service servicing the Shuffle must be configured in the
+ NodeManagers configuration.
+
+** Job Configuration Properties (on per job basis):
+
+*--------------------------------------+---------------------+-----------------+
+| <> | <> | <> |
+*--------------------------------------+---------------------+-----------------+
+| <<>> | <<>> | The <<>> implementation to use |
+*--------------------------------------+---------------------+-----------------+
+| <<>> | <<>> | The <<>> implementation to use |
+*--------------------------------------+---------------------+-----------------+
+
+ These properties can also be set in the <<>> to change the default values for all jobs.
+
+** NodeManager Configuration properties, <<>> in all nodes:
+
+*--------------------------------------+---------------------+-----------------+
+| <> | <> | <> |
+*--------------------------------------+---------------------+-----------------+
+| <<>> | <<<...,mapreduce.shuffle>>> | The auxiliary service name |
+*--------------------------------------+---------------------+-----------------+
+| <<>> | <<>> | The auxiliary service class to use |
+*--------------------------------------+---------------------+-----------------+
+
+ <> If setting an auxiliary service in addition the default
+ <<>> service, then a new service key should be added to the
+ <<>> property, for example <<>>.
+ Then the property defining the corresponding class must be
+ <<>>.
+
\ No newline at end of file
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/CommonStub.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/CommonStub.java
new file mode 100644
index 00000000000..f1f11a17be4
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/CommonStub.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.pipes;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.Socket;
+
+import javax.crypto.SecretKey;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
+
+public class CommonStub {
+
+ protected Socket socket = null;
+ protected DataInputStream dataInput;
+ protected DataOutputStream dataOut;
+
+ protected String createDigest(byte[] password, String data) throws IOException {
+ SecretKey key = JobTokenSecretManager.createSecretKey(password);
+
+ return SecureShuffleUtils.hashFromString(data, key);
+
+ }
+
+ protected void readObject(Writable obj, DataInputStream inStream) throws IOException {
+ int numBytes = WritableUtils.readVInt(inStream);
+ byte[] buffer;
+ // For BytesWritable and Text, use the specified length to set the length
+ // this causes the "obvious" translations to work. So that if you emit
+ // a string "abc" from C++, it shows up as "abc".
+ if (obj instanceof BytesWritable) {
+ buffer = new byte[numBytes];
+ inStream.readFully(buffer);
+ ((BytesWritable) obj).set(buffer, 0, numBytes);
+ } else if (obj instanceof Text) {
+ buffer = new byte[numBytes];
+ inStream.readFully(buffer);
+ ((Text) obj).set(buffer);
+ } else {
+ obj.readFields(inStream);
+ }
+ }
+
+
+ protected void writeObject(Writable obj, DataOutputStream stream)
+ throws IOException {
+ // For Text and BytesWritable, encode them directly, so that they end up
+ // in C++ as the natural translations.
+ DataOutputBuffer buffer = new DataOutputBuffer();
+ if (obj instanceof Text) {
+ Text t = (Text) obj;
+ int len = t.getLength();
+ WritableUtils.writeVLong(stream, len);
+ stream.flush();
+
+ stream.write(t.getBytes(), 0, len);
+ stream.flush();
+
+ } else if (obj instanceof BytesWritable) {
+ BytesWritable b = (BytesWritable) obj;
+ int len = b.getLength();
+ WritableUtils.writeVLong(stream, len);
+ stream.write(b.getBytes(), 0, len);
+ } else {
+ buffer.reset();
+ obj.write(buffer);
+ int length = buffer.getLength();
+
+ WritableUtils.writeVInt(stream, length);
+ stream.write(buffer.getData(), 0, length);
+ }
+ stream.flush();
+
+ }
+
+ protected void initSoket() throws Exception {
+ int port = Integer.parseInt(System.getenv("mapreduce.pipes.command.port"));
+
+ java.net.InetAddress address = java.net.InetAddress.getLocalHost();
+
+ socket = new Socket(address.getHostName(), port);
+ InputStream input = socket.getInputStream();
+ OutputStream output = socket.getOutputStream();
+
+ // try to read
+ dataInput = new DataInputStream(input);
+
+ WritableUtils.readVInt(dataInput);
+
+ String str = Text.readString(dataInput);
+
+ Text.readString(dataInput);
+
+ dataOut = new DataOutputStream(output);
+ WritableUtils.writeVInt(dataOut, 57);
+ String s = createDigest("password".getBytes(), str);
+
+ Text.writeString(dataOut, s);
+
+ // start
+ WritableUtils.readVInt(dataInput);
+ int cuttentAnswer = WritableUtils.readVInt(dataInput);
+ System.out.println("CURRENT_PROTOCOL_VERSION:" + cuttentAnswer);
+
+ // get configuration
+ // should be MessageType.SET_JOB_CONF.code
+ WritableUtils.readVInt(dataInput);
+
+ // array length
+
+ int j = WritableUtils.readVInt(dataInput);
+ for (int i = 0; i < j; i++) {
+ Text.readString(dataInput);
+ i++;
+ Text.readString(dataInput);
+ }
+ }
+
+ protected void closeSoket() {
+ if (socket != null) {
+ try {
+ socket.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/PipeApplicationRunnableStub.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/PipeApplicationRunnableStub.java
new file mode 100644
index 00000000000..c2cc794c56d
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/PipeApplicationRunnableStub.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.pipes;
+
+
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+
+/*
+ Stub for TestPipeApplication test. This stub produced test data for main test. Main test checks data
+ */
+
+public class PipeApplicationRunnableStub extends CommonStub {
+
+ public static void main(String[] args) {
+ PipeApplicationRunnableStub client = new PipeApplicationRunnableStub();
+ client.binaryProtocolStub();
+ }
+
+ public void binaryProtocolStub() {
+ try {
+
+ initSoket();
+ System.out.println("start OK");
+
+ // RUN_MAP.code
+ // should be 3
+
+ int answer = WritableUtils.readVInt(dataInput);
+ System.out.println("RunMap:" + answer);
+ TestPipeApplication.FakeSplit split = new TestPipeApplication.FakeSplit();
+ readObject(split, dataInput);
+
+ WritableUtils.readVInt(dataInput);
+ WritableUtils.readVInt(dataInput);
+ // end runMap
+ // get InputTypes
+ WritableUtils.readVInt(dataInput);
+ String inText = Text.readString(dataInput);
+ System.out.println("Key class:" + inText);
+ inText = Text.readString(dataInput);
+ System.out.println("Value class:" + inText);
+
+ @SuppressWarnings("unused")
+ int inCode = 0;
+
+ // read all data from sender and write to output
+ while ((inCode = WritableUtils.readVInt(dataInput)) == 4) {
+ FloatWritable key = new FloatWritable();
+ NullWritable value = NullWritable.get();
+ readObject(key, dataInput);
+ System.out.println("value:" + key.get());
+ readObject(value, dataInput);
+ }
+
+ WritableUtils.writeVInt(dataOut, 54);
+
+ dataOut.flush();
+ dataOut.close();
+
+ } catch (Exception x) {
+ x.printStackTrace();
+ } finally {
+ closeSoket();
+ }
+
+ }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/PipeApplicationStub.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/PipeApplicationStub.java
new file mode 100644
index 00000000000..33e0b813eb3
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/PipeApplicationStub.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.pipes;
+
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+/*
+Stub for TestPipeApplication test. This stub produced test data for main test. Main test checks data
+ */
+
+public class PipeApplicationStub extends CommonStub {
+
+ public static void main(String[] args) {
+ PipeApplicationStub client = new PipeApplicationStub();
+ client.binaryProtocolStub();
+ }
+
+ public void binaryProtocolStub() {
+ try {
+
+ initSoket();
+
+ // output code
+ WritableUtils.writeVInt(dataOut, 50);
+ IntWritable wt = new IntWritable();
+ wt.set(123);
+ writeObject(wt, dataOut);
+ writeObject(new Text("value"), dataOut);
+
+ // PARTITIONED_OUTPUT
+ WritableUtils.writeVInt(dataOut, 51);
+ WritableUtils.writeVInt(dataOut, 0);
+ writeObject(wt, dataOut);
+ writeObject(new Text("value"), dataOut);
+
+
+ // STATUS
+ WritableUtils.writeVInt(dataOut, 52);
+ Text.writeString(dataOut, "PROGRESS");
+ dataOut.flush();
+
+ // progress
+ WritableUtils.writeVInt(dataOut, 53);
+ dataOut.writeFloat(0.55f);
+ // register counter
+ WritableUtils.writeVInt(dataOut, 55);
+ // id
+ WritableUtils.writeVInt(dataOut, 0);
+ Text.writeString(dataOut, "group");
+ Text.writeString(dataOut, "name");
+ // increment counter
+ WritableUtils.writeVInt(dataOut, 56);
+ WritableUtils.writeVInt(dataOut, 0);
+
+ WritableUtils.writeVLong(dataOut, 2);
+
+ // map item
+ int intValue = WritableUtils.readVInt(dataInput);
+ System.out.println("intValue:" + intValue);
+ IntWritable iw = new IntWritable();
+ readObject(iw, dataInput);
+ System.out.println("key:" + iw.get());
+ Text txt = new Text();
+ readObject(txt, dataInput);
+ System.out.println("value:" + txt.toString());
+
+ // done
+ // end of session
+ WritableUtils.writeVInt(dataOut, 54);
+
+ System.out.println("finish");
+ dataOut.flush();
+ dataOut.close();
+
+ } catch (Exception x) {
+ x.printStackTrace();
+ } finally {
+ closeSoket();
+ }
+ }
+
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/PipeReducerStub.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/PipeReducerStub.java
new file mode 100644
index 00000000000..a0afdbc9477
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/PipeReducerStub.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.pipes;
+
+
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+
+/*
+Stub for TestPipeApplication test. This stub produced test data for main test. Main test checks data
+ */
+
+public class PipeReducerStub extends CommonStub {
+
+ public static void main(String[] args) {
+ PipeReducerStub client = new PipeReducerStub();
+ client.binaryProtocolStub();
+ }
+
+ public void binaryProtocolStub() {
+ try {
+
+ initSoket();
+
+ //should be 5
+ //RUN_REDUCE boolean
+ WritableUtils.readVInt(dataInput);
+ WritableUtils.readVInt(dataInput);
+ int intValue = WritableUtils.readVInt(dataInput);
+ System.out.println("getIsJavaRecordWriter:" + intValue);
+
+ // reduce key
+ WritableUtils.readVInt(dataInput);
+ // value of reduce key
+ BooleanWritable value = new BooleanWritable();
+ readObject(value, dataInput);
+ System.out.println("reducer key :" + value);
+ // reduce value code:
+
+ // reduce values
+ while ((intValue = WritableUtils.readVInt(dataInput)) == 7) {
+ Text txt = new Text();
+ // value
+ readObject(txt, dataInput);
+ System.out.println("reduce value :" + txt);
+ }
+
+
+ // done
+ WritableUtils.writeVInt(dataOut, 54);
+
+ dataOut.flush();
+ dataOut.close();
+
+ } catch (Exception x) {
+ x.printStackTrace();
+ } finally {
+ closeSoket();
+
+ }
+ }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
new file mode 100644
index 00000000000..aa345c42c72
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
@@ -0,0 +1,747 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.pipes;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.IFile.Writer;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapred.Counters.Group;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskLog;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestPipeApplication {
+ private static File workSpace = new File("target",
+ TestPipeApplication.class.getName() + "-workSpace");
+
+ private static String taskName = "attempt_001_02_r03_04_05";
+
+ /**
+ * test PipesMapRunner test the transfer data from reader
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testRunner() throws Exception {
+
+ // clean old password files
+ File[] psw = cleanTokenPasswordFile();
+ try {
+ RecordReader rReader = new ReaderPipesMapRunner();
+ JobConf conf = new JobConf();
+ conf.set(Submitter.IS_JAVA_RR, "true");
+ // for stdour and stderror
+
+ conf.set(MRJobConfig.TASK_ATTEMPT_ID, taskName);
+
+ CombineOutputCollector output = new CombineOutputCollector(
+ new Counters.Counter(), new Progress());
+ FileSystem fs = new RawLocalFileSystem();
+ fs.setConf(conf);
+ Writer wr = new Writer(conf, fs,
+ new Path(workSpace + File.separator + "outfile"), IntWritable.class,
+ Text.class, null, null);
+ output.setWriter(wr);
+ // stub for client
+ File fCommand = getFileCommand("org.apache.hadoop.mapred.pipes.PipeApplicationRunnableStub");
+
+ conf.set(MRJobConfig.CACHE_LOCALFILES, fCommand.getAbsolutePath());
+ // token for authorization
+ Token token = new Token(
+ "user".getBytes(), "password".getBytes(), new Text("kind"), new Text(
+ "service"));
+ conf.getCredentials().addToken(new Text("ShuffleAndJobToken"), token);
+ conf.setBoolean(MRJobConfig.SKIP_RECORDS, true);
+ TestTaskReporter reporter = new TestTaskReporter();
+ PipesMapRunner runner = new PipesMapRunner();
+
+ initStdOut(conf);
+
+ runner.configure(conf);
+ runner.run(rReader, output, reporter);
+
+ String stdOut = readStdOut(conf);
+
+ // test part of translated data. As common file for client and test -
+ // clients stdOut
+ // check version
+ assertTrue(stdOut.contains("CURRENT_PROTOCOL_VERSION:0"));
+ // check key and value classes
+ assertTrue(stdOut
+ .contains("Key class:org.apache.hadoop.io.FloatWritable"));
+ assertTrue(stdOut
+ .contains("Value class:org.apache.hadoop.io.NullWritable"));
+ // test have sent all data from reader
+ assertTrue(stdOut.contains("value:0.0"));
+ assertTrue(stdOut.contains("value:9.0"));
+
+ } finally {
+ if (psw != null) {
+ // remove password files
+ for (File file : psw) {
+ file.deleteOnExit();
+ }
+ }
+
+ }
+ }
+
+ /**
+ * test org.apache.hadoop.mapred.pipes.Application
+ * test a internal functions: MessageType.REGISTER_COUNTER, INCREMENT_COUNTER, STATUS, PROGRESS...
+ *
+ * @throws Throwable
+ */
+
+ @Test
+ public void testApplication() throws Throwable {
+ JobConf conf = new JobConf();
+
+ RecordReader rReader = new Reader();
+
+ // client for test
+ File fCommand = getFileCommand("org.apache.hadoop.mapred.pipes.PipeApplicationStub");
+
+ TestTaskReporter reporter = new TestTaskReporter();
+
+ File[] psw = cleanTokenPasswordFile();
+ try {
+
+ conf.set(MRJobConfig.TASK_ATTEMPT_ID, taskName);
+ conf.set(MRJobConfig.CACHE_LOCALFILES, fCommand.getAbsolutePath());
+
+ // token for authorization
+ Token token = new Token(
+ "user".getBytes(), "password".getBytes(), new Text("kind"), new Text(
+ "service"));
+
+ conf.getCredentials().addToken(new Text("ShuffleAndJobToken"), token);
+ FakeCollector output = new FakeCollector(new Counters.Counter(),
+ new Progress());
+ FileSystem fs = new RawLocalFileSystem();
+ fs.setConf(conf);
+ Writer wr = new Writer(conf, fs,
+ new Path(workSpace.getAbsolutePath() + File.separator + "outfile"),
+ IntWritable.class, Text.class, null, null);
+ output.setWriter(wr);
+ conf.set(Submitter.PRESERVE_COMMANDFILE, "true");
+
+ Application, Writable, IntWritable, Text> application = new Application, Writable, IntWritable, Text>(
+ conf, rReader, output, reporter, IntWritable.class, Text.class);
+ application.getDownlink().flush();
+
+ application.getDownlink().mapItem(new IntWritable(3), new Text("txt"));
+
+ application.getDownlink().flush();
+
+ application.waitForFinish();
+
+ wr.close();
+
+ // test getDownlink().mapItem();
+ String stdOut = readStdOut(conf);
+ assertTrue(stdOut.contains("key:3"));
+ assertTrue(stdOut.contains("value:txt"));
+
+ // reporter test counter, and status should be sended
+ // test MessageType.REGISTER_COUNTER and INCREMENT_COUNTER
+ assertEquals(1.0, reporter.getProgress(), 0.01);
+ assertNotNull(reporter.getCounter("group", "name"));
+ // test status MessageType.STATUS
+ assertEquals(reporter.getStatus(), "PROGRESS");
+ stdOut = readFile(new File(workSpace.getAbsolutePath() + File.separator
+ + "outfile"));
+ // check MessageType.PROGRESS
+ assertEquals(0.55f, rReader.getProgress(), 0.001);
+ application.getDownlink().close();
+ // test MessageType.OUTPUT
+ Entry entry = output.getCollect().entrySet()
+ .iterator().next();
+ assertEquals(123, entry.getKey().get());
+ assertEquals("value", entry.getValue().toString());
+ try {
+ // try to abort
+ application.abort(new Throwable());
+ fail();
+ } catch (IOException e) {
+ // abort works ?
+ assertEquals("pipe child exception", e.getMessage());
+ }
+ } finally {
+ if (psw != null) {
+ // remove password files
+ for (File file : psw) {
+ file.deleteOnExit();
+ }
+ }
+ }
+ }
+
+ /**
+ * test org.apache.hadoop.mapred.pipes.Submitter
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testSubmitter() throws Exception {
+
+ JobConf conf = new JobConf();
+
+ File[] psw = cleanTokenPasswordFile();
+
+ System.setProperty("test.build.data",
+ "target/tmp/build/TEST_SUBMITTER_MAPPER/data");
+ conf.set("hadoop.log.dir", "target/tmp");
+
+ // prepare configuration
+ Submitter.setIsJavaMapper(conf, false);
+ Submitter.setIsJavaReducer(conf, false);
+ Submitter.setKeepCommandFile(conf, false);
+ Submitter.setIsJavaRecordReader(conf, false);
+ Submitter.setIsJavaRecordWriter(conf, false);
+ PipesPartitioner partitioner = new PipesPartitioner();
+ partitioner.configure(conf);
+
+ Submitter.setJavaPartitioner(conf, partitioner.getClass());
+
+ assertEquals(PipesPartitioner.class, (Submitter.getJavaPartitioner(conf)));
+ // test going to call main method with System.exit(). Change Security
+ SecurityManager securityManager = System.getSecurityManager();
+ // store System.out
+ PrintStream oldps = System.out;
+ ByteArrayOutputStream out = new ByteArrayOutputStream();
+ ExitUtil.disableSystemExit();
+ // test without parameters
+ try {
+ System.setOut(new PrintStream(out));
+ Submitter.main(new String[0]);
+ fail();
+ } catch (ExitUtil.ExitException e) {
+ // System.exit prohibited! output message test
+ assertTrue(out.toString().contains(""));
+ assertTrue(out.toString().contains("bin/hadoop pipes"));
+ assertTrue(out.toString().contains("[-input ] // Input directory"));
+ assertTrue(out.toString()
+ .contains("[-output ] // Output directory"));
+ assertTrue(out.toString().contains("[-jar // jar filename"));
+ assertTrue(out.toString().contains(
+ "[-inputformat ] // InputFormat class"));
+ assertTrue(out.toString().contains("[-map ] // Java Map class"));
+ assertTrue(out.toString().contains(
+ "[-partitioner ] // Java Partitioner"));
+ assertTrue(out.toString().contains(
+ "[-reduce ] // Java Reduce class"));
+ assertTrue(out.toString().contains(
+ "[-writer ] // Java RecordWriter"));
+ assertTrue(out.toString().contains(
+ "[-program ] // executable URI"));
+ assertTrue(out.toString().contains(
+ "[-reduces ] // number of reduces"));
+ assertTrue(out.toString().contains(
+ "[-lazyOutput ] // createOutputLazily"));
+
+ assertTrue(out
+ .toString()
+ .contains(
+ "-conf specify an application configuration file"));
+ assertTrue(out.toString().contains(
+ "-D use value for given property"));
+ assertTrue(out.toString().contains(
+ "-fs specify a namenode"));
+ assertTrue(out.toString().contains(
+ "-jt specify a job tracker"));
+ assertTrue(out
+ .toString()
+ .contains(
+ "-files specify comma separated files to be copied to the map reduce cluster"));
+ assertTrue(out
+ .toString()
+ .contains(
+ "-libjars specify comma separated jar files to include in the classpath."));
+ assertTrue(out
+ .toString()
+ .contains(
+ "-archives specify comma separated archives to be unarchived on the compute machines."));
+ } finally {
+ System.setOut(oldps);
+ // restore
+ System.setSecurityManager(securityManager);
+ if (psw != null) {
+ // remove password files
+ for (File file : psw) {
+ file.deleteOnExit();
+ }
+ }
+ }
+ // test call Submitter form command line
+ try {
+ File fCommand = getFileCommand(null);
+ String[] args = new String[22];
+ File input = new File(workSpace + File.separator + "input");
+ if (!input.exists()) {
+ Assert.assertTrue(input.createNewFile());
+ }
+ File outPut = new File(workSpace + File.separator + "output");
+ FileUtil.fullyDelete(outPut);
+
+ args[0] = "-input";
+ args[1] = input.getAbsolutePath();// "input";
+ args[2] = "-output";
+ args[3] = outPut.getAbsolutePath();// "output";
+ args[4] = "-inputformat";
+ args[5] = "org.apache.hadoop.mapred.TextInputFormat";
+ args[6] = "-map";
+ args[7] = "org.apache.hadoop.mapred.lib.IdentityMapper";
+ args[8] = "-partitioner";
+ args[9] = "org.apache.hadoop.mapred.pipes.PipesPartitioner";
+ args[10] = "-reduce";
+ args[11] = "org.apache.hadoop.mapred.lib.IdentityReducer";
+ args[12] = "-writer";
+ args[13] = "org.apache.hadoop.mapred.TextOutputFormat";
+ args[14] = "-program";
+ args[15] = fCommand.getAbsolutePath();// "program";
+ args[16] = "-reduces";
+ args[17] = "2";
+ args[18] = "-lazyOutput";
+ args[19] = "lazyOutput";
+ args[20] = "-jobconf";
+ args[21] = "mapreduce.pipes.isjavarecordwriter=false,mapreduce.pipes.isjavarecordreader=false";
+
+ Submitter.main(args);
+ fail();
+ } catch (ExitUtil.ExitException e) {
+ // status should be 0
+ assertEquals(e.status, 0);
+
+ } finally {
+ System.setOut(oldps);
+ System.setSecurityManager(securityManager);
+ }
+
+ }
+
+ /**
+ * test org.apache.hadoop.mapred.pipes.PipesReducer
+ * test the transfer of data: key and value
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testPipesReduser() throws Exception {
+
+ File[] psw = cleanTokenPasswordFile();
+ JobConf conf = new JobConf();
+ try {
+ Token token = new Token(
+ "user".getBytes(), "password".getBytes(), new Text("kind"), new Text(
+ "service"));
+ conf.getCredentials().addToken(new Text("ShuffleAndJobToken"), token);
+
+ File fCommand = getFileCommand("org.apache.hadoop.mapred.pipes.PipeReducerStub");
+ conf.set(MRJobConfig.CACHE_LOCALFILES, fCommand.getAbsolutePath());
+
+ PipesReducer reducer = new PipesReducer();
+ reducer.configure(conf);
+ BooleanWritable bw = new BooleanWritable(true);
+
+ conf.set(MRJobConfig.TASK_ATTEMPT_ID, taskName);
+ initStdOut(conf);
+ conf.setBoolean(MRJobConfig.SKIP_RECORDS, true);
+ CombineOutputCollector output = new CombineOutputCollector(
+ new Counters.Counter(), new Progress());
+ Reporter reporter = new TestTaskReporter();
+ List texts = new ArrayList();
+ texts.add(new Text("first"));
+ texts.add(new Text("second"));
+ texts.add(new Text("third"));
+
+ reducer.reduce(bw, texts.iterator(), output, reporter);
+ reducer.close();
+ String stdOut = readStdOut(conf);
+ // test data: key
+ assertTrue(stdOut.contains("reducer key :true"));
+ // and values
+ assertTrue(stdOut.contains("reduce value :first"));
+ assertTrue(stdOut.contains("reduce value :second"));
+ assertTrue(stdOut.contains("reduce value :third"));
+
+ } finally {
+ if (psw != null) {
+ // remove password files
+ for (File file : psw) {
+ file.deleteOnExit();
+ }
+ }
+ }
+
+ }
+
+ /**
+ * test PipesPartitioner
+ * test set and get data from PipesPartitioner
+ */
+ @Test
+ public void testPipesPartitioner() {
+
+ PipesPartitioner partitioner = new PipesPartitioner();
+ JobConf configuration = new JobConf();
+ Submitter.getJavaPartitioner(configuration);
+ partitioner.configure(new JobConf());
+ IntWritable iw = new IntWritable(4);
+ // the cache empty
+ assertEquals(0, partitioner.getPartition(iw, new Text("test"), 2));
+ // set data into cache
+ PipesPartitioner.setNextPartition(3);
+ // get data from cache
+ assertEquals(3, partitioner.getPartition(iw, new Text("test"), 2));
+ }
+
+ /**
+ * clean previous std error and outs
+ */
+
+ private void initStdOut(JobConf configuration) {
+ TaskAttemptID taskId = TaskAttemptID.forName(configuration
+ .get(MRJobConfig.TASK_ATTEMPT_ID));
+ File stdOut = TaskLog.getTaskLogFile(taskId, false, TaskLog.LogName.STDOUT);
+ File stdErr = TaskLog.getTaskLogFile(taskId, false, TaskLog.LogName.STDERR);
+ // prepare folder
+ if (!stdOut.getParentFile().exists()) {
+ stdOut.getParentFile().mkdirs();
+ } else { // clean logs
+ stdOut.deleteOnExit();
+ stdErr.deleteOnExit();
+ }
+ }
+
+ private String readStdOut(JobConf conf) throws Exception {
+ TaskAttemptID taskId = TaskAttemptID.forName(conf
+ .get(MRJobConfig.TASK_ATTEMPT_ID));
+ File stdOut = TaskLog.getTaskLogFile(taskId, false, TaskLog.LogName.STDOUT);
+
+ return readFile(stdOut);
+
+ }
+
+ private String readFile(File file) throws Exception {
+ ByteArrayOutputStream out = new ByteArrayOutputStream();
+ InputStream is = new FileInputStream(file);
+ byte[] buffer = new byte[1024];
+ int counter = 0;
+ while ((counter = is.read(buffer)) >= 0) {
+ out.write(buffer, 0, counter);
+ }
+
+ is.close();
+
+ return out.toString();
+
+ }
+
+ private class Progress implements Progressable {
+
+ @Override
+ public void progress() {
+
+ }
+
+ }
+
+ private File[] cleanTokenPasswordFile() throws Exception {
+ File[] result = new File[2];
+ result[0] = new File("./jobTokenPassword");
+ if (result[0].exists()) {
+ FileUtil.chmod(result[0].getAbsolutePath(), "700");
+ assertTrue(result[0].delete());
+ }
+ result[1] = new File("./.jobTokenPassword.crc");
+ if (result[1].exists()) {
+ FileUtil.chmod(result[1].getAbsolutePath(), "700");
+ result[1].delete();
+ }
+ return result;
+ }
+
+ private File getFileCommand(String clazz) throws Exception {
+ String classpath = System.getProperty("java.class.path");
+ File fCommand = new File(workSpace + File.separator + "cache.sh");
+ fCommand.deleteOnExit();
+ if (!fCommand.getParentFile().exists()) {
+ fCommand.getParentFile().mkdirs();
+ }
+ fCommand.createNewFile();
+ OutputStream os = new FileOutputStream(fCommand);
+ os.write("#!/bin/sh \n".getBytes());
+ if (clazz == null) {
+ os.write(("ls ").getBytes());
+ } else {
+ os.write(("java -cp " + classpath + " " + clazz).getBytes());
+ }
+ os.flush();
+ os.close();
+ FileUtil.chmod(fCommand.getAbsolutePath(), "700");
+ return fCommand;
+ }
+
+ private class CombineOutputCollector implements
+ OutputCollector {
+ private Writer writer;
+ private Counters.Counter outCounter;
+ private Progressable progressable;
+
+ public CombineOutputCollector(Counters.Counter outCounter,
+ Progressable progressable) {
+ this.outCounter = outCounter;
+ this.progressable = progressable;
+ }
+
+ public synchronized void setWriter(Writer writer) {
+ this.writer = writer;
+ }
+
+ public synchronized void collect(K key, V value) throws IOException {
+ outCounter.increment(1);
+ writer.append(key, value);
+ progressable.progress();
+ }
+ }
+
+ public static class FakeSplit implements InputSplit {
+ public void write(DataOutput out) throws IOException {
+ }
+
+ public void readFields(DataInput in) throws IOException {
+ }
+
+ public long getLength() {
+ return 0L;
+ }
+
+ public String[] getLocations() {
+ return new String[0];
+ }
+ }
+
+ private class TestTaskReporter implements Reporter {
+ private int recordNum = 0; // number of records processed
+ private String status = null;
+ private Counters counters = new Counters();
+ private InputSplit split = new FakeSplit();
+
+ @Override
+ public void progress() {
+
+ recordNum++;
+ }
+
+ @Override
+ public void setStatus(String status) {
+ this.status = status;
+
+ }
+
+ public String getStatus() {
+ return this.status;
+
+ }
+
+ public Counters.Counter getCounter(String group, String name) {
+ Counters.Counter counter = null;
+ if (counters != null) {
+ counter = counters.findCounter(group, name);
+ if (counter == null) {
+ Group grp = counters.addGroup(group, group);
+ counter = grp.addCounter(name, name, 10);
+ }
+ }
+ return counter;
+ }
+
+ public Counters.Counter getCounter(Enum> name) {
+ return counters == null ? null : counters.findCounter(name);
+ }
+
+ public void incrCounter(Enum> key, long amount) {
+ if (counters != null) {
+ counters.incrCounter(key, amount);
+ }
+ }
+
+ public void incrCounter(String group, String counter, long amount) {
+
+ if (counters != null) {
+ counters.incrCounter(group, counter, amount);
+ }
+
+ }
+
+ @Override
+ public InputSplit getInputSplit() throws UnsupportedOperationException {
+ return split;
+ }
+
+ @Override
+ public float getProgress() {
+ return recordNum;
+ }
+
+ }
+
+ private class Reader implements RecordReader {
+ private int index = 0;
+ private FloatWritable progress;
+
+ @Override
+ public boolean next(FloatWritable key, NullWritable value)
+ throws IOException {
+ progress = key;
+ index++;
+ return index <= 10;
+ }
+
+ @Override
+ public float getProgress() throws IOException {
+ return progress.get();
+ }
+
+ @Override
+ public long getPos() throws IOException {
+
+ return index;
+ }
+
+ @Override
+ public NullWritable createValue() {
+
+ return NullWritable.get();
+ }
+
+ @Override
+ public FloatWritable createKey() {
+ FloatWritable result = new FloatWritable(index);
+ return result;
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
+ }
+
+
+ private class ReaderPipesMapRunner implements RecordReader {
+ private int index = 0;
+
+ @Override
+ public boolean next(FloatWritable key, NullWritable value)
+ throws IOException {
+ key.set(index++);
+ return index <= 10;
+ }
+
+ @Override
+ public float getProgress() throws IOException {
+ return index;
+ }
+
+ @Override
+ public long getPos() throws IOException {
+
+ return index;
+ }
+
+ @Override
+ public NullWritable createValue() {
+
+ return NullWritable.get();
+ }
+
+ @Override
+ public FloatWritable createKey() {
+ FloatWritable result = new FloatWritable(index);
+ return result;
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
+ }
+
+ private class FakeCollector extends
+ CombineOutputCollector {
+
+ final private Map collect = new HashMap();
+
+ public FakeCollector(Counter outCounter, Progressable progressable) {
+ super(outCounter, progressable);
+ }
+
+ @Override
+ public synchronized void collect(IntWritable key, Text value)
+ throws IOException {
+ collect.put(key, value);
+ super.collect(key, value);
+ }
+
+ public Map getCollect() {
+ return collect;
+ }
+ }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipesNonJavaInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipesNonJavaInputFormat.java
new file mode 100644
index 00000000000..b3277854d18
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipesNonJavaInputFormat.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.pipes;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.pipes.TestPipeApplication.FakeSplit;
+import org.junit.Assert;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+public class TestPipesNonJavaInputFormat {
+ private static File workSpace = new File("target",
+ TestPipesNonJavaInputFormat.class.getName() + "-workSpace");
+
+ /**
+ * test PipesNonJavaInputFormat
+ */
+
+ @Test
+ public void testFormat() throws IOException {
+
+ PipesNonJavaInputFormat inputFormat = new PipesNonJavaInputFormat();
+ JobConf conf = new JobConf();
+
+ Reporter reporter= mock(Reporter.class);
+ RecordReader reader = inputFormat
+ .getRecordReader(new FakeSplit(), conf, reporter);
+ assertEquals(0.0f, reader.getProgress(), 0.001);
+
+ // input and output files
+ File input1 = new File(workSpace + File.separator + "input1");
+ if (!input1.getParentFile().exists()) {
+ Assert.assertTrue(input1.getParentFile().mkdirs());
+ }
+
+ if (!input1.exists()) {
+ Assert.assertTrue(input1.createNewFile());
+ }
+
+ File input2 = new File(workSpace + File.separator + "input2");
+ if (!input2.exists()) {
+ Assert.assertTrue(input2.createNewFile());
+ }
+ // set data for splits
+ conf.set(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR,
+ input1.getAbsolutePath() + "," + input2.getAbsolutePath());
+ InputSplit[] splits = inputFormat.getSplits(conf, 2);
+ assertEquals(2, splits.length);
+
+ PipesNonJavaInputFormat.PipesDummyRecordReader dummyRecordReader = new PipesNonJavaInputFormat.PipesDummyRecordReader(
+ conf, splits[0]);
+ // empty dummyRecordReader
+ assertNull(dummyRecordReader.createKey());
+ assertNull(dummyRecordReader.createValue());
+ assertEquals(0, dummyRecordReader.getPos());
+ assertEquals(0.0, dummyRecordReader.getProgress(), 0.001);
+ // test method next
+ assertTrue(dummyRecordReader.next(new FloatWritable(2.0f), NullWritable.get()));
+ assertEquals(2.0, dummyRecordReader.getProgress(), 0.001);
+ dummyRecordReader.close();
+ }
+
+}
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 2cfe2e8c456..a199ce35022 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -65,6 +65,7 @@
+
+ How long to wait between aggregated log retention checks.
+ If set to 0 or a negative value then the value is computed as one-tenth
+ of the aggregated log retention time. Be careful set this too small and
+ you will spam the name node.
+ yarn.log-aggregation.retain-check-interval-seconds
+ -1
+
+
Time in seconds to retain user logs. Only applicable if
log aggregation is disabled
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/InlineDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/InlineDispatcher.java
similarity index 92%
rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/InlineDispatcher.java
rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/InlineDispatcher.java
index d771a61d864..eb1aa9dc990 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/InlineDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/InlineDispatcher.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.yarn.server.resourcemanager.resourcetracker;
+package org.apache.hadoop.yarn.event;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
+@SuppressWarnings({"unchecked", "rawtypes"})
public class InlineDispatcher extends AsyncDispatcher {
private static final Log LOG = LogFactory.getLog(InlineDispatcher.class);
@@ -48,7 +49,7 @@ public class InlineDispatcher extends AsyncDispatcher {
return new TestEventHandler();
}
- static class EmptyEventHandler implements EventHandler {
+ public static class EmptyEventHandler implements EventHandler {
@Override
public void handle(Event event) {
//do nothing
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
index c1cf9af3602..035cd9515c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
@@ -28,12 +28,19 @@ import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Before;
import org.junit.Test;
import static org.mockito.Mockito.*;
public class TestAggregatedLogDeletionService {
+ @Before
+ public void closeFilesystems() throws IOException {
+ // prevent the same mockfs instance from being reused due to FS cache
+ FileSystem.closeAll();
+ }
+
@Test
public void testDeletion() throws Exception {
long now = System.currentTimeMillis();
@@ -121,6 +128,70 @@ public class TestAggregatedLogDeletionService {
verify(mockFs).delete(app4Dir, true);
}
+ @Test
+ public void testCheckInterval() throws Exception {
+ long RETENTION_SECS = 10 * 24 * 3600;
+ long now = System.currentTimeMillis();
+ long toDeleteTime = now - RETENTION_SECS*1000;
+
+ String root = "mockfs://foo/";
+ String remoteRootLogDir = root+"tmp/logs";
+ String suffix = "logs";
+ Configuration conf = new Configuration();
+ conf.setClass("fs.mockfs.impl", MockFileSystem.class, FileSystem.class);
+ conf.set(YarnConfiguration.LOG_AGGREGATION_ENABLED, "true");
+ conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, "864000");
+ conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS, "1");
+ conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteRootLogDir);
+ conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, suffix);
+
+ // prevent us from picking up the same mockfs instance from another test
+ FileSystem.closeAll();
+ Path rootPath = new Path(root);
+ FileSystem rootFs = rootPath.getFileSystem(conf);
+ FileSystem mockFs = ((FilterFileSystem)rootFs).getRawFileSystem();
+
+ Path remoteRootLogPath = new Path(remoteRootLogDir);
+
+ Path userDir = new Path(remoteRootLogPath, "me");
+ FileStatus userDirStatus = new FileStatus(0, true, 0, 0, now, userDir);
+
+ when(mockFs.listStatus(remoteRootLogPath)).thenReturn(
+ new FileStatus[]{userDirStatus});
+
+ Path userLogDir = new Path(userDir, suffix);
+ Path app1Dir = new Path(userLogDir, "application_1_1");
+ FileStatus app1DirStatus = new FileStatus(0, true, 0, 0, now, app1Dir);
+
+ when(mockFs.listStatus(userLogDir)).thenReturn(
+ new FileStatus[]{app1DirStatus});
+
+ Path app1Log1 = new Path(app1Dir, "host1");
+ FileStatus app1Log1Status = new FileStatus(10, false, 1, 1, now, app1Log1);
+
+ when(mockFs.listStatus(app1Dir)).thenReturn(
+ new FileStatus[]{app1Log1Status});
+
+ AggregatedLogDeletionService deletionSvc =
+ new AggregatedLogDeletionService();
+ deletionSvc.init(conf);
+ deletionSvc.start();
+
+ verify(mockFs, timeout(10000).atLeast(4)).listStatus(any(Path.class));
+ verify(mockFs, never()).delete(app1Dir, true);
+
+ // modify the timestamp of the logs and verify it's picked up quickly
+ app1DirStatus = new FileStatus(0, true, 0, 0, toDeleteTime, app1Dir);
+ app1Log1Status = new FileStatus(10, false, 1, 1, toDeleteTime, app1Log1);
+ when(mockFs.listStatus(userLogDir)).thenReturn(
+ new FileStatus[]{app1DirStatus});
+ when(mockFs.listStatus(app1Dir)).thenReturn(
+ new FileStatus[]{app1Log1Status});
+
+ verify(mockFs, timeout(10000)).delete(app1Dir, true);
+
+ deletionSvc.stop();
+ }
static class MockFileSystem extends FilterFileSystem {
MockFileSystem() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index 62d6afc557d..d73f52c588d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -28,7 +28,7 @@ import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -125,10 +125,10 @@ public class ContainersMonitorImpl extends AbstractService implements
this.maxPmemAllottedForContainers >
totalPhysicalMemoryOnNM * 0.80f) {
LOG.warn("NodeManager configured with " +
- StringUtils.humanReadableInt(maxPmemAllottedForContainers) +
+ TraditionalBinaryPrefix.long2String(maxPmemAllottedForContainers, "", 1) +
" physical memory allocated to containers, which is more than " +
"80% of the total physical memory available (" +
- StringUtils.humanReadableInt(totalPhysicalMemoryOnNM) +
+ TraditionalBinaryPrefix.long2String(totalPhysicalMemoryOnNM, "", 1) +
"). Thrashing might happen.");
}
@@ -493,12 +493,12 @@ public class ContainersMonitorImpl extends AbstractService implements
private String formatUsageString(long currentVmemUsage, long vmemLimit,
long currentPmemUsage, long pmemLimit) {
- return String.format("%sb of %sb physical memory used; " +
- "%sb of %sb virtual memory used",
- StringUtils.humanReadableInt(currentPmemUsage),
- StringUtils.humanReadableInt(pmemLimit),
- StringUtils.humanReadableInt(currentVmemUsage),
- StringUtils.humanReadableInt(vmemLimit));
+ return String.format("%sB of %sB physical memory used; " +
+ "%sB of %sB virtual memory used",
+ TraditionalBinaryPrefix.long2String(currentPmemUsage, "", 1),
+ TraditionalBinaryPrefix.long2String(pmemLimit, "", 1),
+ TraditionalBinaryPrefix.long2String(currentVmemUsage, "", 1),
+ TraditionalBinaryPrefix.long2String(vmemLimit, "", 1));
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
index 16db4a7fd91..41456fde9bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
@@ -267,8 +267,8 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
String expectedMsgPattern =
"Container \\[pid=" + pid + ",containerID=" + cId
+ "\\] is running beyond virtual memory limits. Current usage: "
- + "[0-9.]+m?b of [0-9.]+m?b physical memory used; "
- + "[0-9.]+m?b of [0-9.]+m?b virtual memory used. "
+ + "[0-9.]+ ?[KMGTPE]?B of [0-9.]+ ?[KMGTPE]?B physical memory used; "
+ + "[0-9.]+ ?[KMGTPE]?B of [0-9.]+ ?[KMGTPE]?B virtual memory used. "
+ "Killing container.\nDump of the process-tree for "
+ cId + " :\n";
Pattern pat = Pattern.compile(expectedMsgPattern);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index a464b3ae000..e0522a33fdc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -157,6 +157,10 @@ public class ClientRMService extends AbstractService implements
this.server.start();
clientBindAddress = conf.updateConnectAddr(YarnConfiguration.RM_ADDRESS,
server.getListenerAddress());
+ // enable RM to short-circuit token operations directly to itself
+ RMDelegationTokenIdentifier.Renewer.setSecretManager(
+ rmDTSecretManager, clientBindAddress);
+
super.start();
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 52b4d2892a4..70fd2576ab0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -228,7 +228,7 @@ public class RMAppManager implements EventHandler,
}
@SuppressWarnings("unchecked")
- protected synchronized void submitApplication(
+ protected void submitApplication(
ApplicationSubmissionContext submissionContext, long submitTime) {
ApplicationId applicationId = submissionContext.getApplicationId();
RMApp application = null;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
index e1bb437763e..f651566d657 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
@@ -108,7 +108,8 @@ public class SchedulerUtils {
Resource normalized =
Resources.normalize(
resourceCalculator, ask.getCapability(), minimumResource);
- ask.setCapability(normalized);
+ ask.getCapability().setMemory(normalized.getMemory());
+ ask.getCapability().setVirtualCores(normalized.getVirtualCores());
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index 9232190ba3b..066a0a5b969 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -261,7 +261,7 @@ public class DelegationTokenRenewer extends AbstractService {
* done else false.
* @throws IOException
*/
- public synchronized void addApplication(
+ public void addApplication(
ApplicationId applicationId, Credentials ts, boolean shouldCancelAtEnd)
throws IOException {
if (ts == null) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 8479c2c87a3..871755c8f5f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -27,7 +27,9 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import java.security.PrivilegedExceptionAction;
import java.util.List;
+import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CyclicBarrier;
import junit.framework.Assert;
@@ -37,6 +39,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.ClientRMProtocol;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
@@ -44,28 +47,36 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.DelegationToken;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
import org.apache.hadoop.yarn.server.RMDelegationTokenSecretManager;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.apache.hadoop.yarn.util.Records;
-import org.junit.Test;
import org.junit.AfterClass;
import org.junit.BeforeClass;
-
+import org.junit.Test;
public class TestClientRMService {
@@ -235,6 +246,88 @@ public class TestClientRMService {
rmService.renewDelegationToken(request);
}
+ @Test(timeout=4000)
+ public void testConcurrentAppSubmit()
+ throws IOException, InterruptedException, BrokenBarrierException {
+ YarnScheduler yarnScheduler = mock(YarnScheduler.class);
+ RMContext rmContext = mock(RMContext.class);
+ mockRMContext(yarnScheduler, rmContext);
+ RMStateStore stateStore = mock(RMStateStore.class);
+ when(rmContext.getStateStore()).thenReturn(stateStore);
+ RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler,
+ null, mock(ApplicationACLsManager.class), new Configuration());
+
+ final ApplicationId appId1 = getApplicationId(100);
+ final ApplicationId appId2 = getApplicationId(101);
+ final SubmitApplicationRequest submitRequest1 = mockSubmitAppRequest(appId1);
+ final SubmitApplicationRequest submitRequest2 = mockSubmitAppRequest(appId2);
+
+ final CyclicBarrier startBarrier = new CyclicBarrier(2);
+ final CyclicBarrier endBarrier = new CyclicBarrier(2);
+
+ @SuppressWarnings("rawtypes")
+ EventHandler eventHandler = new EventHandler() {
+ @Override
+ public void handle(Event rawEvent) {
+ if (rawEvent instanceof RMAppEvent) {
+ RMAppEvent event = (RMAppEvent) rawEvent;
+ if (event.getApplicationId().equals(appId1)) {
+ try {
+ startBarrier.await();
+ endBarrier.await();
+ } catch (BrokenBarrierException e) {
+ LOG.warn("Broken Barrier", e);
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted while awaiting barriers", e);
+ }
+ }
+ }
+ }
+ };
+
+ when(rmContext.getDispatcher().getEventHandler()).thenReturn(eventHandler);
+
+ final ClientRMService rmService =
+ new ClientRMService(rmContext, yarnScheduler, appManager, null, null);
+
+ // submit an app and wait for it to block while in app submission
+ Thread t = new Thread() {
+ @Override
+ public void run() {
+ try {
+ rmService.submitApplication(submitRequest1);
+ } catch (YarnRemoteException e) {}
+ }
+ };
+ t.start();
+
+ // submit another app, so go through while the first app is blocked
+ startBarrier.await();
+ rmService.submitApplication(submitRequest2);
+ endBarrier.await();
+ t.join();
+ }
+
+ private SubmitApplicationRequest mockSubmitAppRequest(ApplicationId appId) {
+ String user = MockApps.newUserName();
+ String queue = MockApps.newQueue();
+
+ ContainerLaunchContext amContainerSpec = mock(ContainerLaunchContext.class);
+ Resource resource = mock(Resource.class);
+ when(amContainerSpec.getResource()).thenReturn(resource);
+
+ ApplicationSubmissionContext submissionContext = mock(ApplicationSubmissionContext.class);
+ when(submissionContext.getUser()).thenReturn(user);
+ when(submissionContext.getQueue()).thenReturn(queue);
+ when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec);
+ when(submissionContext.getApplicationId()).thenReturn(appId);
+
+ SubmitApplicationRequest submitRequest =
+ recordFactory.newRecordInstance(SubmitApplicationRequest.class);
+ submitRequest.setApplicationSubmissionContext(submissionContext);
+ return submitRequest;
+ }
+
private void mockRMContext(YarnScheduler yarnScheduler, RMContext rmContext)
throws IOException {
Dispatcher dispatcher = mock(Dispatcher.class);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
index 3f786964745..5ee851b8438 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
@@ -17,13 +17,12 @@
package org.apache.hadoop.yarn.server.resourcemanager;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException;
+import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.security.PrivilegedAction;
import java.security.PrivilegedExceptionAction;
@@ -34,9 +33,15 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.api.ClientRMProtocol;
import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
@@ -46,12 +51,14 @@ import org.apache.hadoop.yarn.api.records.DelegationToken;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
import org.apache.hadoop.yarn.server.RMDelegationTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.apache.hadoop.yarn.util.ProtoUtils;
import org.apache.hadoop.yarn.util.Records;
+import org.junit.Before;
import org.junit.Test;
@@ -59,6 +66,10 @@ public class TestClientRMTokens {
private static final Log LOG = LogFactory.getLog(TestClientRMTokens.class);
+ @Before
+ public void resetSecretManager() {
+ RMDelegationTokenIdentifier.Renewer.setSecretManager(null, null);
+ }
@Test
public void testDelegationToken() throws IOException, InterruptedException {
@@ -200,7 +211,122 @@ public class TestClientRMTokens {
RPC.stopProxy(clientRMWithDT);
}
}
+ }
+
+ @Test
+ public void testShortCircuitRenewCancel()
+ throws IOException, InterruptedException {
+ InetSocketAddress addr =
+ new InetSocketAddress(InetAddress.getLocalHost(), 123);
+ checkShortCircuitRenewCancel(addr, addr, true);
+ }
+
+ @Test
+ public void testShortCircuitRenewCancelWildcardAddress()
+ throws IOException, InterruptedException {
+ InetSocketAddress rmAddr = new InetSocketAddress(123);
+ checkShortCircuitRenewCancel(
+ rmAddr,
+ new InetSocketAddress(InetAddress.getLocalHost(), rmAddr.getPort()),
+ true);
+ }
+
+ @Test
+ public void testShortCircuitRenewCancelSameHostDifferentPort()
+ throws IOException, InterruptedException {
+ InetSocketAddress rmAddr =
+ new InetSocketAddress(InetAddress.getLocalHost(), 123);
+ checkShortCircuitRenewCancel(
+ rmAddr,
+ new InetSocketAddress(rmAddr.getAddress(), rmAddr.getPort()+1),
+ false);
+ }
+
+ @Test
+ public void testShortCircuitRenewCancelDifferentHostSamePort()
+ throws IOException, InterruptedException {
+ InetSocketAddress rmAddr =
+ new InetSocketAddress(InetAddress.getLocalHost(), 123);
+ checkShortCircuitRenewCancel(
+ rmAddr,
+ new InetSocketAddress("1.1.1.1", rmAddr.getPort()),
+ false);
+ }
+
+ @Test
+ public void testShortCircuitRenewCancelDifferentHostDifferentPort()
+ throws IOException, InterruptedException {
+ InetSocketAddress rmAddr =
+ new InetSocketAddress(InetAddress.getLocalHost(), 123);
+ checkShortCircuitRenewCancel(
+ rmAddr,
+ new InetSocketAddress("1.1.1.1", rmAddr.getPort()+1),
+ false);
+ }
+
+ @SuppressWarnings("unchecked")
+ private void checkShortCircuitRenewCancel(InetSocketAddress rmAddr,
+ InetSocketAddress serviceAddr,
+ boolean shouldShortCircuit
+ ) throws IOException, InterruptedException {
+ Configuration conf = new Configuration();
+ conf.setClass(YarnConfiguration.IPC_RPC_IMPL,
+ YarnBadRPC.class, YarnRPC.class);
+ RMDelegationTokenSecretManager secretManager =
+ mock(RMDelegationTokenSecretManager.class);
+ RMDelegationTokenIdentifier.Renewer.setSecretManager(secretManager, rmAddr);
+
+ RMDelegationTokenIdentifier ident = new RMDelegationTokenIdentifier(
+ new Text("owner"), new Text("renewer"), null);
+ Token token =
+ new Token(ident, secretManager);
+
+ SecurityUtil.setTokenService(token, serviceAddr);
+ if (shouldShortCircuit) {
+ token.renew(conf);
+ verify(secretManager).renewToken(eq(token), eq("renewer"));
+ reset(secretManager);
+ token.cancel(conf);
+ verify(secretManager).cancelToken(eq(token), eq("renewer"));
+ } else {
+ try {
+ token.renew(conf);
+ fail();
+ } catch (RuntimeException e) {
+ assertEquals("getProxy", e.getMessage());
+ }
+ verify(secretManager, never()).renewToken(any(Token.class), anyString());
+ try {
+ token.cancel(conf);
+ fail();
+ } catch (RuntimeException e) {
+ assertEquals("getProxy", e.getMessage());
+ }
+ verify(secretManager, never()).cancelToken(any(Token.class), anyString());
+ }
+ }
+
+ @SuppressWarnings("rawtypes")
+ public static class YarnBadRPC extends YarnRPC {
+ @Override
+ public Object getProxy(Class protocol, InetSocketAddress addr,
+ Configuration conf) {
+ throw new RuntimeException("getProxy");
+ }
+
+ @Override
+ public void stopProxy(Object proxy, Configuration conf) {
+ throw new RuntimeException("stopProxy");
+ }
+
+ @Override
+ public Server getServer(Class protocol, Object instance,
+ InetSocketAddress addr, Configuration conf,
+ SecretManager extends TokenIdentifier> secretManager,
+ int numHandlers, String portRangeConfig) {
+ throw new RuntimeException("getServer");
+ }
}
// Get the delegation token directly as it is a little difficult to setup
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
index 9ae8bf02902..6c14008626a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.event.InlineDispatcher;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
-import org.apache.hadoop.yarn.server.resourcemanager.resourcetracker.InlineDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
index 1c4b6f9e0cc..6ec3f5403b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.InlineDispatcher;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java
index a7a52c29ab6..7d7f99d054e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.event.InlineDispatcher;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
index 25a4b968fd1..c907df389b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.event.InlineDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
@@ -54,7 +55,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
-import org.apache.hadoop.yarn.server.resourcemanager.resourcetracker.InlineDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
index 681e8d01698..bc806f60a2e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -42,23 +43,35 @@ public class TestSchedulerUtils {
// case negative memory
ask.setCapability(Resources.createResource(-1024));
+ Resource before = ask.getCapability();
SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
+ Resource after = ask.getCapability();
assertEquals(minMemory, ask.getCapability().getMemory());
+ assertTrue(before == after);
// case zero memory
ask.setCapability(Resources.createResource(0));
+ before = ask.getCapability();
SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
+ after = ask.getCapability();
assertEquals(minMemory, ask.getCapability().getMemory());
+ assertTrue(before == after);
// case memory is a multiple of minMemory
ask.setCapability(Resources.createResource(2 * minMemory));
+ before = ask.getCapability();
SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
+ after = ask.getCapability();
assertEquals(2 * minMemory, ask.getCapability().getMemory());
+ assertTrue(before == after);
// case memory is not a multiple of minMemory
ask.setCapability(Resources.createResource(minMemory + 10));
+ before = ask.getCapability();
SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
+ after = ask.getCapability();
assertEquals(2 * minMemory, ask.getCapability().getMemory());
+ assertTrue(before == after);
}
@@ -73,24 +86,33 @@ public class TestSchedulerUtils {
// case negative memory/vcores
ask.setCapability(Resources.createResource(-1024, -1));
+ Resource before = ask.getCapability();
SchedulerUtils.normalizeRequest(
ask, resourceCalculator, clusterResource, minResource);
+ Resource after = ask.getCapability();
assertEquals(minResource, ask.getCapability());
+ assertTrue(before == after);
// case zero memory/vcores
ask.setCapability(Resources.createResource(0, 0));
+ before = ask.getCapability();
SchedulerUtils.normalizeRequest(
ask, resourceCalculator, clusterResource, minResource);
+ after = ask.getCapability();
assertEquals(minResource, ask.getCapability());
assertEquals(1, ask.getCapability().getVirtualCores());
assertEquals(1024, ask.getCapability().getMemory());
+ assertTrue(before == after);
// case non-zero memory & zero cores
ask.setCapability(Resources.createResource(1536, 0));
+ before = ask.getCapability();
SchedulerUtils.normalizeRequest(
ask, resourceCalculator, clusterResource, minResource);
+ after = ask.getCapability();
assertEquals(Resources.createResource(2048, 1), ask.getCapability());
assertEquals(1, ask.getCapability().getVirtualCores());
assertEquals(2048, ask.getCapability().getMemory());
+ assertTrue(before == after);
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index 97310093cbb..c4dbe876c78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -33,13 +33,13 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.InlineDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.Application;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.Task;
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
-import org.apache.hadoop.yarn.server.resourcemanager.resourcetracker.InlineDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
index ad127a9264d..c59625361ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
@@ -21,11 +21,17 @@ package org.apache.hadoop.yarn.server.resourcemanager.security;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Collections;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -50,6 +56,8 @@ import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
/**
* unit test -
@@ -541,4 +549,54 @@ public class TestDelegationTokenRenewer {
fail("Renewal of cancelled token should have failed");
} catch (InvalidToken ite) {}
}
+
+ @Test(timeout=2000)
+ public void testConncurrentAddApplication()
+ throws IOException, InterruptedException, BrokenBarrierException {
+ final CyclicBarrier startBarrier = new CyclicBarrier(2);
+ final CyclicBarrier endBarrier = new CyclicBarrier(2);
+
+ // this token uses barriers to block during renew
+ final Credentials creds1 = new Credentials();
+ final Token> token1 = mock(Token.class);
+ creds1.addToken(new Text("token"), token1);
+ doReturn(true).when(token1).isManaged();
+ doAnswer(new Answer() {
+ public Long answer(InvocationOnMock invocation)
+ throws InterruptedException, BrokenBarrierException {
+ startBarrier.await();
+ endBarrier.await();
+ return Long.MAX_VALUE;
+ }}).when(token1).renew(any(Configuration.class));
+
+ // this dummy token fakes renewing
+ final Credentials creds2 = new Credentials();
+ final Token> token2 = mock(Token.class);
+ creds2.addToken(new Text("token"), token2);
+ doReturn(true).when(token2).isManaged();
+ doReturn(Long.MAX_VALUE).when(token2).renew(any(Configuration.class));
+
+ // fire up the renewer
+ final DelegationTokenRenewer dtr = new DelegationTokenRenewer();
+ dtr.init(conf);
+ dtr.start();
+
+ // submit a job that blocks during renewal
+ Thread submitThread = new Thread() {
+ @Override
+ public void run() {
+ try {
+ dtr.addApplication(mock(ApplicationId.class), creds1, false);
+ } catch (IOException e) {}
+ }
+ };
+ submitThread.start();
+
+ // wait till 1st submit blocks, then submit another
+ startBarrier.await();
+ dtr.addApplication(mock(ApplicationId.class), creds2, false);
+ // signal 1st to complete
+ endBarrier.await();
+ submitThread.join();
+ }
}
diff --git a/hadoop-yarn-project/pom.xml b/hadoop-yarn-project/pom.xml
index a0eed4838c2..4366b387e13 100644
--- a/hadoop-yarn-project/pom.xml
+++ b/hadoop-yarn-project/pom.xml
@@ -33,7 +33,6 @@
true
600000
once
- ${basedir}
yarn
true
@@ -200,16 +199,6 @@
- org.codehaus.mojo
- findbugs-maven-plugin
-
- true
- true
- ${mr.basedir}/dev-support/findbugs-exclude.xml
- Max
-
-
-
org.apache.rat
apache-rat-plugin
|