diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml
index eed27a88570..fff40cb43d1 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml
@@ -155,6 +155,9 @@
org.apache.hadoop:hadoop-common
org.apache.hadoop:hadoop-hdfs
+
+ org.slf4j:slf4j-api
+ org.slf4j:slf4j-log4j12
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index d32dc611f9a..32b2c2a4e60 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -140,6 +140,9 @@ Release 2.0.0 - UNRELEASED
HADOOP-8206. Common portion of a ZK-based failover controller (todd)
+ HADOOP-8210. Common side of HDFS-3148: The client should be able
+ to use multiple local interfaces for data transfer. (eli)
+
IMPROVEMENTS
HADOOP-7524. Change RPC to allow multiple protocols including multuple
@@ -239,6 +242,9 @@ Release 2.0.0 - UNRELEASED
HADOOP-8236. haadmin should have configurable timeouts for failover
commands. (todd)
+ HADOOP-8242. AbstractDelegationTokenIdentifier: add getter methods
+ for owner and realuser. (Colin Patrick McCabe via eli)
+
OPTIMIZATIONS
BUG FIXES
@@ -308,6 +314,9 @@ Release 2.0.0 - UNRELEASED
HADOOP-8238. NetUtils#getHostNameOfIP blows up if given ip:port
string w/o port. (eli)
+ HADOOP-8243. Security support broken in CLI (manual) failover controller
+ (todd)
+
BREAKDOWN OF HADOOP-7454 SUBTASKS
HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
@@ -360,6 +369,18 @@ Release 2.0.0 - UNRELEASED
HADOOP-8116. RetriableCommand is using RetryPolicy incorrectly after
HADOOP-7896. (atm)
+Release 0.23.3 - UNRELEASED
+
+ INCOMPATIBLE CHANGES
+
+ NEW FEATURES
+
+ IMPROVEMENTS
+
+ OPTIMIZATIONS
+
+ BUG FIXES
+
Release 0.23.2 - UNRELEASED
INCOMPATIBLE CHANGES
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java
index d1dad189545..22f245a9c84 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java
@@ -52,14 +52,22 @@ public class FailoverController {
public FailoverController(Configuration conf) {
this.conf = conf;
- this.gracefulFenceTimeout = conf.getInt(
+ this.gracefulFenceTimeout = getGracefulFenceTimeout(conf);
+ this.rpcTimeoutToNewActive = getRpcTimeoutToNewActive(conf);
+ }
+
+ static int getGracefulFenceTimeout(Configuration conf) {
+ return conf.getInt(
CommonConfigurationKeys.HA_FC_GRACEFUL_FENCE_TIMEOUT_KEY,
CommonConfigurationKeys.HA_FC_GRACEFUL_FENCE_TIMEOUT_DEFAULT);
- this.rpcTimeoutToNewActive = conf.getInt(
+ }
+
+ static int getRpcTimeoutToNewActive(Configuration conf) {
+ return conf.getInt(
CommonConfigurationKeys.HA_FC_NEW_ACTIVE_TIMEOUT_KEY,
CommonConfigurationKeys.HA_FC_NEW_ACTIVE_TIMEOUT_DEFAULT);
}
-
+
/**
* Perform pre-failover checks on the given service we plan to
* failover to, eg to prevent failing over to a service (eg due
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
index 75c0c11eee7..a3d898cf3e5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
@@ -114,7 +114,8 @@ public abstract class HAAdmin extends Configured implements Tool {
return -1;
}
- HAServiceProtocol proto = resolveTarget(argv[1]).getProxy();
+ HAServiceProtocol proto = resolveTarget(argv[1]).getProxy(
+ getConf(), 0);
HAServiceProtocolHelper.transitionToActive(proto);
return 0;
}
@@ -127,7 +128,8 @@ public abstract class HAAdmin extends Configured implements Tool {
return -1;
}
- HAServiceProtocol proto = resolveTarget(argv[1]).getProxy();
+ HAServiceProtocol proto = resolveTarget(argv[1]).getProxy(
+ getConf(), 0);
HAServiceProtocolHelper.transitionToStandby(proto);
return 0;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java
index e3f887611c2..90c887d2923 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java
@@ -68,11 +68,4 @@ public abstract class HAServiceTarget {
getAddress(),
confCopy, factory, timeoutMs);
}
-
- /**
- * @return a proxy to connect to the target HA Service.
- */
- public final HAServiceProtocol getProxy() throws IOException {
- return getProxy(new Configuration(), 0); // default conf, timeout
- }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
index 2e6dd48b21b..da98842b9ce 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
@@ -250,7 +250,8 @@ public abstract class ZKFailoverController implements Tool {
private synchronized void becomeActive() throws ServiceFailedException {
LOG.info("Trying to make " + localTarget + " active...");
try {
- HAServiceProtocolHelper.transitionToActive(localTarget.getProxy());
+ HAServiceProtocolHelper.transitionToActive(localTarget.getProxy(
+ conf, FailoverController.getRpcTimeoutToNewActive(conf)));
LOG.info("Successfully transitioned " + localTarget +
" to active state");
} catch (Throwable t) {
@@ -277,7 +278,8 @@ public abstract class ZKFailoverController implements Tool {
LOG.info("ZK Election indicated that " + localTarget +
" should become standby");
try {
- localTarget.getProxy().transitionToStandby();
+ int timeout = FailoverController.getGracefulFenceTimeout(conf);
+ localTarget.getProxy(conf, timeout).transitionToStandby();
LOG.info("Successfully transitioned " + localTarget +
" to standby state");
} catch (Exception e) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java
index 6a5c6481cb7..0a9ff7301bf 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java
@@ -27,7 +27,10 @@ import java.net.InetAddress;
import java.net.NetworkInterface;
import java.net.SocketException;
import java.net.UnknownHostException;
+import java.util.Collections;
import java.util.Enumeration;
+import java.util.LinkedHashSet;
+import java.util.Set;
import java.util.Vector;
import javax.naming.NamingException;
@@ -113,6 +116,31 @@ public class DNS {
return null;
}
+ /**
+ * @param nif network interface to get addresses for
+ * @return set containing addresses for each subinterface of nif,
+ * see below for the rationale for using an ordered set
+ */
+ private static LinkedHashSet getSubinterfaceInetAddrs(
+ NetworkInterface nif) {
+ LinkedHashSet addrs = new LinkedHashSet();
+ Enumeration subNifs = nif.getSubInterfaces();
+ while (subNifs.hasMoreElements()) {
+ NetworkInterface subNif = subNifs.nextElement();
+ addrs.addAll(Collections.list(subNif.getInetAddresses()));
+ }
+ return addrs;
+ }
+
+ /**
+ * Like {@link DNS#getIPs(String, boolean), but returns all
+ * IPs associated with the given interface and its subinterfaces.
+ */
+ public static String[] getIPs(String strInterface)
+ throws UnknownHostException {
+ return getIPs(strInterface, true);
+ }
+
/**
* Returns all the IPs associated with the provided interface, if any, in
* textual form.
@@ -120,6 +148,9 @@ public class DNS {
* @param strInterface
* The name of the network interface or sub-interface to query
* (eg eth0 or eth0:0) or the string "default"
+ * @param returnSubinterfaces
+ * Whether to return IPs associated with subinterfaces of
+ * the given interface
* @return A string vector of all the IPs associated with the provided
* interface. The local host IP is returned if the interface
* name "default" is specified or there is an I/O error looking
@@ -128,8 +159,8 @@ public class DNS {
* If the given interface is invalid
*
*/
- public static String[] getIPs(String strInterface)
- throws UnknownHostException {
+ public static String[] getIPs(String strInterface,
+ boolean returnSubinterfaces) throws UnknownHostException {
if ("default".equals(strInterface)) {
return new String[] { cachedHostAddress };
}
@@ -147,12 +178,22 @@ public class DNS {
if (netIf == null) {
throw new UnknownHostException("No such interface " + strInterface);
}
- Vector ips = new Vector();
- Enumeration addrs = netIf.getInetAddresses();
- while (addrs.hasMoreElements()) {
- ips.add(addrs.nextElement().getHostAddress());
+
+ // NB: Using a LinkedHashSet to preserve the order for callers
+ // that depend on a particular element being 1st in the array.
+ // For example, getDefaultIP always returns the first element.
+ LinkedHashSet allAddrs = new LinkedHashSet();
+ allAddrs.addAll(Collections.list(netIf.getInetAddresses()));
+ if (!returnSubinterfaces) {
+ allAddrs.removeAll(getSubinterfaceInetAddrs(netIf));
}
- return ips.toArray(new String[] {});
+
+ String ips[] = new String[allAddrs.size()];
+ int i = 0;
+ for (InetAddress addr : allAddrs) {
+ ips[i++] = addr.getHostAddress();
+ }
+ return ips;
}
@@ -256,7 +297,7 @@ public class DNS {
return address;
}
- /**
+ /**
* Returns all the host names associated by the default nameserver with the
* address bound to the specified network interface
*
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 90c5f169c9f..b84bcb6a2a4 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
@@ -43,6 +43,8 @@ import javax.net.SocketFactory;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.commons.net.util.SubnetUtils;
+import org.apache.commons.net.util.SubnetUtils.SubnetInfo;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@@ -51,6 +53,8 @@ import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.util.ReflectionUtils;
+import com.google.common.base.Preconditions;
+
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Unstable
public class NetUtils {
@@ -469,11 +473,27 @@ public class NetUtils {
* @see java.net.Socket#connect(java.net.SocketAddress, int)
*
* @param socket
- * @param endpoint
- * @param timeout - timeout in milliseconds
+ * @param address the remote address
+ * @param timeout timeout in milliseconds
+ */
+ public static void connect(Socket socket,
+ SocketAddress address,
+ int timeout) throws IOException {
+ connect(socket, address, null, timeout);
+ }
+
+ /**
+ * Like {@link NetUtils#connect(Socket, SocketAddress, int)} but
+ * also takes a local address and port to bind the socket to.
+ *
+ * @param socket
+ * @param address the remote address
+ * @param localAddr the local address to bind the socket to
+ * @param timeout timeout in milliseconds
*/
public static void connect(Socket socket,
- SocketAddress endpoint,
+ SocketAddress endpoint,
+ SocketAddress localAddr,
int timeout) throws IOException {
if (socket == null || endpoint == null || timeout < 0) {
throw new IllegalArgumentException("Illegal argument for connect()");
@@ -481,6 +501,15 @@ public class NetUtils {
SocketChannel ch = socket.getChannel();
+ if (localAddr != null) {
+ Class localClass = localAddr.getClass();
+ Class remoteClass = endpoint.getClass();
+ Preconditions.checkArgument(localClass.equals(remoteClass),
+ "Local address %s must be of same family as remote address %s.",
+ localAddr, endpoint);
+ socket.bind(localAddr);
+ }
+
if (ch == null) {
// let the default implementation handle it.
socket.connect(endpoint, timeout);
@@ -769,4 +798,70 @@ public class NetUtils {
("\"" + hostname + "\"")
: UNKNOWN_HOST;
}
+
+ /**
+ * @return true if the given string is a subnet specified
+ * using CIDR notation, false otherwise
+ */
+ public static boolean isValidSubnet(String subnet) {
+ try {
+ new SubnetUtils(subnet);
+ return true;
+ } catch (IllegalArgumentException iae) {
+ return false;
+ }
+ }
+
+ /**
+ * Add all addresses associated with the given nif in the
+ * given subnet to the given list.
+ */
+ private static void addMatchingAddrs(NetworkInterface nif,
+ SubnetInfo subnetInfo, List addrs) {
+ Enumeration ifAddrs = nif.getInetAddresses();
+ while (ifAddrs.hasMoreElements()) {
+ InetAddress ifAddr = ifAddrs.nextElement();
+ if (subnetInfo.isInRange(ifAddr.getHostAddress())) {
+ addrs.add(ifAddr);
+ }
+ }
+ }
+
+ /**
+ * Return an InetAddress for each interface that matches the
+ * given subnet specified using CIDR notation.
+ *
+ * @param subnet subnet specified using CIDR notation
+ * @param returnSubinterfaces
+ * whether to return IPs associated with subinterfaces
+ * @throws IllegalArgumentException if subnet is invalid
+ */
+ public static List getIPs(String subnet,
+ boolean returnSubinterfaces) {
+ List addrs = new ArrayList();
+ SubnetInfo subnetInfo = new SubnetUtils(subnet).getInfo();
+ Enumeration nifs;
+
+ try {
+ nifs = NetworkInterface.getNetworkInterfaces();
+ } catch (SocketException e) {
+ LOG.error("Unable to get host interfaces", e);
+ return addrs;
+ }
+
+ while (nifs.hasMoreElements()) {
+ NetworkInterface nif = nifs.nextElement();
+ // NB: adding addresses even if the nif is not up
+ addMatchingAddrs(nif, subnetInfo, addrs);
+
+ if (!returnSubinterfaces) {
+ continue;
+ }
+ Enumeration subNifs = nif.getSubInterfaces();
+ while (subNifs.hasMoreElements()) {
+ addMatchingAddrs(subNifs.nextElement(), subnetInfo, addrs);
+ }
+ }
+ return addrs;
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenIdentifier.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenIdentifier.java
index a3147c117a0..c7738086fb3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenIdentifier.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenIdentifier.java
@@ -96,10 +96,18 @@ extends TokenIdentifier {
}
}
+ public Text getOwner() {
+ return owner;
+ }
+
public Text getRenewer() {
return renewer;
}
+ public Text getRealUser() {
+ return realUser;
+ }
+
public void setIssueDate(long issueDate) {
this.issueDate = issueDate;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 89dfe3a9295..897f79d352a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -109,6 +109,9 @@ Trunk (unreleased changes)
HDFS-3116. Typo in fetchdt error message. (AOE Takashi via atm)
+ HDFS-3126. Journal stream from Namenode to BackupNode needs to have
+ timeout. (Hari Mankude via suresh)
+
Release 2.0.0 - UNRELEASED
INCOMPATIBLE CHANGES
@@ -179,6 +182,9 @@ Release 2.0.0 - UNRELEASED
HDFS-3167. CLI-based driver for MiniDFSCluster. (Henry Robinson via atm)
+ HDFS-3148. The client should be able to use multiple local interfaces
+ for data transfer. (eli)
+
IMPROVEMENTS
HDFS-2018. Move all journal stream management code into one place.
@@ -296,6 +302,10 @@ Release 2.0.0 - UNRELEASED
HDFS-3144. Refactor DatanodeID#getName by use. (eli)
+ HDFS-3130. Move fsdataset implementation to a package. (szetszwo)
+
+ HDFS-3120. Enable hsync and hflush by default. (eli)
+
OPTIMIZATIONS
HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
@@ -708,7 +718,24 @@ Release 2.0.0 - UNRELEASED
HDFS-3039. Address findbugs and javadoc warnings on branch. (todd via atm)
-Release 0.23.2 - UNRELEASED
+Release 0.23.3 - UNRELEASED
+
+ INCOMPATIBLE CHANGES
+
+ NEW FEATURES
+
+ IMPROVEMENTS
+
+ OPTIMIZATIONS
+
+ BUG FIXES
+
+ HDFS-3166. Add timeout to Hftp connections. (Daryn Sharp via szetszwo)
+
+ HDFS-3176. Use MD5MD5CRC32FileChecksum.readFields() in JsonUtil . (Kihwal
+ Lee via szetszwo)
+
+Release 0.23.2 - UNRELEASED
INCOMPATIBLE CHANGES
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 350796d3319..8710aaee08e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -218,7 +218,7 @@
the OBL warning.
-->
-
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 37575e55faa..70285262bfe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -57,12 +57,16 @@ import java.io.OutputStream;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.Socket;
+import java.net.SocketAddress;
import java.net.URI;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Random;
import javax.net.SocketFactory;
@@ -123,6 +127,7 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.Client;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.DNS;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
@@ -132,7 +137,9 @@ import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
+import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
+import com.google.common.net.InetAddresses;
/********************************************************
* DFSClient can connect to a Hadoop Filesystem and
@@ -168,6 +175,8 @@ public class DFSClient implements java.io.Closeable {
final LeaseRenewer leaserenewer;
final SocketCache socketCache;
final Conf dfsClientConf;
+ private Random r = new Random();
+ private SocketAddress[] localInterfaceAddrs;
/**
* DFSClient configuration
@@ -361,6 +370,68 @@ public class DFSClient implements java.io.Closeable {
if (LOG.isDebugEnabled()) {
LOG.debug("Short circuit read is " + shortCircuitLocalReads);
}
+ String localInterfaces[] =
+ conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
+ localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
+ if (LOG.isDebugEnabled() && 0 != localInterfaces.length) {
+ LOG.debug("Using local interfaces [" +
+ Joiner.on(',').join(localInterfaces)+ "] with addresses [" +
+ Joiner.on(',').join(localInterfaceAddrs) + "]");
+ }
+ }
+
+ /**
+ * Return the socket addresses to use with each configured
+ * local interface. Local interfaces may be specified by IP
+ * address, IP address range using CIDR notation, interface
+ * name (e.g. eth0) or sub-interface name (e.g. eth0:0).
+ * The socket addresses consist of the IPs for the interfaces
+ * and the ephemeral port (port 0). If an IP, IP range, or
+ * interface name matches an interface with sub-interfaces
+ * only the IP of the interface is used. Sub-interfaces can
+ * be used by specifying them explicitly (by IP or name).
+ *
+ * @return SocketAddresses for the configured local interfaces,
+ * or an empty array if none are configured
+ * @throws UnknownHostException if a given interface name is invalid
+ */
+ private static SocketAddress[] getLocalInterfaceAddrs(
+ String interfaceNames[]) throws UnknownHostException {
+ List localAddrs = new ArrayList();
+ for (String interfaceName : interfaceNames) {
+ if (InetAddresses.isInetAddress(interfaceName)) {
+ localAddrs.add(new InetSocketAddress(interfaceName, 0));
+ } else if (NetUtils.isValidSubnet(interfaceName)) {
+ for (InetAddress addr : NetUtils.getIPs(interfaceName, false)) {
+ localAddrs.add(new InetSocketAddress(addr, 0));
+ }
+ } else {
+ for (String ip : DNS.getIPs(interfaceName, false)) {
+ localAddrs.add(new InetSocketAddress(ip, 0));
+ }
+ }
+ }
+ return localAddrs.toArray(new SocketAddress[localAddrs.size()]);
+ }
+
+ /**
+ * Select one of the configured local interfaces at random. We use a random
+ * interface because other policies like round-robin are less effective
+ * given that we cache connections to datanodes.
+ *
+ * @return one of the local interface addresses at random, or null if no
+ * local interfaces are configured
+ */
+ SocketAddress getRandomLocalInterfaceAddr() {
+ if (localInterfaceAddrs.length == 0) {
+ return null;
+ }
+ final int idx = r.nextInt(localInterfaceAddrs.length);
+ final SocketAddress addr = localInterfaceAddrs[idx];
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Using local interface " + addr);
+ }
+ return addr;
}
/**
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 4a945b5d5a7..3d8c6287500 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
@@ -197,6 +197,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY = "dfs.namenode.checkpoint.edits.dir";
public static final String DFS_HOSTS = "dfs.hosts";
public static final String DFS_HOSTS_EXCLUDE = "dfs.hosts.exclude";
+ public static final String DFS_CLIENT_LOCAL_INTERFACES = "dfs.client.local.interfaces";
// Much code in hdfs is not yet updated to use these keys.
public static final String DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY = "dfs.client.block.write.locateFollowingBlock.retries";
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 58f807f3200..0d00acd57e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -850,7 +850,9 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
// disaster.
sock.setTcpNoDelay(true);
- NetUtils.connect(sock, dnAddr, dfsClient.getConf().socketTimeout);
+ NetUtils.connect(sock, dnAddr,
+ dfsClient.getRandomLocalInterfaceAddr(),
+ dfsClient.getConf().socketTimeout);
sock.setSoTimeout(dfsClient.getConf().socketTimeout);
}
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 1d4a45c8483..9322143a9ec 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
@@ -1171,7 +1171,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
NetUtils.createSocketAddr(first.getXferAddr());
final Socket sock = client.socketFactory.createSocket();
final int timeout = client.getDatanodeReadTimeout(length);
- NetUtils.connect(sock, isa, timeout);
+ NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), timeout);
sock.setSoTimeout(timeout);
sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
if(DFSClient.LOG.isDebugEnabled()) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
index 8fe8cba60d8..829190623a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
+import org.apache.hadoop.hdfs.web.URLUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
@@ -293,15 +294,6 @@ public class HftpFileSystem extends FileSystem
}
return ugiParamenter.toString();
}
-
- static Void throwIOExceptionFromConnection(
- final HttpURLConnection connection, final IOException ioe
- ) throws IOException {
- final int code = connection.getResponseCode();
- final String s = connection.getResponseMessage();
- throw s == null? ioe:
- new IOException(s + " (error code=" + code + ")", ioe);
- }
/**
* Open an HTTP connection to the namenode to read file data and metadata.
@@ -312,13 +304,10 @@ public class HftpFileSystem extends FileSystem
throws IOException {
query = addDelegationTokenParam(query);
final URL url = getNamenodeURL(path, query);
- final HttpURLConnection connection = (HttpURLConnection)url.openConnection();
- try {
- connection.setRequestMethod("GET");
- connection.connect();
- } catch (IOException ioe) {
- throwIOExceptionFromConnection(connection, ioe);
- }
+ final HttpURLConnection connection =
+ (HttpURLConnection)URLUtils.openConnection(url);
+ connection.setRequestMethod("GET");
+ connection.connect();
return connection;
}
@@ -342,7 +331,7 @@ public class HftpFileSystem extends FileSystem
@Override
protected HttpURLConnection openConnection() throws IOException {
- return (HttpURLConnection)url.openConnection();
+ return (HttpURLConnection)URLUtils.openConnection(url);
}
/** Use HTTP Range header for specifying offset. */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
index 97e3b2414a3..b6fe12d66b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
@@ -41,6 +41,7 @@ import javax.net.ssl.X509TrustManager;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.URLUtils;
/**
* An implementation of a protocol for accessing filesystems over HTTPS. The
@@ -137,15 +138,11 @@ public class HsftpFileSystem extends HftpFileSystem {
query = addDelegationTokenParam(query);
final URL url = new URL("https", nnAddr.getHostName(),
nnAddr.getPort(), path + '?' + query);
- HttpsURLConnection conn = (HttpsURLConnection)url.openConnection();
+ HttpsURLConnection conn = (HttpsURLConnection)URLUtils.openConnection(url);
// bypass hostname verification
- try {
- conn.setHostnameVerifier(new DummyHostnameVerifier());
- conn.setRequestMethod("GET");
- conn.connect();
- } catch (IOException ioe) {
- throwIOExceptionFromConnection(conn, ioe);
- }
+ conn.setHostnameVerifier(new DummyHostnameVerifier());
+ conn.setRequestMethod("GET");
+ conn.connect();
// check cert expiration date
final int warnDays = ExpWarnDays;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
index 3a40fe78817..27eddeb6e52 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
@@ -190,7 +190,7 @@ public class NameNodeProxies {
InetSocketAddress address, Configuration conf, UserGroupInformation ugi)
throws IOException {
JournalProtocolPB proxy = (JournalProtocolPB) createNameNodeProxy(address,
- conf, ugi, JournalProtocolPB.class);
+ conf, ugi, JournalProtocolPB.class, 30000);
return new JournalProtocolTranslatorPB(proxy);
}
@@ -198,7 +198,7 @@ public class NameNodeProxies {
createNNProxyWithRefreshAuthorizationPolicyProtocol(InetSocketAddress address,
Configuration conf, UserGroupInformation ugi) throws IOException {
RefreshAuthorizationPolicyProtocolPB proxy = (RefreshAuthorizationPolicyProtocolPB)
- createNameNodeProxy(address, conf, ugi, RefreshAuthorizationPolicyProtocolPB.class);
+ createNameNodeProxy(address, conf, ugi, RefreshAuthorizationPolicyProtocolPB.class, 0);
return new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(proxy);
}
@@ -206,7 +206,7 @@ public class NameNodeProxies {
createNNProxyWithRefreshUserMappingsProtocol(InetSocketAddress address,
Configuration conf, UserGroupInformation ugi) throws IOException {
RefreshUserMappingsProtocolPB proxy = (RefreshUserMappingsProtocolPB)
- createNameNodeProxy(address, conf, ugi, RefreshUserMappingsProtocolPB.class);
+ createNameNodeProxy(address, conf, ugi, RefreshUserMappingsProtocolPB.class, 0);
return new RefreshUserMappingsProtocolClientSideTranslatorPB(proxy);
}
@@ -214,7 +214,7 @@ public class NameNodeProxies {
InetSocketAddress address, Configuration conf, UserGroupInformation ugi)
throws IOException {
GetUserMappingsProtocolPB proxy = (GetUserMappingsProtocolPB)
- createNameNodeProxy(address, conf, ugi, GetUserMappingsProtocolPB.class);
+ createNameNodeProxy(address, conf, ugi, GetUserMappingsProtocolPB.class, 0);
return new GetUserMappingsProtocolClientSideTranslatorPB(proxy);
}
@@ -222,7 +222,7 @@ public class NameNodeProxies {
InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
boolean withRetries) throws IOException {
NamenodeProtocolPB proxy = (NamenodeProtocolPB) createNameNodeProxy(
- address, conf, ugi, NamenodeProtocolPB.class);
+ address, conf, ugi, NamenodeProtocolPB.class, 0);
if (withRetries) { // create the proxy with retries
RetryPolicy timeoutPolicy = RetryPolicies.exponentialBackoffRetry(5, 200,
TimeUnit.MILLISECONDS);
@@ -244,7 +244,7 @@ public class NameNodeProxies {
InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
boolean withRetries) throws IOException {
ClientNamenodeProtocolPB proxy = (ClientNamenodeProtocolPB) NameNodeProxies
- .createNameNodeProxy(address, conf, ugi, ClientNamenodeProtocolPB.class);
+ .createNameNodeProxy(address, conf, ugi, ClientNamenodeProtocolPB.class, 0);
if (withRetries) { // create the proxy with retries
RetryPolicy createPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(5,
@@ -275,11 +275,11 @@ public class NameNodeProxies {
@SuppressWarnings("unchecked")
private static Object createNameNodeProxy(InetSocketAddress address,
- Configuration conf, UserGroupInformation ugi, Class xface)
+ Configuration conf, UserGroupInformation ugi, Class xface, int rpcTimeout)
throws IOException {
RPC.setProtocolEngine(conf, xface, ProtobufRpcEngine.class);
Object proxy = RPC.getProxy(xface, RPC.getProtocolVersion(xface), address,
- ugi, conf, NetUtils.getDefaultSocketFactory(conf));
+ ugi, conf, NetUtils.getDefaultSocketFactory(conf), rpcTimeout);
return proxy;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
index e97b668d9ee..870f8e58839 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
@@ -86,7 +86,7 @@ public class BlockMetadataHeader {
* @return
* @throws IOException
*/
- static BlockMetadataHeader readHeader(File file) throws IOException {
+ public static BlockMetadataHeader readHeader(File file) throws IOException {
DataInputStream in = null;
try {
in = new DataInputStream(new BufferedInputStream(
@@ -144,7 +144,7 @@ public class BlockMetadataHeader {
/**
* Returns the size of the header
*/
- static int getHeaderSize() {
+ public static int getHeaderSize() {
return Short.SIZE/Byte.SIZE + DataChecksum.getChecksumHeaderSize();
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index b1635a9ae3f..6c2d939c119 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -540,8 +540,8 @@ public class DataNode extends Configured
}
}
- // calls specific to BP
- protected void notifyNamenodeDeletedBlock(ExtendedBlock block) {
+ /** Notify the corresponding namenode to delete the block. */
+ public void notifyNamenodeDeletedBlock(ExtendedBlock block) {
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if (bpos != null) {
bpos.notifyNamenodeDeletedBlock(block);
@@ -1134,9 +1134,8 @@ public class DataNode extends Configured
/**
* Check if there is a disk failure and if so, handle the error
- *
- **/
- protected void checkDiskError( ) {
+ */
+ public void checkDiskError() {
try {
data.checkDataDir();
} catch (DiskErrorException de) {
@@ -1691,6 +1690,12 @@ public class DataNode extends Configured
return data;
}
+ /** @return the block scanner. */
+ public DataBlockScanner getBlockScanner() {
+ return blockScanner;
+ }
+
+
public static void secureMain(String args[], SecureResources resources) {
try {
StringUtils.startupShutdownMessage(DataNode.class, args, LOG);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 9b5f2bb31f5..6c9957ff9cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -30,28 +30,25 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Properties;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.HardLink;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.HardLink;
-import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon;
@@ -65,7 +62,7 @@ import org.apache.hadoop.util.DiskChecker;
@InterfaceAudience.Private
public class DataStorage extends Storage {
// Constants
- final static String BLOCK_SUBDIR_PREFIX = "subdir";
+ public final static String BLOCK_SUBDIR_PREFIX = "subdir";
final static String BLOCK_FILE_PREFIX = "blk_";
final static String COPY_FILE_PREFIX = "dncp_";
final static String STORAGE_DIR_DETACHED = "detach";
@@ -98,15 +95,17 @@ public class DataStorage extends Storage {
this.storageID = strgID;
}
- synchronized String getStorageID() {
+ /** @return storage ID. */
+ public synchronized String getStorageID() {
return storageID;
}
synchronized void setStorageID(String newStorageID) {
this.storageID = newStorageID;
}
-
- synchronized void createStorageID(int datanodePort) {
+
+ /** Create an ID for this storage. */
+ public synchronized void createStorageID(int datanodePort) {
if (storageID != null && !storageID.isEmpty()) {
return;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
index e3eaa6126ea..0a0d57bd6e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
@@ -18,21 +18,17 @@
package org.apache.hadoop.hdfs.server.datanode;
import java.io.File;
-import java.io.FilenameFilter;
import java.io.IOException;
-import java.util.Arrays;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.Block;
/** Provide utility methods for Datanode. */
@InterfaceAudience.Private
-class DatanodeUtil {
- static final String METADATA_EXTENSION = ".meta";
+public class DatanodeUtil {
+ public static final String UNLINK_BLOCK_SUFFIX = ".unlinked";
- static final String UNLINK_BLOCK_SUFFIX = ".unlinked";
-
- private static final String DISK_ERROR = "Possible disk error: ";
+ public static final String DISK_ERROR = "Possible disk error: ";
/** Get the cause of an I/O exception if caused by a possible disk error
* @param ioe an I/O exception
@@ -52,55 +48,34 @@ class DatanodeUtil {
* @throws IOException
* if the file already exists or if the file cannot be created.
*/
- static File createTmpFile(Block b, File f) throws IOException {
+ public static File createTmpFile(Block b, File f) throws IOException {
if (f.exists()) {
- throw new IOException("Unexpected problem in creating temporary file for "
- + b + ". File " + f + " should not be present, but is.");
+ throw new IOException("Failed to create temporary file for " + b
+ + ". File " + f + " should not be present, but is.");
}
// Create the zero-length temp file
final boolean fileCreated;
try {
fileCreated = f.createNewFile();
} catch (IOException ioe) {
- throw (IOException)new IOException(DISK_ERROR + f).initCause(ioe);
+ throw new IOException(DISK_ERROR + "Failed to create " + f, ioe);
}
if (!fileCreated) {
- throw new IOException("Unexpected problem in creating temporary file for "
- + b + ". File " + f + " should be creatable, but is already present.");
+ throw new IOException("Failed to create temporary file for " + b
+ + ". File " + f + " should be creatable, but is already present.");
}
return f;
}
- static String getMetaFileName(String blockFileName, long genStamp) {
- return blockFileName + "_" + genStamp + METADATA_EXTENSION;
- }
-
- static File getMetaFile(File f, long genStamp) {
- return new File(getMetaFileName(f.getAbsolutePath(), genStamp));
+ /**
+ * @return the meta name given the block name and generation stamp.
+ */
+ public static String getMetaName(String blockName, long generationStamp) {
+ return blockName + "_" + generationStamp + Block.METADATA_EXTENSION;
}
- /** Find the corresponding meta data file from a given block file */
- static File findMetaFile(final File blockFile) throws IOException {
- final String prefix = blockFile.getName() + "_";
- final File parent = blockFile.getParentFile();
- File[] matches = parent.listFiles(new FilenameFilter() {
- public boolean accept(File dir, String name) {
- return dir.equals(parent)
- && name.startsWith(prefix) && name.endsWith(METADATA_EXTENSION);
- }
- });
-
- if (matches == null || matches.length == 0) {
- throw new IOException("Meta file not found, blockFile=" + blockFile);
- }
- else if (matches.length > 1) {
- throw new IOException("Found more than one meta files: "
- + Arrays.asList(matches));
- }
- return matches[0];
- }
-
- static File getUnlinkTmpFile(File f) {
+ /** @return the unlink file. */
+ public static File getUnlinkTmpFile(File f) {
return new File(f.getParentFile(), f.getName()+UNLINK_BLOCK_SUFFIX);
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
index 2969b181088..90658733ecd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
/**
* This class describes a replica that has been finalized.
*/
-class FinalizedReplica extends ReplicaInfo {
+public class FinalizedReplica extends ReplicaInfo {
private boolean unlinked; // copy-on-write done for block
/**
@@ -37,7 +37,7 @@ class FinalizedReplica extends ReplicaInfo {
* @param vol volume where replica is located
* @param dir directory path where block and meta files are located
*/
- FinalizedReplica(long blockId, long len, long genStamp,
+ public FinalizedReplica(long blockId, long len, long genStamp,
FsVolumeSpi vol, File dir) {
super(blockId, len, genStamp, vol, dir);
}
@@ -48,7 +48,7 @@ class FinalizedReplica extends ReplicaInfo {
* @param vol volume where replica is located
* @param dir directory path where block and meta files are located
*/
- FinalizedReplica(Block block, FsVolumeSpi vol, File dir) {
+ public FinalizedReplica(Block block, FsVolumeSpi vol, File dir) {
super(block, vol, dir);
}
@@ -56,7 +56,7 @@ class FinalizedReplica extends ReplicaInfo {
* Copy constructor.
* @param from
*/
- FinalizedReplica(FinalizedReplica from) {
+ public FinalizedReplica(FinalizedReplica from) {
super(from);
this.unlinked = from.isUnlinked();
}
@@ -67,12 +67,12 @@ class FinalizedReplica extends ReplicaInfo {
}
@Override // ReplicaInfo
- boolean isUnlinked() {
+ public boolean isUnlinked() {
return unlinked;
}
@Override // ReplicaInfo
- void setUnlinked() {
+ public void setUnlinked() {
unlinked = true;
}
@@ -99,6 +99,6 @@ class FinalizedReplica extends ReplicaInfo {
@Override
public String toString() {
return super.toString()
- + "\n unlinked=" + unlinked;
+ + "\n unlinked =" + unlinked;
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaAlreadyExistsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaAlreadyExistsException.java
index 6c66277f897..124c155dcbe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaAlreadyExistsException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaAlreadyExistsException.java
@@ -25,7 +25,7 @@ import java.io.IOException;
* Exception indicating that the target block already exists
* and is not set to be recovered/overwritten.
*/
-class ReplicaAlreadyExistsException extends IOException {
+public class ReplicaAlreadyExistsException extends IOException {
private static final long serialVersionUID = 1L;
public ReplicaAlreadyExistsException() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java
index e9427a2ad8c..c61807b2d87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
* Those are the replicas that
* are created in a pipeline initiated by a dfs client.
*/
-class ReplicaBeingWritten extends ReplicaInPipeline {
+public class ReplicaBeingWritten extends ReplicaInPipeline {
/**
* Constructor for a zero length replica
* @param blockId block id
@@ -35,7 +35,7 @@ class ReplicaBeingWritten extends ReplicaInPipeline {
* @param vol volume where replica is located
* @param dir directory path where block and meta files are located
*/
- ReplicaBeingWritten(long blockId, long genStamp,
+ public ReplicaBeingWritten(long blockId, long genStamp,
FsVolumeSpi vol, File dir) {
super( blockId, genStamp, vol, dir);
}
@@ -47,7 +47,7 @@ class ReplicaBeingWritten extends ReplicaInPipeline {
* @param dir directory path where block and meta files are located
* @param writer a thread that is writing to this replica
*/
- ReplicaBeingWritten(Block block,
+ public ReplicaBeingWritten(Block block,
FsVolumeSpi vol, File dir, Thread writer) {
super( block, vol, dir, writer);
}
@@ -61,7 +61,7 @@ class ReplicaBeingWritten extends ReplicaInPipeline {
* @param dir directory path where block and meta files are located
* @param writer a thread that is writing to this replica
*/
- ReplicaBeingWritten(long blockId, long len, long genStamp,
+ public ReplicaBeingWritten(long blockId, long len, long genStamp,
FsVolumeSpi vol, File dir, Thread writer ) {
super( blockId, len, genStamp, vol, dir, writer);
}
@@ -70,7 +70,7 @@ class ReplicaBeingWritten extends ReplicaInPipeline {
* Copy constructor.
* @param from
*/
- ReplicaBeingWritten(ReplicaBeingWritten from) {
+ public ReplicaBeingWritten(ReplicaBeingWritten from) {
super(from);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
index b29d1c12b3c..a9ef36dd969 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.util.DataChecksum;
*
* The base class implements a temporary replica
*/
-class ReplicaInPipeline extends ReplicaInfo
+public class ReplicaInPipeline extends ReplicaInfo
implements ReplicaInPipelineInterface {
private long bytesAcked;
private long bytesOnDisk;
@@ -50,9 +50,8 @@ class ReplicaInPipeline extends ReplicaInfo
* @param genStamp replica generation stamp
* @param vol volume where replica is located
* @param dir directory path where block and meta files are located
- * @param state replica state
*/
- ReplicaInPipeline(long blockId, long genStamp,
+ public ReplicaInPipeline(long blockId, long genStamp,
FsVolumeSpi vol, File dir) {
this( blockId, 0L, genStamp, vol, dir, Thread.currentThread());
}
@@ -91,7 +90,7 @@ class ReplicaInPipeline extends ReplicaInfo
* Copy constructor.
* @param from
*/
- ReplicaInPipeline(ReplicaInPipeline from) {
+ public ReplicaInPipeline(ReplicaInPipeline from) {
super(from);
this.bytesAcked = from.getBytesAcked();
this.bytesOnDisk = from.getBytesOnDisk();
@@ -151,7 +150,7 @@ class ReplicaInPipeline extends ReplicaInfo
* Interrupt the writing thread and wait until it dies
* @throws IOException the waiting is interrupted
*/
- void stopWriter() throws IOException {
+ public void stopWriter() throws IOException {
if (writer != null && writer != Thread.currentThread() && writer.isAlive()) {
writer.interrupt();
try {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
index b77eff19d9f..61822424cd3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
@@ -84,20 +84,12 @@ abstract public class ReplicaInfo extends Block implements Replica {
ReplicaInfo(ReplicaInfo from) {
this(from, from.getVolume(), from.getDir());
}
-
- /**
- * Get this replica's meta file name
- * @return this replica's meta file name
- */
- private String getMetaFileName() {
- return getBlockName() + "_" + getGenerationStamp() + METADATA_EXTENSION;
- }
/**
* Get the full path of this replica's data file
* @return the full path of this replica's data file
*/
- File getBlockFile() {
+ public File getBlockFile() {
return new File(getDir(), getBlockName());
}
@@ -105,15 +97,16 @@ abstract public class ReplicaInfo extends Block implements Replica {
* Get the full path of this replica's meta file
* @return the full path of this replica's meta file
*/
- File getMetaFile() {
- return new File(getDir(), getMetaFileName());
+ public File getMetaFile() {
+ return new File(getDir(),
+ DatanodeUtil.getMetaName(getBlockName(), getGenerationStamp()));
}
/**
* Get the volume where this replica is located on disk
* @return the volume where this replica is located on disk
*/
- FsVolumeSpi getVolume() {
+ public FsVolumeSpi getVolume() {
return volume;
}
@@ -136,7 +129,7 @@ abstract public class ReplicaInfo extends Block implements Replica {
* Set the parent directory where this replica is located
* @param dir the parent directory where the replica is located
*/
- void setDir(File dir) {
+ public void setDir(File dir) {
this.dir = dir;
}
@@ -145,14 +138,14 @@ abstract public class ReplicaInfo extends Block implements Replica {
* @return true if the replica has already been unlinked
* or no need to be detached; false otherwise
*/
- boolean isUnlinked() {
+ public boolean isUnlinked() {
return true; // no need to be unlinked
}
/**
* set that this replica is unlinked
*/
- void setUnlinked() {
+ public void setUnlinked() {
// no need to be unlinked
}
@@ -201,7 +194,7 @@ abstract public class ReplicaInfo extends Block implements Replica {
* false if it is already detached or no need to be detached
* @throws IOException if there is any copy error
*/
- boolean unlinkBlock(int numLinks) throws IOException {
+ public boolean unlinkBlock(int numLinks) throws IOException {
if (isUnlinked()) {
return false;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
index 9bb86e72ca4..124574b842f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
@@ -29,21 +29,21 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
*/
public class ReplicaNotFoundException extends IOException {
private static final long serialVersionUID = 1L;
- final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
- final static String UNFINALIZED_REPLICA =
+ public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
+ public final static String UNFINALIZED_REPLICA =
"Cannot append to an unfinalized replica ";
- final static String UNFINALIZED_AND_NONRBW_REPLICA =
+ public final static String UNFINALIZED_AND_NONRBW_REPLICA =
"Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
- final static String NON_EXISTENT_REPLICA =
+ public final static String NON_EXISTENT_REPLICA =
"Cannot append to a non-existent replica ";
- final static String UNEXPECTED_GS_REPLICA =
+ public final static String UNEXPECTED_GS_REPLICA =
"Cannot append to a replica with unexpeted generation stamp ";
public ReplicaNotFoundException() {
super();
}
- ReplicaNotFoundException(ExtendedBlock b) {
+ public ReplicaNotFoundException(ExtendedBlock b) {
super("Replica not found for " + b);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
index d5bbf04227a..e92a6b5a1eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
@@ -31,12 +31,12 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
* A recovery with higher recovery id preempts recoveries with a lower id.
*
*/
-class ReplicaUnderRecovery extends ReplicaInfo {
+public class ReplicaUnderRecovery extends ReplicaInfo {
private ReplicaInfo original; // the original replica that needs to be recovered
private long recoveryId; // recovery id; it is also the generation stamp
// that the replica will be bumped to after recovery
- ReplicaUnderRecovery(ReplicaInfo replica, long recoveryId) {
+ public ReplicaUnderRecovery(ReplicaInfo replica, long recoveryId) {
super(replica.getBlockId(), replica.getNumBytes(), replica.getGenerationStamp(),
replica.getVolume(), replica.getDir());
if ( replica.getState() != ReplicaState.FINALIZED &&
@@ -52,7 +52,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
* Copy constructor.
* @param from
*/
- ReplicaUnderRecovery(ReplicaUnderRecovery from) {
+ public ReplicaUnderRecovery(ReplicaUnderRecovery from) {
super(from);
this.original = from.getOriginalReplica();
this.recoveryId = from.getRecoveryID();
@@ -62,7 +62,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
* Get the recovery id
* @return the generation stamp that the replica will be bumped to
*/
- long getRecoveryID() {
+ public long getRecoveryID() {
return recoveryId;
}
@@ -70,7 +70,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
* Set the recovery id
* @param recoveryId the new recoveryId
*/
- void setRecoveryID(long recoveryId) {
+ public void setRecoveryID(long recoveryId) {
if (recoveryId > this.recoveryId) {
this.recoveryId = recoveryId;
} else {
@@ -83,17 +83,17 @@ class ReplicaUnderRecovery extends ReplicaInfo {
* Get the original replica that's under recovery
* @return the original replica under recovery
*/
- ReplicaInfo getOriginalReplica() {
+ public ReplicaInfo getOriginalReplica() {
return original;
}
@Override //ReplicaInfo
- boolean isUnlinked() {
+ public boolean isUnlinked() {
return original.isUnlinked();
}
@Override //ReplicaInfo
- void setUnlinked() {
+ public void setUnlinked() {
original.setUnlinked();
}
@@ -131,7 +131,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
}
@Override //ReplicaInfo
- void setDir(File dir) {
+ public void setDir(File dir) {
super.setDir(dir);
original.setDir(dir);
}
@@ -159,7 +159,7 @@ class ReplicaUnderRecovery extends ReplicaInfo {
+ "\n original=" + original;
}
- ReplicaRecoveryInfo createInfo() {
+ public ReplicaRecoveryInfo createInfo() {
return new ReplicaRecoveryInfo(original.getBlockId(),
original.getBytesOnDisk(), original.getGenerationStamp(),
original.getState());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
index b26b77ecade..49c8d3dbadf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
* client continues to write or be recovered as a result of
* lease recovery.
*/
-class ReplicaWaitingToBeRecovered extends ReplicaInfo {
+public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
private boolean unlinked; // copy-on-write done for block
/**
@@ -43,7 +43,7 @@ class ReplicaWaitingToBeRecovered extends ReplicaInfo {
* @param vol volume where replica is located
* @param dir directory path where block and meta files are located
*/
- ReplicaWaitingToBeRecovered(long blockId, long len, long genStamp,
+ public ReplicaWaitingToBeRecovered(long blockId, long len, long genStamp,
FsVolumeSpi vol, File dir) {
super(blockId, len, genStamp, vol, dir);
}
@@ -54,7 +54,7 @@ class ReplicaWaitingToBeRecovered extends ReplicaInfo {
* @param vol volume where replica is located
* @param dir directory path where block and meta files are located
*/
- ReplicaWaitingToBeRecovered(Block block, FsVolumeSpi vol, File dir) {
+ public ReplicaWaitingToBeRecovered(Block block, FsVolumeSpi vol, File dir) {
super(block, vol, dir);
}
@@ -62,7 +62,7 @@ class ReplicaWaitingToBeRecovered extends ReplicaInfo {
* Copy constructor.
* @param from
*/
- ReplicaWaitingToBeRecovered(ReplicaWaitingToBeRecovered from) {
+ public ReplicaWaitingToBeRecovered(ReplicaWaitingToBeRecovered from) {
super(from);
this.unlinked = from.isUnlinked();
}
@@ -73,12 +73,12 @@ class ReplicaWaitingToBeRecovered extends ReplicaInfo {
}
@Override //ReplicaInfo
- boolean isUnlinked() {
+ public boolean isUnlinked() {
return unlinked;
}
@Override //ReplicaInfo
- void setUnlinked() {
+ public void setUnlinked() {
unlinked = true;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 0e3242e49bf..c0fe3906c9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -33,9 +33,9 @@ import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset;
import org.apache.hadoop.hdfs.server.datanode.Replica;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
@@ -58,7 +58,7 @@ public interface FsDatasetSpi extends FSDatasetMBean {
@SuppressWarnings("rawtypes")
final Class extends Factory> clazz = conf.getClass(
DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
- FSDataset.Factory.class,
+ FsDatasetFactory.class,
Factory.class);
return ReflectionUtils.newInstance(clazz, conf);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
new file mode 100644
index 00000000000..ce3dccde71b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -0,0 +1,286 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DU;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+/**
+ * A block pool slice represents a portion of a block pool stored on a volume.
+ * Taken together, all BlockPoolSlices sharing a block pool ID across a
+ * cluster represent a single block pool.
+ *
+ * This class is synchronized by {@link FsVolumeImpl}.
+ */
+class BlockPoolSlice {
+ private final String bpid;
+ private final FsVolumeImpl volume; // volume to which this BlockPool belongs to
+ private final File currentDir; // StorageDirectory/current/bpid/current
+ private final LDir finalizedDir; // directory store Finalized replica
+ private final File rbwDir; // directory store RBW replica
+ private final File tmpDir; // directory store Temporary replica
+
+ // TODO:FEDERATION scalability issue - a thread per DU is needed
+ private final DU dfsUsage;
+
+ /**
+ * Create a blook pool slice
+ * @param bpid Block pool Id
+ * @param volume {@link FsVolumeImpl} to which this BlockPool belongs to
+ * @param bpDir directory corresponding to the BlockPool
+ * @param conf
+ * @throws IOException
+ */
+ BlockPoolSlice(String bpid, FsVolumeImpl volume, File bpDir,
+ Configuration conf) throws IOException {
+ this.bpid = bpid;
+ this.volume = volume;
+ this.currentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
+ final File finalizedDir = new File(
+ currentDir, DataStorage.STORAGE_DIR_FINALIZED);
+
+ // Files that were being written when the datanode was last shutdown
+ // are now moved back to the data directory. It is possible that
+ // in the future, we might want to do some sort of datanode-local
+ // recovery for these blocks. For example, crc validation.
+ //
+ this.tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP);
+ if (tmpDir.exists()) {
+ FileUtil.fullyDelete(tmpDir);
+ }
+ this.rbwDir = new File(currentDir, DataStorage.STORAGE_DIR_RBW);
+ final boolean supportAppends = conf.getBoolean(
+ DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
+ DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
+ if (rbwDir.exists() && !supportAppends) {
+ FileUtil.fullyDelete(rbwDir);
+ }
+ final int maxBlocksPerDir = conf.getInt(
+ DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_KEY,
+ DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_DEFAULT);
+ this.finalizedDir = new LDir(finalizedDir, maxBlocksPerDir);
+ if (!rbwDir.mkdirs()) { // create rbw directory if not exist
+ if (!rbwDir.isDirectory()) {
+ throw new IOException("Mkdirs failed to create " + rbwDir.toString());
+ }
+ }
+ if (!tmpDir.mkdirs()) {
+ if (!tmpDir.isDirectory()) {
+ throw new IOException("Mkdirs failed to create " + tmpDir.toString());
+ }
+ }
+ this.dfsUsage = new DU(bpDir, conf);
+ this.dfsUsage.start();
+ }
+
+ File getDirectory() {
+ return currentDir.getParentFile();
+ }
+
+ File getFinalizedDir() {
+ return finalizedDir.dir;
+ }
+
+ File getRbwDir() {
+ return rbwDir;
+ }
+
+ /** Run DU on local drives. It must be synchronized from caller. */
+ void decDfsUsed(long value) {
+ dfsUsage.decDfsUsed(value);
+ }
+
+ long getDfsUsed() throws IOException {
+ return dfsUsage.getUsed();
+ }
+
+ /**
+ * Temporary files. They get moved to the finalized block directory when
+ * the block is finalized.
+ */
+ File createTmpFile(Block b) throws IOException {
+ File f = new File(tmpDir, b.getBlockName());
+ return DatanodeUtil.createTmpFile(b, f);
+ }
+
+ /**
+ * RBW files. They get moved to the finalized block directory when
+ * the block is finalized.
+ */
+ File createRbwFile(Block b) throws IOException {
+ File f = new File(rbwDir, b.getBlockName());
+ return DatanodeUtil.createTmpFile(b, f);
+ }
+
+ File addBlock(Block b, File f) throws IOException {
+ File blockFile = finalizedDir.addBlock(b, f);
+ File metaFile = FsDatasetUtil.getMetaFile(blockFile, b.getGenerationStamp());
+ dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
+ return blockFile;
+ }
+
+ void checkDirs() throws DiskErrorException {
+ finalizedDir.checkDirTree();
+ DiskChecker.checkDir(tmpDir);
+ DiskChecker.checkDir(rbwDir);
+ }
+
+ void getVolumeMap(ReplicaMap volumeMap) throws IOException {
+ // add finalized replicas
+ finalizedDir.getVolumeMap(bpid, volumeMap, volume);
+ // add rbw replicas
+ addToReplicasMap(volumeMap, rbwDir, false);
+ }
+
+ /**
+ * Add replicas under the given directory to the volume map
+ * @param volumeMap the replicas map
+ * @param dir an input directory
+ * @param isFinalized true if the directory has finalized replicas;
+ * false if the directory has rbw replicas
+ */
+ void addToReplicasMap(ReplicaMap volumeMap, File dir, boolean isFinalized
+ ) throws IOException {
+ File blockFiles[] = FileUtil.listFiles(dir);
+ for (File blockFile : blockFiles) {
+ if (!Block.isBlockFilename(blockFile))
+ continue;
+
+ long genStamp = FsDatasetUtil.getGenerationStampFromFile(
+ blockFiles, blockFile);
+ long blockId = Block.filename2id(blockFile.getName());
+ ReplicaInfo newReplica = null;
+ if (isFinalized) {
+ newReplica = new FinalizedReplica(blockId,
+ blockFile.length(), genStamp, volume, blockFile.getParentFile());
+ } else {
+ newReplica = new ReplicaWaitingToBeRecovered(blockId,
+ validateIntegrity(blockFile, genStamp),
+ genStamp, volume, blockFile.getParentFile());
+ }
+
+ ReplicaInfo oldReplica = volumeMap.add(bpid, newReplica);
+ if (oldReplica != null) {
+ FsDatasetImpl.LOG.warn("Two block files with the same block id exist " +
+ "on disk: " + oldReplica.getBlockFile() + " and " + blockFile );
+ }
+ }
+ }
+
+ /**
+ * Find out the number of bytes in the block that match its crc.
+ *
+ * This algorithm assumes that data corruption caused by unexpected
+ * datanode shutdown occurs only in the last crc chunk. So it checks
+ * only the last chunk.
+ *
+ * @param blockFile the block file
+ * @param genStamp generation stamp of the block
+ * @return the number of valid bytes
+ */
+ private long validateIntegrity(File blockFile, long genStamp) {
+ DataInputStream checksumIn = null;
+ InputStream blockIn = null;
+ try {
+ final File metaFile = FsDatasetUtil.getMetaFile(blockFile, genStamp);
+ long blockFileLen = blockFile.length();
+ long metaFileLen = metaFile.length();
+ int crcHeaderLen = DataChecksum.getChecksumHeaderSize();
+ if (!blockFile.exists() || blockFileLen == 0 ||
+ !metaFile.exists() || metaFileLen < crcHeaderLen) {
+ return 0;
+ }
+ checksumIn = new DataInputStream(
+ new BufferedInputStream(new FileInputStream(metaFile),
+ HdfsConstants.IO_FILE_BUFFER_SIZE));
+
+ // read and handle the common header here. For now just a version
+ BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
+ short version = header.getVersion();
+ if (version != BlockMetadataHeader.VERSION) {
+ FsDatasetImpl.LOG.warn("Wrong version (" + version + ") for metadata file "
+ + metaFile + " ignoring ...");
+ }
+ DataChecksum checksum = header.getChecksum();
+ int bytesPerChecksum = checksum.getBytesPerChecksum();
+ int checksumSize = checksum.getChecksumSize();
+ long numChunks = Math.min(
+ (blockFileLen + bytesPerChecksum - 1)/bytesPerChecksum,
+ (metaFileLen - crcHeaderLen)/checksumSize);
+ if (numChunks == 0) {
+ return 0;
+ }
+ IOUtils.skipFully(checksumIn, (numChunks-1)*checksumSize);
+ blockIn = new FileInputStream(blockFile);
+ long lastChunkStartPos = (numChunks-1)*bytesPerChecksum;
+ IOUtils.skipFully(blockIn, lastChunkStartPos);
+ int lastChunkSize = (int)Math.min(
+ bytesPerChecksum, blockFileLen-lastChunkStartPos);
+ byte[] buf = new byte[lastChunkSize+checksumSize];
+ checksumIn.readFully(buf, lastChunkSize, checksumSize);
+ IOUtils.readFully(blockIn, buf, 0, lastChunkSize);
+
+ checksum.update(buf, 0, lastChunkSize);
+ if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
+ return lastChunkStartPos + lastChunkSize;
+ } else { // last chunck is corrupt
+ return lastChunkStartPos;
+ }
+ } catch (IOException e) {
+ FsDatasetImpl.LOG.warn(e);
+ return 0;
+ } finally {
+ IOUtils.closeStream(checksumIn);
+ IOUtils.closeStream(blockIn);
+ }
+ }
+
+ void clearPath(File f) {
+ finalizedDir.clearPath(f);
+ }
+
+ @Override
+ public String toString() {
+ return currentDir.getAbsolutePath();
+ }
+
+ void shutdown() {
+ dfsUsage.shutdown();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
similarity index 78%
rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java
rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
index 89272b2ecfa..0f171805ab6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import java.io.File;
import java.util.HashMap;
@@ -29,29 +29,25 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
-/*
+/**
* This class is a container of multiple thread pools, each for a volume,
* so that we can schedule async disk operations easily.
*
- * Examples of async disk operations are deletion of block files in FSDataset.
+ * Examples of async disk operations are deletion of block files.
* We don't want to create a new thread for each of the deletion request, and
* we don't want to do all deletions in the heartbeat thread since deletion
* can be slow, and we don't want to use a single thread pool because that
* is inefficient when we have more than 1 volume. AsyncDiskService is the
* solution for these.
*
- * This class is used inside FSDataset.
- *
- * In the future, we should extract AsyncDiskService and put it into common.
- * The FSDataset-specific logic should reside here.
+ * This class and {@link org.apache.hadoop.util.AsyncDiskService} are similar.
+ * They should be combined.
*/
-class FSDatasetAsyncDiskService {
-
- final FSDataset dataset;
-
- public static final Log LOG = LogFactory.getLog(FSDatasetAsyncDiskService.class);
+class FsDatasetAsyncDiskService {
+ public static final Log LOG = LogFactory.getLog(FsDatasetAsyncDiskService.class);
// ThreadPool core pool size
private static final int CORE_THREADS_PER_VOLUME = 1;
@@ -60,9 +56,8 @@ class FSDatasetAsyncDiskService {
// ThreadPool keep-alive time for threads over core pool size
private static final long THREADS_KEEP_ALIVE_SECONDS = 60;
- private final ThreadGroup threadGroup = new ThreadGroup("async disk service");
-
- private HashMap executors
+ private final DataNode datanode;
+ private Map executors
= new HashMap();
/**
@@ -74,8 +69,10 @@ class FSDatasetAsyncDiskService {
*
* @param volumes The roots of the data volumes.
*/
- FSDatasetAsyncDiskService(FSDataset dataset, File[] volumes) {
- this.dataset = dataset;
+ FsDatasetAsyncDiskService(DataNode datanode, File[] volumes) {
+ this.datanode = datanode;
+
+ final ThreadGroup threadGroup = new ThreadGroup(getClass().getSimpleName());
// Create one ThreadPool per volume
for (int v = 0 ; v < volumes.length; v++) {
final File vol = volumes[v];
@@ -136,16 +133,12 @@ class FSDatasetAsyncDiskService {
* tasks to finish.
*/
synchronized void shutdown() {
-
if (executors == null) {
-
LOG.warn("AsyncDiskService has already shut down.");
-
} else {
LOG.info("Shutting down all async disk service threads...");
- for (Map.Entry e
- : executors.entrySet()) {
+ for (Map.Entry e : executors.entrySet()) {
e.getValue().shutdown();
}
// clear the executor map so that calling execute again will fail.
@@ -159,11 +152,11 @@ class FSDatasetAsyncDiskService {
* Delete the block file and meta file from the disk asynchronously, adjust
* dfsUsed statistics accordingly.
*/
- void deleteAsync(FSDataset.FSVolume volume, File blockFile, File metaFile,
+ void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile,
ExtendedBlock block) {
- DataNode.LOG.info("Scheduling block " + block.getLocalBlock().toString()
+ LOG.info("Scheduling block " + block.getLocalBlock()
+ " file " + blockFile + " for deletion");
- ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(dataset,
+ ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
volume, blockFile, metaFile, block);
execute(volume.getCurrentDir(), deletionTask);
}
@@ -171,51 +164,43 @@ class FSDatasetAsyncDiskService {
/** A task for deleting a block file and its associated meta file, as well
* as decrement the dfs usage of the volume.
*/
- static class ReplicaFileDeleteTask implements Runnable {
- final FSDataset dataset;
- final FSDataset.FSVolume volume;
+ class ReplicaFileDeleteTask implements Runnable {
+ final FsVolumeImpl volume;
final File blockFile;
final File metaFile;
final ExtendedBlock block;
- ReplicaFileDeleteTask(FSDataset dataset, FSDataset.FSVolume volume, File blockFile,
+ ReplicaFileDeleteTask(FsVolumeImpl volume, File blockFile,
File metaFile, ExtendedBlock block) {
- this.dataset = dataset;
this.volume = volume;
this.blockFile = blockFile;
this.metaFile = metaFile;
this.block = block;
}
-
- FSDataset.FSVolume getVolume() {
- return volume;
- }
@Override
public String toString() {
// Called in AsyncDiskService.execute for displaying error messages.
return "deletion of block " + block.getBlockPoolId() + " "
- + block.getLocalBlock().toString() + " with block file " + blockFile
+ + block.getLocalBlock() + " with block file " + blockFile
+ " and meta file " + metaFile + " from volume " + volume;
}
@Override
public void run() {
long dfsBytes = blockFile.length() + metaFile.length();
- if ( !blockFile.delete() || ( !metaFile.delete() && metaFile.exists() ) ) {
- DataNode.LOG.warn("Unexpected error trying to delete block "
- + block.getBlockPoolId() + " " + block.getLocalBlock().toString()
+ if (!blockFile.delete() || (!metaFile.delete() && metaFile.exists())) {
+ LOG.warn("Unexpected error trying to delete block "
+ + block.getBlockPoolId() + " " + block.getLocalBlock()
+ " at file " + blockFile + ". Ignored.");
} else {
if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){
- dataset.notifyNamenodeDeletedBlock(block);
+ datanode.notifyNamenodeDeletedBlock(block);
}
volume.decDfsUsed(block.getBlockPoolId(), dfsBytes);
- DataNode.LOG.info("Deleted block " + block.getBlockPoolId() + " "
- + block.getLocalBlock().toString() + " at file " + blockFile);
+ LOG.info("Deleted block " + block.getBlockPoolId() + " "
+ + block.getLocalBlock() + " at file " + blockFile);
}
}
- };
-
-
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetFactory.java
new file mode 100644
index 00000000000..52e385bff66
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetFactory.java
@@ -0,0 +1,36 @@
+/**
+ * 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.datanode.fsdataset.impl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+
+/**
+ * A factory for creating {@link FsDatasetImpl} objects.
+ */
+public class FsDatasetFactory extends FsDatasetSpi.Factory {
+ @Override
+ public FsDatasetImpl newInstance(DataNode datanode,
+ DataStorage storage, Configuration conf) throws IOException {
+ return new FsDatasetImpl(datanode, storage, conf);
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
similarity index 53%
rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index f8699630f28..567ad90710c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -15,52 +15,53 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
-import java.io.BufferedInputStream;
-import java.io.BufferedReader;
-import java.io.DataInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
-import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
-import java.io.PrintStream;
import java.io.RandomAccessFile;
import java.nio.channels.FileChannel;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
-import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
import javax.management.NotCompliantMBeanException;
import javax.management.ObjectName;
import javax.management.StandardMBean;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.DF;
-import org.apache.hadoop.fs.DU;
-import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.Replica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
@@ -72,10 +73,8 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
-import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
import org.apache.hadoop.util.ReflectionUtils;
@@ -86,951 +85,18 @@ import org.apache.hadoop.util.ReflectionUtils;
*
***************************************************/
@InterfaceAudience.Private
-public class FSDataset implements FsDatasetSpi {
- /**
- * A factory for creating FSDataset objects.
- */
- public static class Factory extends FsDatasetSpi.Factory {
- @Override
- public FSDataset newInstance(DataNode datanode,
- DataStorage storage, Configuration conf) throws IOException {
- return new FSDataset(datanode, storage, conf);
- }
- }
-
- /**
- * A node type that can be built into a tree reflecting the
- * hierarchy of blocks on the local disk.
- */
- private static class FSDir {
- final int maxBlocksPerDir;
- final File dir;
- int numBlocks = 0;
- FSDir children[];
- int lastChildIdx = 0;
-
- private FSDir(File dir, int maxBlocksPerDir)
- throws IOException {
- this.dir = dir;
- this.maxBlocksPerDir = maxBlocksPerDir;
-
- this.children = null;
- if (!dir.exists()) {
- if (!dir.mkdirs()) {
- throw new IOException("Mkdirs failed to create " +
- dir.toString());
- }
- } else {
- File[] files = FileUtil.listFiles(dir);
- List dirList = new ArrayList();
- for (int idx = 0; idx < files.length; idx++) {
- if (files[idx].isDirectory()) {
- dirList.add(new FSDir(files[idx], maxBlocksPerDir));
- } else if (Block.isBlockFilename(files[idx])) {
- numBlocks++;
- }
- }
- if (dirList.size() > 0) {
- children = dirList.toArray(new FSDir[dirList.size()]);
- }
- }
- }
-
- private File addBlock(Block b, File src) throws IOException {
- //First try without creating subdirectories
- File file = addBlock(b, src, false, false);
- return (file != null) ? file : addBlock(b, src, true, true);
- }
-
- private File addBlock(Block b, File src, boolean createOk,
- boolean resetIdx) throws IOException {
- if (numBlocks < maxBlocksPerDir) {
- final File dest = moveBlockFiles(b, src, dir);
- numBlocks += 1;
- return dest;
- }
-
- if (lastChildIdx < 0 && resetIdx) {
- //reset so that all children will be checked
- lastChildIdx = DFSUtil.getRandom().nextInt(children.length);
- }
-
- if (lastChildIdx >= 0 && children != null) {
- //Check if any child-tree has room for a block.
- for (int i=0; i < children.length; i++) {
- int idx = (lastChildIdx + i)%children.length;
- File file = children[idx].addBlock(b, src, false, resetIdx);
- if (file != null) {
- lastChildIdx = idx;
- return file;
- }
- }
- lastChildIdx = -1;
- }
-
- if (!createOk) {
- return null;
- }
-
- if (children == null || children.length == 0) {
- children = new FSDir[maxBlocksPerDir];
- for (int idx = 0; idx < maxBlocksPerDir; idx++) {
- final File sub = new File(dir, DataStorage.BLOCK_SUBDIR_PREFIX+idx);
- children[idx] = new FSDir(sub, maxBlocksPerDir);
- }
- }
-
- //now pick a child randomly for creating a new set of subdirs.
- lastChildIdx = DFSUtil.getRandom().nextInt(children.length);
- return children[ lastChildIdx ].addBlock(b, src, true, false);
- }
-
- private void getVolumeMap(String bpid, ReplicasMap volumeMap, FSVolume volume)
- throws IOException {
- if (children != null) {
- for (int i = 0; i < children.length; i++) {
- children[i].getVolumeMap(bpid, volumeMap, volume);
- }
- }
-
- recoverTempUnlinkedBlock();
- volume.addToReplicasMap(bpid, volumeMap, dir, true);
- }
-
- /**
- * Recover unlinked tmp files on datanode restart. If the original block
- * does not exist, then the tmp file is renamed to be the
- * original file name; otherwise the tmp file is deleted.
- */
- private void recoverTempUnlinkedBlock() throws IOException {
- File files[] = FileUtil.listFiles(dir);
- for (File file : files) {
- if (!FSDataset.isUnlinkTmpFile(file)) {
- continue;
- }
- File blockFile = getOrigFile(file);
- if (blockFile.exists()) {
- //
- // If the original block file still exists, then no recovery
- // is needed.
- //
- if (!file.delete()) {
- throw new IOException("Unable to cleanup unlinked tmp file " +
- file);
- }
- } else {
- if (!file.renameTo(blockFile)) {
- throw new IOException("Unable to cleanup detached file " +
- file);
- }
- }
- }
- }
-
- /**
- * check if a data diretory is healthy
- * @throws DiskErrorException
- */
- private void checkDirTree() throws DiskErrorException {
- DiskChecker.checkDir(dir);
-
- if (children != null) {
- for (int i = 0; i < children.length; i++) {
- children[i].checkDirTree();
- }
- }
- }
-
- private void clearPath(File f) {
- String root = dir.getAbsolutePath();
- String dir = f.getAbsolutePath();
- if (dir.startsWith(root)) {
- String[] dirNames = dir.substring(root.length()).
- split(File.separator + "subdir");
- if (clearPath(f, dirNames, 1))
- return;
- }
- clearPath(f, null, -1);
- }
-
- /*
- * dirNames is an array of string integers derived from
- * usual directory structure data/subdirN/subdirXY/subdirM ...
- * If dirName array is non-null, we only check the child at
- * the children[dirNames[idx]]. This avoids iterating over
- * children in common case. If directory structure changes
- * in later versions, we need to revisit this.
- */
- private boolean clearPath(File f, String[] dirNames, int idx) {
- if ((dirNames == null || idx == dirNames.length) &&
- dir.compareTo(f) == 0) {
- numBlocks--;
- return true;
- }
-
- if (dirNames != null) {
- //guess the child index from the directory name
- if (idx > (dirNames.length - 1) || children == null) {
- return false;
- }
- int childIdx;
- try {
- childIdx = Integer.parseInt(dirNames[idx]);
- } catch (NumberFormatException ignored) {
- // layout changed? we could print a warning.
- return false;
- }
- return (childIdx >= 0 && childIdx < children.length) ?
- children[childIdx].clearPath(f, dirNames, idx+1) : false;
- }
-
- //guesses failed. back to blind iteration.
- if (children != null) {
- for(int i=0; i < children.length; i++) {
- if (children[i].clearPath(f, null, -1)){
- return true;
- }
- }
- }
- return false;
- }
-
- @Override
- public String toString() {
- return "FSDir{" +
- "dir=" + dir +
- ", children=" + (children == null ? null : Arrays.asList(children)) +
- "}";
- }
- }
-
- /**
- * A BlockPoolSlice represents a portion of a BlockPool stored on a volume.
- * Taken together, all BlockPoolSlices sharing a block pool ID across a
- * cluster represent a single block pool.
- *
- * This class is synchronized by {@link FSVolume}.
- */
- private static class BlockPoolSlice {
- private final String bpid;
- private final FSVolume volume; // volume to which this BlockPool belongs to
- private final File currentDir; // StorageDirectory/current/bpid/current
- private final FSDir finalizedDir; // directory store Finalized replica
- private final File rbwDir; // directory store RBW replica
- private final File tmpDir; // directory store Temporary replica
-
- // TODO:FEDERATION scalability issue - a thread per DU is needed
- private final DU dfsUsage;
-
- /**
- *
- * @param bpid Block pool Id
- * @param volume {@link FSVolume} to which this BlockPool belongs to
- * @param bpDir directory corresponding to the BlockPool
- * @param conf
- * @throws IOException
- */
- BlockPoolSlice(String bpid, FSVolume volume, File bpDir, Configuration conf)
- throws IOException {
- this.bpid = bpid;
- this.volume = volume;
- this.currentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
- final File finalizedDir = new File(
- currentDir, DataStorage.STORAGE_DIR_FINALIZED);
-
- // Files that were being written when the datanode was last shutdown
- // are now moved back to the data directory. It is possible that
- // in the future, we might want to do some sort of datanode-local
- // recovery for these blocks. For example, crc validation.
- //
- this.tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP);
- if (tmpDir.exists()) {
- FileUtil.fullyDelete(tmpDir);
- }
- this.rbwDir = new File(currentDir, DataStorage.STORAGE_DIR_RBW);
- final boolean supportAppends = conf.getBoolean(
- DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
- DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
- if (rbwDir.exists() && !supportAppends) {
- FileUtil.fullyDelete(rbwDir);
- }
- final int maxBlocksPerDir = conf.getInt(
- DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_KEY,
- DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_DEFAULT);
- this.finalizedDir = new FSDir(finalizedDir, maxBlocksPerDir);
- if (!rbwDir.mkdirs()) { // create rbw directory if not exist
- if (!rbwDir.isDirectory()) {
- throw new IOException("Mkdirs failed to create " + rbwDir.toString());
- }
- }
- if (!tmpDir.mkdirs()) {
- if (!tmpDir.isDirectory()) {
- throw new IOException("Mkdirs failed to create " + tmpDir.toString());
- }
- }
- this.dfsUsage = new DU(bpDir, conf);
- this.dfsUsage.start();
- }
-
- File getDirectory() {
- return currentDir.getParentFile();
- }
-
- File getFinalizedDir() {
- return finalizedDir.dir;
- }
-
- File getRbwDir() {
- return rbwDir;
- }
-
- /**
- * This should be used only by {@link FSVolume#decDfsUsed(String, long)}
- * and it will be synchronized there.
- */
- void decDfsUsed(long value) {
- dfsUsage.decDfsUsed(value);
- }
-
- long getDfsUsed() throws IOException {
- return dfsUsage.getUsed();
- }
-
- /**
- * Temporary files. They get moved to the finalized block directory when
- * the block is finalized.
- */
- File createTmpFile(Block b) throws IOException {
- File f = new File(tmpDir, b.getBlockName());
- return DatanodeUtil.createTmpFile(b, f);
- }
-
- /**
- * RBW files. They get moved to the finalized block directory when
- * the block is finalized.
- */
- File createRbwFile(Block b) throws IOException {
- File f = new File(rbwDir, b.getBlockName());
- return DatanodeUtil.createTmpFile(b, f);
- }
-
- File addBlock(Block b, File f) throws IOException {
- File blockFile = finalizedDir.addBlock(b, f);
- File metaFile = DatanodeUtil.getMetaFile(blockFile, b.getGenerationStamp());
- dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
- return blockFile;
- }
-
- void checkDirs() throws DiskErrorException {
- finalizedDir.checkDirTree();
- DiskChecker.checkDir(tmpDir);
- DiskChecker.checkDir(rbwDir);
- }
-
- void getVolumeMap(ReplicasMap volumeMap) throws IOException {
- // add finalized replicas
- finalizedDir.getVolumeMap(bpid, volumeMap, volume);
- // add rbw replicas
- addToReplicasMap(volumeMap, rbwDir, false);
- }
-
- /**
- * Add replicas under the given directory to the volume map
- * @param volumeMap the replicas map
- * @param dir an input directory
- * @param isFinalized true if the directory has finalized replicas;
- * false if the directory has rbw replicas
- */
- private void addToReplicasMap(ReplicasMap volumeMap, File dir,
- boolean isFinalized) throws IOException {
- File blockFiles[] = FileUtil.listFiles(dir);
- for (File blockFile : blockFiles) {
- if (!Block.isBlockFilename(blockFile))
- continue;
-
- long genStamp = getGenerationStampFromFile(blockFiles, blockFile);
- long blockId = Block.filename2id(blockFile.getName());
- ReplicaInfo newReplica = null;
- if (isFinalized) {
- newReplica = new FinalizedReplica(blockId,
- blockFile.length(), genStamp, volume, blockFile.getParentFile());
- } else {
- newReplica = new ReplicaWaitingToBeRecovered(blockId,
- validateIntegrity(blockFile, genStamp),
- genStamp, volume, blockFile.getParentFile());
- }
-
- ReplicaInfo oldReplica = volumeMap.add(bpid, newReplica);
- if (oldReplica != null) {
- DataNode.LOG.warn("Two block files with the same block id exist " +
- "on disk: " + oldReplica.getBlockFile() +
- " and " + blockFile );
- }
- }
- }
-
- /**
- * Find out the number of bytes in the block that match its crc.
- *
- * This algorithm assumes that data corruption caused by unexpected
- * datanode shutdown occurs only in the last crc chunk. So it checks
- * only the last chunk.
- *
- * @param blockFile the block file
- * @param genStamp generation stamp of the block
- * @return the number of valid bytes
- */
- private long validateIntegrity(File blockFile, long genStamp) {
- DataInputStream checksumIn = null;
- InputStream blockIn = null;
- try {
- final File metaFile = DatanodeUtil.getMetaFile(blockFile, genStamp);
- long blockFileLen = blockFile.length();
- long metaFileLen = metaFile.length();
- int crcHeaderLen = DataChecksum.getChecksumHeaderSize();
- if (!blockFile.exists() || blockFileLen == 0 ||
- !metaFile.exists() || metaFileLen < crcHeaderLen) {
- return 0;
- }
- checksumIn = new DataInputStream(
- new BufferedInputStream(new FileInputStream(metaFile),
- HdfsConstants.IO_FILE_BUFFER_SIZE));
-
- // read and handle the common header here. For now just a version
- BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
- short version = header.getVersion();
- if (version != BlockMetadataHeader.VERSION) {
- DataNode.LOG.warn("Wrong version (" + version + ") for metadata file "
- + metaFile + " ignoring ...");
- }
- DataChecksum checksum = header.getChecksum();
- int bytesPerChecksum = checksum.getBytesPerChecksum();
- int checksumSize = checksum.getChecksumSize();
- long numChunks = Math.min(
- (blockFileLen + bytesPerChecksum - 1)/bytesPerChecksum,
- (metaFileLen - crcHeaderLen)/checksumSize);
- if (numChunks == 0) {
- return 0;
- }
- IOUtils.skipFully(checksumIn, (numChunks-1)*checksumSize);
- blockIn = new FileInputStream(blockFile);
- long lastChunkStartPos = (numChunks-1)*bytesPerChecksum;
- IOUtils.skipFully(blockIn, lastChunkStartPos);
- int lastChunkSize = (int)Math.min(
- bytesPerChecksum, blockFileLen-lastChunkStartPos);
- byte[] buf = new byte[lastChunkSize+checksumSize];
- checksumIn.readFully(buf, lastChunkSize, checksumSize);
- IOUtils.readFully(blockIn, buf, 0, lastChunkSize);
-
- checksum.update(buf, 0, lastChunkSize);
- if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
- return lastChunkStartPos + lastChunkSize;
- } else { // last chunck is corrupt
- return lastChunkStartPos;
- }
- } catch (IOException e) {
- DataNode.LOG.warn(e);
- return 0;
- } finally {
- IOUtils.closeStream(checksumIn);
- IOUtils.closeStream(blockIn);
- }
- }
-
- void clearPath(File f) {
- finalizedDir.clearPath(f);
- }
-
- public String toString() {
- return currentDir.getAbsolutePath();
- }
-
- public void shutdown() {
- dfsUsage.shutdown();
- }
- }
-
- /**
- * The underlying volume used to store replica.
- *
- * It uses the {@link FSDataset} object for synchronization.
- */
- static class FSVolume implements FsVolumeSpi {
- private final FSDataset dataset;
- private final String storageID;
- private final Map map = new HashMap();
- private final File currentDir; // /current
- private final DF usage;
- private final long reserved;
-
- FSVolume(FSDataset dataset, String storageID, File currentDir,
- Configuration conf) throws IOException {
- this.dataset = dataset;
- this.storageID = storageID;
- this.reserved = conf.getLong(DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY,
- DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT);
- this.currentDir = currentDir;
- File parent = currentDir.getParentFile();
- this.usage = new DF(parent, conf);
- }
-
- File getCurrentDir() {
- return currentDir;
- }
-
- File getRbwDir(String bpid) throws IOException {
- BlockPoolSlice bp = getBlockPoolSlice(bpid);
- return bp.getRbwDir();
- }
-
- void decDfsUsed(String bpid, long value) {
- synchronized(dataset) {
- BlockPoolSlice bp = map.get(bpid);
- if (bp != null) {
- bp.decDfsUsed(value);
- }
- }
- }
-
- long getDfsUsed() throws IOException {
- long dfsUsed = 0;
- synchronized(dataset) {
- for(BlockPoolSlice s : map.values()) {
- dfsUsed += s.getDfsUsed();
- }
- }
- return dfsUsed;
- }
-
- long getBlockPoolUsed(String bpid) throws IOException {
- return getBlockPoolSlice(bpid).getDfsUsed();
- }
-
- /**
- * Calculate the capacity of the filesystem, after removing any
- * reserved capacity.
- * @return the unreserved number of bytes left in this filesystem. May be zero.
- */
- long getCapacity() {
- long remaining = usage.getCapacity() - reserved;
- return remaining > 0 ? remaining : 0;
- }
-
- @Override
- public long getAvailable() throws IOException {
- long remaining = getCapacity()-getDfsUsed();
- long available = usage.getAvailable();
- if (remaining>available) {
- remaining = available;
- }
- return (remaining > 0) ? remaining : 0;
- }
-
- long getReserved(){
- return reserved;
- }
-
- String getMount() throws IOException {
- return usage.getMount();
- }
-
- private BlockPoolSlice getBlockPoolSlice(String bpid) throws IOException {
- BlockPoolSlice bp = map.get(bpid);
- if (bp == null) {
- throw new IOException("block pool " + bpid + " is not found");
- }
- return bp;
- }
-
- @Override
- public String getPath(String bpid) throws IOException {
- return getBlockPoolSlice(bpid).getDirectory().getAbsolutePath();
- }
-
- @Override
- public File getFinalizedDir(String bpid) throws IOException {
- return getBlockPoolSlice(bpid).getFinalizedDir();
- }
-
- /**
- * Make a deep copy of the list of currently active BPIDs
- */
- @Override
- public String[] getBlockPoolList() {
- synchronized(dataset) {
- return map.keySet().toArray(new String[map.keySet().size()]);
- }
- }
-
- /**
- * Temporary files. They get moved to the finalized block directory when
- * the block is finalized.
- */
- File createTmpFile(String bpid, Block b) throws IOException {
- BlockPoolSlice bp = getBlockPoolSlice(bpid);
- return bp.createTmpFile(b);
- }
-
- /**
- * RBW files. They get moved to the finalized block directory when
- * the block is finalized.
- */
- File createRbwFile(String bpid, Block b) throws IOException {
- BlockPoolSlice bp = getBlockPoolSlice(bpid);
- return bp.createRbwFile(b);
- }
-
- File addBlock(String bpid, Block b, File f) throws IOException {
- BlockPoolSlice bp = getBlockPoolSlice(bpid);
- return bp.addBlock(b, f);
- }
-
- /**
- * This should be used only by {@link FSVolumeSet#checkDirs()}
- * and it will be synchronized there.
- */
- void checkDirs() throws DiskErrorException {
- // TODO:FEDERATION valid synchronization
- for(BlockPoolSlice s : map.values()) {
- s.checkDirs();
- }
- }
-
- void getVolumeMap(ReplicasMap volumeMap) throws IOException {
- Set> set = map.entrySet();
- for (Entry entry : set) {
- entry.getValue().getVolumeMap(volumeMap);
- }
- }
-
- void getVolumeMap(String bpid, ReplicasMap volumeMap) throws IOException {
- BlockPoolSlice bp = getBlockPoolSlice(bpid);
- bp.getVolumeMap(volumeMap);
- }
-
- /**
- * Add replicas under the given directory to the volume map
- * @param volumeMap the replicas map
- * @param dir an input directory
- * @param isFinalized true if the directory has finalized replicas;
- * false if the directory has rbw replicas
- * @throws IOException
- */
- private void addToReplicasMap(String bpid, ReplicasMap volumeMap,
- File dir, boolean isFinalized) throws IOException {
- BlockPoolSlice bp = getBlockPoolSlice(bpid);
- // TODO move this up
- // dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
- bp.addToReplicasMap(volumeMap, dir, isFinalized);
- }
-
- void clearPath(String bpid, File f) throws IOException {
- BlockPoolSlice bp = getBlockPoolSlice(bpid);
- bp.clearPath(f);
- }
-
- @Override
- public String toString() {
- return currentDir.getAbsolutePath();
- }
-
- public void shutdown() {
- Set> set = map.entrySet();
- for (Entry entry : set) {
- entry.getValue().shutdown();
- }
- }
-
- public void addBlockPool(String bpid, Configuration conf)
- throws IOException {
- File bpdir = new File(currentDir, bpid);
- BlockPoolSlice bp = new BlockPoolSlice(bpid, this, bpdir, conf);
- map.put(bpid, bp);
- }
-
- public void shutdownBlockPool(String bpid) {
- BlockPoolSlice bp = map.get(bpid);
- if (bp!=null) {
- bp.shutdown();
- }
- map.remove(bpid);
- }
-
- private boolean isBPDirEmpty(String bpid)
- throws IOException {
- File volumeCurrentDir = this.getCurrentDir();
- File bpDir = new File(volumeCurrentDir, bpid);
- File bpCurrentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
- File finalizedDir = new File(bpCurrentDir,
- DataStorage.STORAGE_DIR_FINALIZED);
- File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
- if (finalizedDir.exists() && FileUtil.list(finalizedDir).length != 0) {
- return false;
- }
- if (rbwDir.exists() && FileUtil.list(rbwDir).length != 0) {
- return false;
- }
- return true;
- }
-
- private void deleteBPDirectories(String bpid, boolean force)
- throws IOException {
- File volumeCurrentDir = this.getCurrentDir();
- File bpDir = new File(volumeCurrentDir, bpid);
- if (!bpDir.isDirectory()) {
- // nothing to be deleted
- return;
- }
- File tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP);
- File bpCurrentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
- File finalizedDir = new File(bpCurrentDir,
- DataStorage.STORAGE_DIR_FINALIZED);
- File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
- if (force) {
- FileUtil.fullyDelete(bpDir);
- } else {
- if (!rbwDir.delete()) {
- throw new IOException("Failed to delete " + rbwDir);
- }
- if (!finalizedDir.delete()) {
- throw new IOException("Failed to delete " + finalizedDir);
- }
- FileUtil.fullyDelete(tmpDir);
- for (File f : FileUtil.listFiles(bpCurrentDir)) {
- if (!f.delete()) {
- throw new IOException("Failed to delete " + f);
- }
- }
- if (!bpCurrentDir.delete()) {
- throw new IOException("Failed to delete " + bpCurrentDir);
- }
- for (File f : FileUtil.listFiles(bpDir)) {
- if (!f.delete()) {
- throw new IOException("Failed to delete " + f);
- }
- }
- if (!bpDir.delete()) {
- throw new IOException("Failed to delete " + bpDir);
- }
- }
- }
-
- String getStorageID() {
- return storageID;
- }
- }
-
- static class FSVolumeSet {
- /*
- * Read access to this unmodifiable list is not synchronized.
- * This list is replaced on modification holding "this" lock.
- */
- private volatile List volumes = null;
-
- final VolumeChoosingPolicy blockChooser;
- int numFailedVolumes;
-
- FSVolumeSet(List volumes, int failedVols,
- VolumeChoosingPolicy blockChooser) {
- this.volumes = Collections.unmodifiableList(volumes);
- this.blockChooser = blockChooser;
- this.numFailedVolumes = failedVols;
- }
-
- private int numberOfFailedVolumes() {
- return numFailedVolumes;
- }
-
- /**
- * Get next volume. Synchronized to ensure {@link #curVolume} is updated
- * by a single thread and next volume is chosen with no concurrent
- * update to {@link #volumes}.
- * @param blockSize free space needed on the volume
- * @return next volume to store the block in.
- */
- synchronized FSVolume getNextVolume(long blockSize) throws IOException {
- return blockChooser.chooseVolume(volumes, blockSize);
- }
-
- private long getDfsUsed() throws IOException {
- long dfsUsed = 0L;
- for (FSVolume v : volumes) {
- dfsUsed += v.getDfsUsed();
- }
- return dfsUsed;
- }
-
- private long getBlockPoolUsed(String bpid) throws IOException {
- long dfsUsed = 0L;
- for (FSVolume v : volumes) {
- dfsUsed += v.getBlockPoolUsed(bpid);
- }
- return dfsUsed;
- }
-
- private long getCapacity() {
- long capacity = 0L;
- for (FSVolume v : volumes) {
- capacity += v.getCapacity();
- }
- return capacity;
- }
-
- private long getRemaining() throws IOException {
- long remaining = 0L;
- for (FsVolumeSpi vol : volumes) {
- remaining += vol.getAvailable();
- }
- return remaining;
- }
-
- private void getVolumeMap(ReplicasMap volumeMap) throws IOException {
- for (FSVolume v : volumes) {
- v.getVolumeMap(volumeMap);
- }
- }
-
- private void getVolumeMap(String bpid, ReplicasMap volumeMap)
- throws IOException {
- for (FSVolume v : volumes) {
- v.getVolumeMap(bpid, volumeMap);
- }
- }
-
- /**
- * Calls {@link FSVolume#checkDirs()} on each volume, removing any
- * volumes from the active list that result in a DiskErrorException.
- *
- * This method is synchronized to allow only one instance of checkDirs()
- * call
- * @return list of all the removed volumes.
- */
- private synchronized List checkDirs() {
- ArrayList removedVols = null;
-
- // Make a copy of volumes for performing modification
- final List volumeList = new ArrayList(volumes);
-
- for(Iterator i = volumeList.iterator(); i.hasNext(); ) {
- final FSVolume fsv = i.next();
- try {
- fsv.checkDirs();
- } catch (DiskErrorException e) {
- DataNode.LOG.warn("Removing failed volume " + fsv + ": ",e);
- if (removedVols == null) {
- removedVols = new ArrayList(2);
- }
- removedVols.add(fsv);
- fsv.shutdown();
- i.remove(); // Remove the volume
- numFailedVolumes++;
- }
- }
-
- if (removedVols != null && removedVols.size() > 0) {
- // Replace volume list
- volumes = Collections.unmodifiableList(volumeList);
- DataNode.LOG.info("Completed FSVolumeSet.checkDirs. Removed "
- + removedVols.size() + " volumes. List of current volumes: "
- + this);
- }
-
- return removedVols;
- }
-
- @Override
- public String toString() {
- return volumes.toString();
- }
-
-
- private void addBlockPool(String bpid, Configuration conf)
- throws IOException {
- for (FSVolume v : volumes) {
- v.addBlockPool(bpid, conf);
- }
- }
-
- private void removeBlockPool(String bpid) {
- for (FSVolume v : volumes) {
- v.shutdownBlockPool(bpid);
- }
- }
-
- private void shutdown() {
- for (FSVolume volume : volumes) {
- if(volume != null) {
- volume.shutdown();
- }
- }
- }
- }
-
- //////////////////////////////////////////////////////
- //
- // FSDataSet
- //
- //////////////////////////////////////////////////////
-
- private static boolean isUnlinkTmpFile(File f) {
- String name = f.getName();
- return name.endsWith(DatanodeUtil.UNLINK_BLOCK_SUFFIX);
- }
-
- private static File getOrigFile(File unlinkTmpFile) {
- String fileName = unlinkTmpFile.getName();
- return new File(unlinkTmpFile.getParentFile(),
- fileName.substring(0,
- fileName.length() - DatanodeUtil.UNLINK_BLOCK_SUFFIX.length()));
- }
-
- protected File getMetaFile(ExtendedBlock b) throws IOException {
- return DatanodeUtil.getMetaFile(getBlockFile(b), b.getGenerationStamp());
- }
-
- /** Find the metadata file for the specified block file.
- * Return the generation stamp from the name of the metafile.
- */
- private static long getGenerationStampFromFile(File[] listdir, File blockFile) {
- String blockName = blockFile.getName();
- for (int j = 0; j < listdir.length; j++) {
- String path = listdir[j].getName();
- if (!path.startsWith(blockName)) {
- continue;
- }
- if (blockFile == listdir[j]) {
- continue;
- }
- return Block.getGenerationStamp(listdir[j].getName());
- }
- DataNode.LOG.warn("Block " + blockFile +
- " does not have a metafile!");
- return GenerationStamp.GRANDFATHER_GENERATION_STAMP;
- }
-
- /** Find the corresponding meta data file from a given block file */
- private static long parseGenerationStamp(File blockFile, File metaFile
- ) throws IOException {
- String metaname = metaFile.getName();
- String gs = metaname.substring(blockFile.getName().length() + 1,
- metaname.length() - DatanodeUtil.METADATA_EXTENSION.length());
- try {
- return Long.parseLong(gs);
- } catch(NumberFormatException nfe) {
- throw (IOException)new IOException("blockFile=" + blockFile
- + ", metaFile=" + metaFile).initCause(nfe);
- }
- }
+class FsDatasetImpl implements FsDatasetSpi {
+ static final Log LOG = LogFactory.getLog(FsDatasetImpl.class);
@Override // FsDatasetSpi
- public List getVolumes() {
+ public List getVolumes() {
return volumes.volumes;
}
@Override
- public synchronized FSVolume getVolume(final ExtendedBlock b) {
+ public synchronized FsVolumeImpl getVolume(final ExtendedBlock b) {
final ReplicaInfo r = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock());
- return r != null? (FSVolume)r.getVolume(): null;
+ return r != null? (FsVolumeImpl)r.getVolume(): null;
}
@Override // FsDatasetSpi
@@ -1040,11 +106,12 @@ public class FSDataset implements FsDatasetSpi {
if (blockfile == null) {
return null;
}
- final File metafile = DatanodeUtil.findMetaFile(blockfile);
- return new Block(blkid, blockfile.length(),
- parseGenerationStamp(blockfile, metafile));
+ final File metafile = FsDatasetUtil.findMetaFile(blockfile);
+ final long gs = FsDatasetUtil.parseGenerationStamp(blockfile, metafile);
+ return new Block(blkid, blockfile.length(), gs);
}
+
/**
* Returns a clone of a replica stored in data-node memory.
* Should be primarily used for testing.
@@ -1069,21 +136,21 @@ public class FSDataset implements FsDatasetSpi {
}
return null;
}
-
+
@Override // FsDatasetSpi
public LengthInputStream getMetaDataInputStream(ExtendedBlock b)
throws IOException {
- final File meta = getMetaFile(b);
+ File meta = FsDatasetUtil.getMetaFile(getBlockFile(b), b.getGenerationStamp());
if (meta == null || !meta.exists()) {
return null;
}
return new LengthInputStream(new FileInputStream(meta), meta.length());
}
- private final DataNode datanode;
- final FSVolumeSet volumes;
- final ReplicasMap volumeMap;
- final FSDatasetAsyncDiskService asyncDiskService;
+ final DataNode datanode;
+ final FsVolumeList volumes;
+ final ReplicaMap volumeMap;
+ final FsDatasetAsyncDiskService asyncDiskService;
private final int validVolsRequired;
// Used for synchronizing access to usage stats
@@ -1092,7 +159,7 @@ public class FSDataset implements FsDatasetSpi {
/**
* An FSDataset has a directory where it loads its data files.
*/
- private FSDataset(DataNode datanode, DataStorage storage, Configuration conf
+ FsDatasetImpl(DataNode datanode, DataStorage storage, Configuration conf
) throws IOException {
this.datanode = datanode;
// The number of volumes required for operation is the total number
@@ -1119,29 +186,29 @@ public class FSDataset implements FsDatasetSpi {
+ ", volume failures tolerated: " + volFailuresTolerated);
}
- final List volArray = new ArrayList(
+ final List volArray = new ArrayList(
storage.getNumStorageDirs());
for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
final File dir = storage.getStorageDir(idx).getCurrentDir();
- volArray.add(new FSVolume(this, storage.getStorageID(), dir, conf));
- DataNode.LOG.info("FSDataset added volume - " + dir);
+ volArray.add(new FsVolumeImpl(this, storage.getStorageID(), dir, conf));
+ LOG.info("Added volume - " + dir);
}
- volumeMap = new ReplicasMap(this);
+ volumeMap = new ReplicaMap(this);
@SuppressWarnings("unchecked")
- final VolumeChoosingPolicy blockChooserImpl =
+ final VolumeChoosingPolicy blockChooserImpl =
ReflectionUtils.newInstance(conf.getClass(
DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_POLICY_KEY,
RoundRobinVolumeChoosingPolicy.class,
VolumeChoosingPolicy.class), conf);
- volumes = new FSVolumeSet(volArray, volsFailed, blockChooserImpl);
+ volumes = new FsVolumeList(volArray, volsFailed, blockChooserImpl);
volumes.getVolumeMap(volumeMap);
File[] roots = new File[storage.getNumStorageDirs()];
for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
roots[idx] = storage.getStorageDir(idx).getCurrentDir();
}
- asyncDiskService = new FSDatasetAsyncDiskService(this, roots);
+ asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots);
registerMBean(storage.getStorageID());
}
@@ -1221,8 +288,8 @@ public class FSDataset implements FsDatasetSpi {
File getBlockFile(String bpid, Block b) throws IOException {
File f = validateBlockFile(bpid, b);
if(f == null) {
- if (DataNode.LOG.isDebugEnabled()) {
- DataNode.LOG.debug("b=" + b + ", volumeMap=" + volumeMap);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("b=" + b + ", volumeMap=" + volumeMap);
}
throw new IOException("Block " + b + " is not valid.");
}
@@ -1322,27 +389,12 @@ public class FSDataset implements FsDatasetSpi {
return new ReplicaInputStreams(new FileInputStream(blockInFile.getFD()),
new FileInputStream(metaInFile.getFD()));
}
-
- /**
- * Make a copy of the block if this block is linked to an existing
- * snapshot. This ensures that modifying this block does not modify
- * data in any existing snapshots.
- * @param block Block
- * @param numLinks Unlink if the number of links exceed this value
- * @throws IOException
- * @return - true if the specified block was unlinked or the block
- * is not in any snapshot.
- */
- public boolean unlinkBlock(ExtendedBlock block, int numLinks) throws IOException {
- ReplicaInfo info = getReplicaInfo(block);
- return info.unlinkBlock(numLinks);
- }
- private static File moveBlockFiles(Block b, File srcfile, File destdir
+ static File moveBlockFiles(Block b, File srcfile, File destdir
) throws IOException {
final File dstfile = new File(destdir, b.getBlockName());
- final File srcmeta = DatanodeUtil.getMetaFile(srcfile, b.getGenerationStamp());
- final File dstmeta = DatanodeUtil.getMetaFile(dstfile, b.getGenerationStamp());
+ final File srcmeta = FsDatasetUtil.getMetaFile(srcfile, b.getGenerationStamp());
+ final File dstmeta = FsDatasetUtil.getMetaFile(dstfile, b.getGenerationStamp());
if (!srcmeta.renameTo(dstmeta)) {
throw new IOException("Failed to move meta file for " + b
+ " from " + srcmeta + " to " + dstmeta);
@@ -1351,16 +403,16 @@ public class FSDataset implements FsDatasetSpi {
throw new IOException("Failed to move block file for " + b
+ " from " + srcfile + " to " + dstfile.getAbsolutePath());
}
- if (DataNode.LOG.isDebugEnabled()) {
- DataNode.LOG.debug("addBlock: Moved " + srcmeta + " to " + dstmeta);
- DataNode.LOG.debug("addBlock: Moved " + srcfile + " to " + dstfile);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("addBlock: Moved " + srcmeta + " to " + dstmeta
+ + " and " + srcfile + " to " + dstfile);
}
return dstfile;
}
static private void truncateBlock(File blockFile, File metaFile,
long oldlen, long newlen) throws IOException {
- DataNode.LOG.info("truncateBlock: blockFile=" + blockFile
+ LOG.info("truncateBlock: blockFile=" + blockFile
+ ", metaFile=" + metaFile
+ ", oldlen=" + oldlen
+ ", newlen=" + newlen);
@@ -1411,7 +463,7 @@ public class FSDataset implements FsDatasetSpi {
@Override // FsDatasetSpi
- public synchronized ReplicaInPipelineInterface append(ExtendedBlock b,
+ public synchronized ReplicaInPipeline append(ExtendedBlock b,
long newGS, long expectedBlockLen) throws IOException {
// If the block was successfully finalized because all packets
// were successfully processed at the Datanode but the ack for
@@ -1425,7 +477,7 @@ public class FSDataset implements FsDatasetSpi {
" should be greater than the replica " + b + "'s generation stamp");
}
ReplicaInfo replicaInfo = getReplicaInfo(b);
- DataNode.LOG.info("Appending to replica " + replicaInfo);
+ LOG.info("Appending to replica " + replicaInfo);
if (replicaInfo.getState() != ReplicaState.FINALIZED) {
throw new ReplicaNotFoundException(
ReplicaNotFoundException.UNFINALIZED_REPLICA + b);
@@ -1460,7 +512,7 @@ public class FSDataset implements FsDatasetSpi {
// construct a RBW replica with the new GS
File blkfile = replicaInfo.getBlockFile();
- FSVolume v = (FSVolume)replicaInfo.getVolume();
+ FsVolumeImpl v = (FsVolumeImpl)replicaInfo.getVolume();
if (v.getAvailable() < estimateBlockLen - replicaInfo.getNumBytes()) {
throw new DiskOutOfSpaceException("Insufficient space for appending to "
+ replicaInfo);
@@ -1473,8 +525,8 @@ public class FSDataset implements FsDatasetSpi {
File newmeta = newReplicaInfo.getMetaFile();
// rename meta file to rbw directory
- if (DataNode.LOG.isDebugEnabled()) {
- DataNode.LOG.debug("Renaming " + oldmeta + " to " + newmeta);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Renaming " + oldmeta + " to " + newmeta);
}
if (!oldmeta.renameTo(newmeta)) {
throw new IOException("Block " + replicaInfo + " reopen failed. " +
@@ -1483,13 +535,13 @@ public class FSDataset implements FsDatasetSpi {
}
// rename block file to rbw directory
- if (DataNode.LOG.isDebugEnabled()) {
- DataNode.LOG.debug("Renaming " + blkfile + " to " + newBlkFile);
- DataNode.LOG.debug("Old block file length is " + blkfile.length());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Renaming " + blkfile + " to " + newBlkFile
+ + ", file length=" + blkfile.length());
}
if (!blkfile.renameTo(newBlkFile)) {
if (!newmeta.renameTo(oldmeta)) { // restore the meta file
- DataNode.LOG.warn("Cannot move meta file " + newmeta +
+ LOG.warn("Cannot move meta file " + newmeta +
"back to the finalized directory " + oldmeta);
}
throw new IOException("Block " + replicaInfo + " reopen failed. " +
@@ -1552,9 +604,9 @@ public class FSDataset implements FsDatasetSpi {
}
@Override // FsDatasetSpi
- public synchronized ReplicaInPipelineInterface recoverAppend(ExtendedBlock b,
+ public synchronized ReplicaInPipeline recoverAppend(ExtendedBlock b,
long newGS, long expectedBlockLen) throws IOException {
- DataNode.LOG.info("Recover failed append to " + b);
+ LOG.info("Recover failed append to " + b);
ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen);
@@ -1571,10 +623,9 @@ public class FSDataset implements FsDatasetSpi {
@Override // FsDatasetSpi
public void recoverClose(ExtendedBlock b, long newGS,
long expectedBlockLen) throws IOException {
- DataNode.LOG.info("Recover failed close " + b);
+ LOG.info("Recover failed close " + b);
// check replica's state
- ReplicaInfo replicaInfo = recoverCheck(b, newGS,
- expectedBlockLen);
+ ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen);
// bump the replica's GS
bumpReplicaGS(replicaInfo, newGS);
// finalize the replica if RBW
@@ -1599,8 +650,8 @@ public class FSDataset implements FsDatasetSpi {
File newmeta = replicaInfo.getMetaFile();
// rename meta file to new GS
- if (DataNode.LOG.isDebugEnabled()) {
- DataNode.LOG.debug("Renaming " + oldmeta + " to " + newmeta);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Renaming " + oldmeta + " to " + newmeta);
}
if (!oldmeta.renameTo(newmeta)) {
replicaInfo.setGenerationStamp(oldGS); // restore old GS
@@ -1611,7 +662,7 @@ public class FSDataset implements FsDatasetSpi {
}
@Override // FsDatasetSpi
- public synchronized ReplicaInPipelineInterface createRbw(ExtendedBlock b)
+ public synchronized ReplicaInPipeline createRbw(ExtendedBlock b)
throws IOException {
ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(),
b.getBlockId());
@@ -1621,7 +672,7 @@ public class FSDataset implements FsDatasetSpi {
" and thus cannot be created.");
}
// create a new block
- FSVolume v = volumes.getNextVolume(b.getNumBytes());
+ FsVolumeImpl v = volumes.getNextVolume(b.getNumBytes());
// create a rbw file to hold block in the designated volume
File f = v.createRbwFile(b.getBlockPoolId(), b.getLocalBlock());
ReplicaBeingWritten newReplicaInfo = new ReplicaBeingWritten(b.getBlockId(),
@@ -1631,10 +682,10 @@ public class FSDataset implements FsDatasetSpi {
}
@Override // FsDatasetSpi
- public synchronized ReplicaInPipelineInterface recoverRbw(ExtendedBlock b,
+ public synchronized ReplicaInPipeline recoverRbw(ExtendedBlock b,
long newGS, long minBytesRcvd, long maxBytesRcvd)
throws IOException {
- DataNode.LOG.info("Recover the RBW replica " + b);
+ LOG.info("Recover the RBW replica " + b);
ReplicaInfo replicaInfo = getReplicaInfo(b.getBlockPoolId(), b.getBlockId());
@@ -1645,7 +696,7 @@ public class FSDataset implements FsDatasetSpi {
}
ReplicaBeingWritten rbw = (ReplicaBeingWritten)replicaInfo;
- DataNode.LOG.info("Recovering replica " + rbw);
+ LOG.info("Recovering replica " + rbw);
// Stop the previous writer
rbw.stopWriter();
@@ -1676,12 +727,12 @@ public class FSDataset implements FsDatasetSpi {
}
@Override // FsDatasetSpi
- public synchronized ReplicaInPipelineInterface convertTemporaryToRbw(
+ public synchronized ReplicaInPipeline convertTemporaryToRbw(
final ExtendedBlock b) throws IOException {
final long blockId = b.getBlockId();
final long expectedGs = b.getGenerationStamp();
final long visible = b.getNumBytes();
- DataNode.LOG.info("Convert replica " + b
+ LOG.info("Convert replica " + b
+ " from Temporary to RBW, visible length=" + visible);
final ReplicaInPipeline temp;
@@ -1717,7 +768,7 @@ public class FSDataset implements FsDatasetSpi {
+ visible + ", temp=" + temp);
}
// check volume
- final FSVolume v = (FSVolume)temp.getVolume();
+ final FsVolumeImpl v = (FsVolumeImpl)temp.getVolume();
if (v == null) {
throw new IOException("r.getVolume() = null, temp=" + temp);
}
@@ -1737,7 +788,7 @@ public class FSDataset implements FsDatasetSpi {
}
@Override // FsDatasetSpi
- public synchronized ReplicaInPipelineInterface createTemporary(ExtendedBlock b)
+ public synchronized ReplicaInPipeline createTemporary(ExtendedBlock b)
throws IOException {
ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(), b.getBlockId());
if (replicaInfo != null) {
@@ -1746,7 +797,7 @@ public class FSDataset implements FsDatasetSpi {
" and thus cannot be created.");
}
- FSVolume v = volumes.getNextVolume(b.getNumBytes());
+ FsVolumeImpl v = volumes.getNextVolume(b.getNumBytes());
// create a temporary file to hold block in the designated volume
File f = v.createTmpFile(b.getBlockPoolId(), b.getLocalBlock());
ReplicaInPipeline newReplicaInfo = new ReplicaInPipeline(b.getBlockId(),
@@ -1763,12 +814,12 @@ public class FSDataset implements FsDatasetSpi {
@Override // FsDatasetSpi
public void adjustCrcChannelPosition(ExtendedBlock b, ReplicaOutputStreams streams,
int checksumSize) throws IOException {
- FileOutputStream file = (FileOutputStream) streams.getChecksumOut();
+ FileOutputStream file = (FileOutputStream)streams.getChecksumOut();
FileChannel channel = file.getChannel();
long oldPos = channel.position();
long newPos = oldPos - checksumSize;
- if (DataNode.LOG.isDebugEnabled()) {
- DataNode.LOG.debug("Changing meta file offset of block " + b + " from " +
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Changing meta file offset of block " + b + " from " +
oldPos + " to " + newPos);
}
channel.position(newPos);
@@ -1805,7 +856,7 @@ public class FSDataset implements FsDatasetSpi {
newReplicaInfo = (FinalizedReplica)
((ReplicaUnderRecovery)replicaInfo).getOriginalReplica();
} else {
- FSVolume v = (FSVolume)replicaInfo.getVolume();
+ FsVolumeImpl v = (FsVolumeImpl)replicaInfo.getVolume();
File f = replicaInfo.getBlockFile();
if (v == null) {
throw new IOException("No volume for temporary file " + f +
@@ -1833,7 +884,7 @@ public class FSDataset implements FsDatasetSpi {
// delete the on-disk temp file
if (delBlockFromDisk(replicaInfo.getBlockFile(),
replicaInfo.getMetaFile(), b.getLocalBlock())) {
- DataNode.LOG.warn("Block " + b + " unfinalized and removed. " );
+ LOG.warn("Block " + b + " unfinalized and removed. " );
}
}
}
@@ -1847,17 +898,16 @@ public class FSDataset implements FsDatasetSpi {
*/
private boolean delBlockFromDisk(File blockFile, File metaFile, Block b) {
if (blockFile == null) {
- DataNode.LOG.warn("No file exists for block: " + b);
+ LOG.warn("No file exists for block: " + b);
return true;
}
if (!blockFile.delete()) {
- DataNode.LOG.warn("Not able to delete the block file: " + blockFile);
+ LOG.warn("Not able to delete the block file: " + blockFile);
return false;
} else { // remove the meta file
if (metaFile != null && !metaFile.delete()) {
- DataNode.LOG.warn(
- "Not able to delete the meta block file: " + metaFile);
+ LOG.warn("Not able to delete the meta block file: " + metaFile);
return false;
}
}
@@ -1958,8 +1008,8 @@ public class FSDataset implements FsDatasetSpi {
datanode.checkDiskError();
}
- if (DataNode.LOG.isDebugEnabled()) {
- DataNode.LOG.debug("b=" + b + ", f=" + f);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("b=" + b + ", f=" + f);
}
return null;
}
@@ -1977,7 +1027,7 @@ public class FSDataset implements FsDatasetSpi {
}
//check replica's meta file
- final File metafile = DatanodeUtil.getMetaFile(f, r.getGenerationStamp());
+ final File metafile = FsDatasetUtil.getMetaFile(f, r.getGenerationStamp());
if (!metafile.exists()) {
throw new IOException("Metafile " + metafile + " does not exist, r=" + r);
}
@@ -1995,69 +1045,64 @@ public class FSDataset implements FsDatasetSpi {
public void invalidate(String bpid, Block invalidBlks[]) throws IOException {
boolean error = false;
for (int i = 0; i < invalidBlks.length; i++) {
- File f = null;
- final FSVolume v;
+ final File f;
+ final FsVolumeImpl v;
synchronized (this) {
f = getFile(bpid, invalidBlks[i].getBlockId());
- ReplicaInfo dinfo = volumeMap.get(bpid, invalidBlks[i]);
- if (dinfo == null ||
- dinfo.getGenerationStamp() != invalidBlks[i].getGenerationStamp()) {
- DataNode.LOG.warn("Unexpected error trying to delete block "
- + invalidBlks[i] +
- ". BlockInfo not found in volumeMap.");
+ ReplicaInfo info = volumeMap.get(bpid, invalidBlks[i]);
+ if (info == null) {
+ LOG.warn("Failed to delete replica " + invalidBlks[i]
+ + ": ReplicaInfo not found.");
error = true;
continue;
}
- v = (FSVolume)dinfo.getVolume();
+ if (info.getGenerationStamp() != invalidBlks[i].getGenerationStamp()) {
+ LOG.warn("Failed to delete replica " + invalidBlks[i]
+ + ": GenerationStamp not matched, info=" + info);
+ error = true;
+ continue;
+ }
+ v = (FsVolumeImpl)info.getVolume();
if (f == null) {
- DataNode.LOG.warn("Unexpected error trying to delete block "
- + invalidBlks[i] +
- ". Block not found in blockMap." +
- ((v == null) ? " " : " Block found in volumeMap."));
+ LOG.warn("Failed to delete replica " + invalidBlks[i]
+ + ": File not found, volume=" + v);
error = true;
continue;
}
if (v == null) {
- DataNode.LOG.warn("Unexpected error trying to delete block "
- + invalidBlks[i] +
- ". No volume for this block." +
- " Block found in blockMap. " + f + ".");
+ LOG.warn("Failed to delete replica " + invalidBlks[i]
+ + ". No volume for this replica, file=" + f + ".");
error = true;
continue;
}
File parent = f.getParentFile();
if (parent == null) {
- DataNode.LOG.warn("Unexpected error trying to delete block "
- + invalidBlks[i] +
- ". Parent not found for file " + f + ".");
+ LOG.warn("Failed to delete replica " + invalidBlks[i]
+ + ". Parent not found for file " + f + ".");
error = true;
continue;
}
- ReplicaState replicaState = dinfo.getState();
+ ReplicaState replicaState = info.getState();
if (replicaState == ReplicaState.FINALIZED ||
(replicaState == ReplicaState.RUR &&
- ((ReplicaUnderRecovery)dinfo).getOriginalReplica().getState() ==
+ ((ReplicaUnderRecovery)info).getOriginalReplica().getState() ==
ReplicaState.FINALIZED)) {
v.clearPath(bpid, parent);
}
volumeMap.remove(bpid, invalidBlks[i]);
}
- File metaFile = DatanodeUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp());
// Delete the block asynchronously to make sure we can do it fast enough
- asyncDiskService.deleteAsync(v, f, metaFile,
+ asyncDiskService.deleteAsync(v, f,
+ FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()),
new ExtendedBlock(bpid, invalidBlks[i]));
}
if (error) {
throw new IOException("Error in deleting blocks.");
}
}
-
- public void notifyNamenodeDeletedBlock(ExtendedBlock block){
- datanode.notifyNamenodeDeletedBlock(block);
- }
- @Override // {@link FsDatasetSpi}
+ @Override // FsDatasetSpi
public synchronized boolean contains(final ExtendedBlock block) {
final long blockId = block.getLocalBlock().getBlockId();
return getFile(block.getBlockPoolId(), blockId) != null;
@@ -2085,7 +1130,7 @@ public class FSDataset implements FsDatasetSpi {
@Override // FsDatasetSpi
public void checkDataDir() throws DiskErrorException {
long totalBlocks=0, removedBlocks=0;
- List failedVols = volumes.checkDirs();
+ List failedVols = volumes.checkDirs();
// If there no failed volumes return
if (failedVols == null) {
@@ -2095,16 +1140,16 @@ public class FSDataset implements FsDatasetSpi {
// Otherwise remove blocks for the failed volumes
long mlsec = System.currentTimeMillis();
synchronized (this) {
- for (FSVolume fv: failedVols) {
- for (String bpid : fv.map.keySet()) {
+ for (FsVolumeImpl fv: failedVols) {
+ for (String bpid : fv.getBlockPoolList()) {
Iterator ib = volumeMap.replicas(bpid).iterator();
while(ib.hasNext()) {
ReplicaInfo b = ib.next();
totalBlocks++;
// check if the volume block belongs to still valid
if(b.getVolume() == fv) {
- DataNode.LOG.warn("Removing replica " + bpid + ":" + b.getBlockId()
- + " on failed volume " + fv.currentDir.getAbsolutePath());
+ LOG.warn("Removing replica " + bpid + ":" + b.getBlockId()
+ + " on failed volume " + fv.getCurrentDir().getAbsolutePath());
ib.remove();
removedBlocks++;
}
@@ -2113,16 +1158,15 @@ public class FSDataset implements FsDatasetSpi {
}
} // end of sync
mlsec = System.currentTimeMillis() - mlsec;
- DataNode.LOG.warn("Removed " + removedBlocks + " out of " + totalBlocks +
+ LOG.warn("Removed " + removedBlocks + " out of " + totalBlocks +
"(took " + mlsec + " millisecs)");
// report the error
StringBuilder sb = new StringBuilder();
- for (FSVolume fv : failedVols) {
- sb.append(fv.currentDir.getAbsolutePath() + ";");
+ for (FsVolumeImpl fv : failedVols) {
+ sb.append(fv.getCurrentDir().getAbsolutePath() + ";");
}
-
- throw new DiskErrorException("DataNode failed volumes:" + sb);
+ throw new DiskErrorException("DataNode failed volumes:" + sb);
}
@@ -2152,9 +1196,9 @@ public class FSDataset implements FsDatasetSpi {
bean = new StandardMBean(this,FSDatasetMBean.class);
mbeanName = MBeans.register("DataNode", "FSDatasetState-" + storageName, bean);
} catch (NotCompliantMBeanException e) {
- DataNode.LOG.warn("Error registering FSDatasetState MBean", e);
+ LOG.warn("Error registering FSDatasetState MBean", e);
}
- DataNode.LOG.info("Registered FSDatasetState MBean");
+ LOG.info("Registered FSDatasetState MBean");
}
@Override // FsDatasetSpi
@@ -2221,7 +1265,7 @@ public class FSDataset implements FsDatasetSpi {
// If metadata file exists then delete it
if (diskMetaFile != null && diskMetaFile.exists()
&& diskMetaFile.delete()) {
- DataNode.LOG.warn("Deleted a metadata file without a block "
+ LOG.warn("Deleted a metadata file without a block "
+ diskMetaFile.getAbsolutePath());
}
return;
@@ -2230,15 +1274,16 @@ public class FSDataset implements FsDatasetSpi {
// Block is in memory and not on the disk
// Remove the block from volumeMap
volumeMap.remove(bpid, blockId);
- if (datanode.blockScanner != null) {
- datanode.blockScanner.deleteBlock(bpid, new Block(blockId));
+ final DataBlockScanner blockScanner = datanode.getBlockScanner();
+ if (blockScanner != null) {
+ blockScanner.deleteBlock(bpid, new Block(blockId));
}
- DataNode.LOG.warn("Removed block " + blockId
+ LOG.warn("Removed block " + blockId
+ " from memory with missing block file on the disk");
// Finally remove the metadata file
if (diskMetaFile != null && diskMetaFile.exists()
&& diskMetaFile.delete()) {
- DataNode.LOG.warn("Deleted a metadata file for the deleted block "
+ LOG.warn("Deleted a metadata file for the deleted block "
+ diskMetaFile.getAbsolutePath());
}
}
@@ -2252,10 +1297,11 @@ public class FSDataset implements FsDatasetSpi {
ReplicaInfo diskBlockInfo = new FinalizedReplica(blockId,
diskFile.length(), diskGS, vol, diskFile.getParentFile());
volumeMap.add(bpid, diskBlockInfo);
- if (datanode.blockScanner != null) {
- datanode.blockScanner.addBlock(new ExtendedBlock(bpid, diskBlockInfo));
+ final DataBlockScanner blockScanner = datanode.getBlockScanner();
+ if (blockScanner != null) {
+ blockScanner.addBlock(new ExtendedBlock(bpid, diskBlockInfo));
}
- DataNode.LOG.warn("Added missing block to memory " + diskBlockInfo);
+ LOG.warn("Added missing block to memory " + diskBlockInfo);
return;
}
/*
@@ -2265,7 +1311,7 @@ public class FSDataset implements FsDatasetSpi {
File memFile = memBlockInfo.getBlockFile();
if (memFile.exists()) {
if (memFile.compareTo(diskFile) != 0) {
- DataNode.LOG.warn("Block file " + memFile.getAbsolutePath()
+ LOG.warn("Block file " + memFile.getAbsolutePath()
+ " does not match file found by scan "
+ diskFile.getAbsolutePath());
// TODO: Should the diskFile be deleted?
@@ -2275,25 +1321,25 @@ public class FSDataset implements FsDatasetSpi {
// Update the block with the file found on the disk. Since the block
// file and metadata file are found as a pair on the disk, update
// the block based on the metadata file found on the disk
- DataNode.LOG.warn("Block file in volumeMap "
+ LOG.warn("Block file in volumeMap "
+ memFile.getAbsolutePath()
+ " does not exist. Updating it to the file found during scan "
+ diskFile.getAbsolutePath());
memBlockInfo.setDir(diskFile.getParentFile());
memFile = diskFile;
- DataNode.LOG.warn("Updating generation stamp for block " + blockId
+ LOG.warn("Updating generation stamp for block " + blockId
+ " from " + memBlockInfo.getGenerationStamp() + " to " + diskGS);
memBlockInfo.setGenerationStamp(diskGS);
}
// Compare generation stamp
if (memBlockInfo.getGenerationStamp() != diskGS) {
- File memMetaFile = DatanodeUtil.getMetaFile(diskFile,
+ File memMetaFile = FsDatasetUtil.getMetaFile(diskFile,
memBlockInfo.getGenerationStamp());
if (memMetaFile.exists()) {
if (memMetaFile.compareTo(diskMetaFile) != 0) {
- DataNode.LOG.warn("Metadata file in memory "
+ LOG.warn("Metadata file in memory "
+ memMetaFile.getAbsolutePath()
+ " does not match file found by scan "
+ (diskMetaFile == null? null: diskMetaFile.getAbsolutePath()));
@@ -2306,7 +1352,7 @@ public class FSDataset implements FsDatasetSpi {
&& diskMetaFile.getParent().equals(memFile.getParent()) ? diskGS
: GenerationStamp.GRANDFATHER_GENERATION_STAMP;
- DataNode.LOG.warn("Updating generation stamp for block " + blockId
+ LOG.warn("Updating generation stamp for block " + blockId
+ " from " + memBlockInfo.getGenerationStamp() + " to " + gs);
memBlockInfo.setGenerationStamp(gs);
@@ -2317,7 +1363,7 @@ public class FSDataset implements FsDatasetSpi {
if (memBlockInfo.getNumBytes() != memFile.length()) {
// Update the length based on the block file
corruptBlock = new Block(memBlockInfo);
- DataNode.LOG.warn("Updating size of block " + blockId + " from "
+ LOG.warn("Updating size of block " + blockId + " from "
+ memBlockInfo.getNumBytes() + " to " + memFile.length());
memBlockInfo.setNumBytes(memFile.length());
}
@@ -2325,12 +1371,12 @@ public class FSDataset implements FsDatasetSpi {
// Send corrupt block report outside the lock
if (corruptBlock != null) {
- DataNode.LOG.warn("Reporting the block " + corruptBlock
+ LOG.warn("Reporting the block " + corruptBlock
+ " as corrupt due to length mismatch");
try {
datanode.reportBadBlocks(new ExtendedBlock(bpid, corruptBlock));
} catch (IOException e) {
- DataNode.LOG.warn("Failed to repot bad block " + corruptBlock, e);
+ LOG.warn("Failed to repot bad block " + corruptBlock, e);
}
}
}
@@ -2359,9 +1405,9 @@ public class FSDataset implements FsDatasetSpi {
/** static version of {@link #initReplicaRecovery(Block, long)}. */
static ReplicaRecoveryInfo initReplicaRecovery(String bpid,
- ReplicasMap map, Block block, long recoveryId) throws IOException {
+ ReplicaMap map, Block block, long recoveryId) throws IOException {
final ReplicaInfo replica = map.get(bpid, block.getBlockId());
- DataNode.LOG.info("initReplicaRecovery: block=" + block
+ LOG.info("initReplicaRecovery: block=" + block
+ ", recoveryId=" + recoveryId
+ ", replica=" + replica);
@@ -2410,13 +1456,13 @@ public class FSDataset implements FsDatasetSpi {
}
final long oldRecoveryID = rur.getRecoveryID();
rur.setRecoveryID(recoveryId);
- DataNode.LOG.info("initReplicaRecovery: update recovery id for " + block
+ LOG.info("initReplicaRecovery: update recovery id for " + block
+ " from " + oldRecoveryID + " to " + recoveryId);
}
else {
rur = new ReplicaUnderRecovery(replica, recoveryId);
map.add(bpid, rur);
- DataNode.LOG.info("initReplicaRecovery: changing replica state for "
+ LOG.info("initReplicaRecovery: changing replica state for "
+ block + " from " + replica.getState()
+ " to " + rur.getState());
}
@@ -2431,7 +1477,7 @@ public class FSDataset implements FsDatasetSpi {
//get replica
final String bpid = oldBlock.getBlockPoolId();
final ReplicaInfo replica = volumeMap.get(bpid, oldBlock.getBlockId());
- DataNode.LOG.info("updateReplica: block=" + oldBlock
+ LOG.info("updateReplica: block=" + oldBlock
+ ", recoveryId=" + recoveryId
+ ", length=" + newlength
+ ", replica=" + replica);
@@ -2518,16 +1564,18 @@ public class FSDataset implements FsDatasetSpi {
return replica.getVisibleLength();
}
+ @Override
public synchronized void addBlockPool(String bpid, Configuration conf)
throws IOException {
- DataNode.LOG.info("Adding block pool " + bpid);
+ LOG.info("Adding block pool " + bpid);
volumes.addBlockPool(bpid, conf);
volumeMap.initBlockPool(bpid);
volumes.getVolumeMap(bpid, volumeMap);
}
-
+
+ @Override
public synchronized void shutdownBlockPool(String bpid) {
- DataNode.LOG.info("Removing block pool " + bpid);
+ LOG.info("Removing block pool " + bpid);
volumeMap.cleanUpBlockPool(bpid);
volumes.removeBlockPool(bpid);
}
@@ -2546,30 +1594,29 @@ public class FSDataset implements FsDatasetSpi {
final long freeSpace;
final long reservedSpace;
- VolumeInfo(String dir, long usedSpace, long freeSpace, long reservedSpace) {
- this.directory = dir;
+ VolumeInfo(FsVolumeImpl v, long usedSpace, long freeSpace) {
+ this.directory = v.toString();
this.usedSpace = usedSpace;
this.freeSpace = freeSpace;
- this.reservedSpace = reservedSpace;
+ this.reservedSpace = v.getReserved();
}
}
private Collection getVolumeInfo() {
Collection info = new ArrayList();
- for (FSVolume volume : volumes.volumes) {
+ for (FsVolumeImpl volume : volumes.volumes) {
long used = 0;
long free = 0;
try {
used = volume.getDfsUsed();
free = volume.getAvailable();
} catch (IOException e) {
- DataNode.LOG.warn(e.getMessage());
+ LOG.warn(e.getMessage());
used = 0;
free = 0;
}
- info.add(new VolumeInfo(volume.toString(), used, free,
- volume.getReserved()));
+ info.add(new VolumeInfo(volume, used, free));
}
return info;
}
@@ -2592,16 +1639,15 @@ public class FSDataset implements FsDatasetSpi {
public synchronized void deleteBlockPool(String bpid, boolean force)
throws IOException {
if (!force) {
- for (FSVolume volume : volumes.volumes) {
+ for (FsVolumeImpl volume : volumes.volumes) {
if (!volume.isBPDirEmpty(bpid)) {
- DataNode.LOG.warn(bpid
- + " has some block files, cannot delete unless forced");
+ LOG.warn(bpid + " has some block files, cannot delete unless forced");
throw new IOException("Cannot delete block pool, "
+ "it contains some block files");
}
}
}
- for (FSVolume volume : volumes.volumes) {
+ for (FsVolumeImpl volume : volumes.volumes) {
volume.deleteBPDirectories(bpid, force);
}
}
@@ -2610,7 +1656,7 @@ public class FSDataset implements FsDatasetSpi {
public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block)
throws IOException {
File datafile = getBlockFile(block);
- File metafile = DatanodeUtil.getMetaFile(datafile, block.getGenerationStamp());
+ File metafile = FsDatasetUtil.getMetaFile(datafile, block.getGenerationStamp());
BlockLocalPathInfo info = new BlockLocalPathInfo(block,
datafile.getAbsolutePath(), metafile.getAbsolutePath());
return info;
@@ -2620,8 +1666,8 @@ public class FSDataset implements FsDatasetSpi {
public RollingLogs createRollingLogs(String bpid, String prefix
) throws IOException {
String dir = null;
- final List volumes = getVolumes();
- for (FSVolume vol : volumes) {
+ final List volumes = getVolumes();
+ for (FsVolumeImpl vol : volumes) {
String bpDir = vol.getPath(bpid);
if (RollingLogsImpl.isFilePresent(bpDir, prefix)) {
dir = bpDir;
@@ -2633,202 +1679,4 @@ public class FSDataset implements FsDatasetSpi {
}
return new RollingLogsImpl(dir, prefix);
}
-
- static class RollingLogsImpl implements RollingLogs {
- private static final String CURR_SUFFIX = ".curr";
- private static final String PREV_SUFFIX = ".prev";
-
- static boolean isFilePresent(String dir, String filePrefix) {
- return new File(dir, filePrefix + CURR_SUFFIX).exists() ||
- new File(dir, filePrefix + PREV_SUFFIX).exists();
- }
-
- private final File curr;
- private final File prev;
- private PrintStream out; //require synchronized access
-
- private Appender appender = new Appender() {
- @Override
- public Appendable append(CharSequence csq) {
- synchronized(RollingLogsImpl.this) {
- if (out == null) {
- throw new IllegalStateException(RollingLogsImpl.this
- + " is not yet opened.");
- }
- out.print(csq);
- }
- return this;
- }
-
- @Override
- public Appendable append(char c) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Appendable append(CharSequence csq, int start, int end) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void close() {
- synchronized(RollingLogsImpl.this) {
- if (out != null) {
- out.close();
- out = null;
- }
- }
- }
- };
-
-
- private final AtomicInteger numReaders = new AtomicInteger();
-
- private RollingLogsImpl(String dir, String filePrefix) throws FileNotFoundException{
- curr = new File(dir, filePrefix + CURR_SUFFIX);
- prev = new File(dir, filePrefix + PREV_SUFFIX);
- out = new PrintStream(new FileOutputStream(curr, true));
- }
-
- @Override
- public Reader iterator(boolean skipPrevFile) throws IOException {
- numReaders.incrementAndGet();
- return new Reader(skipPrevFile);
- }
-
- @Override
- public Appender appender() {
- return appender;
- }
-
- @Override
- public boolean roll() throws IOException {
- if (numReaders.get() > 0) {
- return false;
- }
- if (!prev.delete() && prev.exists()) {
- throw new IOException("Failed to delete " + prev);
- }
-
- synchronized(this) {
- appender.close();
- final boolean renamed = curr.renameTo(prev);
- out = new PrintStream(new FileOutputStream(curr, true));
- if (!renamed) {
- throw new IOException("Failed to rename " + curr + " to " + prev);
- }
- }
- return true;
- }
-
- @Override
- public String toString() {
- return curr.toString();
- }
-
- /**
- * This is used to read the lines in order.
- * If the data is not read completely (i.e, untill hasNext() returns
- * false), it needs to be explicitly
- */
- private class Reader implements RollingLogs.LineIterator {
- private File file;
- private BufferedReader reader;
- private String line;
- private boolean closed = false;
-
- private Reader(boolean skipPrevFile) throws IOException {
- reader = null;
- file = skipPrevFile? curr : prev;
- readNext();
- }
-
- @Override
- public boolean isPrevious() {
- return file == prev;
- }
-
- private boolean openFile() throws IOException {
-
- for(int i=0; i<2; i++) {
- if (reader != null || i > 0) {
- // move to next file
- file = isPrevious()? curr : null;
- }
- if (file == null) {
- return false;
- }
- if (file.exists()) {
- break;
- }
- }
-
- if (reader != null ) {
- reader.close();
- reader = null;
- }
-
- reader = new BufferedReader(new FileReader(file));
- return true;
- }
-
- // read next line if possible.
- private void readNext() throws IOException {
- line = null;
- try {
- if (reader != null && (line = reader.readLine()) != null) {
- return;
- }
- if (line == null) {
- // move to the next file.
- if (openFile()) {
- readNext();
- }
- }
- } finally {
- if (!hasNext()) {
- close();
- }
- }
- }
-
- @Override
- public boolean hasNext() {
- return line != null;
- }
-
- @Override
- public String next() {
- String curLine = line;
- try {
- readNext();
- } catch (IOException e) {
- DataBlockScanner.LOG.warn("Failed to read next line.", e);
- }
- return curLine;
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void close() throws IOException {
- if (!closed) {
- try {
- if (reader != null) {
- reader.close();
- }
- } finally {
- file = null;
- reader = null;
- closed = true;
- final int n = numReaders.decrementAndGet();
- assert(n >= 0);
- }
- }
- }
- }
- }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
new file mode 100644
index 00000000000..9750a9c4324
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
@@ -0,0 +1,107 @@
+/**
+ * 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.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+
+/** Utility methods. */
+@InterfaceAudience.Private
+public class FsDatasetUtil {
+ static boolean isUnlinkTmpFile(File f) {
+ return f.getName().endsWith(DatanodeUtil.UNLINK_BLOCK_SUFFIX);
+ }
+
+ static File getOrigFile(File unlinkTmpFile) {
+ final String name = unlinkTmpFile.getName();
+ if (!name.endsWith(DatanodeUtil.UNLINK_BLOCK_SUFFIX)) {
+ throw new IllegalArgumentException("unlinkTmpFile=" + unlinkTmpFile
+ + " does not end with " + DatanodeUtil.UNLINK_BLOCK_SUFFIX);
+ }
+ final int n = name.length() - DatanodeUtil.UNLINK_BLOCK_SUFFIX.length();
+ return new File(unlinkTmpFile.getParentFile(), name.substring(0, n));
+ }
+
+ static File getMetaFile(File f, long gs) {
+ return new File(f.getParent(),
+ DatanodeUtil.getMetaName(f.getName(), gs));
+ }
+
+ /** Find the corresponding meta data file from a given block file */
+ static File findMetaFile(final File blockFile) throws IOException {
+ final String prefix = blockFile.getName() + "_";
+ final File parent = blockFile.getParentFile();
+ final File[] matches = parent.listFiles(new FilenameFilter() {
+ @Override
+ public boolean accept(File dir, String name) {
+ return dir.equals(parent) && name.startsWith(prefix)
+ && name.endsWith(Block.METADATA_EXTENSION);
+ }
+ });
+
+ if (matches == null || matches.length == 0) {
+ throw new IOException("Meta file not found, blockFile=" + blockFile);
+ }
+ if (matches.length > 1) {
+ throw new IOException("Found more than one meta files: "
+ + Arrays.asList(matches));
+ }
+ return matches[0];
+ }
+
+ /**
+ * Find the meta-file for the specified block file
+ * and then return the generation stamp from the name of the meta-file.
+ */
+ static long getGenerationStampFromFile(File[] listdir, File blockFile) {
+ String blockName = blockFile.getName();
+ for (int j = 0; j < listdir.length; j++) {
+ String path = listdir[j].getName();
+ if (!path.startsWith(blockName)) {
+ continue;
+ }
+ if (blockFile == listdir[j]) {
+ continue;
+ }
+ return Block.getGenerationStamp(listdir[j].getName());
+ }
+ FsDatasetImpl.LOG.warn("Block " + blockFile + " does not have a metafile!");
+ return GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+ }
+
+ /** Find the corresponding meta data file from a given block file */
+ static long parseGenerationStamp(File blockFile, File metaFile
+ ) throws IOException {
+ final String metaname = metaFile.getName();
+ final String gs = metaname.substring(blockFile.getName().length() + 1,
+ metaname.length() - Block.METADATA_EXTENSION.length());
+ try {
+ return Long.parseLong(gs);
+ } catch(NumberFormatException nfe) {
+ throw new IOException("Failed to parse generation stamp: blockFile="
+ + blockFile + ", metaFile=" + metaFile, nfe);
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
new file mode 100644
index 00000000000..f9e82960230
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -0,0 +1,288 @@
+/**
+ * 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.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DF;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+/**
+ * The underlying volume used to store replica.
+ *
+ * It uses the {@link FsDatasetImpl} object for synchronization.
+ */
+@InterfaceAudience.Private
+class FsVolumeImpl implements FsVolumeSpi {
+ private final FsDatasetImpl dataset;
+ private final String storageID;
+ private final Map bpSlices
+ = new HashMap();
+ private final File currentDir; // /current
+ private final DF usage;
+ private final long reserved;
+
+ FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir,
+ Configuration conf) throws IOException {
+ this.dataset = dataset;
+ this.storageID = storageID;
+ this.reserved = conf.getLong(
+ DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY,
+ DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT);
+ this.currentDir = currentDir;
+ File parent = currentDir.getParentFile();
+ this.usage = new DF(parent, conf);
+ }
+
+ File getCurrentDir() {
+ return currentDir;
+ }
+
+ File getRbwDir(String bpid) throws IOException {
+ return getBlockPoolSlice(bpid).getRbwDir();
+ }
+
+ void decDfsUsed(String bpid, long value) {
+ synchronized(dataset) {
+ BlockPoolSlice bp = bpSlices.get(bpid);
+ if (bp != null) {
+ bp.decDfsUsed(value);
+ }
+ }
+ }
+
+ long getDfsUsed() throws IOException {
+ long dfsUsed = 0;
+ synchronized(dataset) {
+ for(BlockPoolSlice s : bpSlices.values()) {
+ dfsUsed += s.getDfsUsed();
+ }
+ }
+ return dfsUsed;
+ }
+
+ long getBlockPoolUsed(String bpid) throws IOException {
+ return getBlockPoolSlice(bpid).getDfsUsed();
+ }
+
+ /**
+ * Calculate the capacity of the filesystem, after removing any
+ * reserved capacity.
+ * @return the unreserved number of bytes left in this filesystem. May be zero.
+ */
+ long getCapacity() {
+ long remaining = usage.getCapacity() - reserved;
+ return remaining > 0 ? remaining : 0;
+ }
+
+ @Override
+ public long getAvailable() throws IOException {
+ long remaining = getCapacity()-getDfsUsed();
+ long available = usage.getAvailable();
+ if (remaining > available) {
+ remaining = available;
+ }
+ return (remaining > 0) ? remaining : 0;
+ }
+
+ long getReserved(){
+ return reserved;
+ }
+
+ BlockPoolSlice getBlockPoolSlice(String bpid) throws IOException {
+ BlockPoolSlice bp = bpSlices.get(bpid);
+ if (bp == null) {
+ throw new IOException("block pool " + bpid + " is not found");
+ }
+ return bp;
+ }
+
+ @Override
+ public String getPath(String bpid) throws IOException {
+ return getBlockPoolSlice(bpid).getDirectory().getAbsolutePath();
+ }
+
+ @Override
+ public File getFinalizedDir(String bpid) throws IOException {
+ return getBlockPoolSlice(bpid).getFinalizedDir();
+ }
+
+ /**
+ * Make a deep copy of the list of currently active BPIDs
+ */
+ @Override
+ public String[] getBlockPoolList() {
+ return bpSlices.keySet().toArray(new String[bpSlices.keySet().size()]);
+ }
+
+ /**
+ * Temporary files. They get moved to the finalized block directory when
+ * the block is finalized.
+ */
+ File createTmpFile(String bpid, Block b) throws IOException {
+ return getBlockPoolSlice(bpid).createTmpFile(b);
+ }
+
+ /**
+ * RBW files. They get moved to the finalized block directory when
+ * the block is finalized.
+ */
+ File createRbwFile(String bpid, Block b) throws IOException {
+ return getBlockPoolSlice(bpid).createRbwFile(b);
+ }
+
+ File addBlock(String bpid, Block b, File f) throws IOException {
+ return getBlockPoolSlice(bpid).addBlock(b, f);
+ }
+
+ void checkDirs() throws DiskErrorException {
+ // TODO:FEDERATION valid synchronization
+ for(BlockPoolSlice s : bpSlices.values()) {
+ s.checkDirs();
+ }
+ }
+
+ void getVolumeMap(ReplicaMap volumeMap) throws IOException {
+ for(BlockPoolSlice s : bpSlices.values()) {
+ s.getVolumeMap(volumeMap);
+ }
+ }
+
+ void getVolumeMap(String bpid, ReplicaMap volumeMap) throws IOException {
+ getBlockPoolSlice(bpid).getVolumeMap(volumeMap);
+ }
+
+ /**
+ * Add replicas under the given directory to the volume map
+ * @param volumeMap the replicas map
+ * @param dir an input directory
+ * @param isFinalized true if the directory has finalized replicas;
+ * false if the directory has rbw replicas
+ * @throws IOException
+ */
+ void addToReplicasMap(String bpid, ReplicaMap volumeMap,
+ File dir, boolean isFinalized) throws IOException {
+ BlockPoolSlice bp = getBlockPoolSlice(bpid);
+ // TODO move this up
+ // dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
+ bp.addToReplicasMap(volumeMap, dir, isFinalized);
+ }
+
+ void clearPath(String bpid, File f) throws IOException {
+ getBlockPoolSlice(bpid).clearPath(f);
+ }
+
+ @Override
+ public String toString() {
+ return currentDir.getAbsolutePath();
+ }
+
+ void shutdown() {
+ Set> set = bpSlices.entrySet();
+ for (Entry entry : set) {
+ entry.getValue().shutdown();
+ }
+ }
+
+ void addBlockPool(String bpid, Configuration conf) throws IOException {
+ File bpdir = new File(currentDir, bpid);
+ BlockPoolSlice bp = new BlockPoolSlice(bpid, this, bpdir, conf);
+ bpSlices.put(bpid, bp);
+ }
+
+ void shutdownBlockPool(String bpid) {
+ BlockPoolSlice bp = bpSlices.get(bpid);
+ if (bp != null) {
+ bp.shutdown();
+ }
+ bpSlices.remove(bpid);
+ }
+
+ boolean isBPDirEmpty(String bpid) throws IOException {
+ File volumeCurrentDir = this.getCurrentDir();
+ File bpDir = new File(volumeCurrentDir, bpid);
+ File bpCurrentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
+ File finalizedDir = new File(bpCurrentDir,
+ DataStorage.STORAGE_DIR_FINALIZED);
+ File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
+ if (finalizedDir.exists() && FileUtil.list(finalizedDir).length != 0) {
+ return false;
+ }
+ if (rbwDir.exists() && FileUtil.list(rbwDir).length != 0) {
+ return false;
+ }
+ return true;
+ }
+
+ void deleteBPDirectories(String bpid, boolean force) throws IOException {
+ File volumeCurrentDir = this.getCurrentDir();
+ File bpDir = new File(volumeCurrentDir, bpid);
+ if (!bpDir.isDirectory()) {
+ // nothing to be deleted
+ return;
+ }
+ File tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP);
+ File bpCurrentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT);
+ File finalizedDir = new File(bpCurrentDir,
+ DataStorage.STORAGE_DIR_FINALIZED);
+ File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
+ if (force) {
+ FileUtil.fullyDelete(bpDir);
+ } else {
+ if (!rbwDir.delete()) {
+ throw new IOException("Failed to delete " + rbwDir);
+ }
+ if (!finalizedDir.delete()) {
+ throw new IOException("Failed to delete " + finalizedDir);
+ }
+ FileUtil.fullyDelete(tmpDir);
+ for (File f : FileUtil.listFiles(bpCurrentDir)) {
+ if (!f.delete()) {
+ throw new IOException("Failed to delete " + f);
+ }
+ }
+ if (!bpCurrentDir.delete()) {
+ throw new IOException("Failed to delete " + bpCurrentDir);
+ }
+ for (File f : FileUtil.listFiles(bpDir)) {
+ if (!f.delete()) {
+ throw new IOException("Failed to delete " + f);
+ }
+ }
+ if (!bpDir.delete()) {
+ throw new IOException("Failed to delete " + bpDir);
+ }
+ }
+ }
+
+ String getStorageID() {
+ return storageID;
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
new file mode 100644
index 00000000000..107ce5d2f8c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
@@ -0,0 +1,172 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+class FsVolumeList {
+ /**
+ * Read access to this unmodifiable list is not synchronized.
+ * This list is replaced on modification holding "this" lock.
+ */
+ volatile List volumes = null;
+
+ private final VolumeChoosingPolicy blockChooser;
+ private volatile int numFailedVolumes;
+
+ FsVolumeList(List volumes, int failedVols,
+ VolumeChoosingPolicy blockChooser) {
+ this.volumes = Collections.unmodifiableList(volumes);
+ this.blockChooser = blockChooser;
+ this.numFailedVolumes = failedVols;
+ }
+
+ int numberOfFailedVolumes() {
+ return numFailedVolumes;
+ }
+
+ /**
+ * Get next volume. Synchronized to ensure {@link #curVolume} is updated
+ * by a single thread and next volume is chosen with no concurrent
+ * update to {@link #volumes}.
+ * @param blockSize free space needed on the volume
+ * @return next volume to store the block in.
+ */
+ synchronized FsVolumeImpl getNextVolume(long blockSize) throws IOException {
+ return blockChooser.chooseVolume(volumes, blockSize);
+ }
+
+ long getDfsUsed() throws IOException {
+ long dfsUsed = 0L;
+ for (FsVolumeImpl v : volumes) {
+ dfsUsed += v.getDfsUsed();
+ }
+ return dfsUsed;
+ }
+
+ long getBlockPoolUsed(String bpid) throws IOException {
+ long dfsUsed = 0L;
+ for (FsVolumeImpl v : volumes) {
+ dfsUsed += v.getBlockPoolUsed(bpid);
+ }
+ return dfsUsed;
+ }
+
+ long getCapacity() {
+ long capacity = 0L;
+ for (FsVolumeImpl v : volumes) {
+ capacity += v.getCapacity();
+ }
+ return capacity;
+ }
+
+ long getRemaining() throws IOException {
+ long remaining = 0L;
+ for (FsVolumeSpi vol : volumes) {
+ remaining += vol.getAvailable();
+ }
+ return remaining;
+ }
+
+ void getVolumeMap(ReplicaMap volumeMap) throws IOException {
+ for (FsVolumeImpl v : volumes) {
+ v.getVolumeMap(volumeMap);
+ }
+ }
+
+ void getVolumeMap(String bpid, ReplicaMap volumeMap) throws IOException {
+ for (FsVolumeImpl v : volumes) {
+ v.getVolumeMap(bpid, volumeMap);
+ }
+ }
+
+ /**
+ * Calls {@link FsVolumeImpl#checkDirs()} on each volume, removing any
+ * volumes from the active list that result in a DiskErrorException.
+ *
+ * This method is synchronized to allow only one instance of checkDirs()
+ * call
+ * @return list of all the removed volumes.
+ */
+ synchronized List checkDirs() {
+ ArrayList removedVols = null;
+
+ // Make a copy of volumes for performing modification
+ final List volumeList = new ArrayList(volumes);
+
+ for(Iterator i = volumeList.iterator(); i.hasNext(); ) {
+ final FsVolumeImpl fsv = i.next();
+ try {
+ fsv.checkDirs();
+ } catch (DiskErrorException e) {
+ FsDatasetImpl.LOG.warn("Removing failed volume " + fsv + ": ",e);
+ if (removedVols == null) {
+ removedVols = new ArrayList(1);
+ }
+ removedVols.add(fsv);
+ fsv.shutdown();
+ i.remove(); // Remove the volume
+ numFailedVolumes++;
+ }
+ }
+
+ if (removedVols != null && removedVols.size() > 0) {
+ // Replace volume list
+ volumes = Collections.unmodifiableList(volumeList);
+ FsDatasetImpl.LOG.info("Completed checkDirs. Removed " + removedVols.size()
+ + " volumes. Current volumes: " + this);
+ }
+
+ return removedVols;
+ }
+
+ @Override
+ public String toString() {
+ return volumes.toString();
+ }
+
+
+ void addBlockPool(String bpid, Configuration conf) throws IOException {
+ for (FsVolumeImpl v : volumes) {
+ v.addBlockPool(bpid, conf);
+ }
+ }
+
+ void removeBlockPool(String bpid) {
+ for (FsVolumeImpl v : volumes) {
+ v.shutdownBlockPool(bpid);
+ }
+ }
+
+ void shutdown() {
+ for (FsVolumeImpl volume : volumes) {
+ if(volume != null) {
+ volume.shutdown();
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LDir.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LDir.java
new file mode 100644
index 00000000000..991b58b3ae2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LDir.java
@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+/**
+ * A node type that can be built into a tree reflecting the
+ * hierarchy of replicas on the local disk.
+ */
+class LDir {
+ final File dir;
+ final int maxBlocksPerDir;
+
+ private int numBlocks = 0;
+ private LDir[] children = null;
+ private int lastChildIdx = 0;
+
+ LDir(File dir, int maxBlocksPerDir) throws IOException {
+ this.dir = dir;
+ this.maxBlocksPerDir = maxBlocksPerDir;
+
+ if (!dir.exists()) {
+ if (!dir.mkdirs()) {
+ throw new IOException("Failed to mkdirs " + dir);
+ }
+ } else {
+ File[] files = FileUtil.listFiles(dir);
+ List dirList = new ArrayList();
+ for (int idx = 0; idx < files.length; idx++) {
+ if (files[idx].isDirectory()) {
+ dirList.add(new LDir(files[idx], maxBlocksPerDir));
+ } else if (Block.isBlockFilename(files[idx])) {
+ numBlocks++;
+ }
+ }
+ if (dirList.size() > 0) {
+ children = dirList.toArray(new LDir[dirList.size()]);
+ }
+ }
+ }
+
+ File addBlock(Block b, File src) throws IOException {
+ //First try without creating subdirectories
+ File file = addBlock(b, src, false, false);
+ return (file != null) ? file : addBlock(b, src, true, true);
+ }
+
+ private File addBlock(Block b, File src, boolean createOk, boolean resetIdx
+ ) throws IOException {
+ if (numBlocks < maxBlocksPerDir) {
+ final File dest = FsDatasetImpl.moveBlockFiles(b, src, dir);
+ numBlocks += 1;
+ return dest;
+ }
+
+ if (lastChildIdx < 0 && resetIdx) {
+ //reset so that all children will be checked
+ lastChildIdx = DFSUtil.getRandom().nextInt(children.length);
+ }
+
+ if (lastChildIdx >= 0 && children != null) {
+ //Check if any child-tree has room for a block.
+ for (int i=0; i < children.length; i++) {
+ int idx = (lastChildIdx + i)%children.length;
+ File file = children[idx].addBlock(b, src, false, resetIdx);
+ if (file != null) {
+ lastChildIdx = idx;
+ return file;
+ }
+ }
+ lastChildIdx = -1;
+ }
+
+ if (!createOk) {
+ return null;
+ }
+
+ if (children == null || children.length == 0) {
+ children = new LDir[maxBlocksPerDir];
+ for (int idx = 0; idx < maxBlocksPerDir; idx++) {
+ final File sub = new File(dir, DataStorage.BLOCK_SUBDIR_PREFIX+idx);
+ children[idx] = new LDir(sub, maxBlocksPerDir);
+ }
+ }
+
+ //now pick a child randomly for creating a new set of subdirs.
+ lastChildIdx = DFSUtil.getRandom().nextInt(children.length);
+ return children[ lastChildIdx ].addBlock(b, src, true, false);
+ }
+
+ void getVolumeMap(String bpid, ReplicaMap volumeMap, FsVolumeImpl volume
+ ) throws IOException {
+ if (children != null) {
+ for (int i = 0; i < children.length; i++) {
+ children[i].getVolumeMap(bpid, volumeMap, volume);
+ }
+ }
+
+ recoverTempUnlinkedBlock();
+ volume.addToReplicasMap(bpid, volumeMap, dir, true);
+ }
+
+ /**
+ * Recover unlinked tmp files on datanode restart. If the original block
+ * does not exist, then the tmp file is renamed to be the
+ * original file name; otherwise the tmp file is deleted.
+ */
+ private void recoverTempUnlinkedBlock() throws IOException {
+ File files[] = FileUtil.listFiles(dir);
+ for (File file : files) {
+ if (!FsDatasetUtil.isUnlinkTmpFile(file)) {
+ continue;
+ }
+ File blockFile = FsDatasetUtil.getOrigFile(file);
+ if (blockFile.exists()) {
+ // If the original block file still exists, then no recovery is needed.
+ if (!file.delete()) {
+ throw new IOException("Unable to cleanup unlinked tmp file " + file);
+ }
+ } else {
+ if (!file.renameTo(blockFile)) {
+ throw new IOException("Unable to cleanup detached file " + file);
+ }
+ }
+ }
+ }
+
+ /**
+ * check if a data diretory is healthy
+ * @throws DiskErrorException
+ */
+ void checkDirTree() throws DiskErrorException {
+ DiskChecker.checkDir(dir);
+
+ if (children != null) {
+ for (int i = 0; i < children.length; i++) {
+ children[i].checkDirTree();
+ }
+ }
+ }
+
+ void clearPath(File f) {
+ String root = dir.getAbsolutePath();
+ String dir = f.getAbsolutePath();
+ if (dir.startsWith(root)) {
+ String[] dirNames = dir.substring(root.length()).
+ split(File.separator + DataStorage.BLOCK_SUBDIR_PREFIX);
+ if (clearPath(f, dirNames, 1))
+ return;
+ }
+ clearPath(f, null, -1);
+ }
+
+ /**
+ * dirNames is an array of string integers derived from
+ * usual directory structure data/subdirN/subdirXY/subdirM ...
+ * If dirName array is non-null, we only check the child at
+ * the children[dirNames[idx]]. This avoids iterating over
+ * children in common case. If directory structure changes
+ * in later versions, we need to revisit this.
+ */
+ private boolean clearPath(File f, String[] dirNames, int idx) {
+ if ((dirNames == null || idx == dirNames.length) &&
+ dir.compareTo(f) == 0) {
+ numBlocks--;
+ return true;
+ }
+
+ if (dirNames != null) {
+ //guess the child index from the directory name
+ if (idx > (dirNames.length - 1) || children == null) {
+ return false;
+ }
+ int childIdx;
+ try {
+ childIdx = Integer.parseInt(dirNames[idx]);
+ } catch (NumberFormatException ignored) {
+ // layout changed? we could print a warning.
+ return false;
+ }
+ return (childIdx >= 0 && childIdx < children.length) ?
+ children[childIdx].clearPath(f, dirNames, idx+1) : false;
+ }
+
+ //guesses failed. back to blind iteration.
+ if (children != null) {
+ for(int i=0; i < children.length; i++) {
+ if (children[i].clearPath(f, null, -1)){
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "FSDir{dir=" + dir + ", children="
+ + (children == null ? null : Arrays.asList(children)) + "}";
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java
similarity index 96%
rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicasMap.java
rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java
index a1b9279dd1b..aa81c727071 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import java.util.Collection;
import java.util.HashMap;
@@ -23,11 +23,12 @@ import java.util.Map;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
/**
- * Maintains the replicas map.
+ * Maintains the replica map.
*/
-class ReplicasMap {
+class ReplicaMap {
// Object using which this class is synchronized
private final Object mutex;
@@ -35,7 +36,7 @@ class ReplicasMap {
private Map> map =
new HashMap>();
- ReplicasMap(Object mutex) {
+ ReplicaMap(Object mutex) {
if (mutex == null) {
throw new HadoopIllegalArgumentException(
"Object to synchronize on cannot be null");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java
new file mode 100644
index 00000000000..48a9829d626
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java
@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
+
+class RollingLogsImpl implements RollingLogs {
+ private static final String CURR_SUFFIX = ".curr";
+ private static final String PREV_SUFFIX = ".prev";
+
+ static boolean isFilePresent(String dir, String filePrefix) {
+ return new File(dir, filePrefix + CURR_SUFFIX).exists() ||
+ new File(dir, filePrefix + PREV_SUFFIX).exists();
+ }
+
+ private final File curr;
+ private final File prev;
+ private PrintStream out; //require synchronized access
+
+ private Appender appender = new Appender() {
+ @Override
+ public Appendable append(CharSequence csq) {
+ synchronized(RollingLogsImpl.this) {
+ if (out == null) {
+ throw new IllegalStateException(RollingLogsImpl.this
+ + " is not yet opened.");
+ }
+ out.print(csq);
+ }
+ return this;
+ }
+
+ @Override
+ public Appendable append(char c) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Appendable append(CharSequence csq, int start, int end) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void close() {
+ synchronized(RollingLogsImpl.this) {
+ if (out != null) {
+ out.close();
+ out = null;
+ }
+ }
+ }
+ };
+
+
+ private final AtomicInteger numReaders = new AtomicInteger();
+
+ RollingLogsImpl(String dir, String filePrefix) throws FileNotFoundException{
+ curr = new File(dir, filePrefix + CURR_SUFFIX);
+ prev = new File(dir, filePrefix + PREV_SUFFIX);
+ out = new PrintStream(new FileOutputStream(curr, true));
+ }
+
+ @Override
+ public Reader iterator(boolean skipPrevFile) throws IOException {
+ numReaders.incrementAndGet();
+ return new Reader(skipPrevFile);
+ }
+
+ @Override
+ public Appender appender() {
+ return appender;
+ }
+
+ @Override
+ public boolean roll() throws IOException {
+ if (numReaders.get() > 0) {
+ return false;
+ }
+ if (!prev.delete() && prev.exists()) {
+ throw new IOException("Failed to delete " + prev);
+ }
+
+ synchronized(this) {
+ appender.close();
+ final boolean renamed = curr.renameTo(prev);
+ out = new PrintStream(new FileOutputStream(curr, true));
+ if (!renamed) {
+ throw new IOException("Failed to rename " + curr + " to " + prev);
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return curr.toString();
+ }
+
+ /**
+ * This is used to read the lines in order.
+ * If the data is not read completely (i.e, untill hasNext() returns
+ * false), it needs to be explicitly
+ */
+ private class Reader implements RollingLogs.LineIterator {
+ private File file;
+ private BufferedReader reader;
+ private String line;
+ private boolean closed = false;
+
+ private Reader(boolean skipPrevFile) throws IOException {
+ reader = null;
+ file = skipPrevFile? curr : prev;
+ readNext();
+ }
+
+ @Override
+ public boolean isPrevious() {
+ return file == prev;
+ }
+
+ private boolean openFile() throws IOException {
+
+ for(int i=0; i<2; i++) {
+ if (reader != null || i > 0) {
+ // move to next file
+ file = isPrevious()? curr : null;
+ }
+ if (file == null) {
+ return false;
+ }
+ if (file.exists()) {
+ break;
+ }
+ }
+
+ if (reader != null ) {
+ reader.close();
+ reader = null;
+ }
+
+ reader = new BufferedReader(new FileReader(file));
+ return true;
+ }
+
+ // read next line if possible.
+ private void readNext() throws IOException {
+ line = null;
+ try {
+ if (reader != null && (line = reader.readLine()) != null) {
+ return;
+ }
+ if (line == null) {
+ // move to the next file.
+ if (openFile()) {
+ readNext();
+ }
+ }
+ } finally {
+ if (!hasNext()) {
+ close();
+ }
+ }
+ }
+
+ @Override
+ public boolean hasNext() {
+ return line != null;
+ }
+
+ @Override
+ public String next() {
+ String curLine = line;
+ try {
+ readNext();
+ } catch (IOException e) {
+ DataBlockScanner.LOG.warn("Failed to read next line.", e);
+ }
+ return curLine;
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (!closed) {
+ try {
+ if (reader != null) {
+ reader.close();
+ }
+ } finally {
+ file = null;
+ reader = null;
+ closed = true;
+ final int n = numReaders.decrementAndGet();
+ assert(n >= 0);
+ }
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
index 9cad4eb0432..706696fd662 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
+import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingService;
/**
@@ -171,6 +172,12 @@ public class BackupNode extends NameNode {
@Override // NameNode
public void stop() {
+ stop(true);
+ }
+
+ @VisibleForTesting
+ void stop(boolean reportError) {
+
if(checkpointManager != null) {
// Prevent from starting a new checkpoint.
// Checkpoints that has already been started may proceed until
@@ -180,7 +187,10 @@ public class BackupNode extends NameNode {
// ClosedByInterruptException.
checkpointManager.shouldRun = false;
}
- if(namenode != null && getRegistration() != null) {
+
+ // reportError is a test hook to simulate backupnode crashing and not
+ // doing a clean exit w.r.t active namenode
+ if (reportError && namenode != null && getRegistration() != null) {
// Exclude this node from the list of backup streams on the name-node
try {
namenode.errorReport(getRegistration(), NamenodeProtocol.FATAL,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 7c630d70db6..9753b7f4906 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -1020,7 +1020,7 @@ public class FSEditLog {
LOG.info("Registering new backup node: " + bnReg);
BackupJournalManager bjm = new BackupJournalManager(bnReg, nnReg);
- journalSet.add(bjm, true);
+ journalSet.add(bjm, false);
}
synchronized void releaseBackupStream(NamenodeRegistration registration)
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 06e613aee0e..dcc78dd306c 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
@@ -300,8 +300,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
NameNodeResourceChecker nnResourceChecker;
private FsServerDefaults serverDefaults;
- // allow appending to hdfs files
- private boolean supportAppends = true;
+
+ private boolean supportAppends;
private ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure =
ReplaceDatanodeOnFailure.DEFAULT;
@@ -1813,9 +1813,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throws AccessControlException, SafeModeException,
FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, IOException {
- if (supportAppends == false) {
- throw new UnsupportedOperationException("Append to hdfs not supported." +
- " Please refer to dfs.support.append configuration parameter.");
+ if (!supportAppends) {
+ throw new UnsupportedOperationException(
+ "Append is not enabled on this NameNode. Use the " +
+ DFS_SUPPORT_APPEND_KEY + " configuration option to enable it.");
}
LocatedBlock lb = null;
writeLock();
@@ -2896,9 +2897,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
//remove lease, close file
finalizeINodeFileUnderConstruction(src, pendingFile);
- } else if (supportAppends) {
+ } else {
// If this commit does not want to close the file, persist blocks
- // only if append is supported or we're explicitly told to
dir.persistBlocks(src, pendingFile);
}
} finally {
@@ -4481,9 +4481,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
} finally {
writeUnlock();
}
- if (supportAppends || persistBlocks) {
- getEditLog().logSync();
- }
+ getEditLog().logSync();
LOG.info("updatePipeline(" + oldBlock + ") successfully to " + newBlock);
}
@@ -4522,11 +4520,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
blockinfo.setExpectedLocations(descriptors);
- // persist blocks only if append is supported
String src = leaseManager.findPath(pendingFile);
- if (supportAppends) {
- dir.persistBlocks(src, pendingFile);
- }
+ dir.persistBlocks(src, pendingFile);
}
// rename was successful. If any part of the renamed subtree had
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
index 22d7b297a69..3e652c13a7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
import org.apache.hadoop.hdfs.server.namenode.CancelDelegationTokenServlet;
import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet;
import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet;
+import org.apache.hadoop.hdfs.web.URLUtils;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.net.NetUtils;
@@ -224,8 +225,7 @@ public class DelegationTokenFetcher {
URL remoteURL = new URL(url.toString());
SecurityUtil.fetchServiceTicket(remoteURL);
- URLConnection connection = remoteURL.openConnection();
-
+ URLConnection connection = URLUtils.openConnection(remoteURL);
InputStream in = connection.getInputStream();
Credentials ts = new Credentials();
dis = new DataInputStream(in);
@@ -265,7 +265,7 @@ public class DelegationTokenFetcher {
try {
URL url = new URL(buf.toString());
SecurityUtil.fetchServiceTicket(url);
- connection = (HttpURLConnection) url.openConnection();
+ connection = (HttpURLConnection)URLUtils.openConnection(url);
if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
throw new IOException("Error renewing token: " +
connection.getResponseMessage());
@@ -359,7 +359,7 @@ public class DelegationTokenFetcher {
try {
URL url = new URL(buf.toString());
SecurityUtil.fetchServiceTicket(url);
- connection = (HttpURLConnection) url.openConnection();
+ connection = (HttpURLConnection)URLUtils.openConnection(url);
if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
throw new IOException("Error cancelling token: " +
connection.getResponseMessage());
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 d27b664f6d7..45c4445e84b 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
@@ -513,18 +513,13 @@ public class JsonUtil {
final byte[] bytes = StringUtils.hexStringToByte((String)m.get("bytes"));
final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
- final int bytesPerCRC = in.readInt();
- final long crcPerBlock = in.readLong();
- final MD5Hash md5 = MD5Hash.read(in);
- final MD5MD5CRC32FileChecksum checksum = new MD5MD5CRC32FileChecksum(
- bytesPerCRC, crcPerBlock, md5);
+ final MD5MD5CRC32FileChecksum checksum = new MD5MD5CRC32FileChecksum();
+ checksum.readFields(in);
//check algorithm name
- final String alg = "MD5-of-" + crcPerBlock + "MD5-of-" + bytesPerCRC + "CRC32";
- if (!alg.equals(algorithm)) {
- throw new IOException("Algorithm not matched: algorithm=" + algorithm
- + ", crcPerBlock=" + crcPerBlock
- + ", bytesPerCRC=" + bytesPerCRC);
+ if (!checksum.getAlgorithmName().equals(algorithm)) {
+ throw new IOException("Algorithm not matched. Expected " + algorithm
+ + ", Received " + checksum.getAlgorithmName());
}
//check length
if (length != checksum.getLength()) {
@@ -534,4 +529,4 @@ public class JsonUtil {
return checksum;
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLUtils.java
new file mode 100644
index 00000000000..7e4edd2c81a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLUtils.java
@@ -0,0 +1,51 @@
+/**
+ * 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.web;
+
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLConnection;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Utilities for handling URLs
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+@InterfaceStability.Unstable
+public class URLUtils {
+ /**
+ * Timeout for socket connects and reads
+ */
+ public static int SOCKET_TIMEOUT = 1*60*1000; // 1 minute
+
+ /**
+ * Opens a url with read and connect timeouts
+ * @param url to open
+ * @return URLConnection
+ * @throws IOException
+ */
+ public static URLConnection openConnection(URL url) throws IOException {
+ URLConnection connection = url.openConnection();
+ connection.setConnectTimeout(SOCKET_TIMEOUT);
+ connection.setReadTimeout(SOCKET_TIMEOUT);
+ return connection;
+ }
+}
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 b9391449f1e..b084d4b4c8b 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
@@ -836,4 +836,26 @@
+
+ dfs.support.append
+ true
+
+ Does HDFS allow appends to files?
+
+
+
+
+ dfs.client.local.interfaces
+
+ A comma separated list of network interface names to use
+ for data transfer between the client and datanodes. When creating
+ a connection to read from or write to a datanode, the client
+ chooses one of the specified interfaces at random and binds its
+ socket to the IP of that interface. Individual names may be
+ specified as either an interface name (eg "eth0"), a subinterface
+ name (eg "eth0:0"), or an IP address (which may be specified using
+ CIDR notation to match a range of IPs).
+
+
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java
index 61ab20061fc..ddeb57d3ff3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java
@@ -230,7 +230,6 @@ public class TestFiPipelines {
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, customBlockSize / 2);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 0);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
index 593134350d7..ca4862bd010 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
@@ -163,7 +163,6 @@ public class TestStickyBit extends TestCase {
try {
Configuration conf = new HdfsConfiguration();
conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
FileSystem hdfs = cluster.getFileSystem();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java
index d837c0f71af..d26e2cb4e2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java
@@ -55,7 +55,6 @@ public class FileAppendTest4 {
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, BYTES_PER_CHECKSUM);
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, PACKET_SIZE);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
}
@BeforeClass
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
index 259d26a3049..a0b7b0d151c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
@@ -40,7 +40,6 @@ public class TestClientProtocolForPipelineRecovery {
@Test public void testGetNewStamp() throws IOException {
int numDataNodes = 1;
Configuration conf = new HdfsConfiguration();
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
try {
cluster.waitActive();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
index 71ad9afa950..3ef892b4f19 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
@@ -207,7 +207,6 @@ public class TestDataTransferProtocol extends TestCase {
@Test public void testOpWrite() throws IOException {
int numDataNodes = 1;
Configuration conf = new HdfsConfiguration();
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
try {
cluster.waitActive();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java
index 3b116d4940c..0b59086dae8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java
@@ -85,7 +85,6 @@ public class TestFileAppend2 extends TestCase {
SimulatedFSDataset.setFactory(conf);
}
conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
FileSystem fs = cluster.getFileSystem();
@@ -338,7 +337,6 @@ public class TestFileAppend2 extends TestCase {
conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 30000);
conf.setInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY, 30000);
conf.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 50);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(numDatanodes)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
index aedbac30ac2..40c0a1a3ed4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
@@ -70,7 +70,6 @@ public class TestFileAppend3 extends junit.framework.TestCase {
AppendTestUtil.LOG.info("setUp()");
conf = new HdfsConfiguration();
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
buffersize = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
fs = (DistributedFileSystem)cluster.getFileSystem();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
index ab60c4c2210..f35bc465ed3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
@@ -79,7 +79,6 @@ public class TestFileAppend4 {
if (simulatedStorage) {
SimulatedFSDataset.setFactory(conf);
}
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
// lower heartbeat interval for fast recognition of DN death
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
index 88d5d4bc2f6..632d8cc35e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
@@ -311,7 +311,6 @@ public class TestFileConcurrentReader extends junit.framework.TestCase {
final int writeSize,
Configuration conf
) throws IOException {
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, syncType == SyncType.APPEND);
conf.setBoolean(DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_KEY,
transferToAllowed);
init(conf);
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 87848f33a1f..357b79e7bb5 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
@@ -38,6 +38,7 @@ import java.io.FileNotFoundException;
import java.io.FileReader;
import java.io.IOException;
import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
import java.util.EnumSet;
import org.apache.commons.logging.LogFactory;
@@ -67,6 +68,7 @@ import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
import org.apache.hadoop.io.IOUtils;
import org.apache.log4j.Level;
+import static org.junit.Assume.assumeTrue;
/**
* This class tests various cases during file creation.
@@ -140,11 +142,34 @@ public class TestFileCreation extends junit.framework.TestCase {
}
}
+ public void testFileCreation() throws IOException {
+ checkFileCreation(null);
+ }
+
+ /** Same test but the client should bind to a local interface */
+ public void testFileCreationSetLocalInterface() throws IOException {
+ assumeTrue(System.getProperty("os.name").startsWith("Linux"));
+
+ // The mini cluster listens on the loopback so we can use it here
+ checkFileCreation("lo");
+
+ try {
+ checkFileCreation("bogus-interface");
+ fail("Able to specify a bogus interface");
+ } catch (UnknownHostException e) {
+ assertEquals("No such interface bogus-interface", e.getMessage());
+ }
+ }
+
/**
* Test if file creation and disk space consumption works right
+ * @param netIf the local interface, if any, clients should use to access DNs
*/
- public void testFileCreation() throws IOException {
+ public void checkFileCreation(String netIf) throws IOException {
Configuration conf = new HdfsConfiguration();
+ if (netIf != null) {
+ conf.set(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES, netIf);
+ }
if (simulatedStorage) {
SimulatedFSDataset.setFactory(conf);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java
index 379e83d9191..6517391885d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java
@@ -43,7 +43,6 @@ public class TestFileCreationDelete extends junit.framework.TestCase {
conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
// create cluster
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java
new file mode 100644
index 00000000000..483d184d928
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java
@@ -0,0 +1,128 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.InetAddress;
+import java.net.ServerSocket;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.net.URLConnection;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.web.URLUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestHftpURLTimeouts {
+ @BeforeClass
+ public static void setup() {
+ URLUtils.SOCKET_TIMEOUT = 1;
+ }
+
+ @Test
+ public void testHftpSocketTimeout() throws Exception {
+ Configuration conf = new Configuration();
+ ServerSocket socket = new ServerSocket(0,1);
+ URI uri = new URI("hftp", null,
+ InetAddress.getByName(null).getHostAddress(),
+ socket.getLocalPort(),
+ null, null, null);
+ boolean timedout = false;
+
+ HftpFileSystem fs = (HftpFileSystem)FileSystem.get(uri, conf);
+ HttpURLConnection conn = fs.openConnection("/", "");
+ timedout = false;
+ try {
+ // this will consume the only slot in the backlog
+ conn.getInputStream();
+ } catch (SocketTimeoutException ste) {
+ timedout = true;
+ assertEquals("Read timed out", ste.getMessage());
+ } finally {
+ if (conn != null) conn.disconnect();
+ }
+ assertTrue("read timedout", timedout);
+ assertTrue("connect timedout", checkConnectTimeout(fs, false));
+ }
+
+ @Test
+ public void testHsftpSocketTimeout() throws Exception {
+ Configuration conf = new Configuration();
+ ServerSocket socket = new ServerSocket(0,1);
+ URI uri = new URI("hsftp", null,
+ InetAddress.getByName(null).getHostAddress(),
+ socket.getLocalPort(),
+ null, null, null);
+ boolean timedout = false;
+
+ HsftpFileSystem fs = (HsftpFileSystem)FileSystem.get(uri, conf);
+ HttpURLConnection conn = null;
+ timedout = false;
+ try {
+ // this will consume the only slot in the backlog
+ conn = fs.openConnection("/", "");
+ } catch (SocketTimeoutException ste) {
+ // SSL expects a negotiation, so it will timeout on read, unlike hftp
+ timedout = true;
+ assertEquals("Read timed out", ste.getMessage());
+ } finally {
+ if (conn != null) conn.disconnect();
+ }
+ assertTrue("ssl read connect timedout", timedout);
+ assertTrue("connect timedout", checkConnectTimeout(fs, true));
+ }
+
+ private boolean checkConnectTimeout(HftpFileSystem fs, boolean ignoreReadTimeout)
+ throws IOException {
+ boolean timedout = false;
+ List conns = new LinkedList();
+ try {
+ // with a listen backlog of 1, should only have to make one connection
+ // to trigger a connection timeout. however... linux doesn't honor the
+ // socket's listen backlog so we have to try a bunch of times
+ for (int n=32; !timedout && n > 0; n--) {
+ try {
+ conns.add(fs.openConnection("/", ""));
+ } catch (SocketTimeoutException ste) {
+ String message = ste.getMessage();
+ // https will get a read timeout due to SSL negotiation, but
+ // a normal http will not, so need to ignore SSL read timeouts
+ // until a connect timeout occurs
+ if (!(ignoreReadTimeout && message.equals("Read timed out"))) {
+ timedout = true;
+ assertEquals("connect timed out", message);
+ }
+ }
+ }
+ } finally {
+ for (HttpURLConnection conn : conns) {
+ conn.disconnect();
+ }
+ }
+ return timedout;
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
index c7fbee3b188..98b8bdf5924 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -70,7 +70,6 @@ public class TestLeaseRecovery extends junit.framework.TestCase {
final int ORG_FILE_SIZE = 3000;
Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
MiniDFSCluster cluster = null;
try {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPipelines.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPipelines.java
index 0d2ebc96af6..2c813a9582a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPipelines.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPipelines.java
@@ -155,7 +155,6 @@ public class TestPipelines {
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, customBlockSize / 2);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 0);
}
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 9be29bfa11f..b1d31da9304 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
@@ -84,7 +84,6 @@ public class TestQuota {
// Space quotas
final int DEFAULT_BLOCK_SIZE = 512;
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
final FileSystem fs = cluster.getFileSystem();
assertTrue("Not a HDFS: "+fs.getUri(),
@@ -533,7 +532,6 @@ public class TestQuota {
// set a smaller block size so that we can test with smaller
// diskspace quotas
conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "512");
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
final FileSystem fs = cluster.getFileSystem();
assertTrue("Not a HDFS: "+fs.getUri(),
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadWhileWriting.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadWhileWriting.java
index 2bf6cbba3e3..23e6a414f55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadWhileWriting.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadWhileWriting.java
@@ -56,8 +56,6 @@ public class TestReadWhileWriting {
@Test
public void pipeline_02_03() throws Exception {
final Configuration conf = new HdfsConfiguration();
- //enable append
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
// create cluster
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
index 5882f1a5a10..6189a96f81c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
@@ -54,7 +54,6 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
// create cluster
System.out.println("Test 1*****************************");
@@ -140,7 +139,6 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
System.out.println("Test 2************************************");
// create cluster
@@ -215,7 +213,6 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
System.out.println("Test 3************************************");
// create cluster
@@ -280,7 +277,6 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
System.out.println("Test 4************************************");
// create cluster
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
index c9436e4f332..687e813e25c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
@@ -182,8 +182,6 @@ public class TestBlockTokenWithDFS {
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, numDataNodes);
conf.setInt("ipc.client.connect.max.retries", 0);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
- DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
return conf;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
index f05bf653c46..74be37d986b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
@@ -22,12 +22,16 @@ package org.apache.hadoop.hdfs.server.datanode;
import java.io.File;
import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
import org.mockito.Mockito;
import com.google.common.base.Preconditions;
@@ -100,6 +104,19 @@ public class DataNodeTestUtils {
return spy;
}
+ public static InterDatanodeProtocol createInterDatanodeProtocolProxy(
+ DataNode dn, DatanodeID datanodeid, final Configuration conf
+ ) throws IOException {
+ return DataNode.createInterDataNodeProtocolProxy(datanodeid, conf,
+ dn.getDnConf().socketTimeout);
+ }
+
+ public static void shutdownBlockScanner(DataNode dn) {
+ if (dn.blockScanner != null) {
+ dn.blockScanner.shutdown();
+ }
+ }
+
/**
* This method is used for testing.
* Examples are adding and deleting blocks directly.
@@ -111,26 +128,22 @@ public class DataNodeTestUtils {
return dn.getFSDataset();
}
- public static FSDataset getFsDatasetImpl(DataNode dn) {
- return (FSDataset)dn.getFSDataset();
- }
-
public static File getFile(DataNode dn, String bpid, long bid) {
- return getFsDatasetImpl(dn).getFile(bpid, bid);
+ return FsDatasetTestUtil.getFile(dn.getFSDataset(), bpid, bid);
}
public static File getBlockFile(DataNode dn, String bpid, Block b
) throws IOException {
- return getFsDatasetImpl(dn).getBlockFile(bpid, b);
+ return FsDatasetTestUtil.getBlockFile(dn.getFSDataset(), bpid, b);
}
- public static boolean unlinkBlock(DataNode dn, ExtendedBlock block, int numLinks
+ public static boolean unlinkBlock(DataNode dn, ExtendedBlock bk, int numLinks
) throws IOException {
- return getFsDatasetImpl(dn).getReplicaInfo(block).unlinkBlock(numLinks);
+ return FsDatasetTestUtil.unlinkBlock(dn.getFSDataset(), bk, numLinks);
}
public static long getPendingAsyncDeletions(DataNode dn) {
- return getFsDatasetImpl(dn).asyncDiskService.countPendingDeletions();
+ return FsDatasetTestUtil.getPendingAsyncDeletions(dn.getFSDataset());
}
/**
@@ -142,6 +155,6 @@ public class DataNodeTestUtils {
*/
public static ReplicaInfo fetchReplicaInfo(final DataNode dn,
final String bpid, final long blkId) {
- return getFsDatasetImpl(dn).fetchReplicaInfo(bpid, blkId);
+ return FsDatasetTestUtil.fetchReplicaInfo(dn.getFSDataset(), bpid, blkId);
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
index f2cb248ae14..8dcf12a0645 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
@@ -613,14 +613,13 @@ public class TestBlockReport {
// Look about specified DN for the replica of the block from 1st DN
final DataNode dn1 = cluster.getDataNodes().get(DN_N1);
- final FSDataset dataset1 = (FSDataset)DataNodeTestUtils.getFSDataset(dn1);
String bpid = cluster.getNamesystem().getBlockPoolId();
- Replica r = dataset1.fetchReplicaInfo(bpid, bl.getBlockId());
+ Replica r = DataNodeTestUtils.fetchReplicaInfo(dn1, bpid, bl.getBlockId());
long start = System.currentTimeMillis();
int count = 0;
while (r == null) {
waitTil(5);
- r = dataset1.fetchReplicaInfo(bpid, bl.getBlockId());
+ r = DataNodeTestUtils.fetchReplicaInfo(dn1, bpid, bl.getBlockId());
long waiting_period = System.currentTimeMillis() - start;
if (count++ % 100 == 0)
if(LOG.isDebugEnabled()) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index 04aa71d8b80..202d2cd581f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -37,6 +37,7 @@ 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.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -376,7 +377,7 @@ public class TestDataNodeVolumeFailure {
new FilenameFilter() {
public boolean accept(File dir, String name) {
return name.startsWith("blk_") &&
- name.endsWith(DatanodeUtil.METADATA_EXTENSION);
+ name.endsWith(Block.METADATA_EXTENSION);
}
}
);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index 8707805613c..ea0143eacdf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -38,7 +38,9 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
/**
* Tests {@link DirectoryScanner} handling of differences
@@ -51,7 +53,7 @@ public class TestDirectoryScanner extends TestCase {
private MiniDFSCluster cluster;
private String bpid;
- private FSDataset fds = null;
+ private FsDatasetSpi extends FsVolumeSpi> fds = null;
private DirectoryScanner scanner = null;
private Random rand = new Random();
private Random r = new Random();
@@ -72,7 +74,7 @@ public class TestDirectoryScanner extends TestCase {
/** Truncate a block file */
private long truncateBlockFile() throws IOException {
synchronized (fds) {
- for (ReplicaInfo b : fds.volumeMap.replicas(bpid)) {
+ for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
File f = b.getBlockFile();
File mf = b.getMetaFile();
// Truncate a block file that has a corresponding metadata file
@@ -91,7 +93,7 @@ public class TestDirectoryScanner extends TestCase {
/** Delete a block file */
private long deleteBlockFile() {
synchronized(fds) {
- for (ReplicaInfo b : fds.volumeMap.replicas(bpid)) {
+ for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
File f = b.getBlockFile();
File mf = b.getMetaFile();
// Delete a block file that has corresponding metadata file
@@ -107,7 +109,7 @@ public class TestDirectoryScanner extends TestCase {
/** Delete block meta file */
private long deleteMetaFile() {
synchronized(fds) {
- for (ReplicaInfo b : fds.volumeMap.replicas(bpid)) {
+ for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) {
File file = b.getMetaFile();
// Delete a metadata file
if (file.exists() && file.delete()) {
@@ -124,7 +126,7 @@ public class TestDirectoryScanner extends TestCase {
long id = rand.nextLong();
while (true) {
id = rand.nextLong();
- if (fds.fetchReplicaInfo(bpid, id) == null) {
+ if (FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, id) == null) {
break;
}
}
@@ -142,7 +144,7 @@ public class TestDirectoryScanner extends TestCase {
/** Create a block file in a random volume*/
private long createBlockFile() throws IOException {
- List volumes = fds.getVolumes();
+ List extends FsVolumeSpi> volumes = fds.getVolumes();
int index = rand.nextInt(volumes.size() - 1);
long id = getFreeBlockId();
File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
@@ -155,7 +157,7 @@ public class TestDirectoryScanner extends TestCase {
/** Create a metafile in a random volume*/
private long createMetaFile() throws IOException {
- List volumes = fds.getVolumes();
+ List extends FsVolumeSpi> volumes = fds.getVolumes();
int index = rand.nextInt(volumes.size() - 1);
long id = getFreeBlockId();
File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
@@ -168,7 +170,7 @@ public class TestDirectoryScanner extends TestCase {
/** Create block file and corresponding metafile in a rondom volume */
private long createBlockMetaFile() throws IOException {
- List volumes = fds.getVolumes();
+ List extends FsVolumeSpi> volumes = fds.getVolumes();
int index = rand.nextInt(volumes.size() - 1);
long id = getFreeBlockId();
File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
@@ -228,8 +230,7 @@ public class TestDirectoryScanner extends TestCase {
try {
cluster.waitActive();
bpid = cluster.getNamesystem().getBlockPoolId();
- fds = (FSDataset)DataNodeTestUtils.getFSDataset(
- cluster.getDataNodes().get(0));
+ fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
parallelism);
DataNode dn = cluster.getDataNodes().get(0);
@@ -348,12 +349,13 @@ public class TestDirectoryScanner extends TestCase {
private void verifyAddition(long blockId, long genStamp, long size) {
final ReplicaInfo replicainfo;
- replicainfo = fds.fetchReplicaInfo(bpid, blockId);
+ replicainfo = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
assertNotNull(replicainfo);
// Added block has the same file as the one created by the test
File file = new File(getBlockFile(blockId));
- assertEquals(file.getName(), fds.getFile(bpid, blockId).getName());
+ assertEquals(file.getName(),
+ FsDatasetTestUtil.getFile(fds, bpid, blockId).getName());
// Generation stamp is same as that of created file
assertEquals(genStamp, replicainfo.getGenerationStamp());
@@ -364,12 +366,12 @@ public class TestDirectoryScanner extends TestCase {
private void verifyDeletion(long blockId) {
// Ensure block does not exist in memory
- assertNull(fds.fetchReplicaInfo(bpid, blockId));
+ assertNull(FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId));
}
private void verifyGenStamp(long blockId, long genStamp) {
final ReplicaInfo memBlock;
- memBlock = fds.fetchReplicaInfo(bpid, blockId);
+ memBlock = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
assertNotNull(memBlock);
assertEquals(genStamp, memBlock.getGenerationStamp());
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
index b319dd90465..25c9fcaea77 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
import org.apache.hadoop.util.DataChecksum;
/**
@@ -90,7 +91,7 @@ public class TestSimulatedFSDataset extends TestCase {
public void testFSDatasetFactory() {
final Configuration conf = new Configuration();
FsDatasetSpi.Factory> f = FsDatasetSpi.Factory.getFactory(conf);
- assertEquals(FSDataset.Factory.class, f.getClass());
+ assertEquals(FsDatasetFactory.class, f.getClass());
assertFalse(f.isSimulated());
SimulatedFSDataset.setFactory(conf);
@@ -243,7 +244,7 @@ public class TestSimulatedFSDataset extends TestCase {
}
}
- public void checkInvalidBlock(ExtendedBlock b) throws IOException {
+ public void checkInvalidBlock(ExtendedBlock b) {
final SimulatedFSDataset fsdataset = getSimulatedFSDataset();
assertFalse(fsdataset.isValidBlock(b));
try {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
index 7237f2a93ea..55a87bd8ce4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.log4j.Level;
import org.junit.Assert;
@@ -58,8 +59,8 @@ public class TestTransferRbw {
}
private static ReplicaInPipeline getReplica(final DataNode datanode,
final String bpid, final ReplicaState expectedState) throws InterruptedException {
- final FSDataset dataset = ((FSDataset)datanode.data);
- final Collection replicas = dataset.volumeMap.replicas(bpid);
+ final Collection replicas = FsDatasetTestUtil.getReplicas(
+ datanode.getFSDataset(), bpid);
for(int i = 0; i < 5 && replicas.size() == 0; i++) {
LOG.info("wait since replicas.size() == 0; i=" + i);
Thread.sleep(1000);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java
new file mode 100644
index 00000000000..211737fa73e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+
+public class FsDatasetTestUtil {
+
+ public static File getFile(FsDatasetSpi> fsd, String bpid, long bid) {
+ return ((FsDatasetImpl)fsd).getFile(bpid, bid);
+ }
+
+ public static File getBlockFile(FsDatasetSpi> fsd, String bpid, Block b
+ ) throws IOException {
+ return ((FsDatasetImpl)fsd).getBlockFile(bpid, b);
+ }
+
+ public static boolean unlinkBlock(FsDatasetSpi> fsd,
+ ExtendedBlock block, int numLinks) throws IOException {
+ final ReplicaInfo info = ((FsDatasetImpl)fsd).getReplicaInfo(block);
+ return info.unlinkBlock(numLinks);
+ }
+
+ public static ReplicaInfo fetchReplicaInfo (final FsDatasetSpi> fsd,
+ final String bpid, final long blockId) {
+ return ((FsDatasetImpl)fsd).fetchReplicaInfo(bpid, blockId);
+ }
+
+ public static long getPendingAsyncDeletions(FsDatasetSpi> fsd) {
+ return ((FsDatasetImpl)fsd).asyncDiskService.countPendingDeletions();
+ }
+
+ public static Collection getReplicas(FsDatasetSpi> fsd,
+ String bpid) {
+ return ((FsDatasetImpl)fsd).volumeMap.replicas(bpid);
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
similarity index 90%
rename from hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java
rename to hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
index 615732d8d4a..773864e2f3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import java.io.File;
import java.io.FileInputStream;
@@ -36,7 +36,10 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.io.IOUtils;
import org.junit.Assert;
@@ -73,7 +76,6 @@ public class TestDatanodeRestart {
Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
cluster.waitActive();
try {
@@ -98,8 +100,8 @@ public class TestDatanodeRestart {
out.write(writeBuf);
out.hflush();
DataNode dn = cluster.getDataNodes().get(0);
- for (FsVolumeSpi v : dn.data.getVolumes()) {
- FSVolume volume = (FSVolume)v;
+ for (FsVolumeSpi v : dataset(dn).getVolumes()) {
+ final FsVolumeImpl volume = (FsVolumeImpl)v;
File currentDir = volume.getCurrentDir().getParentFile().getParentFile();
File rbwDir = new File(currentDir, "rbw");
for (File file : rbwDir.listFiles()) {
@@ -114,7 +116,7 @@ public class TestDatanodeRestart {
// check volumeMap: one rwr replica
String bpid = cluster.getNamesystem().getBlockPoolId();
- ReplicasMap replicas = ((FSDataset)(dn.data)).volumeMap;
+ ReplicaMap replicas = dataset(dn).volumeMap;
Assert.assertEquals(1, replicas.size(bpid));
ReplicaInfo replica = replicas.replicas(bpid).iterator().next();
Assert.assertEquals(ReplicaState.RWR, replica.getState());
@@ -123,7 +125,7 @@ public class TestDatanodeRestart {
} else {
Assert.assertEquals(fileLen, replica.getNumBytes());
}
- dn.data.invalidate(bpid, new Block[]{replica});
+ dataset(dn).invalidate(bpid, new Block[]{replica});
} finally {
IOUtils.closeStream(out);
if (fs.exists(src)) {
@@ -138,7 +140,6 @@ public class TestDatanodeRestart {
Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
- conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
cluster.waitActive();
try {
@@ -151,7 +152,7 @@ public class TestDatanodeRestart {
String bpid = cluster.getNamesystem().getBlockPoolId();
DataNode dn = cluster.getDataNodes().get(0);
Iterator replicasItor =
- ((FSDataset)dn.data).volumeMap.replicas(bpid).iterator();
+ dataset(dn).volumeMap.replicas(bpid).iterator();
ReplicaInfo replica = replicasItor.next();
createUnlinkTmpFile(replica, true, true); // rename block file
createUnlinkTmpFile(replica, false, true); // rename meta file
@@ -167,8 +168,7 @@ public class TestDatanodeRestart {
dn = cluster.getDataNodes().get(0);
// check volumeMap: 4 finalized replica
- Collection replicas =
- ((FSDataset)(dn.data)).volumeMap.replicas(bpid);
+ Collection replicas = dataset(dn).volumeMap.replicas(bpid);
Assert.assertEquals(4, replicas.size());
replicasItor = replicas.iterator();
while (replicasItor.hasNext()) {
@@ -180,6 +180,10 @@ public class TestDatanodeRestart {
}
}
+ private static FsDatasetImpl dataset(DataNode dn) {
+ return (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
+ }
+
private static void createUnlinkTmpFile(ReplicaInfo replicaInfo,
boolean changeBlockFile,
boolean isRename) throws IOException {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
similarity index 87%
rename from hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
rename to hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
index 9f96eac70f5..599521f1793 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
@@ -15,22 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
-import java.util.List;
import java.net.InetSocketAddress;
-
import java.net.SocketTimeoutException;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.NetUtils;
+import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@@ -41,16 +34,28 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+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.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
import org.junit.Assert;
import org.junit.Test;
@@ -103,14 +108,14 @@ public class TestInterDatanodeProtocol {
}
public static void checkMetaInfo(ExtendedBlock b, DataNode dn) throws IOException {
- Block metainfo = dn.data.getStoredBlock(b.getBlockPoolId(), b.getBlockId());
+ Block metainfo = DataNodeTestUtils.getFSDataset(dn).getStoredBlock(
+ b.getBlockPoolId(), b.getBlockId());
Assert.assertEquals(b.getBlockId(), metainfo.getBlockId());
Assert.assertEquals(b.getNumBytes(), metainfo.getNumBytes());
}
public static LocatedBlock getLastLocatedBlock(
- ClientProtocol namenode, String src
- ) throws IOException {
+ ClientProtocol namenode, String src) throws IOException {
//get block info for the last block
LocatedBlocks locations = namenode.getBlockLocations(src, 0, Long.MAX_VALUE);
List blocks = locations.getLocatedBlocks();
@@ -148,13 +153,11 @@ public class TestInterDatanodeProtocol {
//connect to a data node
DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
- InterDatanodeProtocol idp = DataNode.createInterDataNodeProtocolProxy(
- datanodeinfo[0], conf, datanode.getDnConf().socketTimeout);
+ InterDatanodeProtocol idp = DataNodeTestUtils.createInterDatanodeProtocolProxy(
+ datanode, datanodeinfo[0], conf);
//stop block scanner, so we could compare lastScanTime
- if (datanode.blockScanner != null) {
- datanode.blockScanner.shutdown();
- }
+ DataNodeTestUtils.shutdownBlockScanner(datanode);
//verify BlockMetaDataInfo
ExtendedBlock b = locatedblock.getBlock();
@@ -187,14 +190,14 @@ public class TestInterDatanodeProtocol {
}
/** Test
- * {@link FSDataset#initReplicaRecovery(String, ReplicasMap, Block, long)}
+ * {@link FsDatasetImpl#initReplicaRecovery(String, ReplicaMap, Block, long)}
*/
@Test
public void testInitReplicaRecovery() throws IOException {
final long firstblockid = 10000L;
final long gs = 7777L;
final long length = 22L;
- final ReplicasMap map = new ReplicasMap(this);
+ final ReplicaMap map = new ReplicaMap(this);
String bpid = "BP-TEST";
final Block[] blocks = new Block[5];
for(int i = 0; i < blocks.length; i++) {
@@ -208,7 +211,8 @@ public class TestInterDatanodeProtocol {
final ReplicaInfo originalInfo = map.get(bpid, b);
final long recoveryid = gs + 1;
- final ReplicaRecoveryInfo recoveryInfo = FSDataset.initReplicaRecovery(bpid, map, blocks[0], recoveryid);
+ final ReplicaRecoveryInfo recoveryInfo = FsDatasetImpl.initReplicaRecovery(
+ bpid, map, blocks[0], recoveryid);
assertEquals(originalInfo, recoveryInfo);
final ReplicaUnderRecovery updatedInfo = (ReplicaUnderRecovery)map.get(bpid, b);
@@ -217,7 +221,7 @@ public class TestInterDatanodeProtocol {
//recover one more time
final long recoveryid2 = gs + 2;
- final ReplicaRecoveryInfo recoveryInfo2 = FSDataset.initReplicaRecovery(bpid, map, blocks[0], recoveryid2);
+ final ReplicaRecoveryInfo recoveryInfo2 = FsDatasetImpl.initReplicaRecovery(bpid, map, blocks[0], recoveryid2);
assertEquals(originalInfo, recoveryInfo2);
final ReplicaUnderRecovery updatedInfo2 = (ReplicaUnderRecovery)map.get(bpid, b);
@@ -226,7 +230,7 @@ public class TestInterDatanodeProtocol {
//case RecoveryInProgressException
try {
- FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
+ FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
Assert.fail();
}
catch(RecoveryInProgressException ripe) {
@@ -237,7 +241,7 @@ public class TestInterDatanodeProtocol {
{ // BlockRecoveryFI_01: replica not found
final long recoveryid = gs + 1;
final Block b = new Block(firstblockid - 1, length, gs);
- ReplicaRecoveryInfo r = FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
+ ReplicaRecoveryInfo r = FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
Assert.assertNull("Data-node should not have this replica.", r);
}
@@ -245,7 +249,7 @@ public class TestInterDatanodeProtocol {
final long recoveryid = gs - 1;
final Block b = new Block(firstblockid + 1, length, gs);
try {
- FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
+ FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
Assert.fail();
}
catch(IOException ioe) {
@@ -258,7 +262,7 @@ public class TestInterDatanodeProtocol {
final long recoveryid = gs + 1;
final Block b = new Block(firstblockid, length, gs+1);
try {
- FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
+ FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
fail("InitReplicaRecovery should fail because replica's " +
"gs is less than the block's gs");
} catch (IOException e) {
@@ -270,7 +274,7 @@ public class TestInterDatanodeProtocol {
/**
* Test for
- * {@link FSDataset#updateReplicaUnderRecovery(ExtendedBlock, long, long)}
+ * {@link FsDatasetImpl#updateReplicaUnderRecovery(ExtendedBlock, long, long)}
* */
@Test
public void testUpdateReplicaUnderRecovery() throws IOException {
@@ -296,22 +300,22 @@ public class TestInterDatanodeProtocol {
//get DataNode and FSDataset objects
final DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
Assert.assertTrue(datanode != null);
- Assert.assertTrue(datanode.data instanceof FSDataset);
- final FSDataset fsdataset = (FSDataset)datanode.data;
//initReplicaRecovery
final ExtendedBlock b = locatedblock.getBlock();
final long recoveryid = b.getGenerationStamp() + 1;
final long newlength = b.getNumBytes() - 1;
+ final FsDatasetSpi> fsdataset = DataNodeTestUtils.getFSDataset(datanode);
final ReplicaRecoveryInfo rri = fsdataset.initReplicaRecovery(
new RecoveringBlock(b, null, recoveryid));
//check replica
- final ReplicaInfo replica = fsdataset.fetchReplicaInfo(bpid, b.getBlockId());
+ final ReplicaInfo replica = FsDatasetTestUtil.fetchReplicaInfo(
+ fsdataset, bpid, b.getBlockId());
Assert.assertEquals(ReplicaState.RUR, replica.getState());
//check meta data before update
- FSDataset.checkReplicaFiles(replica);
+ FsDatasetImpl.checkReplicaFiles(replica);
//case "THIS IS NOT SUPPOSED TO HAPPEN"
//with (block length) != (stored replica's on disk length).
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReplicasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReplicaMap.java
similarity index 91%
rename from hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReplicasMap.java
rename to hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReplicaMap.java
index cacbfd62f6b..e35447391c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReplicasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReplicaMap.java
@@ -15,21 +15,23 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.fail;
import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReplicaMap;
import org.junit.Before;
import org.junit.Test;
/**
* Unit test for ReplicasMap class
*/
-public class TestReplicasMap {
- private final ReplicasMap map = new ReplicasMap(TestReplicasMap.class);
+public class TestReplicaMap {
+ private final ReplicaMap map = new ReplicaMap(TestReplicaMap.class);
private final String bpid = "BP-TEST";
private final Block block = new Block(1234, 1234, 1234);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestWriteToReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
similarity index 90%
rename from hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestWriteToReplica.java
rename to hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
index 4e5bd5dbee8..dfefc1e8437 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestWriteToReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
@@ -15,14 +15,23 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdfs.server.datanode;
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import java.io.IOException;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
import org.junit.Assert;
import org.junit.Test;
@@ -45,7 +54,7 @@ public class TestWriteToReplica {
try {
cluster.waitActive();
DataNode dn = cluster.getDataNodes().get(0);
- FSDataset dataSet = (FSDataset)dn.data;
+ FsDatasetImpl dataSet = (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
// set up replicasMap
String bpid = cluster.getNamesystem().getBlockPoolId();
@@ -66,7 +75,7 @@ public class TestWriteToReplica {
try {
cluster.waitActive();
DataNode dn = cluster.getDataNodes().get(0);
- FSDataset dataSet = (FSDataset)dn.data;
+ FsDatasetImpl dataSet = (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
// set up replicasMap
String bpid = cluster.getNamesystem().getBlockPoolId();
@@ -86,7 +95,7 @@ public class TestWriteToReplica {
try {
cluster.waitActive();
DataNode dn = cluster.getDataNodes().get(0);
- FSDataset dataSet = (FSDataset)dn.data;
+ FsDatasetImpl dataSet = (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
// set up replicasMap
String bpid = cluster.getNamesystem().getBlockPoolId();
@@ -106,7 +115,7 @@ public class TestWriteToReplica {
try {
cluster.waitActive();
DataNode dn = cluster.getDataNodes().get(0);
- FSDataset dataSet = (FSDataset)dn.data;
+ FsDatasetImpl dataSet = (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
// set up replicasMap
String bpid = cluster.getNamesystem().getBlockPoolId();
@@ -128,7 +137,7 @@ public class TestWriteToReplica {
* @return Contrived blocks for further testing.
* @throws IOException
*/
- private ExtendedBlock[] setup(String bpid, FSDataset dataSet) throws IOException {
+ private ExtendedBlock[] setup(String bpid, FsDatasetImpl dataSet) throws IOException {
// setup replicas map
ExtendedBlock[] blocks = new ExtendedBlock[] {
@@ -137,8 +146,8 @@ public class TestWriteToReplica {
new ExtendedBlock(bpid, 5, 1, 2005), new ExtendedBlock(bpid, 6, 1, 2006)
};
- ReplicasMap replicasMap = dataSet.volumeMap;
- FSVolume vol = dataSet.volumes.getNextVolume(0);
+ ReplicaMap replicasMap = dataSet.volumeMap;
+ FsVolumeImpl vol = dataSet.volumes.getNextVolume(0);
ReplicaInfo replicaInfo = new FinalizedReplica(
blocks[FINALIZED].getLocalBlock(), vol, vol.getCurrentDir().getParentFile());
replicasMap.add(bpid, replicaInfo);
@@ -165,9 +174,9 @@ public class TestWriteToReplica {
return blocks;
}
- private void testAppend(String bpid, FSDataset dataSet, ExtendedBlock[] blocks) throws IOException {
+ private void testAppend(String bpid, FsDatasetImpl dataSet, ExtendedBlock[] blocks) throws IOException {
long newGS = blocks[FINALIZED].getGenerationStamp()+1;
- final FSVolume v = (FSVolume)dataSet.volumeMap.get(
+ final FsVolumeImpl v = (FsVolumeImpl)dataSet.volumeMap.get(
bpid, blocks[FINALIZED].getLocalBlock()).getVolume();
long available = v.getCapacity()-v.getDfsUsed();
long expectedLen = blocks[FINALIZED].getNumBytes();
@@ -285,7 +294,7 @@ public class TestWriteToReplica {
}
}
- private void testClose(FSDataset dataSet, ExtendedBlock [] blocks) throws IOException {
+ private void testClose(FsDatasetImpl dataSet, ExtendedBlock [] blocks) throws IOException {
long newGS = blocks[FINALIZED].getGenerationStamp()+1;
dataSet.recoverClose(blocks[FINALIZED], newGS,
blocks[FINALIZED].getNumBytes()); // successful
@@ -335,7 +344,7 @@ public class TestWriteToReplica {
}
}
- private void testWriteToRbw(FSDataset dataSet, ExtendedBlock[] blocks) throws IOException {
+ private void testWriteToRbw(FsDatasetImpl dataSet, ExtendedBlock[] blocks) throws IOException {
try {
dataSet.recoverRbw(blocks[FINALIZED],
blocks[FINALIZED].getGenerationStamp()+1,
@@ -428,7 +437,7 @@ public class TestWriteToReplica {
dataSet.createRbw(blocks[NON_EXISTENT]);
}
- private void testWriteToTemporary(FSDataset dataSet, ExtendedBlock[] blocks) throws IOException {
+ private void testWriteToTemporary(FsDatasetImpl dataSet, ExtendedBlock[] blocks) throws IOException {
try {
dataSet.createTemporary(blocks[FINALIZED]);
Assert.fail("Should not have created a temporary replica that was " +
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
index 5d93b8cf45f..4118be2048c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
@@ -179,12 +179,22 @@ public class TestBackupNode {
// do some edits
assertTrue(fileSys.mkdirs(new Path("/edit-while-bn-down")));
-
+
// start a new backup node
backup = startBackupNode(conf, StartupOption.BACKUP, 1);
testBNInSync(cluster, backup, 4);
assertNotNull(backup.getNamesystem().getFileInfo("/edit-while-bn-down", false));
+
+ // Trigger an unclean shutdown of the backup node. Backup node will not
+ // unregister from the active when this is done simulating a node crash.
+ backup.stop(false);
+
+ // do some edits on the active. This should go through without failing.
+ // This will verify that active is still up and can add entries to
+ // master editlog.
+ assertTrue(fileSys.mkdirs(new Path("/edit-while-bn-down-2")));
+
} finally {
LOG.info("Shutting down...");
if (backup != null) backup.stop();
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index a28d38f2bb0..6241943baf4 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -195,6 +195,51 @@ Release 2.0.0 - UNRELEASED
MAPREDUCE-4091. tools testcases failing because of MAPREDUCE-4082 (tucu)
+ MAPREDUCE-4095. TestJobInProgress#testLocality uses a bogus topology.
+ (Colin Patrick McCabe via eli)
+
+Release 0.23.3 - UNRELEASED
+
+ INCOMPATIBLE CHANGES
+
+ MAPREDUCE-4072. User set java.library.path seems to overwrite default
+ creating problems native lib loading (Anupam Seth via bobby)
+
+ NEW FEATURES
+
+ IMPROVEMENTS
+
+ OPTIMIZATIONS
+
+ BUG FIXES
+
+ MAPREDUCE-4092. commitJob Exception does not fail job (Jon Eagles via
+ bobby)
+
+ MAPREDUCE-4089. Hung Tasks never time out. (Robert Evans via tgraves)
+
+ MAPREDUCE-4024. RM webservices can't query on finalStatus (Tom Graves
+ via bobby)
+
+ MAPREDUCE-4060. Multiple SLF4J binding warning (Jason Lowe via bobby)
+
+ MAPREDUCE-3983. TestTTResourceReporting can fail, and should just be
+ deleted (Ravi Prakash via bobby)
+
+ MAPREDUCE-4012 Hadoop Job setup error leaves no useful info to users
+ (when LinuxTaskController is used). (tgraves)
+
+ MAPREDUCE-4062. AM Launcher thread can hang forever (tgraves via bobby)
+
+ MAPREDUCE-3988. mapreduce.job.local.dir doesn't point to a single directory
+ on a node. (Eric Payne via bobby)
+
+ MAPREDUCE-3999. Tracking link gives an error if the AppMaster hasn't
+ started yet (Ravi Prakash via bobby)
+
+ MAPREDUCE-4020. Web services returns incorrect JSON for deep queue tree
+ (Anupam Seth via tgraves)
+
Release 0.23.2 - UNRELEASED
INCOMPATIBLE CHANGES
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
index 6d78a6a8c03..164f406017e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
@@ -175,7 +175,7 @@ public class TaskAttemptListenerImpl extends CompositeService
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
- taskHeartbeatHandler.receivedPing(attemptID);
+ taskHeartbeatHandler.progressing(attemptID);
Job job = context.getJob(attemptID.getTaskId().getJobId());
Task task = job.getTask(attemptID.getTaskId());
@@ -203,7 +203,7 @@ public class TaskAttemptListenerImpl extends CompositeService
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
- taskHeartbeatHandler.receivedPing(attemptID);
+ taskHeartbeatHandler.progressing(attemptID);
//Ignorable TaskStatus? - since a task will send a LastStatusUpdate
context.getEventHandler().handle(
new TaskAttemptEvent(attemptID,
@@ -217,7 +217,7 @@ public class TaskAttemptListenerImpl extends CompositeService
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
- taskHeartbeatHandler.receivedPing(attemptID);
+ taskHeartbeatHandler.progressing(attemptID);
context.getEventHandler().handle(
new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_DONE));
@@ -270,7 +270,7 @@ public class TaskAttemptListenerImpl extends CompositeService
context.getJob(attemptID.getTaskId().getJobId()).getTaskAttemptCompletionEvents(
fromEventId, maxEvents);
- taskHeartbeatHandler.receivedPing(attemptID);
+ taskHeartbeatHandler.progressing(attemptID);
// filter the events to return only map completion events in old format
List mapEvents = new ArrayList();
@@ -287,7 +287,7 @@ public class TaskAttemptListenerImpl extends CompositeService
@Override
public boolean ping(TaskAttemptID taskAttemptID) throws IOException {
LOG.info("Ping from " + taskAttemptID.toString());
- taskHeartbeatHandler.receivedPing(TypeConverter.toYarn(taskAttemptID));
+ taskHeartbeatHandler.pinged(TypeConverter.toYarn(taskAttemptID));
return true;
}
@@ -299,7 +299,7 @@ public class TaskAttemptListenerImpl extends CompositeService
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
- taskHeartbeatHandler.receivedPing(attemptID);
+ taskHeartbeatHandler.progressing(attemptID);
// This is mainly used for cases where we want to propagate exception traces
// of tasks that fail.
@@ -317,7 +317,7 @@ public class TaskAttemptListenerImpl extends CompositeService
LOG.info("Status update from " + taskAttemptID.toString());
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId yarnAttemptID =
TypeConverter.toYarn(taskAttemptID);
- taskHeartbeatHandler.receivedPing(yarnAttemptID);
+ taskHeartbeatHandler.progressing(yarnAttemptID);
TaskAttemptStatus taskAttemptStatus =
new TaskAttemptStatus();
taskAttemptStatus.id = yarnAttemptID;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
index 0687ab6cbd6..01b29eaf178 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.log4j.LogManager;
@@ -236,11 +237,33 @@ class YarnChild {
job.setStrings(MRConfig.LOCAL_DIR, localSysDirs);
LOG.info(MRConfig.LOCAL_DIR + " for child: " + job.get(MRConfig.LOCAL_DIR));
LocalDirAllocator lDirAlloc = new LocalDirAllocator(MRConfig.LOCAL_DIR);
- Path workDir = lDirAlloc.getLocalPathForWrite("work", job);
- FileSystem lfs = FileSystem.getLocal(job).getRaw();
- if (!lfs.mkdirs(workDir)) {
- throw new IOException("Mkdirs failed to create "
- + workDir.toString());
+ Path workDir = null;
+ // First, try to find the JOB_LOCAL_DIR on this host.
+ try {
+ workDir = lDirAlloc.getLocalPathToRead("work", job);
+ } catch (DiskErrorException e) {
+ // DiskErrorException means dir not found. If not found, it will
+ // be created below.
+ }
+ if (workDir == null) {
+ // JOB_LOCAL_DIR doesn't exist on this host -- Create it.
+ workDir = lDirAlloc.getLocalPathForWrite("work", job);
+ FileSystem lfs = FileSystem.getLocal(job).getRaw();
+ boolean madeDir = false;
+ try {
+ madeDir = lfs.mkdirs(workDir);
+ } catch (FileAlreadyExistsException e) {
+ // Since all tasks will be running in their own JVM, the race condition
+ // exists where multiple tasks could be trying to create this directory
+ // at the same time. If this task loses the race, it's okay because
+ // the directory already exists.
+ madeDir = true;
+ workDir = lDirAlloc.getLocalPathToRead("work", job);
+ }
+ if (!madeDir) {
+ throw new IOException("Mkdirs failed to create "
+ + workDir.toString());
+ }
}
job.set(MRJobConfig.JOB_LOCAL_DIR,workDir.toString());
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
index b827a2cdf3f..b83f663836d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
@@ -44,9 +44,36 @@ import org.apache.hadoop.yarn.service.AbstractService;
*/
@SuppressWarnings({"unchecked", "rawtypes"})
public class TaskHeartbeatHandler extends AbstractService {
-
+
+ private static class ReportTime {
+ private long lastPing;
+ private long lastProgress;
+
+ public ReportTime(long time) {
+ setLastProgress(time);
+ }
+
+ public synchronized void setLastPing(long time) {
+ lastPing = time;
+ }
+
+ public synchronized void setLastProgress(long time) {
+ lastProgress = time;
+ lastPing = time;
+ }
+
+ public synchronized long getLastPing() {
+ return lastPing;
+ }
+
+ public synchronized long getLastProgress() {
+ return lastProgress;
+ }
+ }
+
private static final Log LOG = LogFactory.getLog(TaskHeartbeatHandler.class);
-
+ private static final int PING_TIMEOUT = 5 * 60 * 1000;
+
//thread which runs periodically to see the last time since a heartbeat is
//received from a task.
private Thread lostTaskCheckerThread;
@@ -56,8 +83,8 @@ public class TaskHeartbeatHandler extends AbstractService {
private final EventHandler eventHandler;
private final Clock clock;
-
- private ConcurrentMap runningAttempts;
+
+ private ConcurrentMap runningAttempts;
public TaskHeartbeatHandler(EventHandler eventHandler, Clock clock,
int numThreads) {
@@ -65,7 +92,7 @@ public class TaskHeartbeatHandler extends AbstractService {
this.eventHandler = eventHandler;
this.clock = clock;
runningAttempts =
- new ConcurrentHashMap(16, 0.75f, numThreads);
+ new ConcurrentHashMap(16, 0.75f, numThreads);
}
@Override
@@ -91,14 +118,26 @@ public class TaskHeartbeatHandler extends AbstractService {
super.stop();
}
- public void receivedPing(TaskAttemptId attemptID) {
+ public void progressing(TaskAttemptId attemptID) {
//only put for the registered attempts
//TODO throw an exception if the task isn't registered.
- runningAttempts.replace(attemptID, clock.getTime());
+ ReportTime time = runningAttempts.get(attemptID);
+ if(time != null) {
+ time.setLastProgress(clock.getTime());
+ }
}
+ public void pinged(TaskAttemptId attemptID) {
+ //only put for the registered attempts
+ //TODO throw an exception if the task isn't registered.
+ ReportTime time = runningAttempts.get(attemptID);
+ if(time != null) {
+ time.setLastPing(clock.getTime());
+ }
+ }
+
public void register(TaskAttemptId attemptID) {
- runningAttempts.put(attemptID, clock.getTime());
+ runningAttempts.put(attemptID, new ReportTime(clock.getTime()));
}
public void unregister(TaskAttemptId attemptID) {
@@ -110,30 +149,27 @@ public class TaskHeartbeatHandler extends AbstractService {
@Override
public void run() {
while (!stopped && !Thread.currentThread().isInterrupted()) {
- Iterator> iterator =
+ Iterator> iterator =
runningAttempts.entrySet().iterator();
// avoid calculating current time everytime in loop
long currentTime = clock.getTime();
while (iterator.hasNext()) {
- Map.Entry entry = iterator.next();
- if (currentTime > entry.getValue() + taskTimeOut) {
-
- //In case the iterator isn't picking up the latest.
- // Extra lookup outside of the iterator - but only if the task
- // is considered to be timed out.
- Long taskTime = runningAttempts.get(entry.getKey());
- if (taskTime != null && currentTime > taskTime + taskTimeOut) {
- // task is lost, remove from the list and raise lost event
- iterator.remove();
- eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry
- .getKey(), "AttemptID:" + entry.getKey().toString()
- + " Timed out after " + taskTimeOut / 1000 + " secs"));
- eventHandler.handle(new TaskAttemptEvent(entry.getKey(),
- TaskAttemptEventType.TA_TIMED_OUT));
- }
-
+ Map.Entry entry = iterator.next();
+ boolean taskTimedOut = (taskTimeOut > 0) &&
+ (currentTime > (entry.getValue().getLastProgress() + taskTimeOut));
+ boolean pingTimedOut =
+ (currentTime > (entry.getValue().getLastPing() + PING_TIMEOUT));
+
+ if(taskTimedOut || pingTimedOut) {
+ // task is lost, remove from the list and raise lost event
+ iterator.remove();
+ eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry
+ .getKey(), "AttemptID:" + entry.getKey().toString()
+ + " Timed out after " + taskTimeOut / 1000 + " secs"));
+ eventHandler.handle(new TaskAttemptEvent(entry.getKey(),
+ TaskAttemptEventType.TA_TIMED_OUT));
}
}
try {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
index 0fca3dd6859..98472d33cf1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
@@ -727,7 +727,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
// Commit job & do cleanup
job.getCommitter().commitJob(job.getJobContext());
} catch (IOException e) {
- LOG.warn("Could not do commit for Job", e);
+ LOG.error("Could not do commit for Job", e);
+ job.logJobHistoryFinishedEvent();
+ return job.finished(JobState.FAILED);
}
job.logJobHistoryFinishedEvent();
return job.finished(JobState.SUCCEEDED);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java
index d9f18b3adbf..40ecdb2b3b9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java
@@ -30,11 +30,4 @@ public interface ContainerLauncher
CONTAINER_REMOTE_CLEANUP
}
- // Not a documented config. Only used for tests
- static final String MR_AM_NM_COMMAND_TIMEOUT = MRJobConfig.MR_AM_PREFIX
- + "nm-command-timeout";
- /**
- * Maximum of 1 minute timeout for a Node to react to the command
- */
- static final int DEFAULT_NM_COMMAND_TIMEOUT = 60000;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
index 0befad86427..46a6111d610 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
@@ -23,8 +23,6 @@ import java.nio.ByteBuffer;
import java.security.PrivilegedAction;
import java.util.HashSet;
import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
@@ -72,8 +70,6 @@ public class ContainerLauncherImpl extends AbstractService implements
static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
- int nmTimeOut;
-
private ConcurrentHashMap containers =
new ConcurrentHashMap();
private AppContext context;
@@ -83,7 +79,6 @@ public class ContainerLauncherImpl extends AbstractService implements
private Thread eventHandlingThread;
protected BlockingQueue eventQueue =
new LinkedBlockingQueue();
- final Timer commandTimer = new Timer(true);
YarnRPC rpc;
private Container getContainer(ContainerId id) {
@@ -130,30 +125,18 @@ public class ContainerLauncherImpl extends AbstractService implements
"Container was killed before it was launched");
return;
}
- CommandTimerTask timerTask = new CommandTimerTask(Thread
- .currentThread(), event);
+
final String containerManagerBindAddr = event.getContainerMgrAddress();
ContainerId containerID = event.getContainerID();
ContainerToken containerToken = event.getContainerToken();
ContainerManager proxy = null;
try {
- commandTimer.schedule(timerTask, nmTimeOut);
proxy = getCMProxy(containerID, containerManagerBindAddr,
containerToken);
- // Interrupted during getProxy, but that didn't throw exception
- if (Thread.interrupted()) {
- // The timer canceled the command in the mean while.
- String message = "Container launch failed for " + containerID
- + " : Start-container for " + event.getContainerID()
- + " got interrupted. Returning.";
- this.state = ContainerState.FAILED;
- sendContainerLaunchFailedMsg(taskAttemptID, message);
- return;
- }
// Construct the actual Container
ContainerLaunchContext containerLaunchContext =
event.getContainer();
@@ -164,19 +147,6 @@ public class ContainerLauncherImpl extends AbstractService implements
startRequest.setContainerLaunchContext(containerLaunchContext);
StartContainerResponse response = proxy.startContainer(startRequest);
- // container started properly. Stop the timer
- timerTask.cancel();
- if (Thread.interrupted()) {
- // The timer canceled the command in the mean while, but
- // startContainer didn't throw exception
- String message = "Container launch failed for " + containerID
- + " : Start-container for " + event.getContainerID()
- + " got interrupted. Returning.";
- this.state = ContainerState.FAILED;
- sendContainerLaunchFailedMsg(taskAttemptID, message);
- return;
- }
-
ByteBuffer portInfo = response
.getServiceResponse(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID);
int port = -1;
@@ -198,17 +168,11 @@ public class ContainerLauncherImpl extends AbstractService implements
new TaskAttemptContainerLaunchedEvent(taskAttemptID, port));
this.state = ContainerState.RUNNING;
} catch (Throwable t) {
- if (Thread.interrupted()) {
- // The timer canceled the command in the mean while.
- LOG.info("Start-container for " + event.getContainerID()
- + " got interrupted.");
- }
String message = "Container launch failed for " + containerID + " : "
+ StringUtils.stringifyException(t);
this.state = ContainerState.FAILED;
sendContainerLaunchFailedMsg(taskAttemptID, message);
} finally {
- timerTask.cancel();
if (proxy != null) {
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
}
@@ -220,41 +184,24 @@ public class ContainerLauncherImpl extends AbstractService implements
if(this.state == ContainerState.PREP) {
this.state = ContainerState.KILLED_BEFORE_LAUNCH;
} else {
- CommandTimerTask timerTask = new CommandTimerTask(Thread
- .currentThread(), event);
-
final String containerManagerBindAddr = event.getContainerMgrAddress();
ContainerId containerID = event.getContainerID();
ContainerToken containerToken = event.getContainerToken();
TaskAttemptId taskAttemptID = event.getTaskAttemptID();
LOG.info("KILLING " + taskAttemptID);
- commandTimer.schedule(timerTask, nmTimeOut);
ContainerManager proxy = null;
try {
proxy = getCMProxy(containerID, containerManagerBindAddr,
containerToken);
- if (Thread.interrupted()) {
- // The timer canceled the command in the mean while. No need to
- // return, send cleaned up event anyways.
- LOG.info("Stop-container for " + event.getContainerID()
- + " got interrupted.");
- } else {
// kill the remote container if already launched
StopContainerRequest stopRequest = Records
.newRecord(StopContainerRequest.class);
stopRequest.setContainerId(event.getContainerID());
proxy.stopContainer(stopRequest);
- }
- } catch (Throwable t) {
- if (Thread.interrupted()) {
- // The timer canceled the command in the mean while, clear the
- // interrupt flag
- LOG.info("Stop-container for " + event.getContainerID()
- + " got interrupted.");
- }
+ } catch (Throwable t) {
// ignore the cleanup failure
String message = "cleanup failed for container "
@@ -264,15 +211,6 @@ public class ContainerLauncherImpl extends AbstractService implements
new TaskAttemptDiagnosticsUpdateEvent(taskAttemptID, message));
LOG.warn(message);
} finally {
- timerTask.cancel();
- if (Thread.interrupted()) {
- LOG.info("Stop-container for " + event.getContainerID()
- + " got interrupted.");
- // ignore the cleanup failure
- context.getEventHandler().handle(
- new TaskAttemptDiagnosticsUpdateEvent(taskAttemptID,
- "cleanup failed for container " + event.getContainerID()));
- }
if (proxy != null) {
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
}
@@ -303,8 +241,6 @@ public class ContainerLauncherImpl extends AbstractService implements
MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT,
MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT);
LOG.info("Upper limit on the thread pool size is " + this.limitOnPoolSize);
- this.nmTimeOut = conf.getInt(ContainerLauncher.MR_AM_NM_COMMAND_TIMEOUT,
- ContainerLauncher.DEFAULT_NM_COMMAND_TIMEOUT);
this.rpc = createYarnRPC(conf);
super.init(conf);
}
@@ -409,44 +345,6 @@ public class ContainerLauncherImpl extends AbstractService implements
return proxy;
}
- private static class CommandTimerTask extends TimerTask {
- private final Thread commandThread;
- protected final String message;
- private boolean cancelled = false;
-
- public CommandTimerTask(Thread thread, ContainerLauncherEvent event) {
- super();
- this.commandThread = thread;
- this.message = "Couldn't complete " + event.getType() + " on "
- + event.getContainerID() + "/" + event.getTaskAttemptID()
- + ". Interrupting and returning";
- }
-
- @Override
- public void run() {
- synchronized (this) {
- if (this.cancelled) {
- return;
- }
- LOG.warn(this.message);
- StackTraceElement[] trace = this.commandThread.getStackTrace();
- StringBuilder logMsg = new StringBuilder();
- for (int i = 0; i < trace.length; i++) {
- logMsg.append("\n\tat " + trace[i]);
- }
- LOG.info("Stack trace of the command-thread: \n" + logMsg.toString());
- this.commandThread.interrupt();
- }
- }
-
- @Override
- public boolean cancel() {
- synchronized (this) {
- this.cancelled = true;
- return super.cancel();
- }
- }
- }
/**
* Setup and start the container on remote nodemanager.
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java
new file mode 100644
index 00000000000..d1dea8f06b2
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java
@@ -0,0 +1,69 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.mapreduce.v2.app;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+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.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.junit.Test;
+
+import static org.mockito.Mockito.*;
+
+
+public class TestTaskHeartbeatHandler {
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ @Test
+ public void testTimeout() throws InterruptedException {
+ EventHandler mockHandler = mock(EventHandler.class);
+ Clock clock = new SystemClock();
+ TaskHeartbeatHandler hb = new TaskHeartbeatHandler(mockHandler, clock, 1);
+
+
+ Configuration conf = new Configuration();
+ conf.setInt(MRJobConfig.TASK_TIMEOUT, 10); //10 ms
+ conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 10); //10 ms
+
+ hb.init(conf);
+ hb.start();
+ try {
+ ApplicationId appId = BuilderUtils.newApplicationId(0l, 5);
+ JobId jobId = MRBuilderUtils.newJobId(appId, 4);
+ TaskId tid = MRBuilderUtils.newTaskId(jobId, 3, TaskType.MAP);
+ TaskAttemptId taid = MRBuilderUtils.newTaskAttemptId(tid, 2);
+ hb.register(taid);
+ Thread.sleep(100);
+ //Events only happen when the task is canceled
+ verify(mockHandler, times(2)).handle(any(Event.class));
+ } finally {
+ hb.stop();
+ }
+ }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
index cf429b021c5..fb4dafcdb0f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.app.job.impl;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -72,6 +73,37 @@ public class TestJobImpl {
JobState.ERROR, state);
}
+ @Test
+ public void testCommitJobFailsJob() {
+
+ JobImpl mockJob = mock(JobImpl.class);
+ mockJob.tasks = new HashMap();
+ OutputCommitter mockCommitter = mock(OutputCommitter.class);
+ EventHandler mockEventHandler = mock(EventHandler.class);
+ JobContext mockJobContext = mock(JobContext.class);
+
+ when(mockJob.getCommitter()).thenReturn(mockCommitter);
+ when(mockJob.getEventHandler()).thenReturn(mockEventHandler);
+ when(mockJob.getJobContext()).thenReturn(mockJobContext);
+ doNothing().when(mockJob).setFinishTime();
+ doNothing().when(mockJob).logJobHistoryFinishedEvent();
+ when(mockJob.finished(JobState.KILLED)).thenReturn(JobState.KILLED);
+ when(mockJob.finished(JobState.FAILED)).thenReturn(JobState.FAILED);
+ when(mockJob.finished(JobState.SUCCEEDED)).thenReturn(JobState.SUCCEEDED);
+
+ try {
+ doThrow(new IOException()).when(mockCommitter).commitJob(any(JobContext.class));
+ } catch (IOException e) {
+ // commitJob stubbed out, so this can't happen
+ }
+ doNothing().when(mockEventHandler).handle(any(JobHistoryEvent.class));
+ Assert.assertNotNull("checkJobCompleteSuccess incorrectly returns null " +
+ "for successful job",
+ JobImpl.checkJobCompleteSuccess(mockJob));
+ Assert.assertEquals("checkJobCompleteSuccess returns incorrect state",
+ JobState.FAILED, JobImpl.checkJobCompleteSuccess(mockJob));
+ }
+
@Test
public void testCheckJobCompleteSuccess() {
@@ -98,9 +130,7 @@ public class TestJobImpl {
"for successful job",
JobImpl.checkJobCompleteSuccess(mockJob));
Assert.assertEquals("checkJobCompleteSuccess returns incorrect state",
- JobImpl.checkJobCompleteSuccess(mockJob), JobState.SUCCEEDED);
-
-
+ JobState.SUCCEEDED, JobImpl.checkJobCompleteSuccess(mockJob));
}
@Test
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
index b0ecb5c9bf7..cde1333ec8f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.mapreduce.v2.app.launcher;
import static org.mockito.Mockito.mock;
import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.InetSocketAddress;
import java.util.Map;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.atomic.AtomicInteger;
@@ -30,6 +32,7 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
@@ -44,18 +47,39 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.api.ContainerManager;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+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.factory.providers.YarnRemoteExceptionFactoryProvider;
+import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.junit.Test;
public class TestContainerLauncher {
- static final Log LOG = LogFactory
- .getLog(TestContainerLauncher.class);
+ private static final RecordFactory recordFactory = RecordFactoryProvider
+ .getRecordFactory(null);
+ Configuration conf;
+ Server server;
+
+ static final Log LOG = LogFactory.getLog(TestContainerLauncher.class);
@Test
public void testPoolSize() throws InterruptedException {
@@ -104,10 +128,10 @@ public class TestContainerLauncher {
Assert.assertEquals(10, containerLauncher.numEventsProcessed.get());
containerLauncher.finishEventHandling = false;
for (int i = 0; i < 10; i++) {
- ContainerId containerId =
- BuilderUtils.newContainerId(appAttemptId, i + 10);
- TaskAttemptId taskAttemptId =
- MRBuilderUtils.newTaskAttemptId(taskId, i + 10);
+ ContainerId containerId = BuilderUtils.newContainerId(appAttemptId,
+ i + 10);
+ TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId,
+ i + 10);
containerLauncher.handle(new ContainerLauncherEvent(taskAttemptId,
containerId, "host" + i + ":1234", null,
ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
@@ -119,8 +143,7 @@ public class TestContainerLauncher {
// Different hosts, there should be an increase in core-thread-pool size to
// 21(11hosts+10buffer)
// Core pool size should be 21 but the live pool size should be only 11.
- containerLauncher.expectedCorePoolSize =
- 11 + ContainerLauncherImpl.INITIAL_POOL_SIZE;
+ containerLauncher.expectedCorePoolSize = 11 + ContainerLauncherImpl.INITIAL_POOL_SIZE;
containerLauncher.finishEventHandling = false;
ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 21);
TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, 21);
@@ -200,26 +223,28 @@ public class TestContainerLauncher {
@Test
public void testSlowNM() throws Exception {
- test(false);
+ test();
}
- @Test
- public void testSlowNMWithInterruptsSwallowed() throws Exception {
- test(true);
- }
+ private void test() throws Exception {
- private void test(boolean swallowInterrupts) throws Exception {
-
- MRApp app = new MRAppWithSlowNM(swallowInterrupts);
-
- Configuration conf = new Configuration();
+ conf = new Configuration();
int maxAttempts = 1;
conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, maxAttempts);
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
+ // set timeout low for the test
+ conf.setInt("yarn.rpc.nm-command-timeout", 3000);
+ conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class.getName());
+ YarnRPC rpc = YarnRPC.create(conf);
+ String bindAddr = "localhost:0";
+ InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+ server = rpc.getServer(ContainerManager.class, new DummyContainerManager(),
+ addr, conf, null, 1);
+ server.start();
- // Set low timeout for NM commands
- conf.setInt(ContainerLauncher.MR_AM_NM_COMMAND_TIMEOUT, 3000);
+ MRApp app = new MRAppWithSlowNM();
+ try {
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
@@ -231,8 +256,8 @@ public class TestContainerLauncher {
Map attempts = tasks.values().iterator()
.next().getAttempts();
- Assert.assertEquals("Num attempts is not correct", maxAttempts, attempts
- .size());
+ Assert.assertEquals("Num attempts is not correct", maxAttempts,
+ attempts.size());
TaskAttempt attempt = attempts.values().iterator().next();
app.waitForState(attempt, TaskAttemptState.ASSIGNED);
@@ -241,20 +266,18 @@ public class TestContainerLauncher {
String diagnostics = attempt.getDiagnostics().toString();
LOG.info("attempt.getDiagnostics: " + diagnostics);
- if (swallowInterrupts) {
- Assert.assertEquals("[Container launch failed for "
- + "container_0_0000_01_000000 : Start-container for "
- + "container_0_0000_01_000000 got interrupted. Returning.]",
- diagnostics);
- } else {
+
Assert.assertTrue(diagnostics.contains("Container launch failed for "
+ "container_0_0000_01_000000 : "));
- Assert.assertTrue(diagnostics
- .contains(": java.lang.InterruptedException"));
- }
+ Assert
+ .assertTrue(diagnostics
+ .contains("java.net.SocketTimeoutException: 3000 millis timeout while waiting for channel"));
+ } finally {
+ server.stop();
app.stop();
}
+ }
private final class CustomContainerLauncher extends ContainerLauncherImpl {
@@ -317,13 +340,10 @@ public class TestContainerLauncher {
}
}
- private static class MRAppWithSlowNM extends MRApp {
+ private class MRAppWithSlowNM extends MRApp {
- final boolean swallowInterrupts;
-
- public MRAppWithSlowNM(boolean swallowInterrupts) {
+ public MRAppWithSlowNM() {
super(1, 0, false, "TestContainerLauncher", true);
- this.swallowInterrupts = swallowInterrupts;
}
@Override
@@ -333,20 +353,57 @@ public class TestContainerLauncher {
protected ContainerManager getCMProxy(ContainerId containerID,
String containerManagerBindAddr, ContainerToken containerToken)
throws IOException {
- try {
- synchronized (this) {
- wait(); // Just hang the thread simulating a very slow NM.
- }
- } catch (InterruptedException e) {
- LOG.info(e);
- if (!MRAppWithSlowNM.this.swallowInterrupts) {
- throw new IOException(e);
- }
- Thread.currentThread().interrupt();
- }
- return null;
+ // make proxy connect to our local containerManager server
+ ContainerManager proxy = (ContainerManager) rpc.getProxy(
+ ContainerManager.class,
+ NetUtils.createSocketAddr("localhost:" + server.getPort()), conf);
+ return proxy;
}
};
+
};
}
-}
+
+ public class DummyContainerManager implements ContainerManager {
+
+ private ContainerStatus status = null;
+
+ @Override
+ public GetContainerStatusResponse getContainerStatus(
+ GetContainerStatusRequest request) throws YarnRemoteException {
+ GetContainerStatusResponse response = recordFactory
+ .newRecordInstance(GetContainerStatusResponse.class);
+ response.setStatus(status);
+ return response;
+ }
+
+ @Override
+ public StartContainerResponse startContainer(StartContainerRequest request)
+ throws YarnRemoteException {
+ ContainerLaunchContext container = request.getContainerLaunchContext();
+ StartContainerResponse response = recordFactory
+ .newRecordInstance(StartContainerResponse.class);
+ status = recordFactory.newRecordInstance(ContainerStatus.class);
+ try {
+ // make the thread sleep to look like its not going to respond
+ Thread.sleep(15000);
+ } catch (Exception e) {
+ LOG.error(e);
+ throw new UndeclaredThrowableException(e);
+ }
+ status.setState(ContainerState.RUNNING);
+ status.setContainerId(container.getContainerId());
+ status.setExitStatus(0);
+ return response;
+ }
+
+ @Override
+ public StopContainerResponse stopContainer(StopContainerRequest request)
+ throws YarnRemoteException {
+ Exception e = new Exception("Dummy function", new Exception(
+ "Dummy function cause"));
+ throw YarnRemoteExceptionFactoryProvider.getYarnRemoteExceptionFactory(
+ null).createYarnRemoteException(e);
+ }
+ }
+ }
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index f6fc57ba0fd..a2bfa8080f8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -351,7 +351,7 @@
600000
The number of milliseconds before a task will be
terminated if it neither reads an input, writes an output, nor
- updates its status string.
+ updates its status string. A value of 0 disables the timeout.
@@ -412,7 +412,12 @@
-Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc
The configuration variable mapred.child.ulimit can be used to control the
- maximum virtual memory of the child processes.
+ maximum virtual memory of the child processes.
+
+ Usage of -Djava.library.path can cause programs to no longer function if
+ hadoop native libraries are used. These values should instead be set as part
+ of LD_LIBRARY_PATH in the map / reduce JVM env using the mapreduce.map.env and
+ mapreduce.reduce.env config settings.
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
index 944a28ee8d3..404cfbb22cb 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
@@ -31,6 +31,7 @@ import javax.ws.rs.core.UriInfo;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
+import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
@@ -96,6 +97,7 @@ public class HsWebServices {
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobsInfo getJobs(@QueryParam("user") String userQuery,
@QueryParam("limit") String count,
+ @QueryParam("state") String stateQuery,
@QueryParam("queue") String queueQuery,
@QueryParam("startedTimeBegin") String startedBegin,
@QueryParam("startedTimeEnd") String startedEnd,
@@ -185,6 +187,13 @@ public class HsWebServices {
break;
}
+ if (stateQuery != null && !stateQuery.isEmpty()) {
+ JobState.valueOf(stateQuery);
+ if (!job.getState().toString().equalsIgnoreCase(stateQuery)) {
+ continue;
+ }
+ }
+
// can't really validate queue is a valid one since queues could change
if (queueQuery != null && !queueQuery.isEmpty()) {
if (!job.getQueueName().equals(queueQuery)) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java
index 0ee92f2d3a3..c0110dcd087 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesJobsQuery.java
@@ -32,6 +32,7 @@ import javax.ws.rs.core.MediaType;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.MockJobs;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
@@ -120,7 +121,7 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
public Job getPartialJob(JobId jobID) {
return partialJobs.get(jobID);
}
-
+
@Override
public Map getAllJobs() {
return partialJobs; // OK
@@ -195,6 +196,72 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
.contextPath("jersey-guice-filter").servletPath("/").build());
}
+ @Test
+ public void testJobsQueryStateNone() throws JSONException, Exception {
+ WebResource r = resource();
+ ClientResponse response = r.path("ws").path("v1").path("history")
+ .path("mapreduce").path("jobs").queryParam("state", JobState.KILL_WAIT.toString())
+ .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+ assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+ JSONObject json = response.getEntity(JSONObject.class);
+ assertEquals("incorrect number of elements", 1, json.length());
+ assertEquals("jobs is not null", JSONObject.NULL, json.get("jobs"));
+ }
+
+ @Test
+ public void testJobsQueryState() throws JSONException, Exception {
+ WebResource r = resource();
+ // we only create 3 jobs and it cycles through states so we should have 3 unique states
+ Map jobsMap = appContext.getAllJobs();
+ String queryState = "BOGUS";
+ JobId jid = null;
+ for (Map.Entry entry : jobsMap.entrySet()) {
+ jid = entry.getValue().getID();
+ queryState = entry.getValue().getState().toString();
+ break;
+ }
+ ClientResponse response = r.path("ws").path("v1").path("history")
+ .path("mapreduce").path("jobs").queryParam("state", queryState)
+ .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+ assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+ JSONObject json = response.getEntity(JSONObject.class);
+ assertEquals("incorrect number of elements", 1, json.length());
+ JSONObject jobs = json.getJSONObject("jobs");
+ JSONArray arr = jobs.getJSONArray("job");
+ assertEquals("incorrect number of elements", 1, arr.length());
+ JSONObject info = arr.getJSONObject(0);
+ Job job = appContext.getPartialJob(jid);
+ VerifyJobsUtils.verifyHsJobPartial(info, job);
+ }
+
+ @Test
+ public void testJobsQueryStateInvalid() throws JSONException, Exception {
+ WebResource r = resource();
+
+ ClientResponse response = r.path("ws").path("v1").path("history")
+ .path("mapreduce").path("jobs").queryParam("state", "InvalidState")
+ .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+
+ assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+ assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+ JSONObject msg = response.getEntity(JSONObject.class);
+ JSONObject exception = msg.getJSONObject("RemoteException");
+ assertEquals("incorrect number of elements", 3, exception.length());
+ String message = exception.getString("message");
+ String type = exception.getString("exception");
+ String classname = exception.getString("javaClassName");
+ WebServicesTestUtils
+ .checkStringMatch(
+ "exception message",
+ "No enum const class org.apache.hadoop.mapreduce.v2.api.records.JobState.InvalidState",
+ message);
+ WebServicesTestUtils.checkStringMatch("exception type",
+ "IllegalArgumentException", type);
+ WebServicesTestUtils.checkStringMatch("exception classname",
+ "java.lang.IllegalArgumentException", classname);
+ }
+
+
@Test
public void testJobsQueryUserNone() throws JSONException, Exception {
WebResource r = resource();
@@ -215,6 +282,8 @@ public class TestHsWebServicesJobsQuery extends JerseyTest {
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
JSONObject json = response.getEntity(JSONObject.class);
+ System.out.println(json.toString());
+
assertEquals("incorrect number of elements", 1, json.length());
JSONObject jobs = json.getJSONObject("jobs");
JSONArray arr = jobs.getJSONArray("job");
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
index 2b5ffd4f0db..429bde5f8ce 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.lib.IdentityReducer;
import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
@@ -228,6 +229,10 @@ public class TestMiniMRChildTask {
// check if X=$(tt's X var):/tmp for an old env variable inherited from
// the tt
checkEnv("PATH", path + ":/tmp", "noappend");
+
+ String jobLocalDir = job.get(MRJobConfig.JOB_LOCAL_DIR);
+ assertNotNull(MRJobConfig.JOB_LOCAL_DIR + " is null",
+ jobLocalDir);
}
public void map(WritableComparable key, Writable value,
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java
index e97accedcd2..80d5b09aa0f 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java
@@ -24,6 +24,8 @@ import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.ContainerManager;
import org.apache.hadoop.yarn.api.ContainerManagerPB;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
@@ -38,6 +40,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestP
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponsePBImpl;
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
@@ -48,12 +51,25 @@ import com.google.protobuf.ServiceException;
public class ContainerManagerPBClientImpl implements ContainerManager {
+ // Not a documented config. Only used for tests
+ static final String NM_COMMAND_TIMEOUT = YarnConfiguration.YARN_PREFIX
+ + "rpc.nm-command-timeout";
+
+ /**
+ * Maximum of 1 minute timeout for a Node to react to the command
+ */
+ static final int DEFAULT_COMMAND_TIMEOUT = 60000;
+
private ContainerManagerPB proxy;
public ContainerManagerPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, ContainerManagerPB.class, ProtobufRpcEngine.class);
+ UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
+ int expireIntvl = conf.getInt(NM_COMMAND_TIMEOUT, DEFAULT_COMMAND_TIMEOUT);
proxy = (ContainerManagerPB)RPC.getProxy(
- ContainerManagerPB.class, clientVersion, addr, conf);
+ ContainerManagerPB.class, clientVersion, addr, ugi, conf,
+ NetUtils.getDefaultSocketFactory(conf), expireIntvl);
}
public void close() {
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
new file mode 100644
index 00000000000..7db64ddeda5
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
@@ -0,0 +1,170 @@
+/**
+ * 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.yarn;
+
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.InetSocketAddress;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ContainerManager;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+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.factory.providers.YarnRemoteExceptionFactoryProvider;
+import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.junit.Test;
+
+/*
+ * Test that the container launcher rpc times out properly. This is used
+ * by both RM to launch an AM as well as an AM to launch containers.
+ */
+public class TestContainerLaunchRPC {
+
+ static final Log LOG = LogFactory.getLog(TestContainerLaunchRPC.class);
+
+ private static final String EXCEPTION_CAUSE = "java.net.SocketTimeoutException";
+ private static final RecordFactory recordFactory = RecordFactoryProvider
+ .getRecordFactory(null);
+
+ @Test
+ public void testHadoopProtoRPCTimeout() throws Exception {
+ testRPCTimeout(HadoopYarnProtoRPC.class.getName());
+ }
+
+ private void testRPCTimeout(String rpcClass) throws Exception {
+ Configuration conf = new Configuration();
+ // set timeout low for the test
+ conf.setInt("yarn.rpc.nm-command-timeout", 3000);
+
+ conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass);
+ YarnRPC rpc = YarnRPC.create(conf);
+ String bindAddr = "localhost:0";
+ InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+ Server server = rpc.getServer(ContainerManager.class,
+ new DummyContainerManager(), addr, conf, null, 1);
+ server.start();
+ try {
+
+ ContainerManager proxy = (ContainerManager) rpc.getProxy(
+ ContainerManager.class,
+ NetUtils.createSocketAddr("localhost:" + server.getPort()), conf);
+ ContainerLaunchContext containerLaunchContext = recordFactory
+ .newRecordInstance(ContainerLaunchContext.class);
+ containerLaunchContext.setUser("dummy-user");
+ ContainerId containerId = recordFactory
+ .newRecordInstance(ContainerId.class);
+ ApplicationId applicationId = recordFactory
+ .newRecordInstance(ApplicationId.class);
+ ApplicationAttemptId applicationAttemptId = recordFactory
+ .newRecordInstance(ApplicationAttemptId.class);
+ applicationId.setClusterTimestamp(0);
+ applicationId.setId(0);
+ applicationAttemptId.setApplicationId(applicationId);
+ applicationAttemptId.setAttemptId(0);
+ containerId.setApplicationAttemptId(applicationAttemptId);
+ containerId.setId(100);
+ containerLaunchContext.setContainerId(containerId);
+ containerLaunchContext.setResource(recordFactory
+ .newRecordInstance(Resource.class));
+
+ StartContainerRequest scRequest = recordFactory
+ .newRecordInstance(StartContainerRequest.class);
+ scRequest.setContainerLaunchContext(containerLaunchContext);
+ try {
+ proxy.startContainer(scRequest);
+ } catch (Exception e) {
+ LOG.info(StringUtils.stringifyException(e));
+ Assert.assertTrue("Error, exception does not contain: "
+ + EXCEPTION_CAUSE,
+ e.getCause().getMessage().contains(EXCEPTION_CAUSE));
+
+ return;
+ }
+ } finally {
+ server.stop();
+ }
+
+ Assert.fail("timeout exception should have occurred!");
+ }
+
+ public class DummyContainerManager implements ContainerManager {
+
+ private ContainerStatus status = null;
+
+ @Override
+ public GetContainerStatusResponse getContainerStatus(
+ GetContainerStatusRequest request) throws YarnRemoteException {
+ GetContainerStatusResponse response = recordFactory
+ .newRecordInstance(GetContainerStatusResponse.class);
+ response.setStatus(status);
+ return response;
+ }
+
+ @Override
+ public StartContainerResponse startContainer(StartContainerRequest request)
+ throws YarnRemoteException {
+ ContainerLaunchContext container = request.getContainerLaunchContext();
+ StartContainerResponse response = recordFactory
+ .newRecordInstance(StartContainerResponse.class);
+ status = recordFactory.newRecordInstance(ContainerStatus.class);
+ try {
+ // make the thread sleep to look like its not going to respond
+ Thread.sleep(10000);
+ } catch (Exception e) {
+ LOG.error(e);
+ throw new UndeclaredThrowableException(e);
+ }
+ status.setState(ContainerState.RUNNING);
+ status.setContainerId(container.getContainerId());
+ status.setExitStatus(0);
+ return response;
+ }
+
+ @Override
+ public StopContainerResponse stopContainer(StopContainerRequest request)
+ throws YarnRemoteException {
+ Exception e = new Exception("Dummy function", new Exception(
+ "Dummy function cause"));
+ throw YarnRemoteExceptionFactoryProvider.getYarnRemoteExceptionFactory(
+ null).createYarnRemoteException(e);
+ }
+ }
+}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
index 28f1247bb32..be515377637 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
@@ -172,7 +172,8 @@ public class LinuxContainerExecutor extends ContainerExecutor {
int exitCode = shExec.getExitCode();
LOG.warn("Exit code from container is : " + exitCode);
logOutput(shExec.getOutput());
- throw new IOException("App initialization failed (" + exitCode + ")", e);
+ throw new IOException("App initialization failed (" + exitCode +
+ ") with output: " + shExec.getOutput(), e);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 2bfda630202..110bf8d17a9 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -100,8 +100,8 @@ class CapacitySchedulerPage extends RmView {
@Override
public void render(Block html) {
ArrayList subQueues =
- (csqinfo.qinfo == null) ? csqinfo.csinfo.getSubQueues()
- : csqinfo.qinfo.getSubQueues();
+ (csqinfo.qinfo == null) ? csqinfo.csinfo.getQueues().getQueueInfoList()
+ : csqinfo.qinfo.getQueues().getQueueInfoList();
UL ul = html.ul("#pq");
for (CapacitySchedulerQueueInfo info : subQueues) {
float used = info.getUsedCapacity() / 100;
@@ -122,7 +122,7 @@ class CapacitySchedulerPage extends RmView {
_(join(percent(used), " used"))._();
csqinfo.qinfo = info;
- if (info.getSubQueues() == null) {
+ if (info.getQueues() == null) {
li.ul("#lq").li()._(LeafQueueInfoBlock.class)._()._();
} else {
li._(QueueBlock.class);
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
index edeb1bab04a..c7ce9c87e44 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfoList;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FifoSchedulerInfo;
@@ -57,7 +58,7 @@ public class JAXBContextResolver implements ContextResolver {
SchedulerTypeInfo.class, NodeInfo.class, UserMetricsInfo.class,
CapacitySchedulerInfo.class, ClusterMetricsInfo.class,
SchedulerInfo.class, AppsInfo.class, NodesInfo.class,
- RemoteExceptionData.class};
+ RemoteExceptionData.class, CapacitySchedulerQueueInfoList.class};
public JAXBContextResolver() throws Exception {
this.types = new HashSet(Arrays.asList(cTypes));
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index d745446b46c..eafe94b1800 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -36,6 +36,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -225,6 +226,7 @@ public class RMWebServices {
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public AppsInfo getApps(@Context HttpServletRequest hsr,
@QueryParam("state") String stateQuery,
+ @QueryParam("finalStatus") String finalStatusQuery,
@QueryParam("user") String userQuery,
@QueryParam("queue") String queueQuery,
@QueryParam("limit") String count,
@@ -294,19 +296,25 @@ public class RMWebServices {
.getRMApps();
AppsInfo allApps = new AppsInfo();
for (RMApp rmapp : apps.values()) {
+
if (checkCount && num == countNum) {
break;
}
- AppInfo app = new AppInfo(rmapp, hasAccess(rmapp, hsr));
-
if (stateQuery != null && !stateQuery.isEmpty()) {
RMAppState.valueOf(stateQuery);
- if (!app.getState().equalsIgnoreCase(stateQuery)) {
+ if (!rmapp.getState().toString().equalsIgnoreCase(stateQuery)) {
+ continue;
+ }
+ }
+ if (finalStatusQuery != null && !finalStatusQuery.isEmpty()) {
+ FinalApplicationStatus.valueOf(finalStatusQuery);
+ if (!rmapp.getFinalApplicationStatus().toString()
+ .equalsIgnoreCase(finalStatusQuery)) {
continue;
}
}
if (userQuery != null && !userQuery.isEmpty()) {
- if (!app.getUser().equals(userQuery)) {
+ if (!rmapp.getUser().equals(userQuery)) {
continue;
}
}
@@ -321,19 +329,20 @@ public class RMWebServices {
throw new BadRequestException(e.getMessage());
}
}
- if (!app.getQueue().equals(queueQuery)) {
+ if (!rmapp.getQueue().equals(queueQuery)) {
continue;
}
}
if (checkStart
- && (app.getStartTime() < sBegin || app.getStartTime() > sEnd)) {
+ && (rmapp.getStartTime() < sBegin || rmapp.getStartTime() > sEnd)) {
continue;
}
if (checkEnd
- && (app.getFinishTime() < fBegin || app.getFinishTime() > fEnd)) {
+ && (rmapp.getFinishTime() < fBegin || rmapp.getFinishTime() > fEnd)) {
continue;
}
+ AppInfo app = new AppInfo(rmapp, hasAccess(rmapp, hsr));
allApps.add(app);
num++;
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
index f0a34d405ad..5f323427447 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
-import java.util.ArrayList;
-
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
@@ -38,7 +36,7 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
protected float usedCapacity;
protected float maxCapacity;
protected String queueName;
- protected ArrayList queues;
+ protected CapacitySchedulerQueueInfoList queues;
@XmlTransient
static final float EPSILON = 1e-8f;
@@ -74,22 +72,22 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
return this.queueName;
}
- public ArrayList getSubQueues() {
+ public CapacitySchedulerQueueInfoList getQueues() {
return this.queues;
}
- protected ArrayList getQueues(CSQueue parent) {
+ protected CapacitySchedulerQueueInfoList getQueues(CSQueue parent) {
CSQueue parentQueue = parent;
- ArrayList queuesInfo = new ArrayList();
+ CapacitySchedulerQueueInfoList queuesInfo = new CapacitySchedulerQueueInfoList();
for (CSQueue queue : parentQueue.getChildQueues()) {
CapacitySchedulerQueueInfo info;
if (queue instanceof LeafQueue) {
info = new CapacitySchedulerLeafQueueInfo((LeafQueue)queue);
} else {
info = new CapacitySchedulerQueueInfo(queue);
- info.subQueues = getQueues(queue);
+ info.queues = getQueues(queue);
}
- queuesInfo.add(info);
+ queuesInfo.addToQueueInfoList(info);
}
return queuesInfo;
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
index e3e81500105..6757227c63a 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
@@ -17,8 +17,6 @@
*/
package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
-import java.util.ArrayList;
-
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
@@ -49,7 +47,7 @@ public class CapacitySchedulerQueueInfo {
protected String usedResources;
protected String queueName;
protected QueueState state;
- protected ArrayList subQueues;
+ protected CapacitySchedulerQueueInfoList queues;
CapacitySchedulerQueueInfo() {
};
@@ -117,8 +115,8 @@ public class CapacitySchedulerQueueInfo {
return this.queuePath;
}
- public ArrayList getSubQueues() {
- return this.subQueues;
+ public CapacitySchedulerQueueInfoList getQueues() {
+ return this.queues;
}
/**
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfoList.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfoList.java
new file mode 100644
index 00000000000..f28926c34f6
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfoList.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class CapacitySchedulerQueueInfoList {
+ protected ArrayList queue;
+
+ public CapacitySchedulerQueueInfoList() {
+ queue = new ArrayList();
+ }
+
+ public ArrayList getQueueInfoList() {
+ return this.queue;
+ }
+
+ public boolean addToQueueInfoList(CapacitySchedulerQueueInfo e) {
+ return this.queue.add(e);
+ }
+
+ public CapacitySchedulerQueueInfo getQueueInfo(int i) {
+ return this.queue.get(i);
+ }
+}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
index 2fbf3fdd864..3a30b928873 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
@@ -30,6 +30,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -280,6 +281,85 @@ public class TestRMWebServicesApps extends JerseyTest {
}
}
+ @Test
+ public void testAppsQueryFinalStatus() throws JSONException, Exception {
+ rm.start();
+ MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
+ RMApp app1 = rm.submitApp(1024);
+ amNodeManager.nodeHeartbeat(true);
+ WebResource r = resource();
+
+ ClientResponse response = r.path("ws").path("v1").path("cluster")
+ .path("apps").queryParam("finalStatus", FinalApplicationStatus.UNDEFINED.toString())
+ .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+ assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+ JSONObject json = response.getEntity(JSONObject.class);
+ assertEquals("incorrect number of elements", 1, json.length());
+ System.out.println(json.toString());
+ JSONObject apps = json.getJSONObject("apps");
+ assertEquals("incorrect number of elements", 1, apps.length());
+ JSONArray array = apps.getJSONArray("app");
+ assertEquals("incorrect number of elements", 1, array.length());
+ verifyAppInfo(array.getJSONObject(0), app1);
+ rm.stop();
+ }
+
+ @Test
+ public void testAppsQueryFinalStatusNone() throws JSONException, Exception {
+ rm.start();
+ MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
+ rm.submitApp(1024);
+ amNodeManager.nodeHeartbeat(true);
+ WebResource r = resource();
+
+ ClientResponse response = r.path("ws").path("v1").path("cluster")
+ .path("apps").queryParam("finalStatus", FinalApplicationStatus.KILLED.toString())
+ .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+ assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+ JSONObject json = response.getEntity(JSONObject.class);
+ assertEquals("incorrect number of elements", 1, json.length());
+ assertEquals("apps is not null", JSONObject.NULL, json.get("apps"));
+ rm.stop();
+ }
+
+ @Test
+ public void testAppsQueryFinalStatusInvalid() throws JSONException, Exception {
+ rm.start();
+ MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
+ rm.submitApp(1024);
+ amNodeManager.nodeHeartbeat(true);
+ WebResource r = resource();
+
+ try {
+ r.path("ws").path("v1").path("cluster").path("apps")
+ .queryParam("finalStatus", "INVALID_test")
+ .accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
+ fail("should have thrown exception on invalid state query");
+ } catch (UniformInterfaceException ue) {
+ ClientResponse response = ue.getResponse();
+ assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
+ assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+ JSONObject msg = response.getEntity(JSONObject.class);
+ JSONObject exception = msg.getJSONObject("RemoteException");
+ assertEquals("incorrect number of elements", 3, exception.length());
+ String message = exception.getString("message");
+ String type = exception.getString("exception");
+ String classname = exception.getString("javaClassName");
+ WebServicesTestUtils
+ .checkStringMatch(
+ "exception message",
+ "No enum const class org.apache.hadoop.yarn.api.records.FinalApplicationStatus.INVALID_test",
+ message);
+ WebServicesTestUtils.checkStringMatch("exception type",
+ "IllegalArgumentException", type);
+ WebServicesTestUtils.checkStringMatch("exception classname",
+ "java.lang.IllegalArgumentException", classname);
+
+ } finally {
+ rm.stop();
+ }
+ }
+
@Test
public void testAppsQueryUser() throws JSONException, Exception {
rm.start();
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
index f3275ea0148..b8dc072ba37 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
@@ -151,7 +151,12 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
conf.setUserLimitFactor(B2, 100.0f);
conf.setCapacity(B3, 20);
conf.setUserLimitFactor(B3, 100.0f);
-
+
+ conf.setQueues(A1, new String[] {"a1a", "a1b"});
+ final String A1A = A1 + ".a1a";
+ conf.setCapacity(A1A, 85);
+ final String A1B = A1 + ".a1b";
+ conf.setCapacity(A1B, 15);
}
@Before
@@ -232,12 +237,18 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
WebServicesTestUtils.getXmlFloat(element, "maxCapacity"),
WebServicesTestUtils.getXmlString(element, "queueName"));
- NodeList queues = element.getElementsByTagName("queues");
- for (int j = 0; j < queues.getLength(); j++) {
- Element qElem = (Element) queues.item(j);
- String qName = WebServicesTestUtils.getXmlString(qElem, "queueName");
- String q = CapacitySchedulerConfiguration.ROOT + "." + qName;
- verifySubQueueXML(qElem, q, 100, 100);
+ NodeList children = element.getChildNodes();
+ for (int j = 0; j < children.getLength(); j++) {
+ Element qElem = (Element) children.item(j);
+ if(qElem.getTagName().equals("queues")) {
+ NodeList qListInfos = qElem.getChildNodes();
+ for (int k = 0; k < qListInfos.getLength(); k++) {
+ Element qElem2 = (Element) qListInfos.item(k);
+ String qName2 = WebServicesTestUtils.getXmlString(qElem2, "queueName");
+ String q2 = CapacitySchedulerConfiguration.ROOT + "." + qName2;
+ verifySubQueueXML(qElem2, q2, 100, 100);
+ }
+ }
}
}
}
@@ -245,8 +256,18 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
public void verifySubQueueXML(Element qElem, String q,
float parentAbsCapacity, float parentAbsMaxCapacity)
throws Exception {
- NodeList queues = qElem.getElementsByTagName("subQueues");
- QueueInfo qi = (queues != null) ? new QueueInfo() : new LeafQueueInfo();
+ NodeList children = qElem.getChildNodes();
+ boolean hasSubQueues = false;
+ for (int j = 0; j < children.getLength(); j++) {
+ Element qElem2 = (Element) children.item(j);
+ if(qElem2.getTagName().equals("queues")) {
+ NodeList qListInfos = qElem2.getChildNodes();
+ if (qListInfos.getLength() > 0) {
+ hasSubQueues = true;
+ }
+ }
+ }
+ QueueInfo qi = (hasSubQueues) ? new QueueInfo() : new LeafQueueInfo();
qi.capacity = WebServicesTestUtils.getXmlFloat(qElem, "capacity");
qi.usedCapacity =
WebServicesTestUtils.getXmlFloat(qElem, "usedCapacity");
@@ -263,14 +284,18 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
qi.queueName = WebServicesTestUtils.getXmlString(qElem, "queueName");
qi.state = WebServicesTestUtils.getXmlString(qElem, "state");
verifySubQueueGeneric(q, qi, parentAbsCapacity, parentAbsMaxCapacity);
-
- if (queues != null) {
- for (int j = 0; j < queues.getLength(); j++) {
- Element subqElem = (Element) queues.item(j);
- String qName = WebServicesTestUtils.getXmlString(subqElem, "queueName");
- String q2 = q + "." + qName;
- verifySubQueueXML(subqElem, q2,
- qi.absoluteCapacity, qi.absoluteMaxCapacity);
+ if (hasSubQueues) {
+ for (int j = 0; j < children.getLength(); j++) {
+ Element qElem2 = (Element) children.item(j);
+ if(qElem2.getTagName().equals("queues")) {
+ NodeList qListInfos = qElem2.getChildNodes();
+ for (int k = 0; k < qListInfos.getLength(); k++) {
+ Element qElem3 = (Element) qListInfos.item(k);
+ String qName3 = WebServicesTestUtils.getXmlString(qElem3, "queueName");
+ String q3 = q + "." + qName3;
+ verifySubQueueXML(qElem3, q3, qi.absoluteCapacity, qi.absoluteMaxCapacity);
+ }
+ }
}
} else {
LeafQueueInfo lqi = (LeafQueueInfo) qi;
@@ -307,7 +332,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
(float) info.getDouble("capacity"),
(float) info.getDouble("maxCapacity"), info.getString("queueName"));
- JSONArray arr = info.getJSONArray("queues");
+ JSONArray arr = info.getJSONObject("queues").getJSONArray("queue");
assertEquals("incorrect number of elements", 2, arr.length());
// test subqueues
@@ -333,7 +358,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
throws JSONException, Exception {
int numExpectedElements = 11;
boolean isParentQueue = true;
- if (!info.has("subQueues")) {
+ if (!info.has("queues")) {
numExpectedElements = 20;
isParentQueue = false;
}
@@ -354,7 +379,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
verifySubQueueGeneric(q, qi, parentAbsCapacity, parentAbsMaxCapacity);
if (isParentQueue) {
- JSONArray arr = info.getJSONArray("subQueues");
+ JSONArray arr = info.getJSONObject("queues").getJSONArray("queue");
// test subqueues
for (int i = 0; i < arr.length(); i++) {
JSONObject obj = arr.getJSONObject(i);
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
index 657b466c299..93f6d0d2525 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
@@ -37,9 +37,9 @@ import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.httpclient.Header;
+import org.apache.commons.httpclient.HostConfiguration;
import org.apache.commons.httpclient.HttpClient;
import org.apache.commons.httpclient.HttpMethod;
-import org.apache.commons.httpclient.HostConfiguration;
import org.apache.commons.httpclient.cookie.CookiePolicy;
import org.apache.commons.httpclient.methods.GetMethod;
import org.apache.commons.httpclient.params.HttpClientParams;
@@ -260,7 +260,24 @@ public class WebAppProxyServlet extends HttpServlet {
URI trackingUri = ProxyUriUtils.getUriFromAMUrl(
applicationReport.getOriginalTrackingUrl());
if(applicationReport.getOriginalTrackingUrl().equals("N/A")) {
- notFound(resp, "The MRAppMaster died before writing anything.");
+ String message;
+ switch(applicationReport.getFinalApplicationStatus()) {
+ case FAILED:
+ case KILLED:
+ case SUCCEEDED:
+ message =
+ "The requested application exited before setting a tracking URL.";
+ break;
+ case UNDEFINED:
+ message = "The requested application does not appear to be running "
+ +"yet, and has not set a tracking URL.";
+ break;
+ default:
+ //This should never happen, but just to be safe
+ message = "The requested application has not set a tracking URL.";
+ break;
+ }
+ notFound(resp, message);
return;
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HistoryServerRest.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HistoryServerRest.apt.vm
index 558b2c35ad9..de51c8ad3fb 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HistoryServerRest.apt.vm
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HistoryServerRest.apt.vm
@@ -149,6 +149,7 @@ History Server REST API's.
------
* user - user name
+ * state - the job state
* queue - queue name
* limit - total number of app objects to be returned
* startedTimeBegin - jobs with start time beginning with this time, specified in ms since epoch
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm
index ce489692fe0..977bd52b684 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm
@@ -324,7 +324,7 @@ ResourceManager REST API's.
| queues | array of queues(JSON)/zero or more queue objects(XML) | A collection of queue resources|
*---------------+--------------+-------------------------------+
-** Elements of the queues/subQueues object for a Parent queue
+** Elements of the queues object for a Parent queue
*---------------+--------------+-------------------------------+
|| Item || Data Type || Description |
@@ -349,10 +349,10 @@ ResourceManager REST API's.
*---------------+--------------+-------------------------------+
| state | string of QueueState | The state of the queue |
*---------------+--------------+-------------------------------+
-| subQueues | array of queues(JSON)/zero or more queue objects(XML) | A collection of sub-queue information|
+| queues | array of queues(JSON)/zero or more queue objects(XML) | A collection of sub-queue information|
*---------------+--------------+-------------------------------+
-** Elements of the queues/subQueues object for a Leaf queue - contains all elements in parent plus the following:
+** Elements of the queues object for a Leaf queue - contains all elements in parent plus the following:
*---------------+--------------+-------------------------------+
|| Item || Data Type || Description |
@@ -406,168 +406,190 @@ ResourceManager REST API's.
"queueName" : "root",
"maxCapacity" : 100,
"type" : "capacityScheduler",
- "queues" : [
- {
- "numPendingApplications" : 0,
- "queueName" : "default",
- "userLimitFactor" : 1,
- "maxApplications" : 7000,
- "usedCapacity" : 0,
- "numContainers" : 0,
- "state" : "RUNNING",
- "maxCapacity" : 90,
- "numApplications" : 0,
- "usedResources" : "memory: 0",
- "absoluteMaxCapacity" : 90,
- "maxActiveApplications" : 1,
- "numActiveApplications" : 0,
- "absoluteUsedCapacity" : 0,
- "userLimit" : 100,
- "absoluteCapacity" : 70,
- "maxActiveApplicationsPerUser" : 1,
- "capacity" : 70,
- "type" : "capacitySchedulerLeafQueueInfo",
- "maxApplicationsPerUser" : 7000
- },
- {
- "queueName" : "test",
- "absoluteUsedCapacity" : 0,
- "absoluteCapacity" : 20,
- "usedCapacity" : 0,
- "capacity" : 20,
- "subQueues" : [
- {
- "numPendingApplications" : 0,
- "queueName" : "a1",
- "userLimitFactor" : 1,
- "maxApplications" : 1200,
- "usedCapacity" : 0,
- "numContainers" : 0,
- "state" : "RUNNING",
- "maxCapacity" : 80,
- "numApplications" : 0,
- "usedResources" : "memory: 0",
- "absoluteMaxCapacity" : 16.000002,
- "maxActiveApplications" : 1,
- "numActiveApplications" : 0,
- "absoluteUsedCapacity" : 0,
- "userLimit" : 100,
- "absoluteCapacity" : 12,
- "maxActiveApplicationsPerUser" : 1,
- "capacity" : 60.000004,
- "type" : "capacitySchedulerLeafQueueInfo",
- "maxApplicationsPerUser" : 1200
+ "queues" : {
+ "queue" : [
+ {
+ "numPendingApplications" : 0,
+ "queueName" : "default",
+ "userLimitFactor" : 1,
+ "maxApplications" : 1,
+ "usedCapacity" : 0,
+ "numContainers" : 0,
+ "state" : "RUNNING",
+ "maxCapacity" : 90,
+ "numApplications" : 0,
+ "usedResources" : "memory: 0",
+ "absoluteMaxCapacity" : 90,
+ "maxActiveApplications" : 1,
+ "numActiveApplications" : 0,
+ "absoluteUsedCapacity" : 0,
+ "userLimit" : 100,
+ "absoluteCapacity" : 70,
+ "maxActiveApplicationsPerUser" : 1,
+ "capacity" : 70,
+ "type" : "capacitySchedulerLeafQueueInfo",
+ "maxApplicationsPerUser" : 1
+ },
+ {
+ "queueName" : "test",
+ "absoluteCapacity" : 20,
+ "usedCapacity" : 0,
+ "capacity" : 20,
+ "state" : "RUNNING",
+ "maxCapacity" : 100,
+ "numApplications" : 0,
+ "usedResources" : "memory: 0",
+ "absoluteMaxCapacity" : 100,
+ "queues" : {
+ "queue" : [
+ {
+ "queueName" : "a1",
+ "absoluteCapacity" : 12,
+ "usedCapacity" : 0,
+ "capacity" : 60.000004,
+ "state" : "RUNNING",
+ "maxCapacity" : 100,
+ "numApplications" : 0,
+ "usedResources" : "memory: 0",
+ "absoluteMaxCapacity" : 100,
+ "queues" : {
+ "queue" : [
+ {
+ "numPendingApplications" : 0,
+ "queueName" : "a11",
+ "userLimitFactor" : 1,
+ "maxApplications" : 0,
+ "usedCapacity" : 0,
+ "numContainers" : 0,
+ "state" : "RUNNING",
+ "maxCapacity" : 100,
+ "numApplications" : 0,
+ "usedResources" : "memory: 0",
+ "absoluteMaxCapacity" : 100,
+ "maxActiveApplications" : 1,
+ "numActiveApplications" : 0,
+ "absoluteUsedCapacity" : 0,
+ "userLimit" : 100,
+ "absoluteCapacity" : 10.200001,
+ "maxActiveApplicationsPerUser" : 1,
+ "capacity" : 85,
+ "type" : "capacitySchedulerLeafQueueInfo",
+ "maxApplicationsPerUser" : 0
+ },
+ {
+ "numPendingApplications" : 0,
+ "queueName" : "a12",
+ "userLimitFactor" : 1,
+ "maxApplications" : 0,
+ "usedCapacity" : 0,
+ "numContainers" : 0,
+ "state" : "RUNNING",
+ "maxCapacity" : 100,
+ "numApplications" : 0,
+ "usedResources" : "memory: 0",
+ "absoluteMaxCapacity" : 100,
+ "maxActiveApplications" : 1,
+ "numActiveApplications" : 0,
+ "absoluteUsedCapacity" : 0,
+ "userLimit" : 100,
+ "absoluteCapacity" : 1.8000001,
+ "maxActiveApplicationsPerUser" : 1,
+ "capacity" : 15.000001,
+ "type" : "capacitySchedulerLeafQueueInfo",
+ "maxApplicationsPerUser" : 0
+ }
+ ]
+ },
+ "absoluteUsedCapacity" : 0
+ },
+ {
+ "numPendingApplications" : 0,
+ "queueName" : "a2",
+ "userLimitFactor" : 1,
+ "maxApplications" : 0,
+ "usedCapacity" : 0,
+ "numContainers" : 0,
+ "state" : "RUNNING",
+ "maxCapacity" : 100,
+ "numApplications" : 0,
+ "usedResources" : "memory: 0",
+ "absoluteMaxCapacity" : 100,
+ "maxActiveApplications" : 1,
+ "numActiveApplications" : 0,
+ "absoluteUsedCapacity" : 0,
+ "userLimit" : 100,
+ "absoluteCapacity" : 8.000001,
+ "maxActiveApplicationsPerUser" : 1,
+ "capacity" : 40,
+ "type" : "capacitySchedulerLeafQueueInfo",
+ "maxApplicationsPerUser" : 0
+ }
+ ]
},
- {
- "numPendingApplications" : 0,
- "queueName" : "a2",
- "userLimitFactor" : 1,
- "maxApplications" : 800,
- "usedCapacity" : 0,
- "numContainers" : 0,
- "state" : "RUNNING",
- "maxCapacity" : 100,
- "numApplications" : 0,
- "usedResources" : "memory: 0",
- "absoluteMaxCapacity" : 100,
- "maxActiveApplications" : 1,
- "numActiveApplications" : 0,
- "absoluteUsedCapacity" : 0,
- "userLimit" : 100,
- "absoluteCapacity" : 8.000001,
- "maxActiveApplicationsPerUser" : 1,
- "capacity" : 40,
- "type" : "capacitySchedulerLeafQueueInfo",
- "maxApplicationsPerUser" : 800
- }
- ],
- "state" : "RUNNING",
- "maxCapacity" : 80,
- "numApplications" : 0,
- "usedResources" : "memory: 0",
- "absoluteMaxCapacity" : 80
- },
- {
- "queueName" : "test2",
- "absoluteUsedCapacity" : 0,
- "absoluteCapacity" : 10,
- "usedCapacity" : 0,
- "capacity" : 10,
- "subQueues" : [
- {
- "numPendingApplications" : 0,
- "queueName" : "a5",
- "userLimitFactor" : 1,
- "maxApplications" : 500,
- "usedCapacity" : 0,
- "numContainers" : 0,
- "state" : "RUNNING",
- "maxCapacity" : 100,
- "numApplications" : 0,
- "usedResources" : "memory: 0",
- "absoluteMaxCapacity" : 100,
- "maxActiveApplications" : 1,
- "numActiveApplications" : 0,
- "absoluteUsedCapacity" : 0,
- "userLimit" : 100,
- "absoluteCapacity" : 5,
- "maxActiveApplicationsPerUser" : 1,
- "capacity" : 50,
- "type" : "capacitySchedulerLeafQueueInfo",
- "maxApplicationsPerUser" : 500
+ "absoluteUsedCapacity" : 0
+ },
+ {
+ "queueName" : "test2",
+ "absoluteCapacity" : 10,
+ "usedCapacity" : 0,
+ "capacity" : 10,
+ "state" : "RUNNING",
+ "maxCapacity" : 15.000001,
+ "numApplications" : 0,
+ "usedResources" : "memory: 0",
+ "absoluteMaxCapacity" : 15.000001,
+ "queues" : {
+ "queue" : [
+ {
+ "numPendingApplications" : 0,
+ "queueName" : "a3",
+ "userLimitFactor" : 1,
+ "maxApplications" : 0,
+ "usedCapacity" : 0,
+ "numContainers" : 0,
+ "state" : "RUNNING",
+ "maxCapacity" : 100,
+ "numApplications" : 0,
+ "usedResources" : "memory: 0",
+ "absoluteMaxCapacity" : 15.000001,
+ "maxActiveApplications" : 1,
+ "numActiveApplications" : 0,
+ "absoluteUsedCapacity" : 0,
+ "userLimit" : 100,
+ "absoluteCapacity" : 9,
+ "maxActiveApplicationsPerUser" : 1,
+ "capacity" : 90,
+ "type" : "capacitySchedulerLeafQueueInfo",
+ "maxApplicationsPerUser" : 0
+ },
+ {
+ "numPendingApplications" : 0,
+ "queueName" : "a4",
+ "userLimitFactor" : 1,
+ "maxApplications" : 0,
+ "usedCapacity" : 0,
+ "numContainers" : 0,
+ "state" : "RUNNING",
+ "maxCapacity" : 100,
+ "numApplications" : 0,
+ "usedResources" : "memory: 0",
+ "absoluteMaxCapacity" : 15.000001,
+ "maxActiveApplications" : 1,
+ "numActiveApplications" : 0,
+ "absoluteUsedCapacity" : 0,
+ "userLimit" : 100,
+ "absoluteCapacity" : 1.0000001,
+ "maxActiveApplicationsPerUser" : 1,
+ "capacity" : 10,
+ "type" : "capacitySchedulerLeafQueueInfo",
+ "maxApplicationsPerUser" : 0
+ }
+ ]
},
- {
- "numPendingApplications" : 0,
- "queueName" : "a3",
- "userLimitFactor" : 1,
- "maxApplications" : 400,
- "usedCapacity" : 0,
- "numContainers" : 0,
- "state" : "RUNNING",
- "maxCapacity" : 100,
- "numApplications" : 0,
- "usedResources" : "memory: 0",
- "absoluteMaxCapacity" : 100,
- "maxActiveApplications" : 1,
- "numActiveApplications" : 0,
- "absoluteUsedCapacity" : 0,
- "userLimit" : 100,
- "absoluteCapacity" : 4.0000005,
- "maxActiveApplicationsPerUser" : 1,
- "capacity" : 40,
- "type" : "capacitySchedulerLeafQueueInfo",
- "maxApplicationsPerUser" : 400
- },
- {
- "numPendingApplications" : 0,
- "queueName" : "a4",
- "userLimitFactor" : 1,
- "maxApplications" : 100,
- "usedCapacity" : 0,
- "numContainers" : 0,
- "state" : "RUNNING",
- "maxCapacity" : 100,
- "numApplications" : 0,
- "usedResources" : "memory: 0",
- "absoluteMaxCapacity" : 100,
- "maxActiveApplications" : 1,
- "numActiveApplications" : 0,
- "absoluteUsedCapacity" : 0,
- "userLimit" : 100,
- "absoluteCapacity" : 1.0000001,
- "maxActiveApplicationsPerUser" : 1,
- "capacity" : 10,
- "type" : "capacitySchedulerLeafQueueInfo",
- "maxApplicationsPerUser" : 100
- }
- ],
- "state" : "RUNNING",
- "maxCapacity" : 15.000001,
- "numApplications" : 0,
- "usedResources" : "memory: 0",
- "absoluteMaxCapacity" : 15.000001
- }
- ],
+ "absoluteUsedCapacity" : 0
+ }
+ ]
+ },
"usedCapacity" : 0,
"capacity" : 100
}
@@ -575,7 +597,7 @@ ResourceManager REST API's.
}
+---+
- <>
+ <>
HTTP Request:
@@ -603,155 +625,175 @@ ResourceManager REST API's.
0.0
100.0
root
-
- 70.0
- 0.0
- 90.0
- 70.0
- 90.0
- 0.0
- 0
- memory: 0
- default
- RUNNING
- 0
- 0
- 0
- 7000
- 7000
- 1
- 1
- 100
- 1.0
-
- 20.0
- 0.0
- 80.0
- 20.0
- 80.0
- 0.0
- 0
- memory: 0
- test
- RUNNING
-
- 60.000004
+
+ 70.0
0.0
- 80.0
- 12.0
- 16.000002
+ 90.0
+ 70.0
+ 90.0
0.0
0
memory: 0
- a1
+ default
RUNNING
0
0
0
- 1200
- 1200
+ 1
+ 1
1
1
100
1.0
-
-
- 40.0
+
+
+ 20.0
0.0
100.0
- 8.000001
+ 20.0
100.0
0.0
0
memory: 0
- a2
+ test
RUNNING
- 0
- 0
- 0
- 800
- 800
- 1
- 1
- 100
- 1.0
-
-
-
- 10.0
- 0.0
- 15.000001
- 10.0
- 15.000001
- 0.0
- 0
- memory: 0
- test2
- RUNNING
-
- 50.0
- 0.0
- 100.0
- 5.0
- 100.0
- 0.0
- 0
- memory: 0
- A4
- RUNNING
- 0
- 0
- 0
- 500
- 500
- 1
- 1
- 100
- 1.0
-
-
- 40.0
- 0.0
- 100.0
- 4.0000005
- 100.0
- 0.0
- 0
- memory: 0
- a3
- RUNNING
- 0
- 0
- 0
- 400
- 400
- 1
- 1
- 100
- 1.0
-
-
+
+
+ 60.000004
+ 0.0
+ 100.0
+ 12.0
+ 100.0
+ 0.0
+ 0
+ memory: 0
+ a1
+ RUNNING
+
+
+ 85.0
+ 0.0
+ 100.0
+ 10.200001
+ 100.0
+ 0.0
+ 0
+ memory: 0
+ a11
+ RUNNING
+ 0
+ 0
+ 0
+ 0
+ 0
+ 1
+ 1
+ 100
+ 1.0
+
+
+ 15.000001
+ 0.0
+ 100.0
+ 1.8000001
+ 100.0
+ 0.0
+ 0
+ memory: 0
+ a12
+ RUNNING
+ 0
+ 0
+ 0
+ 0
+ 0
+ 1
+ 1
+ 100
+ 1.0
+
+
+
+
+ 40.0
+ 0.0
+ 100.0
+ 8.000001
+ 100.0
+ 0.0
+ 0
+ memory: 0
+ a2
+ RUNNING
+ 0
+ 0
+ 0
+ 0
+ 0
+ 1
+ 1
+ 100
+ 1.0
+
+
+
+
10.0
0.0
- 100.0
- 1.0000001
- 100.0
+ 15.000001
+ 10.0
+ 15.000001
0.0
0
memory: 0
- a4
+ test2
RUNNING
- 0
- 0
- 0
- 100
- 100
- 1
- 1
- 100
- 1.0
-
+
+
+ 90.0
+ 0.0
+ 100.0
+ 9.0
+ 15.000001
+ 0.0
+ 0
+ memory: 0
+ a3
+ RUNNING
+ 0
+ 0
+ 0
+ 0
+ 0
+ 1
+ 1
+ 100
+ 1.0
+
+
+ 10.0
+ 0.0
+ 100.0
+ 1.0000001
+ 15.000001
+ 0.0
+ 0
+ memory: 0
+ a4
+ RUNNING
+ 0
+ 0
+ 0
+ 0
+ 0
+ 1
+ 1
+ 100
+ 1.0
+
+
+
@@ -890,6 +932,7 @@ ResourceManager REST API's.
------
* state - state of the application
+ * finalStatus - the final status of the application - reported by the application itself
* user - user name
* queue - queue name
* limit - total number of app objects to be returned
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm
index 9048740ee95..7b72322fa14 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm
@@ -779,6 +779,13 @@ Hadoop MapReduce Next Generation - Writing YARN Applications
need to increase the value of the the cluster-wide configuration variable
<<>>.
+** How do I include native libraries?
+
+
+ Setting -Djava.library.path on the command line while launching a container
+ can cause native libraries used by Hadoop to not be loaded correctly and can
+ result in errors. It is cleaner to use LD_LIBRARY_PATH instead.
+
* Useful Links
* {{{https://issues.apache.org/jira/secure/attachment/12486023/MapReduce_NextGen_Architecture.pdf}Map Reduce Next Generation Architecture}}
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestJobInProgress.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestJobInProgress.java
index ea100aab08f..6b16518e42a 100644
--- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestJobInProgress.java
+++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestJobInProgress.java
@@ -354,13 +354,9 @@ public class TestJobInProgress {
Node r2n3 = new NodeBase("/default/rack2/node3");
nt.add(r2n3);
- Node r2n4 = new NodeBase("/default/rack2/s1/node4");
- nt.add(r2n4);
-
LOG.debug("r1n1 parent: " + r1n1.getParent() + "\n" +
"r1n2 parent: " + r1n2.getParent() + "\n" +
- "r2n3 parent: " + r2n3.getParent() + "\n" +
- "r2n4 parent: " + r2n4.getParent());
+ "r2n3 parent: " + r2n3.getParent());
// Same host
assertEquals(0, JobInProgress.getMatchingLevelForNodes(r1n1, r1n1, 3));
@@ -368,8 +364,6 @@ public class TestJobInProgress {
assertEquals(1, JobInProgress.getMatchingLevelForNodes(r1n1, r1n2, 3));
// Different rack
assertEquals(2, JobInProgress.getMatchingLevelForNodes(r1n1, r2n3, 3));
- // Different rack at different depth
- assertEquals(3, JobInProgress.getMatchingLevelForNodes(r1n1, r2n4, 3));
}
}
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestTTResourceReporting.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestTTResourceReporting.java
index 821d4c6de1e..e69de29bb2d 100644
--- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestTTResourceReporting.java
+++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestTTResourceReporting.java
@@ -1,364 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.mapred;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.mapreduce.SleepJob;
-import org.apache.hadoop.mapreduce.TaskCounter;
-import org.apache.hadoop.mapreduce.util.LinuxResourceCalculatorPlugin;
-import org.apache.hadoop.mapreduce.util.ResourceCalculatorPlugin;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
-
-import junit.framework.TestCase;
-import org.junit.Test;
-import org.junit.After;
-
-/**
- * This test class tests the functionality related to configuring, reporting
- * and computing memory related parameters in a Map/Reduce cluster.
- *
- * Each test sets up a {@link MiniMRCluster} with a locally defined
- * {@link org.apache.hadoop.mapred.TaskScheduler}. This scheduler validates
- * the memory related configuration is correctly computed and reported from
- * the tasktracker in
- * {@link org.apache.hadoop.mapred.TaskScheduler#assignTasks(TaskTrackerStatus)}.
- */
-public class TestTTResourceReporting extends TestCase {
-
- static final Log LOG = LogFactory.getLog(TestTTResourceReporting.class);
-
- private MiniMRCluster miniMRCluster;
-
- /**
- * Fake scheduler to test the proper reporting of memory values by TT
- */
- public static class FakeTaskScheduler extends JobQueueTaskScheduler {
-
- private boolean hasPassed = true;
- private boolean hasDynamicValuePassed = true;
- private String message;
-
- public FakeTaskScheduler() {
- super();
- }
-
- public boolean hasTestPassed() {
- return hasPassed;
- }
-
- public boolean hasDynamicTestPassed() {
- return hasDynamicValuePassed;
- }
-
- public String getFailureMessage() {
- return message;
- }
-
- @Override
- public List assignTasks(TaskTracker taskTracker)
- throws IOException {
- TaskTrackerStatus status = taskTracker.getStatus();
- long totalVirtualMemoryOnTT =
- getConf().getLong("totalVmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
- long totalPhysicalMemoryOnTT =
- getConf().getLong("totalPmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
- long mapSlotMemorySize =
- getConf().getLong("mapSlotMemorySize", JobConf.DISABLED_MEMORY_LIMIT);
- long reduceSlotMemorySize =
- getConf()
- .getLong("reduceSlotMemorySize", JobConf.DISABLED_MEMORY_LIMIT);
- long availableVirtualMemoryOnTT =
- getConf().getLong("availableVmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
- long availablePhysicalMemoryOnTT =
- getConf().getLong("availablePmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
- long cumulativeCpuTime =
- getConf().getLong("cumulativeCpuTime", TaskTrackerStatus.UNAVAILABLE);
- long cpuFrequency =
- getConf().getLong("cpuFrequency", TaskTrackerStatus.UNAVAILABLE);
- int numProcessors =
- getConf().getInt("numProcessors", TaskTrackerStatus.UNAVAILABLE);
- float cpuUsage =
- getConf().getFloat("cpuUsage", TaskTrackerStatus.UNAVAILABLE);
-
- long reportedTotalVirtualMemoryOnTT =
- status.getResourceStatus().getTotalVirtualMemory();
- long reportedTotalPhysicalMemoryOnTT =
- status.getResourceStatus().getTotalPhysicalMemory();
- long reportedMapSlotMemorySize =
- status.getResourceStatus().getMapSlotMemorySizeOnTT();
- long reportedReduceSlotMemorySize =
- status.getResourceStatus().getReduceSlotMemorySizeOnTT();
- long reportedAvailableVirtualMemoryOnTT =
- status.getResourceStatus().getAvailabelVirtualMemory();
- long reportedAvailablePhysicalMemoryOnTT =
- status.getResourceStatus().getAvailablePhysicalMemory();
- long reportedCumulativeCpuTime =
- status.getResourceStatus().getCumulativeCpuTime();
- long reportedCpuFrequency = status.getResourceStatus().getCpuFrequency();
- int reportedNumProcessors = status.getResourceStatus().getNumProcessors();
- float reportedCpuUsage = status.getResourceStatus().getCpuUsage();
-
- message =
- "expected values : "
- + "(totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
- + "availableVirtualMemoryOnTT, availablePhysicalMemoryOnTT, "
- + "mapSlotMemSize, reduceSlotMemorySize, cumulativeCpuTime, "
- + "cpuFrequency, numProcessors, cpuUsage) = ("
- + totalVirtualMemoryOnTT + ", "
- + totalPhysicalMemoryOnTT + ","
- + availableVirtualMemoryOnTT + ", "
- + availablePhysicalMemoryOnTT + ","
- + mapSlotMemorySize + ","
- + reduceSlotMemorySize + ","
- + cumulativeCpuTime + ","
- + cpuFrequency + ","
- + numProcessors + ","
- + cpuUsage
- +")";
- message +=
- "\nreported values : "
- + "(totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
- + "availableVirtualMemoryOnTT, availablePhysicalMemoryOnTT, "
- + "reportedMapSlotMemorySize, reportedReduceSlotMemorySize, "
- + "reportedCumulativeCpuTime, reportedCpuFrequency, "
- + "reportedNumProcessors, cpuUsage) = ("
- + reportedTotalVirtualMemoryOnTT + ", "
- + reportedTotalPhysicalMemoryOnTT + ","
- + reportedAvailableVirtualMemoryOnTT + ", "
- + reportedAvailablePhysicalMemoryOnTT + ","
- + reportedMapSlotMemorySize + ","
- + reportedReduceSlotMemorySize + ","
- + reportedCumulativeCpuTime + ","
- + reportedCpuFrequency + ","
- + reportedNumProcessors + ","
- + reportedCpuUsage
- + ")";
- LOG.info(message);
- hasDynamicValuePassed = true;
- // Check task resource status in task reports
- for (TaskStatus taskStatus : status.getTaskReports()) {
- Counters counters = taskStatus.getCounters();
- // This should be zero because the initial CPU time is subtracted.
- long procCumulativeCpuTime = 0;
- long procVirtualMemorySize =
- getConf().getLong("procVirtualMemorySize", -1);
- long procPhysicalMemorySize =
- getConf().getLong("procPhysicalMemorySize", -1);
- long reportedProcCumulativeCpuTime =
- counters.findCounter(TaskCounter.CPU_MILLISECONDS).getValue();
- long reportedProcVirtualMemorySize =
- counters.findCounter(TaskCounter.VIRTUAL_MEMORY_BYTES).getValue();
- long reportedProcPhysicalMemorySize =
- counters.findCounter(TaskCounter.PHYSICAL_MEMORY_BYTES).getValue();
- String procMessage =
- "expected values : "
- + "(procCumulativeCpuTime, procVirtualMemorySize,"
- + " procPhysicalMemorySize) = ("
- + procCumulativeCpuTime + ", "
- + procVirtualMemorySize + ", "
- + procPhysicalMemorySize + ")";
- procMessage +=
- "\nreported values : "
- + "(procCumulativeCpuTime, procVirtualMemorySize,"
- + " procPhysicalMemorySize) = ("
- + reportedProcCumulativeCpuTime + ", "
- + reportedProcVirtualMemorySize + ", "
- + reportedProcPhysicalMemorySize + ")";
- LOG.info(procMessage);
- message += "\n" + procMessage;
- if (procCumulativeCpuTime != reportedProcCumulativeCpuTime ||
- procVirtualMemorySize != reportedProcVirtualMemorySize ||
- procPhysicalMemorySize != reportedProcPhysicalMemorySize) {
- hasDynamicValuePassed = false;
- }
- }
- hasPassed = true;
- if (totalVirtualMemoryOnTT != reportedTotalVirtualMemoryOnTT
- || totalPhysicalMemoryOnTT != reportedTotalPhysicalMemoryOnTT
- || mapSlotMemorySize != reportedMapSlotMemorySize
- || reduceSlotMemorySize != reportedReduceSlotMemorySize
- || numProcessors != reportedNumProcessors) {
- hasPassed = false;
- }
- // These values changes every moment on the node so it can only be
- // tested by DummyMemoryCalculatorPlugin. Need to check them separately
- if (availableVirtualMemoryOnTT != reportedAvailableVirtualMemoryOnTT
- || availablePhysicalMemoryOnTT != reportedAvailablePhysicalMemoryOnTT
- || cumulativeCpuTime != reportedCumulativeCpuTime
- || cpuFrequency != reportedCpuFrequency
- || cpuUsage != reportedCpuUsage) {
- hasDynamicValuePassed = false;
- }
- return super.assignTasks(taskTracker);
- }
- }
-
- /**
- * Test that verifies default values are configured and reported correctly.
- *
- * @throws Exception
- */
- @Test
- public void testDefaultResourceValues()
- throws Exception {
- JobConf conf = new JobConf();
- try {
- // Memory values are disabled by default.
- conf.setClass(
- org.apache.hadoop.mapreduce.server.tasktracker.TTConfig.TT_RESOURCE_CALCULATOR_PLUGIN,
- DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
- setUpCluster(conf);
- JobConf jobConf = miniMRCluster.createJobConf();
- jobConf.setClass(
- org.apache.hadoop.mapreduce.server.tasktracker.TTConfig.TT_RESOURCE_CALCULATOR_PLUGIN,
- DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
- runSleepJob(jobConf);
- verifyTestResults();
- } finally {
- tearDownCluster();
- }
- }
-
- /**
- * Test that verifies that configured values are reported correctly.
- *
- * @throws Exception
- */
- @Test
- public void testConfiguredResourceValues()
- throws Exception {
- JobConf conf = new JobConf();
- conf.setLong("totalVmemOnTT", 4 * 1024 * 1024 * 1024L);
- conf.setLong("totalPmemOnTT", 2 * 1024 * 1024 * 1024L);
- conf.setLong("mapSlotMemorySize", 1 * 512L);
- conf.setLong("reduceSlotMemorySize", 1 * 1024L);
- conf.setLong("availableVmemOnTT", 4 * 1024 * 1024 * 1024L);
- conf.setLong("availablePmemOnTT", 2 * 1024 * 1024 * 1024L);
- conf.setLong("cumulativeCpuTime", 10000L);
- conf.setLong("cpuFrequency", 2000000L);
- conf.setInt("numProcessors", 8);
- conf.setFloat("cpuUsage", 15.5F);
- conf.setLong("procCumulativeCpuTime", 1000L);
- conf.setLong("procVirtualMemorySize", 2 * 1024 * 1024 * 1024L);
- conf.setLong("procPhysicalMemorySize", 1024 * 1024 * 1024L);
-
- conf.setClass(
- org.apache.hadoop.mapreduce.server.tasktracker.TTConfig.TT_RESOURCE_CALCULATOR_PLUGIN,
- DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
- conf.setLong(DummyResourceCalculatorPlugin.MAXVMEM_TESTING_PROPERTY,
- 4 * 1024 * 1024 * 1024L);
- conf.setLong(DummyResourceCalculatorPlugin.MAXPMEM_TESTING_PROPERTY,
- 2 * 1024 * 1024 * 1024L);
- conf.setLong(MRConfig.MAPMEMORY_MB, 512L);
- conf.setLong(MRConfig.REDUCEMEMORY_MB, 1024L);
- conf.setLong(DummyResourceCalculatorPlugin.CUMULATIVE_CPU_TIME, 10000L);
- conf.setLong(DummyResourceCalculatorPlugin.CPU_FREQUENCY, 2000000L);
- conf.setInt(DummyResourceCalculatorPlugin.NUM_PROCESSORS, 8);
- conf.setFloat(DummyResourceCalculatorPlugin.CPU_USAGE, 15.5F);
- try {
- setUpCluster(conf);
- JobConf jobConf = miniMRCluster.createJobConf();
- jobConf.setMemoryForMapTask(1 * 1024L);
- jobConf.setMemoryForReduceTask(2 * 1024L);
- jobConf.setClass(
- org.apache.hadoop.mapreduce.server.tasktracker.TTConfig.TT_RESOURCE_CALCULATOR_PLUGIN,
- DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
- jobConf.setLong(DummyResourceCalculatorPlugin.PROC_CUMULATIVE_CPU_TIME, 1000L);
- jobConf.setLong(DummyResourceCalculatorPlugin.PROC_VMEM_TESTING_PROPERTY,
- 2 * 1024 * 1024 * 1024L);
- jobConf.setLong(DummyResourceCalculatorPlugin.PROC_PMEM_TESTING_PROPERTY,
- 1024 * 1024 * 1024L);
- runSleepJob(jobConf);
- verifyTestResults();
- } finally {
- tearDownCluster();
- }
- }
-
- /**
- * Test that verifies that total memory values are calculated and reported
- * correctly.
- *
- * @throws Exception
- */
- @Test
- public void testResourceValuesOnLinux()
- throws Exception {
- if (!System.getProperty("os.name").startsWith("Linux")) {
- return;
- }
-
- JobConf conf = new JobConf();
- LinuxResourceCalculatorPlugin plugin = new LinuxResourceCalculatorPlugin();
- // In this case, we only check these three fields because they are static
- conf.setLong("totalVmemOnTT", plugin.getVirtualMemorySize());
- conf.setLong("totalPmemOnTT", plugin.getPhysicalMemorySize());
- conf.setLong("numProcessors", plugin.getNumProcessors());
-
- try {
- setUpCluster(conf);
- runSleepJob(miniMRCluster.createJobConf());
- verifyTestResults(true);
- } finally {
- tearDownCluster();
- }
- }
-
- private void setUpCluster(JobConf conf)
- throws Exception {
- conf.setClass(JTConfig.JT_TASK_SCHEDULER,
- TestTTResourceReporting.FakeTaskScheduler.class, TaskScheduler.class);
- conf.set(JTConfig.JT_IPC_HANDLER_COUNT, "1");
- miniMRCluster = new MiniMRCluster(1, "file:///", 3, null, null, conf);
- }
-
- private void runSleepJob(JobConf conf) throws Exception {
- String[] args = { "-m", "1", "-r", "1",
- "-mt", "10", "-rt", "10" };
- ToolRunner.run(conf, new SleepJob(), args);
- }
-
- private void verifyTestResults() {
- verifyTestResults(false);
- }
-
- private void verifyTestResults(boolean excludeDynamic) {
- FakeTaskScheduler scheduler =
- (FakeTaskScheduler)miniMRCluster.getJobTrackerRunner().
- getJobTracker().getTaskScheduler();
- assertTrue(scheduler.getFailureMessage(), scheduler.hasTestPassed());
- if (!excludeDynamic) {
- assertTrue(scheduler.getFailureMessage(),
- scheduler.hasDynamicTestPassed());
- }
- }
-
- @After
- private void tearDownCluster() {
- if (miniMRCluster != null) {
- miniMRCluster.shutdown();
- }
- }
-}