dataDirs = DataNode.getStorageDirs(dn.getConf());
+ return "DataNode: [" +
+ StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "] " +
+ " heartbeating to " + nnAddr;
+ }
+
+ //This must be called only by blockPoolManager.
+ void stop() {
+ shouldServiceRun = false;
+ if (bpThread != null) {
+ bpThread.interrupt();
+ }
+ }
+
+ //This must be called only by blockPoolManager
+ void join() {
+ try {
+ if (bpThread != null) {
+ bpThread.join();
+ }
+ } catch (InterruptedException ie) { }
+ }
+
+ //Cleanup method to be called by current thread before exiting.
+ private synchronized void cleanUp() {
+
+ if(upgradeManager != null)
+ upgradeManager.shutdownUpgrade();
+ shouldServiceRun = false;
+ RPC.stopProxy(bpNamenode);
+ dn.shutdownBlockPool(this);
+ }
+
+ /**
+ * Main loop for each BP thread. Run until shutdown,
+ * forever calling remote NameNode functions.
+ */
+ private void offerService() throws Exception {
+ LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
+ + dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
+ + dnConf.blockReportInterval + "msec" + " Initial delay: "
+ + dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
+ + dnConf.heartBeatInterval);
+
+ //
+ // Now loop for a long time....
+ //
+ while (shouldRun()) {
+ try {
+ long startTime = now();
+
+ //
+ // Every so often, send heartbeat or block-report
+ //
+ if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
+ //
+ // All heartbeat messages include following info:
+ // -- Datanode name
+ // -- data transfer port
+ // -- Total capacity
+ // -- Bytes remaining
+ //
+ lastHeartbeat = startTime;
+ if (!dn.areHeartbeatsDisabledForTests()) {
+ DatanodeCommand[] cmds = sendHeartBeat();
+ dn.metrics.addHeartbeat(now() - startTime);
+
+ long startProcessCommands = now();
+ if (!processCommand(cmds))
+ continue;
+ long endProcessCommands = now();
+ if (endProcessCommands - startProcessCommands > 2000) {
+ LOG.info("Took " + (endProcessCommands - startProcessCommands) +
+ "ms to process " + cmds.length + " commands from NN");
+ }
+ }
+ }
+ if (pendingReceivedRequests > 0
+ || (startTime - lastDeletedReport > dnConf.deleteReportInterval)) {
+ reportReceivedDeletedBlocks();
+ lastDeletedReport = startTime;
+ }
+
+ DatanodeCommand cmd = blockReport();
+ processCommand(cmd);
+
+ // Now safe to start scanning the block pool
+ if (dn.blockScanner != null) {
+ dn.blockScanner.addBlockPool(this.getBlockPoolId());
+ }
+
+ //
+ // There is no work to do; sleep until hearbeat timer elapses,
+ // or work arrives, and then iterate again.
+ //
+ long waitTime = dnConf.heartBeatInterval -
+ (System.currentTimeMillis() - lastHeartbeat);
+ synchronized(receivedAndDeletedBlockList) {
+ if (waitTime > 0 && pendingReceivedRequests == 0) {
+ try {
+ receivedAndDeletedBlockList.wait(waitTime);
+ } catch (InterruptedException ie) {
+ LOG.warn("BPOfferService for " + this + " interrupted");
+ }
+ }
+ } // synchronized
+ } catch(RemoteException re) {
+ String reClass = re.getClassName();
+ if (UnregisteredNodeException.class.getName().equals(reClass) ||
+ DisallowedDatanodeException.class.getName().equals(reClass) ||
+ IncorrectVersionException.class.getName().equals(reClass)) {
+ LOG.warn(this + " is shutting down", re);
+ shouldServiceRun = false;
+ return;
+ }
+ LOG.warn("RemoteException in offerService", re);
+ try {
+ long sleepTime = Math.min(1000, dnConf.heartBeatInterval);
+ Thread.sleep(sleepTime);
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ }
+ } catch (IOException e) {
+ LOG.warn("IOException in offerService", e);
+ }
+ } // while (shouldRun())
+ } // offerService
+
+ /**
+ * Register one bp with the corresponding NameNode
+ *
+ * The bpDatanode needs to register with the namenode on startup in order
+ * 1) to report which storage it is serving now and
+ * 2) to receive a registrationID
+ *
+ * issued by the namenode to recognize registered datanodes.
+ *
+ * @see FSNamesystem#registerDatanode(DatanodeRegistration)
+ * @throws IOException
+ */
+ void register() throws IOException {
+ Preconditions.checkState(bpNSInfo != null,
+ "register() should be called after handshake()");
+
+ // The handshake() phase loaded the block pool storage
+ // off disk - so update the bpRegistration object from that info
+ bpRegistration = dn.createBPRegistration(bpNSInfo);
+
+ LOG.info(this + " beginning handshake with NN");
+
+ while (shouldRun()) {
+ try {
+ // Use returned registration from namenode with updated machine name.
+ bpRegistration = bpNamenode.registerDatanode(bpRegistration);
+ break;
+ } catch(SocketTimeoutException e) { // namenode is busy
+ LOG.info("Problem connecting to server: " + nnAddr);
+ sleepAndLogInterrupts(1000, "connecting to server");
+ }
+ }
+
+ LOG.info("Block pool " + this + " successfully registered with NN");
+ dn.bpRegistrationSucceeded(bpRegistration, getBlockPoolId());
+
+ // random short delay - helps scatter the BR from all DNs
+ scheduleBlockReport(dnConf.initialBlockReportDelay);
+ }
+
+
+ private void sleepAndLogInterrupts(int millis,
+ String stateString) {
+ try {
+ Thread.sleep(millis);
+ } catch (InterruptedException ie) {
+ LOG.info("BPOfferService " + this +
+ " interrupted while " + stateString);
+ }
+ }
+
+ /**
+ * No matter what kind of exception we get, keep retrying to offerService().
+ * That's the loop that connects to the NameNode and provides basic DataNode
+ * functionality.
+ *
+ * Only stop when "shouldRun" or "shouldServiceRun" is turned off, which can
+ * happen either at shutdown or due to refreshNamenodes.
+ */
+ @Override
+ public void run() {
+ LOG.info(this + " starting to offer service");
+
+ try {
+ // init stuff
+ try {
+ // setup storage
+ connectToNNAndHandshake();
+ } catch (IOException ioe) {
+ // Initial handshake, storage recovery or registration failed
+ // End BPOfferService thread
+ LOG.fatal("Initialization failed for block pool " + this, ioe);
+ return;
+ }
+
+ initialized = true; // bp is initialized;
+
+ while (shouldRun()) {
+ try {
+ startDistributedUpgradeIfNeeded();
+ offerService();
+ } catch (Exception ex) {
+ LOG.error("Exception in BPOfferService for " + this, ex);
+ sleepAndLogInterrupts(5000, "offering service");
+ }
+ }
+ } catch (Throwable ex) {
+ LOG.warn("Unexpected exception in block pool " + this, ex);
+ } finally {
+ LOG.warn("Ending block pool service for: " + this);
+ cleanUp();
+ }
+ }
+
+ private boolean shouldRun() {
+ return shouldServiceRun && dn.shouldRun();
+ }
+
+ /**
+ * Process an array of datanode commands
+ *
+ * @param cmds an array of datanode commands
+ * @return true if further processing may be required or false otherwise.
+ */
+ private boolean processCommand(DatanodeCommand[] cmds) {
+ if (cmds != null) {
+ for (DatanodeCommand cmd : cmds) {
+ try {
+ if (processCommand(cmd) == false) {
+ return false;
+ }
+ } catch (IOException ioe) {
+ LOG.warn("Error processing datanode Command", ioe);
+ }
+ }
+ }
+ return true;
+ }
+
+ /**
+ *
+ * @param cmd
+ * @return true if further processing may be required or false otherwise.
+ * @throws IOException
+ */
+ private boolean processCommand(DatanodeCommand cmd) throws IOException {
+ if (cmd == null)
+ return true;
+ final BlockCommand bcmd =
+ cmd instanceof BlockCommand? (BlockCommand)cmd: null;
+
+ switch(cmd.getAction()) {
+ case DatanodeProtocol.DNA_TRANSFER:
+ // Send a copy of a block to another datanode
+ dn.transferBlocks(bcmd.getBlockPoolId(), bcmd.getBlocks(), bcmd.getTargets());
+ dn.metrics.incrBlocksReplicated(bcmd.getBlocks().length);
+ break;
+ case DatanodeProtocol.DNA_INVALIDATE:
+ //
+ // Some local block(s) are obsolete and can be
+ // safely garbage-collected.
+ //
+ Block toDelete[] = bcmd.getBlocks();
+ try {
+ if (dn.blockScanner != null) {
+ dn.blockScanner.deleteBlocks(bcmd.getBlockPoolId(), toDelete);
+ }
+ // using global fsdataset
+ dn.data.invalidate(bcmd.getBlockPoolId(), toDelete);
+ } catch(IOException e) {
+ dn.checkDiskError();
+ throw e;
+ }
+ dn.metrics.incrBlocksRemoved(toDelete.length);
+ break;
+ case DatanodeProtocol.DNA_SHUTDOWN:
+ // shut down the data node
+ shouldServiceRun = false;
+ return false;
+ case DatanodeProtocol.DNA_REGISTER:
+ // namenode requested a registration - at start or if NN lost contact
+ LOG.info("DatanodeCommand action: DNA_REGISTER");
+ if (shouldRun()) {
+ // re-retrieve namespace info to make sure that, if the NN
+ // was restarted, we still match its version (HDFS-2120)
+ retrieveNamespaceInfo();
+ // and re-register
+ register();
+ }
+ break;
+ case DatanodeProtocol.DNA_FINALIZE:
+ String bp = ((FinalizeCommand) cmd).getBlockPoolId();
+ assert getBlockPoolId().equals(bp) :
+ "BP " + getBlockPoolId() + " received DNA_FINALIZE " +
+ "for other block pool " + bp;
+
+ dn.finalizeUpgradeForPool(bp);
+ break;
+ case UpgradeCommand.UC_ACTION_START_UPGRADE:
+ // start distributed upgrade here
+ processDistributedUpgradeCommand((UpgradeCommand)cmd);
+ break;
+ case DatanodeProtocol.DNA_RECOVERBLOCK:
+ dn.recoverBlocks(((BlockRecoveryCommand)cmd).getRecoveringBlocks());
+ break;
+ case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
+ LOG.info("DatanodeCommand action: DNA_ACCESSKEYUPDATE");
+ if (dn.isBlockTokenEnabled) {
+ dn.blockPoolTokenSecretManager.setKeys(getBlockPoolId(),
+ ((KeyUpdateCommand) cmd).getExportedKeys());
+ }
+ break;
+ case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
+ LOG.info("DatanodeCommand action: DNA_BALANCERBANDWIDTHUPDATE");
+ long bandwidth =
+ ((BalancerBandwidthCommand) cmd).getBalancerBandwidthValue();
+ if (bandwidth > 0) {
+ DataXceiverServer dxcs =
+ (DataXceiverServer) dn.dataXceiverServer.getRunnable();
+ dxcs.balanceThrottler.setBandwidth(bandwidth);
+ }
+ break;
+ default:
+ LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
+ }
+ return true;
+ }
+
+ private void processDistributedUpgradeCommand(UpgradeCommand comm)
+ throws IOException {
+ UpgradeManagerDatanode upgradeManager = getUpgradeManager();
+ upgradeManager.processUpgradeCommand(comm);
+ }
+
+ synchronized UpgradeManagerDatanode getUpgradeManager() {
+ if(upgradeManager == null)
+ upgradeManager =
+ new UpgradeManagerDatanode(dn, getBlockPoolId());
+
+ return upgradeManager;
+ }
+
+ /**
+ * Start distributed upgrade if it should be initiated by the data-node.
+ */
+ private void startDistributedUpgradeIfNeeded() throws IOException {
+ UpgradeManagerDatanode um = getUpgradeManager();
+
+ if(!um.getUpgradeState())
+ return;
+ um.setUpgradeState(false, um.getUpgradeVersion());
+ um.startUpgrade();
+ return;
+ }
+
+}
\ No newline at end of file
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 ad5c6d878a..fd8aec7bac 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
@@ -26,9 +26,11 @@
import java.io.IOException;
import java.io.RandomAccessFile;
-import org.apache.commons.httpclient.methods.GetMethod;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
/**
@@ -36,7 +38,9 @@
* This is not related to the Block related functionality in Namenode.
* The biggest part of data block metadata is CRC for the block.
*/
-class BlockMetadataHeader {
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockMetadataHeader {
static final short METADATA_VERSION = FSDataset.METADATA_VERSION;
@@ -52,12 +56,14 @@ class BlockMetadataHeader {
this.checksum = checksum;
this.version = version;
}
-
- short getVersion() {
+
+ /** Get the version */
+ public short getVersion() {
return version;
}
- DataChecksum getChecksum() {
+ /** Get the checksum */
+ public DataChecksum getChecksum() {
return checksum;
}
@@ -68,7 +74,7 @@ DataChecksum getChecksum() {
* @return Metadata Header
* @throws IOException
*/
- static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
+ public static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
return readHeader(in.readShort(), in);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
index 579eb8ed1a..a666149beb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
@@ -34,7 +34,6 @@
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
-import java.util.Random;
import java.util.TreeSet;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@@ -252,8 +251,9 @@ private synchronized long getNewBlockScanTime() {
*/
long period = Math.min(scanPeriod,
Math.max(blockMap.size(),1) * 600 * 1000L);
+ int periodInt = Math.abs((int)period);
return System.currentTimeMillis() - scanPeriod +
- DFSUtil.getRandom().nextInt((int)period);
+ DFSUtil.getRandom().nextInt(periodInt);
}
/** Adds block to list of blocks */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index 4b961522d6..61bc29acf4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -50,7 +50,6 @@
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.PureJavaCrc32;
/** A class that receives a block and writes to its own disk, meanwhile
* may copies it to another site. If a throttler is provided,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
index 535619e096..e14aaf63c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
@@ -31,7 +31,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.datanode.DataNode.BPOfferService;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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 df8a2adcca..65ccba80dc 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
@@ -37,6 +37,7 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
@@ -48,11 +49,12 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTPS_ENABLE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ENABLED_DEFAULT;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
@@ -61,7 +63,6 @@
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.net.Socket;
-import java.net.SocketTimeoutException;
import java.net.URI;
import java.net.UnknownHostException;
import java.nio.channels.ServerSocketChannel;
@@ -74,7 +75,6 @@
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
-import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -90,11 +90,11 @@
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -103,7 +103,6 @@
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
-import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -113,38 +112,28 @@
import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
import org.apache.hadoop.hdfs.server.common.JspHelper;
-import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.datanode.FSDataset.VolumeInfo;
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMethods;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
-import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
-import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
-import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.server.protocolR23Compatible.InterDatanodeProtocolServerSideTranslatorR23;
import org.apache.hadoop.hdfs.server.protocolR23Compatible.InterDatanodeProtocolTranslatorR23;
import org.apache.hadoop.hdfs.server.protocolR23Compatible.InterDatanodeWireProtocol;
@@ -160,8 +149,10 @@
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.net.DNS;
import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
@@ -176,6 +167,8 @@
import org.apache.hadoop.util.VersionInfo;
import org.mortbay.util.ajax.JSON;
+import com.google.common.base.Preconditions;
+
/**********************************************************
* DataNode is a class (and program) that stores a set of
@@ -236,8 +229,7 @@ public class DataNode extends Configured
* Use {@link NetUtils#createSocketAddr(String)} instead.
*/
@Deprecated
- public static InetSocketAddress createSocketAddr(String target
- ) throws IOException {
+ public static InetSocketAddress createSocketAddr(String target) {
return NetUtils.createSocketAddr(target);
}
@@ -331,14 +323,14 @@ public Object run() throws Exception {
}
}
- void joinAll() throws InterruptedException {
+ void joinAll() {
for (BPOfferService bpos: this.getAllNamenodeThreads()) {
bpos.join();
}
}
void refreshNamenodes(Configuration conf)
- throws IOException, InterruptedException {
+ throws IOException {
LOG.info("Refresh request received for nameservices: "
+ conf.get(DFS_FEDERATION_NAMESERVICES));
List newAddresses =
@@ -396,8 +388,6 @@ void refreshNamenodes(Configuration conf)
private volatile String hostName; // Host name of this datanode
- private static String dnThreadName;
-
boolean isBlockTokenEnabled;
BlockPoolTokenSecretManager blockPoolTokenSecretManager;
@@ -414,6 +404,8 @@ void refreshNamenodes(Configuration conf)
private AbstractList dataDirs;
private Configuration conf;
+ private final String userWithLocalPathAccess;
+
/**
* Create the DataNode given a configuration and an array of dataDirs.
* 'dataDirs' is where the blocks are stored.
@@ -432,6 +424,8 @@ void refreshNamenodes(Configuration conf)
final SecureResources resources) throws IOException {
super(conf);
+ this.userWithLocalPathAccess = conf
+ .get(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY);
try {
hostName = getHostName(conf);
startDataNode(conf, dataDirs, resources);
@@ -452,11 +446,8 @@ private synchronized void setClusterId(String cid) throws IOException {
private static String getHostName(Configuration config)
throws UnknownHostException {
- String name = null;
// use configured nameserver & interface to get local hostname
- if (config.get(DFS_DATANODE_HOST_NAME_KEY) != null) {
- name = config.get(DFS_DATANODE_HOST_NAME_KEY);
- }
+ String name = config.get(DFS_DATANODE_HOST_NAME_KEY);
if (name == null) {
name = DNS
.getDefaultHost(config.get(DFS_DATANODE_DNS_INTERFACE_KEY,
@@ -481,11 +472,11 @@ conf, new AccessControlList(conf.get(DFS_ADMIN, " ")),
if(LOG.isDebugEnabled()) {
LOG.debug("Datanode listening on " + infoHost + ":" + tmpInfoPort);
}
- if (conf.getBoolean("dfs.https.enable", false)) {
+ if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT);
InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
- "dfs.datanode.https.address", infoHost + ":" + 0));
+ DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 0));
Configuration sslConf = new HdfsConfiguration(false);
sslConf.addResource(conf.get("dfs.https.server.keystore.resource",
"ssl-server.xml"));
@@ -526,7 +517,7 @@ private void startPlugins(Configuration conf) {
private void initIpcServer(Configuration conf) throws IOException {
InetSocketAddress ipcAddr = NetUtils.createSocketAddr(
- conf.get("dfs.datanode.ipc.address"));
+ conf.get(DFS_DATANODE_IPC_ADDRESS_KEY));
// Add all the RPC protocols that the Datanode implements
ClientDatanodeProtocolServerSideTranslatorR23
@@ -692,679 +683,10 @@ void setHeartbeatsDisabledForTests(
this.heartbeatsDisabledForTests = heartbeatsDisabledForTests;
}
- /**
- * A thread per namenode to perform:
- *
- * Pre-registration handshake with namenode
- * Registration with namenode
- * Send periodic heartbeats to the namenode
- * Handle commands received from the datanode
- *
- */
- @InterfaceAudience.Private
- static class BPOfferService implements Runnable {
- final InetSocketAddress nnAddr;
- DatanodeRegistration bpRegistration;
- NamespaceInfo bpNSInfo;
- long lastBlockReport = 0;
- long lastDeletedReport = 0;
-
- boolean resetBlockReportTime = true;
-
- private Thread bpThread;
- private DatanodeProtocol bpNamenode;
- private String blockPoolId;
- private long lastHeartbeat = 0;
- private volatile boolean initialized = false;
- private final LinkedList receivedAndDeletedBlockList
- = new LinkedList();
- private volatile int pendingReceivedRequests = 0;
- private volatile boolean shouldServiceRun = true;
- UpgradeManagerDatanode upgradeManager = null;
- private final DataNode dn;
- private final DNConf dnConf;
-
- BPOfferService(InetSocketAddress nnAddr, DataNode dn) {
- this.dn = dn;
- this.bpRegistration = dn.createRegistration();
- this.nnAddr = nnAddr;
- this.dnConf = dn.getDnConf();
- }
-
- /**
- * returns true if BP thread has completed initialization of storage
- * and has registered with the corresponding namenode
- * @return true if initialized
- */
- public boolean initialized() {
- return initialized;
- }
-
- public boolean isAlive() {
- return shouldServiceRun && bpThread.isAlive();
- }
-
- public String getBlockPoolId() {
- return blockPoolId;
- }
-
- private InetSocketAddress getNNSocketAddress() {
- return nnAddr;
- }
-
- void setNamespaceInfo(NamespaceInfo nsinfo) {
- bpNSInfo = nsinfo;
- this.blockPoolId = nsinfo.getBlockPoolID();
- }
-
- void setNameNode(DatanodeProtocol dnProtocol) {
- bpNamenode = dnProtocol;
- }
-
- private NamespaceInfo handshake() throws IOException {
- NamespaceInfo nsInfo = new NamespaceInfo();
- while (dn.shouldRun && shouldServiceRun) {
- try {
- nsInfo = bpNamenode.versionRequest();
- // verify build version
- String nsVer = nsInfo.getBuildVersion();
- String stVer = Storage.getBuildVersion();
- LOG.info("handshake: namespace info = " + nsInfo);
-
- if(! nsVer.equals(stVer)) {
- String errorMsg = "Incompatible build versions: bp = " + blockPoolId +
- "namenode BV = " + nsVer + "; datanode BV = " + stVer;
- LOG.warn(errorMsg);
- bpNamenode.errorReport( bpRegistration,
- DatanodeProtocol.NOTIFY, errorMsg );
- } else {
- break;
- }
- } catch(SocketTimeoutException e) { // namenode is busy
- LOG.warn("Problem connecting to server: " + nnAddr);
- } catch(IOException e ) { // namenode is not available
- LOG.warn("Problem connecting to server: " + nnAddr);
- }
-
- // try again in a second
- try {
- Thread.sleep(5000);
- } catch (InterruptedException ie) {}
- }
-
- assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
- "Data-node and name-node layout versions must be the same."
- + "Expected: "+ HdfsConstants.LAYOUT_VERSION
- + " actual "+ nsInfo.getLayoutVersion();
- return nsInfo;
- }
-
- void setupBP(Configuration conf)
- throws IOException {
- // get NN proxy
- DatanodeProtocol dnp =
- (DatanodeProtocol)RPC.waitForProxy(DatanodeProtocol.class,
- DatanodeProtocol.versionID, nnAddr, conf);
- setNameNode(dnp);
-
- // handshake with NN
- NamespaceInfo nsInfo = handshake();
- setNamespaceInfo(nsInfo);
- dn.initBlockPool(this, nsInfo);
-
- bpRegistration.setStorageID(dn.getStorageId());
- StorageInfo storageInfo = dn.storage.getBPStorage(blockPoolId);
- if (storageInfo == null) {
- // it's null in the case of SimulatedDataSet
- bpRegistration.storageInfo.layoutVersion = HdfsConstants.LAYOUT_VERSION;
- bpRegistration.setStorageInfo(nsInfo);
- } else {
- bpRegistration.setStorageInfo(storageInfo);
- }
- }
-
- /**
- * This methods arranges for the data node to send the block report at
- * the next heartbeat.
- */
- void scheduleBlockReport(long delay) {
- if (delay > 0) { // send BR after random delay
- lastBlockReport = System.currentTimeMillis()
- - ( dnConf.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
- } else { // send at next heartbeat
- lastBlockReport = lastHeartbeat - dnConf.blockReportInterval;
- }
- resetBlockReportTime = true; // reset future BRs for randomness
- }
-
- private void reportBadBlocks(ExtendedBlock block) {
- DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) };
- LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) };
-
- try {
- bpNamenode.reportBadBlocks(blocks);
- } catch (IOException e){
- /* One common reason is that NameNode could be in safe mode.
- * Should we keep on retrying in that case?
- */
- LOG.warn("Failed to report bad block " + block + " to namenode : "
- + " Exception", e);
- }
-
- }
-
- /**
- * Report received blocks and delete hints to the Namenode
- *
- * @throws IOException
- */
- private void reportReceivedDeletedBlocks() throws IOException {
-
- // check if there are newly received blocks
- ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
- int currentReceivedRequestsCounter;
- synchronized (receivedAndDeletedBlockList) {
- currentReceivedRequestsCounter = pendingReceivedRequests;
- int numBlocks = receivedAndDeletedBlockList.size();
- if (numBlocks > 0) {
- //
- // Send newly-received and deleted blockids to namenode
- //
- receivedAndDeletedBlockArray = receivedAndDeletedBlockList
- .toArray(new ReceivedDeletedBlockInfo[numBlocks]);
- }
- }
- if (receivedAndDeletedBlockArray != null) {
- bpNamenode.blockReceivedAndDeleted(bpRegistration, blockPoolId,
- receivedAndDeletedBlockArray);
- synchronized (receivedAndDeletedBlockList) {
- for (int i = 0; i < receivedAndDeletedBlockArray.length; i++) {
- receivedAndDeletedBlockList.remove(receivedAndDeletedBlockArray[i]);
- }
- pendingReceivedRequests -= currentReceivedRequestsCounter;
- }
- }
- }
-
- /*
- * Informing the name node could take a long long time! Should we wait
- * till namenode is informed before responding with success to the
- * client? For now we don't.
- */
- void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
- if (block == null || delHint == null) {
- throw new IllegalArgumentException(block == null ? "Block is null"
- : "delHint is null");
- }
-
- if (!block.getBlockPoolId().equals(blockPoolId)) {
- LOG.warn("BlockPool mismatch " + block.getBlockPoolId() + " vs. "
- + blockPoolId);
- return;
- }
-
- synchronized (receivedAndDeletedBlockList) {
- receivedAndDeletedBlockList.add(new ReceivedDeletedBlockInfo(block
- .getLocalBlock(), delHint));
- pendingReceivedRequests++;
- receivedAndDeletedBlockList.notifyAll();
- }
- }
-
- void notifyNamenodeDeletedBlock(ExtendedBlock block) {
- if (block == null) {
- throw new IllegalArgumentException("Block is null");
- }
-
- if (!block.getBlockPoolId().equals(blockPoolId)) {
- LOG.warn("BlockPool mismatch " + block.getBlockPoolId() + " vs. "
- + blockPoolId);
- return;
- }
-
- synchronized (receivedAndDeletedBlockList) {
- receivedAndDeletedBlockList.add(new ReceivedDeletedBlockInfo(block
- .getLocalBlock(), ReceivedDeletedBlockInfo.TODELETE_HINT));
- }
- }
-
-
- /**
- * Report the list blocks to the Namenode
- * @throws IOException
- */
- DatanodeCommand blockReport() throws IOException {
- // send block report if timer has expired.
- DatanodeCommand cmd = null;
- long startTime = now();
- if (startTime - lastBlockReport > dnConf.blockReportInterval) {
-
- // Create block report
- long brCreateStartTime = now();
- BlockListAsLongs bReport = dn.data.getBlockReport(blockPoolId);
-
- // Send block report
- long brSendStartTime = now();
- cmd = bpNamenode.blockReport(bpRegistration, blockPoolId, bReport
- .getBlockListAsLongs());
-
- // Log the block report processing stats from Datanode perspective
- long brSendCost = now() - brSendStartTime;
- long brCreateCost = brSendStartTime - brCreateStartTime;
- dn.metrics.addBlockReport(brSendCost);
- LOG.info("BlockReport of " + bReport.getNumberOfBlocks()
- + " blocks took " + brCreateCost + " msec to generate and "
- + brSendCost + " msecs for RPC and NN processing");
-
- // If we have sent the first block report, then wait a random
- // time before we start the periodic block reports.
- if (resetBlockReportTime) {
- lastBlockReport = startTime - DFSUtil.getRandom().nextInt((int)(dnConf.blockReportInterval));
- resetBlockReportTime = false;
- } else {
- /* say the last block report was at 8:20:14. The current report
- * should have started around 9:20:14 (default 1 hour interval).
- * If current time is :
- * 1) normal like 9:20:18, next report should be at 10:20:14
- * 2) unexpected like 11:35:43, next report should be at 12:20:14
- */
- lastBlockReport += (now() - lastBlockReport) /
- dnConf.blockReportInterval * dnConf.blockReportInterval;
- }
- LOG.info("sent block report, processed command:" + cmd);
- }
- return cmd;
- }
-
-
- DatanodeCommand [] sendHeartBeat() throws IOException {
- return bpNamenode.sendHeartbeat(bpRegistration,
- dn.data.getCapacity(),
- dn.data.getDfsUsed(),
- dn.data.getRemaining(),
- dn.data.getBlockPoolUsed(blockPoolId),
- dn.xmitsInProgress.get(),
- dn.getXceiverCount(), dn.data.getNumFailedVolumes());
- }
-
- //This must be called only by blockPoolManager
- void start() {
- if ((bpThread != null) && (bpThread.isAlive())) {
- //Thread is started already
- return;
- }
- bpThread = new Thread(this, dnThreadName);
- bpThread.setDaemon(true); // needed for JUnit testing
- bpThread.start();
- }
-
- //This must be called only by blockPoolManager.
- void stop() {
- shouldServiceRun = false;
- if (bpThread != null) {
- bpThread.interrupt();
- }
- }
-
- //This must be called only by blockPoolManager
- void join() {
- try {
- if (bpThread != null) {
- bpThread.join();
- }
- } catch (InterruptedException ie) { }
- }
-
- //Cleanup method to be called by current thread before exiting.
- private synchronized void cleanUp() {
-
- if(upgradeManager != null)
- upgradeManager.shutdownUpgrade();
- shouldServiceRun = false;
- RPC.stopProxy(bpNamenode);
- dn.shutdownBlockPool(this);
- }
-
- /**
- * Main loop for each BP thread. Run until shutdown,
- * forever calling remote NameNode functions.
- */
- private void offerService() throws Exception {
- LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
- + dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
- + dnConf.blockReportInterval + "msec" + " Initial delay: "
- + dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
- + dnConf.heartBeatInterval);
-
- //
- // Now loop for a long time....
- //
- while (dn.shouldRun && shouldServiceRun) {
- try {
- long startTime = now();
-
- //
- // Every so often, send heartbeat or block-report
- //
- if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
- //
- // All heartbeat messages include following info:
- // -- Datanode name
- // -- data transfer port
- // -- Total capacity
- // -- Bytes remaining
- //
- lastHeartbeat = startTime;
- if (!dn.heartbeatsDisabledForTests) {
- DatanodeCommand[] cmds = sendHeartBeat();
- dn.metrics.addHeartbeat(now() - startTime);
-
- long startProcessCommands = now();
- if (!processCommand(cmds))
- continue;
- long endProcessCommands = now();
- if (endProcessCommands - startProcessCommands > 2000) {
- LOG.info("Took " + (endProcessCommands - startProcessCommands) +
- "ms to process " + cmds.length + " commands from NN");
- }
- }
- }
- if (pendingReceivedRequests > 0
- || (startTime - lastDeletedReport > dnConf.deleteReportInterval)) {
- reportReceivedDeletedBlocks();
- lastDeletedReport = startTime;
- }
-
- DatanodeCommand cmd = blockReport();
- processCommand(cmd);
-
- // Now safe to start scanning the block pool
- if (dn.blockScanner != null) {
- dn.blockScanner.addBlockPool(this.blockPoolId);
- }
-
- //
- // There is no work to do; sleep until hearbeat timer elapses,
- // or work arrives, and then iterate again.
- //
- long waitTime = dnConf.heartBeatInterval -
- (System.currentTimeMillis() - lastHeartbeat);
- synchronized(receivedAndDeletedBlockList) {
- if (waitTime > 0 && pendingReceivedRequests == 0) {
- try {
- receivedAndDeletedBlockList.wait(waitTime);
- } catch (InterruptedException ie) {
- LOG.warn("BPOfferService for block pool="
- + this.getBlockPoolId() + " received exception:" + ie);
- }
- }
- } // synchronized
- } catch(RemoteException re) {
- String reClass = re.getClassName();
- if (UnregisteredNodeException.class.getName().equals(reClass) ||
- DisallowedDatanodeException.class.getName().equals(reClass) ||
- IncorrectVersionException.class.getName().equals(reClass)) {
- LOG.warn("blockpool " + blockPoolId + " is shutting down", re);
- shouldServiceRun = false;
- return;
- }
- LOG.warn("RemoteException in offerService", re);
- try {
- long sleepTime = Math.min(1000, dnConf.heartBeatInterval);
- Thread.sleep(sleepTime);
- } catch (InterruptedException ie) {
- Thread.currentThread().interrupt();
- }
- } catch (IOException e) {
- LOG.warn("IOException in offerService", e);
- }
- } // while (shouldRun && shouldServiceRun)
- } // offerService
-
- /**
- * Register one bp with the corresponding NameNode
- *
- * The bpDatanode needs to register with the namenode on startup in order
- * 1) to report which storage it is serving now and
- * 2) to receive a registrationID
- *
- * issued by the namenode to recognize registered datanodes.
- *
- * @see FSNamesystem#registerDatanode(DatanodeRegistration)
- * @throws IOException
- */
- void register() throws IOException {
- LOG.info("in register: sid=" + bpRegistration.getStorageID() + ";SI="
- + bpRegistration.storageInfo);
-
- // build and layout versions should match
- String nsBuildVer = bpNamenode.versionRequest().getBuildVersion();
- String stBuildVer = Storage.getBuildVersion();
-
- if (!nsBuildVer.equals(stBuildVer)) {
- LOG.warn("Data-node and name-node Build versions must be " +
- "the same. Namenode build version: " + nsBuildVer + "Datanode " +
- "build version: " + stBuildVer);
- throw new IncorrectVersionException(nsBuildVer, "namenode", stBuildVer);
- }
-
- if (HdfsConstants.LAYOUT_VERSION != bpNSInfo.getLayoutVersion()) {
- LOG.warn("Data-node and name-node layout versions must be " +
- "the same. Expected: "+ HdfsConstants.LAYOUT_VERSION +
- " actual "+ bpNSInfo.getLayoutVersion());
- throw new IncorrectVersionException
- (bpNSInfo.getLayoutVersion(), "namenode");
- }
-
- while(dn.shouldRun && shouldServiceRun) {
- try {
- // Use returned registration from namenode with updated machine name.
- bpRegistration = bpNamenode.registerDatanode(bpRegistration);
-
- LOG.info("bpReg after =" + bpRegistration.storageInfo +
- ";sid=" + bpRegistration.storageID + ";name="+bpRegistration.getName());
-
- break;
- } catch(SocketTimeoutException e) { // namenode is busy
- LOG.info("Problem connecting to server: " + nnAddr);
- try {
- Thread.sleep(1000);
- } catch (InterruptedException ie) {}
- }
- }
-
- dn.bpRegistrationSucceeded(bpRegistration, blockPoolId);
-
- LOG.info("in register:" + ";bpDNR="+bpRegistration.storageInfo);
-
- // random short delay - helps scatter the BR from all DNs
- scheduleBlockReport(dnConf.initialBlockReportDelay);
- }
-
-
- /**
- * No matter what kind of exception we get, keep retrying to offerService().
- * That's the loop that connects to the NameNode and provides basic DataNode
- * functionality.
- *
- * Only stop when "shouldRun" or "shouldServiceRun" is turned off, which can
- * happen either at shutdown or due to refreshNamenodes.
- */
- @Override
- public void run() {
- LOG.info(bpRegistration + "In BPOfferService.run, data = " + dn.data
- + ";bp=" + blockPoolId);
-
- try {
- // init stuff
- try {
- // setup storage
- setupBP(dn.conf);
- register();
- } catch (IOException ioe) {
- // Initial handshake, storage recovery or registration failed
- // End BPOfferService thread
- LOG.fatal(bpRegistration + " initialization failed for block pool "
- + blockPoolId, ioe);
- return;
- }
-
- initialized = true; // bp is initialized;
-
- while (dn.shouldRun && shouldServiceRun) {
- try {
- startDistributedUpgradeIfNeeded();
- offerService();
- } catch (Exception ex) {
- LOG.error("Exception in BPOfferService", ex);
- if (dn.shouldRun && shouldServiceRun) {
- try {
- Thread.sleep(5000);
- } catch (InterruptedException ie) {
- LOG.warn("Received exception", ie);
- }
- }
- }
- }
- } catch (Throwable ex) {
- LOG.warn("Unexpected exception", ex);
- } finally {
- LOG.warn(bpRegistration + " ending block pool service for: "
- + blockPoolId + " thread " + Thread.currentThread().getId());
- cleanUp();
- }
- }
-
- /**
- * Process an array of datanode commands
- *
- * @param cmds an array of datanode commands
- * @return true if further processing may be required or false otherwise.
- */
- private boolean processCommand(DatanodeCommand[] cmds) {
- if (cmds != null) {
- for (DatanodeCommand cmd : cmds) {
- try {
- if (processCommand(cmd) == false) {
- return false;
- }
- } catch (IOException ioe) {
- LOG.warn("Error processing datanode Command", ioe);
- }
- }
- }
- return true;
- }
-
- /**
- *
- * @param cmd
- * @return true if further processing may be required or false otherwise.
- * @throws IOException
- */
- private boolean processCommand(DatanodeCommand cmd) throws IOException {
- if (cmd == null)
- return true;
- final BlockCommand bcmd =
- cmd instanceof BlockCommand? (BlockCommand)cmd: null;
-
- switch(cmd.getAction()) {
- case DatanodeProtocol.DNA_TRANSFER:
- // Send a copy of a block to another datanode
- dn.transferBlocks(bcmd.getBlockPoolId(), bcmd.getBlocks(), bcmd.getTargets());
- dn.metrics.incrBlocksReplicated(bcmd.getBlocks().length);
- break;
- case DatanodeProtocol.DNA_INVALIDATE:
- //
- // Some local block(s) are obsolete and can be
- // safely garbage-collected.
- //
- Block toDelete[] = bcmd.getBlocks();
- try {
- if (dn.blockScanner != null) {
- dn.blockScanner.deleteBlocks(bcmd.getBlockPoolId(), toDelete);
- }
- // using global fsdataset
- dn.data.invalidate(bcmd.getBlockPoolId(), toDelete);
- } catch(IOException e) {
- dn.checkDiskError();
- throw e;
- }
- dn.metrics.incrBlocksRemoved(toDelete.length);
- break;
- case DatanodeProtocol.DNA_SHUTDOWN:
- // shut down the data node
- shouldServiceRun = false;
- return false;
- case DatanodeProtocol.DNA_REGISTER:
- // namenode requested a registration - at start or if NN lost contact
- LOG.info("DatanodeCommand action: DNA_REGISTER");
- if (dn.shouldRun && shouldServiceRun) {
- register();
- }
- break;
- case DatanodeProtocol.DNA_FINALIZE:
- dn.storage.finalizeUpgrade(((FinalizeCommand) cmd)
- .getBlockPoolId());
- break;
- case UpgradeCommand.UC_ACTION_START_UPGRADE:
- // start distributed upgrade here
- processDistributedUpgradeCommand((UpgradeCommand)cmd);
- break;
- case DatanodeProtocol.DNA_RECOVERBLOCK:
- dn.recoverBlocks(((BlockRecoveryCommand)cmd).getRecoveringBlocks());
- break;
- case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
- LOG.info("DatanodeCommand action: DNA_ACCESSKEYUPDATE");
- if (dn.isBlockTokenEnabled) {
- dn.blockPoolTokenSecretManager.setKeys(blockPoolId,
- ((KeyUpdateCommand) cmd).getExportedKeys());
- }
- break;
- case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
- LOG.info("DatanodeCommand action: DNA_BALANCERBANDWIDTHUPDATE");
- long bandwidth =
- ((BalancerBandwidthCommand) cmd).getBalancerBandwidthValue();
- if (bandwidth > 0) {
- DataXceiverServer dxcs =
- (DataXceiverServer) dn.dataXceiverServer.getRunnable();
- dxcs.balanceThrottler.setBandwidth(bandwidth);
- }
- break;
- default:
- LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
- }
- return true;
- }
-
- private void processDistributedUpgradeCommand(UpgradeCommand comm)
- throws IOException {
- UpgradeManagerDatanode upgradeManager = getUpgradeManager();
- upgradeManager.processUpgradeCommand(comm);
- }
-
- synchronized UpgradeManagerDatanode getUpgradeManager() {
- if(upgradeManager == null)
- upgradeManager =
- new UpgradeManagerDatanode(dn, blockPoolId);
-
- return upgradeManager;
- }
-
- /**
- * Start distributed upgrade if it should be initiated by the data-node.
- */
- private void startDistributedUpgradeIfNeeded() throws IOException {
- UpgradeManagerDatanode um = getUpgradeManager();
-
- if(!um.getUpgradeState())
- return;
- um.setUpgradeState(false, um.getUpgradeVersion());
- um.startUpgrade();
- return;
- }
+ boolean areHeartbeatsDisabledForTests() {
+ return this.heartbeatsDisabledForTests;
}
-
+
/**
* This method starts the data node with the specified conf.
*
@@ -1406,13 +728,33 @@ void startDataNode(Configuration conf,
blockPoolManager = new BlockPoolManager(conf);
}
+ /**
+ * Create a DatanodeRegistration for a specific block pool.
+ * @param nsInfo the namespace info from the first part of the NN handshake
+ */
+ DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo) {
+ DatanodeRegistration bpRegistration = createUnknownBPRegistration();
+ String blockPoolId = nsInfo.getBlockPoolID();
+
+ bpRegistration.setStorageID(getStorageId());
+ StorageInfo storageInfo = storage.getBPStorage(blockPoolId);
+ if (storageInfo == null) {
+ // it's null in the case of SimulatedDataSet
+ bpRegistration.storageInfo.layoutVersion = HdfsConstants.LAYOUT_VERSION;
+ bpRegistration.setStorageInfo(nsInfo);
+ } else {
+ bpRegistration.setStorageInfo(storageInfo);
+ }
+ return bpRegistration;
+ }
+
/**
* Check that the registration returned from a NameNode is consistent
* with the information in the storage. If the storage is fresh/unformatted,
* sets the storage ID based on this registration.
* Also updates the block pool's state in the secret manager.
*/
- private synchronized void bpRegistrationSucceeded(DatanodeRegistration bpRegistration,
+ synchronized void bpRegistrationSucceeded(DatanodeRegistration bpRegistration,
String blockPoolId)
throws IOException {
hostName = bpRegistration.getHost();
@@ -1469,7 +811,7 @@ private void registerBlockPoolWithSecretManager(DatanodeRegistration bpRegistrat
/**
* Remove the given block pool from the block scanner, dataset, and storage.
*/
- private void shutdownBlockPool(BPOfferService bpos) {
+ void shutdownBlockPool(BPOfferService bpos) {
blockPoolManager.remove(bpos);
String bpId = bpos.getBlockPoolId();
@@ -1486,11 +828,27 @@ private void shutdownBlockPool(BPOfferService bpos) {
}
}
- void initBlockPool(BPOfferService bpOfferService,
- NamespaceInfo nsInfo) throws IOException {
+ /**
+ * One of the Block Pools has successfully connected to its NN.
+ * This initializes the local storage for that block pool,
+ * checks consistency of the NN's cluster ID, etc.
+ *
+ * If this is the first block pool to register, this also initializes
+ * the datanode-scoped storage.
+ *
+ * @param nsInfo the handshake response from the NN.
+ * @throws IOException if the NN is inconsistent with the local storage.
+ */
+ void initBlockPool(BPOfferService bpos) throws IOException {
+ NamespaceInfo nsInfo = bpos.getNamespaceInfo();
+ Preconditions.checkState(nsInfo != null,
+ "Block pool " + bpos + " should have retrieved " +
+ "its namespace info before calling initBlockPool.");
+
String blockPoolId = nsInfo.getBlockPoolID();
- blockPoolManager.addBlockPool(bpOfferService);
+ // Register the new block pool with the BP manager.
+ blockPoolManager.addBlockPool(bpos);
synchronized (this) {
// we do not allow namenode from different cluster to register
@@ -1521,12 +879,21 @@ void initBlockPool(BPOfferService bpOfferService,
+ blockPoolId + ";lv=" + storage.getLayoutVersion() +
";nsInfo=" + nsInfo);
}
+
+ // In the case that this is the first block pool to connect, initialize
+ // the dataset, block scanners, etc.
initFsDataSet();
- initPeriodicScanners(conf);
- data.addBlockPool(nsInfo.getBlockPoolID(), conf);
+ initPeriodicScanners(conf);
+
+ data.addBlockPool(blockPoolId, conf);
}
- private DatanodeRegistration createRegistration() {
+ /**
+ * Create a DatanodeRegistration object with no valid StorageInfo.
+ * This is used when reporting an error during handshake - ie
+ * before we can load any specific block pool.
+ */
+ private DatanodeRegistration createUnknownBPRegistration() {
DatanodeRegistration reg = new DatanodeRegistration(getMachineName());
reg.setInfoPort(infoServer.getPort());
reg.setIpcPort(getIpcPort());
@@ -1717,6 +1084,68 @@ static String createNewStorageId(int port) {
return "DS-" + rand + "-" + ip + "-" + port + "-"
+ System.currentTimeMillis();
}
+
+ /** Ensure the authentication method is kerberos */
+ private void checkKerberosAuthMethod(String msg) throws IOException {
+ // User invoking the call must be same as the datanode user
+ if (!UserGroupInformation.isSecurityEnabled()) {
+ return;
+ }
+ if (UserGroupInformation.getCurrentUser().getAuthenticationMethod() !=
+ AuthenticationMethod.KERBEROS) {
+ throw new AccessControlException("Error in " + msg
+ + "Only kerberos based authentication is allowed.");
+ }
+ }
+
+ private void checkBlockLocalPathAccess() throws IOException {
+ checkKerberosAuthMethod("getBlockLocalPathInfo()");
+ String currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
+ if (!currentUser.equals(this.userWithLocalPathAccess)) {
+ throw new AccessControlException(
+ "Can't continue with getBlockLocalPathInfo() "
+ + "authorization. The user " + currentUser
+ + " is not allowed to call getBlockLocalPathInfo");
+ }
+ }
+
+ @Override
+ public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
+ Token token) throws IOException {
+ checkBlockLocalPathAccess();
+ checkBlockToken(block, token, BlockTokenSecretManager.AccessMode.READ);
+ BlockLocalPathInfo info = data.getBlockLocalPathInfo(block);
+ if (LOG.isDebugEnabled()) {
+ if (info != null) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("getBlockLocalPathInfo successful block=" + block
+ + " blockfile " + info.getBlockPath() + " metafile "
+ + info.getMetaPath());
+ }
+ } else {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("getBlockLocalPathInfo for block=" + block
+ + " returning null");
+ }
+ }
+ }
+ metrics.incrBlocksGetLocalPathInfo();
+ return info;
+ }
+
+ private void checkBlockToken(ExtendedBlock block, Token token,
+ AccessMode accessMode) throws IOException {
+ if (isBlockTokenEnabled && UserGroupInformation.isSecurityEnabled()) {
+ BlockTokenIdentifier id = new BlockTokenIdentifier();
+ ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
+ DataInputStream in = new DataInputStream(buf);
+ id.readFields(in);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Got: " + id.toString());
+ }
+ blockPoolTokenSecretManager.checkAccess(id, null, block, accessMode);
+ }
+ }
/**
* Shut down this instance of the datanode.
@@ -1917,7 +1346,7 @@ private void transferBlock( ExtendedBlock block,
}
}
- private void transferBlocks(String poolId, Block blocks[],
+ void transferBlocks(String poolId, Block blocks[],
DatanodeInfo xferTargets[][]) {
for (int i = 0; i < blocks.length; i++) {
try {
@@ -2034,7 +1463,7 @@ private class DataTransfer implements Runnable {
* entire target list, the block, and the data.
*/
DataTransfer(DatanodeInfo targets[], ExtendedBlock b, BlockConstructionStage stage,
- final String clientname) throws IOException {
+ final String clientname) {
if (DataTransferProtocol.LOG.isDebugEnabled()) {
DataTransferProtocol.LOG.debug(getClass().getSimpleName() + ": "
+ b + " (numBytes=" + b.getNumBytes() + ")"
@@ -2209,9 +1638,7 @@ public static DataNode instantiateDataNode(String args [], Configuration conf,
System.exit(-1);
}
Collection dataDirs = getStorageDirs(conf);
- dnThreadName = "DataNode: [" +
- StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "]";
- UserGroupInformation.setConfiguration(conf);
+ UserGroupInformation.setConfiguration(conf);
SecurityUtil.login(conf, DFS_DATANODE_KEYTAB_FILE_KEY,
DFS_DATANODE_USER_NAME_KEY);
return makeInstance(dataDirs, conf, resources);
@@ -2554,16 +1981,6 @@ public DatanodeProtocol getBPNamenode(String bpid) throws IOException {
return bpos.bpNamenode;
}
- /**
- * To be used by tests only to set a mock namenode in BPOfferService
- */
- void setBPNamenode(String bpid, DatanodeProtocol namenode) {
- BPOfferService bp = blockPoolManager.get(bpid);
- if (bp != null) {
- bp.setNameNode(namenode);
- }
- }
-
/** Block synchronization */
void syncBlock(RecoveringBlock rBlock,
List syncList) throws IOException {
@@ -2753,6 +2170,14 @@ void transferReplicaForPipelineRecovery(final ExtendedBlock b,
}
}
+ /**
+ * Finalize a pending upgrade in response to DNA_FINALIZE.
+ * @param blockPoolId the block pool to finalize
+ */
+ void finalizeUpgradeForPool(String blockPoolId) throws IOException {
+ storage.finalizeUpgrade(blockPoolId);
+ }
+
// Determine a Datanode's streaming address
public static InetSocketAddress getStreamingAddr(Configuration conf) {
return NetUtils.createSocketAddr(
@@ -2789,7 +2214,7 @@ public String getNamenodeAddresses() {
final Map info = new HashMap();
for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
if (bpos != null && bpos.bpThread != null) {
- info.put(bpos.getNNSocketAddress().getHostName(), bpos.blockPoolId);
+ info.put(bpos.getNNSocketAddress().getHostName(), bpos.getBlockPoolId());
}
}
return JSON.toString(info);
@@ -2820,13 +2245,7 @@ public synchronized String getClusterId() {
}
public void refreshNamenodes(Configuration conf) throws IOException {
- try {
- blockPoolManager.refreshNamenodes(conf);
- } catch (InterruptedException ex) {
- IOException eio = new IOException();
- eio.initCause(ex);
- throw eio;
- }
+ blockPoolManager.refreshNamenodes(conf);
}
@Override //ClientDatanodeProtocol
@@ -2877,7 +2296,7 @@ public boolean isBPServiceAlive(String bpid) {
*/
public boolean isDatanodeFullyStarted() {
for (BPOfferService bp : blockPoolManager.getAllNamenodeThreads()) {
- if (!bp.initialized() || !bp.isAlive()) {
+ if (!bp.isInitialized() || !bp.isAlive()) {
return false;
}
}
@@ -2904,4 +2323,9 @@ public Long getBalancerBandwidth() {
DNConf getDnConf() {
return dnConf;
}
+
+ boolean shouldRun() {
+ return shouldRun;
+ }
+
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 11282a5b7a..fe07754a3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -48,8 +48,6 @@
import org.apache.hadoop.hdfs.protocol.datatransfer.Receiver;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.Builder;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProtoOrBuilder;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
@@ -128,7 +126,7 @@ private void updateCurrentThreadName(String status) {
public void run() {
int opsProcessed = 0;
Op op = null;
- dataXceiverServer.childSockets.put(s, s);
+ dataXceiverServer.childSockets.add(s);
try {
int stdTimeout = s.getSoTimeout();
@@ -165,14 +163,6 @@ public void run() {
s.setSoTimeout(stdTimeout);
}
- // Make sure the xceiver count is not exceeded
- int curXceiverCount = datanode.getXceiverCount();
- if (curXceiverCount > dataXceiverServer.maxXceiverCount) {
- throw new IOException("xceiverCount " + curXceiverCount
- + " exceeds the limit of concurrent xcievers "
- + dataXceiverServer.maxXceiverCount);
- }
-
opStartTime = now();
processOp(op);
++opsProcessed;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
index c0d782a5c7..eed58ecad4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
@@ -23,9 +23,9 @@
import java.net.SocketTimeoutException;
import java.nio.channels.AsynchronousCloseException;
import java.util.Collections;
-import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
-import java.util.Map;
+import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
@@ -48,8 +48,8 @@ class DataXceiverServer implements Runnable {
ServerSocket ss;
DataNode datanode;
// Record all sockets opened for data transfer
- Map childSockets = Collections.synchronizedMap(
- new HashMap());
+ Set childSockets = Collections.synchronizedSet(
+ new HashSet());
/**
* Maximal number of concurrent xceivers per node.
@@ -135,6 +135,15 @@ public void run() {
try {
s = ss.accept();
s.setTcpNoDelay(true);
+
+ // Make sure the xceiver count is not exceeded
+ int curXceiverCount = datanode.getXceiverCount();
+ if (curXceiverCount > maxXceiverCount) {
+ throw new IOException("Xceiver count " + curXceiverCount
+ + " exceeds the limit of concurrent xcievers: "
+ + maxXceiverCount);
+ }
+
new Daemon(datanode.threadGroup, new DataXceiver(s, datanode, this))
.start();
} catch (SocketTimeoutException ignored) {
@@ -184,7 +193,7 @@ void kill() {
// close all the sockets that were accepted earlier
synchronized (childSockets) {
- for (Iterator it = childSockets.values().iterator();
+ for (Iterator it = childSockets.iterator();
it.hasNext();) {
Socket thissock = it.next();
try {
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.java
index e309dc1f47..f885c8b21c 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.java
@@ -52,6 +52,7 @@
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;
@@ -459,7 +460,7 @@ private long validateIntegrity(File blockFile, long genStamp) {
long metaFileLen = metaFile.length();
int crcHeaderLen = DataChecksum.getChecksumHeaderSize();
if (!blockFile.exists() || blockFileLen == 0 ||
- !metaFile.exists() || metaFileLen < (long)crcHeaderLen) {
+ !metaFile.exists() || metaFileLen < crcHeaderLen) {
return 0;
}
checksumIn = new DataInputStream(
@@ -578,7 +579,7 @@ long getBlockPoolUsed(String bpid) throws IOException {
* reserved capacity.
* @return the unreserved number of bytes left in this filesystem. May be zero.
*/
- long getCapacity() throws IOException {
+ long getCapacity() {
long remaining = usage.getCapacity() - reserved;
return remaining > 0 ? remaining : 0;
}
@@ -818,7 +819,7 @@ private long getBlockPoolUsed(String bpid) throws IOException {
return dfsUsed;
}
- private long getCapacity() throws IOException {
+ private long getCapacity() {
long capacity = 0L;
for (FSVolume vol : volumes) {
capacity += vol.getCapacity();
@@ -1667,7 +1668,7 @@ private void bumpReplicaGS(ReplicaInfo replicaInfo,
}
if (!oldmeta.renameTo(newmeta)) {
replicaInfo.setGenerationStamp(oldGS); // restore old GS
- throw new IOException("Block " + (Block)replicaInfo + " reopen failed. " +
+ throw new IOException("Block " + replicaInfo + " reopen failed. " +
" Unable to move meta file " + oldmeta +
" to " + newmeta);
}
@@ -2018,7 +2019,7 @@ private boolean isValid(final ExtendedBlock b, final ReplicaState state) {
/**
* Find the file corresponding to the block and return it if it exists.
*/
- File validateBlockFile(String bpid, Block b) throws IOException {
+ File validateBlockFile(String bpid, Block b) {
//Should we check for metadata file too?
File f = getFile(bpid, b);
@@ -2327,7 +2328,7 @@ public void checkAndUpdate(String bpid, long blockId, File diskFile,
if (datanode.blockScanner != null) {
datanode.blockScanner.addBlock(new ExtendedBlock(bpid, diskBlockInfo));
}
- DataNode.LOG.warn("Added missing block to memory " + (Block)diskBlockInfo);
+ DataNode.LOG.warn("Added missing block to memory " + diskBlockInfo);
return;
}
/*
@@ -2600,7 +2601,7 @@ public synchronized void shutdownBlockPool(String bpid) {
* get list of all bpids
* @return list of bpids
*/
- public String [] getBPIdlist() throws IOException {
+ public String [] getBPIdlist() {
return volumeMap.getBlockPoolList();
}
@@ -2658,4 +2659,14 @@ public synchronized void deleteBlockPool(String bpid, boolean force)
volume.deleteBPDirectories(bpid, force);
}
}
+
+ @Override // FSDatasetInterface
+ public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block)
+ throws IOException {
+ File datafile = getBlockFile(block);
+ File metafile = getMetaFile(datafile, block.getGenerationStamp());
+ BlockLocalPathInfo info = new BlockLocalPathInfo(block,
+ datafile.getAbsolutePath(), metafile.getAbsolutePath());
+ return info;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
index 38017cfdb8..2f05f16c2f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
@@ -19,6 +19,7 @@
import java.io.Closeable;
+import java.io.File;
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
@@ -31,6 +32,7 @@
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
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.io.IOUtils;
import org.apache.hadoop.util.DataChecksum;
@@ -402,4 +404,9 @@ public ReplicaInfo updateReplicaUnderRecovery(
* @throws IOException
*/
public void deleteBlockPool(String bpid, boolean force) throws IOException;
+
+ /**
+ * Get {@link BlockLocalPathInfo} for the given block.
+ **/
+ public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) throws IOException;
}
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 14c1258fe4..c20b0090d2 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
@@ -17,7 +17,6 @@
*/
package org.apache.hadoop.hdfs.server.datanode;
-import java.io.DataInputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
index 4df11d3434..9e18007810 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
@@ -60,6 +60,7 @@ public class DataNodeMetrics {
@Metric MutableCounterLong readsFromRemoteClient;
@Metric MutableCounterLong writesFromLocalClient;
@Metric MutableCounterLong writesFromRemoteClient;
+ @Metric MutableCounterLong blocksGetLocalPathInfo;
@Metric MutableCounterLong volumeFailures;
@@ -165,4 +166,9 @@ public void incrReadsFromClient(boolean local) {
public void incrVolumeFailures() {
volumeFailures.incr();
}
+
+ /** Increment for getBlockLocalPathInfo calls */
+ public void incrBlocksGetLocalPathInfo() {
+ blocksGetLocalPathInfo.incr();
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
index dd68261253..fc1fe14af7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
@@ -17,14 +17,10 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
import java.io.IOException;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
-import java.util.zip.Checksum;
-
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
index 2bd585e236..c655ee75bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
@@ -19,7 +19,6 @@
import java.io.IOException;
-import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
index 84408c0162..39d2abaee7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
@@ -36,7 +36,6 @@
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-import org.apache.hadoop.http.HttpServer;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.util.Daemon;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
index 1c8253f665..6459ffd0e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
@@ -23,8 +23,6 @@
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
index 9db7f8ae66..d05c4fe3d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
@@ -26,10 +26,7 @@
import java.io.DataInputStream;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
import com.google.common.annotations.VisibleForTesting;
/**
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 f80f863346..cb0f88e85a 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
@@ -37,7 +37,6 @@
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
-import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 991fd08c84..30b0b8c151 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -19,7 +19,6 @@
import static org.apache.hadoop.hdfs.server.common.Util.now;
-import java.io.File;
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
@@ -34,7 +33,6 @@
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream.LogHeaderCorruptException;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
@@ -57,8 +55,6 @@
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
import org.apache.hadoop.hdfs.util.Holder;
-import org.apache.hadoop.io.IOUtils;
-
import com.google.common.base.Joiner;
public class FSEditLogLoader {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
index a7c2949f29..a3a516f0ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
@@ -19,8 +19,6 @@
import java.io.File;
import java.io.IOException;
-import java.util.List;
-
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java
index 33d6e90f92..dbf1860a85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java
@@ -23,28 +23,16 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.TreeSet;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
-import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
-import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
-import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-
-import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
public static final Log LOG = LogFactory.getLog(
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 70680e3649..fc56db4752 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
@@ -305,7 +305,20 @@ private static final void logAuditEvent(UserGroupInformation ugi,
* @throws IOException if loading fails
*/
public static FSNamesystem loadFromDisk(Configuration conf) throws IOException {
- FSImage fsImage = new FSImage(conf);
+ Collection namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
+ Collection namespaceEditsDirs =
+ FSNamesystem.getNamespaceEditsDirs(conf);
+
+ if (namespaceDirs.size() == 1) {
+ LOG.warn("Only one " + DFS_NAMENODE_NAME_DIR_KEY
+ + " directory configured , beware data loss!");
+ }
+ if (namespaceEditsDirs.size() == 1) {
+ LOG.warn("Only one " + DFS_NAMENODE_EDITS_DIR_KEY
+ + " directory configured , beware data loss!");
+ }
+
+ FSImage fsImage = new FSImage(conf, namespaceDirs, namespaceEditsDirs);
FSNamesystem namesystem = new FSNamesystem(conf, fsImage);
long loadStart = now();
@@ -2060,10 +2073,12 @@ void removePathAndBlocks(String src, List blocks) {
}
}
- /** Get the file info for a specific file.
+ /**
+ * Get the file info for a specific file.
+ *
* @param src The string representation of the path to the file
* @param resolveLink whether to throw UnresolvedLinkException
- * if src refers to a symlinks
+ * if src refers to a symlink
*
* @throws AccessControlException if access is denied
* @throws UnresolvedLinkException if a symlink is encountered.
@@ -2271,6 +2286,7 @@ boolean internalReleaseLease(Lease lease, String src,
// If the penultimate block is not COMPLETE, then it must be COMMITTED.
if(nrCompleteBlocks < nrBlocks - 2 ||
nrCompleteBlocks == nrBlocks - 2 &&
+ curBlock != null &&
curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
final String message = "DIR* NameSystem.internalReleaseLease: "
+ "attempt to release a create lock on "
@@ -2459,7 +2475,7 @@ void commitBlockSynchronization(ExtendedBlock lastblock,
newtargets[i]);
}
}
- if (closeFile) {
+ if ((closeFile) && (descriptors != null)) {
// the file is getting closed. Insert block locations into blockManager.
// Otherwise fsck will report these blocks as MISSING, especially if the
// blocksReceived from Datanodes take a long time to arrive.
@@ -3283,6 +3299,7 @@ public String toString() {
/**
* Checks consistency of the class state.
* This is costly and currently called only in assert.
+ * @throws IOException
*/
private boolean isConsistent() {
if (blockTotal == -1 && blockSafe == -1) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
index d8bd502597..023e3b612e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
@@ -19,8 +19,6 @@
import java.io.IOException;
import java.io.PrintWriter;
-import java.net.URI;
-import java.net.URISyntaxException;
import java.net.URL;
import javax.net.SocketFactory;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
index 3e630661e9..543033b341 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
-import java.net.URISyntaxException;
import java.net.URL;
import java.security.PrivilegedExceptionAction;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index 83d9858586..9885f23f92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -372,14 +372,16 @@ static String[] getPathNames(String path) {
/**
* Given some components, create a path name.
- * @param components
+ * @param components The path components
+ * @param start index
+ * @param end index
* @return concatenated path
*/
- static String constructPath(byte[][] components, int start) {
+ static String constructPath(byte[][] components, int start, int end) {
StringBuilder buf = new StringBuilder();
- for (int i = start; i < components.length; i++) {
+ for (int i = start; i < end; i++) {
buf.append(DFSUtil.bytes2String(components[i]));
- if (i < components.length - 1) {
+ if (i < end - 1) {
buf.append(Path.SEPARATOR);
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
index 7f0c997ee9..f4d9e78f88 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
@@ -23,8 +23,6 @@
import java.util.List;
import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
@@ -191,18 +189,19 @@ assert compareBytes(this.name, components[0]) == 0 :
existing[index] = curNode;
}
if (curNode.isLink() && (!lastComp || (lastComp && resolveLink))) {
- if(NameNode.stateChangeLog.isDebugEnabled()) {
+ final String path = constructPath(components, 0, components.length);
+ final String preceding = constructPath(components, 0, count);
+ final String remainder =
+ constructPath(components, count + 1, components.length);
+ final String link = DFSUtil.bytes2String(components[count]);
+ final String target = ((INodeSymlink)curNode).getLinkValue();
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("UnresolvedPathException " +
- " count: " + count +
- " componenent: " + DFSUtil.bytes2String(components[count]) +
- " full path: " + constructPath(components, 0) +
- " remaining path: " + constructPath(components, count+1) +
- " symlink: " + ((INodeSymlink)curNode).getLinkValue());
+ " path: " + path + " preceding: " + preceding +
+ " count: " + count + " link: " + link + " target: " + target +
+ " remainder: " + remainder);
}
- final String linkTarget = ((INodeSymlink)curNode).getLinkValue();
- throw new UnresolvedPathException(constructPath(components, 0),
- constructPath(components, count+1),
- linkTarget);
+ throw new UnresolvedPathException(path, preceding, remainder, target);
}
if (lastComp || !curNode.isDirectory()) {
break;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
index 4b59e50924..6f476ec0f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
@@ -17,6 +17,10 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT;
+
import java.io.IOException;
import java.net.InetSocketAddress;
import java.security.PrivilegedExceptionAction;
@@ -145,7 +149,7 @@ private Map getAuthFilterParams(Configuration conf)
}
};
- boolean certSSL = conf.getBoolean("dfs.https.enable", false);
+ boolean certSSL = conf.getBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, false);
boolean useKrb = UserGroupInformation.isSecurityEnabled();
if (certSSL || useKrb) {
boolean needClientAuth = conf.getBoolean(
@@ -156,14 +160,14 @@ private Map getAuthFilterParams(Configuration conf)
DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT));
Configuration sslConf = new HdfsConfiguration(false);
if (certSSL) {
- sslConf.addResource(conf.get(
- "dfs.https.server.keystore.resource", "ssl-server.xml"));
+ sslConf.addResource(conf.get(DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
+ DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT));
}
httpServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth,
useKrb);
// assume same ssl port for all datanodes
InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf
- .get("dfs.datanode.https.address", infoHost + ":" + 50475));
+ .get(DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 50475));
httpServer.setAttribute("datanode.https.port", datanodeSslPort
.getPort());
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 2a2318cb6e..ff05744343 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -819,10 +819,6 @@ public void createSymlink(String target, String link, FsPermission dirPerms,
public String getLinkTarget(String path) throws IOException {
nn.checkOperation(OperationCategory.READ);
metrics.incrGetLinkTargetOps();
- /* Resolves the first symlink in the given path, returning a
- * new path consisting of the target of the symlink and any
- * remaining path components from the original path.
- */
try {
HdfsFileStatus stat = namesystem.getFileInfo(path, false);
if (stat != null) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 74f8b2d5b9..96cc8982df 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -29,7 +29,6 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.Random;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
index 3d2fd8b0be..496423d4a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
@@ -380,7 +380,13 @@ static void redirectToRandomDataNode(ServletContext context,
final NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
final Configuration conf = (Configuration) context
.getAttribute(JspHelper.CURRENT_CONF);
- final DatanodeID datanode = getRandomDatanode(nn);
+ // We can't redirect if there isn't a DN to redirect to.
+ // Lets instead show a proper error message.
+ if (nn.getNamesystem().getNumLiveDataNodes() < 1) {
+ throw new IOException("Can't browse the DFS since there are no " +
+ "live nodes available to redirect to.");
+ }
+ final DatanodeID datanode = getRandomDatanode(nn);;
UserGroupInformation ugi = JspHelper.getUGI(context, request, conf);
String tokenString = getDelegationToken(
nn.getRpcServer(), request, conf, ugi);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RenewDelegationTokenServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RenewDelegationTokenServlet.java
index ddd0acbbfb..d6a42dde55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RenewDelegationTokenServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RenewDelegationTokenServlet.java
@@ -29,7 +29,6 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
index c82494d5ba..b6f58877f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
@@ -20,8 +20,6 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-import java.util.Comparator;
-
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.io.Writable;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockCommandWritable.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockCommandWritable.java
index 990b235c9f..5f22888678 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockCommandWritable.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocolR23Compatible/BlockCommandWritable.java
@@ -20,17 +20,12 @@
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-import java.util.List;
-
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocolR23Compatible.BlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeInfoWritable;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
index b2a84a956a..bc98995af3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
@@ -34,7 +34,6 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
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 eb8af25d26..2a685f73bd 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
@@ -40,7 +40,6 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.HftpFileSystem;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoader.java
index a485871c54..dfa5b1408f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoader.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
import java.io.IOException;
-import java.util.Map;
-
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitor.java
index 6dbca3c4a6..c8a8912741 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitor.java
@@ -22,8 +22,6 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DeprecatedUTF8;
-
import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.ByteToken;
import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.ShortToken;
import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.IntToken;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitorFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitorFactory.java
index b4c9b561e4..43af249a46 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitorFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsVisitorFactory.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
import java.io.IOException;
-import java.util.regex.Pattern;
-
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java
index 8ce57c6802..b9333a841f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TokenizerFactory.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
import java.io.IOException;
-import java.util.regex.Pattern;
-
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DirectBufferPool.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DirectBufferPool.java
index 69b238bbbf..6972a86037 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DirectBufferPool.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DirectBufferPool.java
@@ -19,11 +19,6 @@
import java.lang.ref.WeakReference;
import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java
index 866734f17b..13ca8b45c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java
@@ -22,7 +22,6 @@
import java.util.Collection;
import java.util.ConcurrentModificationException;
import java.util.Iterator;
-import java.util.LinkedList;
import java.util.List;
import java.util.NoSuchElementException;
@@ -615,4 +614,4 @@ public boolean removeAll(Collection> c) {
public boolean retainAll(Collection> c) {
throw new UnsupportedOperationException("retainAll is not supported.");
}
-}
\ No newline at end of file
+}
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 59a8ff645f..c99763e395 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
@@ -122,11 +122,17 @@ creations/deletions), or "all".
dfs.datanode.https.address
0.0.0.0:50475
+ The datanode secure http server address and port.
+ If the port is 0 then the server will start on a free port.
+
dfs.namenode.https-address
0.0.0.0:50470
+ The namenode secure http server address and port.
+ If the port is 0 then the server will start on a free port.
+
@@ -193,6 +199,14 @@ creations/deletions), or "all".
directories, for redundancy.
+
+ dfs.namenode.name.dir.restore
+ false
+ Set to true to enable NameNode to attempt recovering a
+ previously failed dfs.name.dir. When enabled, a recovery of any failed
+ directory is attempted during checkpoint.
+
+
dfs.namenode.fs-limits.max-component-length
0
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/proto/InterDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/proto/InterDatanodeProtocol.proto
new file mode 100644
index 0000000000..9f36e29e0f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/proto/InterDatanodeProtocol.proto
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "InterDatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Block with location information and new generation stamp
+ * to be used for recovery.
+ */
+message InitReplicaRecoveryRequestProto {
+ required RecoveringBlockProto block = 1;
+}
+
+/**
+ * Repica recovery information
+ */
+message InitReplicaRecoveryResponseProto {
+ required ReplicaState state = 1; // State fo the replica
+ required BlockProto block = 2; // block information
+}
+
+/**
+ * Update replica with new generation stamp and length
+ */
+message UpdateReplicaUnderRecoveryRequestProto {
+ required ExtendedBlockProto block = 1; // Block identifier
+ required uint64 recoveryId = 2; // New genstamp of the replica
+ required uint64 newLength = 3; // New length of the replica
+}
+
+/**
+ * Response returns updated block information
+ */
+message UpdateReplicaUnderRecoveryResponseProto {
+ required ExtendedBlockProto block = 1; // Updated block information
+}
+
+/**
+ * Protocol used between datanodes for block recovery.
+ *
+ * See the request and response for details of rpc call.
+ */
+service InterDatanodeProtocolService {
+ /**
+ * Initialize recovery of a replica
+ */
+ rpc initReplicaRecovery(InitReplicaRecoveryRequestProto)
+ returns(InitReplicaRecoveryResponseProto);
+
+ /**
+ * Update a replica with new generation stamp and length
+ */
+ rpc updateReplicaUnderRecovery(UpdateReplicaUnderRecoveryRequestProto)
+ returns(UpdateReplicaUnderRecoveryResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/proto/JournalProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/proto/JournalProtocol.proto
new file mode 100644
index 0000000000..fa7ed15c64
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/proto/JournalProtocol.proto
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "JournalProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * registration - the registration info of the active NameNode
+ * firstTxnId - the first txid in the rolled edit log
+ * numTxns - Number of transactions in editlog
+ * records - bytes containing serialized journal records
+ */
+message JournalRequestProto {
+ required NamenodeRegistrationProto registration = 1; // Registration info
+ required uint64 firstTxnId = 2; // Transaction ID
+ required uint32 numTxns = 3; // Transaction ID
+ required bytes records = 4; // Journal record
+}
+
+/**
+ * void response
+ */
+message JournalResponseProto {
+}
+
+/**
+ * registration - the registration info of the active NameNode
+ * txid - first txid in the new log
+ */
+message StartLogSegmentRequestProto {
+ required NamenodeRegistrationProto registration = 1; // Registration info
+ required uint64 txid = 2; // Transaction ID
+}
+
+/**
+ * void response
+ */
+message StartLogSegmentResponseProto {
+}
+
+/**
+ * Protocol used to journal edits to a remote node. Currently,
+ * this is used to publish edits from the NameNode to a BackupNode.
+ *
+ * See the request and response for details of rpc call.
+ */
+service JournalProtocolService {
+ /**
+ * Request sent by active namenode to backup node via
+ * EditLogBackupOutputStream to stream editlog records.
+ */
+ rpc journal(JournalRequestProto) returns (JournalResponseProto);
+
+ /**
+ * Request sent by active namenode to backup node to notify
+ * that the NameNode has rolled its edit logs and is now writing a
+ * new log segment.
+ */
+ rpc startLogSegment(StartLogSegmentRequestProto)
+ returns (StartLogSegmentResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/proto/NamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/proto/NamenodeProtocol.proto
new file mode 100644
index 0000000000..27fa6d1995
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/proto/NamenodeProtocol.proto
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "NamenodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Get list of blocks for a given datanode with the total length
+ * of adding up to given size
+ * datanode - Datanode ID to get list of block from
+ * size - size to which the block lengths must add up to
+ */
+message GetBlocksRequestProto {
+ required DatanodeIDProto datanode = 1; // Datanode ID
+ required uint64 size = 2; // Size in bytes
+}
+
+
+/**
+ * blocks - List of returned blocks
+ */
+message GetBlocksResponseProto {
+ required BlockWithLocationsProto blocks = 1; // List of blocks
+}
+
+/**
+ * void request
+ */
+message GetBlockKeysRequestProto {
+}
+
+/**
+ * keys - Information about block keys at the active namenode
+ */
+message GetBlockKeysResponseProto {
+ required ExportedBlockKeysProto keys = 1;
+}
+
+/**
+ * void request
+ */
+message GetTransactionIdRequestProto {
+}
+
+/**
+ * txId - Transaction ID of the most recently persisted edit log record
+ */
+message GetTransactionIdResponseProto {
+ required uint64 txId = 1; // Transaction ID
+}
+
+/**
+ * void request
+ */
+message RollEditLogRequestProto {
+}
+
+/**
+ * signature - A unique token to identify checkpoint transaction
+ */
+message RollEditLogResponseProto {
+ required CheckpointSignatureProto signature = 1;
+}
+
+/**
+ * registartion - Namenode reporting the error
+ * errorCode - error code indicating the error
+ * msg - Free text description of the error
+ */
+message ErrorReportRequestProto {
+ required NamenodeRegistrationProto registartion = 1; // Registartion info
+ required uint32 errorCode = 2; // Error code
+ required string msg = 3; // Error message
+}
+
+/**
+ * void response
+ */
+message ErrorReportResponseProto {
+}
+
+/**
+ * registration - Information of the namenode registering with primary namenode
+ */
+message RegisterRequestProto {
+ required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * registration - Updated registration information of the newly registered
+ * datanode.
+ */
+message RegisterResponseProto {
+ required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * Start checkpoint request
+ * registration - Namenode that is starting the checkpoint
+ */
+message StartCheckpointRequestProto {
+ required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * command - Command returned by the active namenode to be
+ * be handled by the caller.
+ */
+message StartCheckpointResponseProto {
+ required NamenodeCommandProto command = 1;
+}
+
+/**
+ * End or finalize the previously started checkpoint
+ * registration - Namenode that is ending the checkpoint
+ * signature - unique token to identify checkpoint transaction,
+ * that was received when checkpoint was started.
+ */
+message EndCheckpointRequestProto {
+ required NamenodeRegistrationProto registration = 1; // Registration info
+ required CheckpointSignatureProto signature = 2;
+}
+
+/**
+ * void response
+ */
+message EndCheckpointResponseProto {
+}
+
+/**
+ * sinceTxId - return the editlog information for transactions >= sinceTxId
+ */
+message GetEditLogManifestRequestProto {
+ required uint64 sinceTxId = 1; // Transaction ID
+}
+
+/**
+ * manifest - Enumeration of editlogs from namenode for
+ * logs >= sinceTxId in the request
+ */
+message GetEditLogManifestResponseProto {
+ required RemoteEditLogManifestProto manifest = 1;
+}
+
+/**
+ * Protocol used by the sub-ordinate namenode to send requests
+ * the active/primary namenode.
+ *
+ * See the request and response for details of rpc call.
+ */
+service NamenodeProtocolService {
+ /**
+ * Get list of blocks for a given datanode with length
+ * of blocks adding up to given size.
+ */
+ rpc getBlocks(GetBlocksRequestProto) returns(GetBlocksResponseProto);
+
+ /**
+ * Get the current block keys
+ */
+ rpc getBlockKeys(GetBlockKeysRequestProto) returns(GetBlockKeysResponseProto);
+
+ /**
+ * Get the transaction ID of the most recently persisted editlog record
+ */
+ rpc getTransationId(GetTransactionIdRequestProto)
+ returns(GetTransactionIdResponseProto);
+
+ /**
+ * Close the current editlog and open a new one for checkpointing purposes
+ */
+ rpc rollEditLog(RollEditLogRequestProto) returns(RollEditLogResponseProto);
+
+ /**
+ * Report from a sub-ordinate namenode of an error to the active namenode.
+ * Active namenode may decide to unregister the reporting namenode
+ * depending on the error.
+ */
+ rpc errorReport(ErrorReportRequestProto) returns(ErrorReportResponseProto);
+
+ /**
+ * Request to register a sub-ordinate namenode
+ */
+ rpc register(RegisterRequestProto) returns(RegisterResponseProto);
+
+ /**
+ * Request to start a checkpoint.
+ */
+ rpc startCheckpoint(StartCheckpointRequestProto)
+ returns(StartCheckpointResponseProto);
+
+ /**
+ * End of finalize the previously started checkpoint
+ */
+ rpc endCheckpoint(EndCheckpointRequestProto)
+ returns(EndCheckpointResponseProto);
+
+ /**
+ * Get editlog manifests from the active namenode for all the editlogs
+ */
+ rpc getEditLogManifest(GetEditLogManifestRequestProto)
+ returns(GetEditLogManifestResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/proto/hdfs.proto
index a77a7c312e..8b86066980 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/proto/hdfs.proto
@@ -53,6 +53,12 @@ message DatanodeIDProto {
required uint32 ipcPort = 4; // the port where the ipc Server is running
}
+/**
+ * DatanodeID array
+ */
+message DatanodeIDsProto {
+ repeated DatanodeIDProto datanodes = 1;
+}
/**
* The status of a Datanode
@@ -76,7 +82,6 @@ message DatanodeInfoProto {
optional AdminState adminState = 10;
}
-
/**
* Summary of a file or directory
*/
@@ -152,10 +157,10 @@ message HdfsFileStatusProto {
required string group = 6;
required uint64 modification_time = 7;
required uint64 access_time = 8;
- //
+
// Optional fields for symlink
- optional bytes symlink = 9; // if symlink, target encoded java UTF8
- //
+ optional bytes symlink = 9; // if symlink, target encoded java UTF8
+
// Optional fields for file
optional uint32 block_replication = 10; // Actually a short - only 16bits used
optional uint64 blocksize = 11;
@@ -169,7 +174,7 @@ message FsServerDefaultsProto {
required uint64 blockSize = 1;
required uint32 bytesPerChecksum = 2;
required uint32 writePacketSize = 3;
- required uint32 replication = 4; // Actually a short - only 16bits used
+ required uint32 replication = 4; // Actually a short - only 16 bits used
required uint32 fileBufferSize = 5;
}
@@ -187,5 +192,156 @@ message DirectoryListingProto {
*/
message UpgradeStatusReportProto {
required uint32 version = 1;;
- required uint32 upgradeStatus = 2; // Between 0 and 100 indicating the % complete
+ required uint32 upgradeStatus = 2; // % completed in range 0 & 100
}
+
+/**
+ * Common node information shared by all the nodes in the cluster
+ */
+message StorageInfoProto {
+ required uint32 layoutVersion = 1; // Layout version of the file system
+ required uint32 namespceID = 2; // File system namespace ID
+ required string clusterID = 3; // ID of the cluster
+ required uint64 cTime = 4; // File system creation time
+}
+
+/**
+ * Information sent by a namenode to identify itself to the primary namenode.
+ */
+message NamenodeRegistrationProto {
+ required string rpcAddress = 1; // host:port of the namenode RPC address
+ required string httpAddress = 2; // host:port of the namenode http server
+ enum NamenodeRoleProto {
+ NAMENODE = 1;
+ BACKUP = 2;
+ CHECKPOINT = 3;
+ }
+ required StorageInfoProto storageInfo = 3; // Node information
+ optional NamenodeRoleProto role = 4; // Namenode role
+}
+
+/**
+ * Unique signature to identify checkpoint transactions.
+ */
+message CheckpointSignatureProto {
+ required string blockPoolId = 1;
+ required uint64 mostRecentCheckpointTxId = 2;
+ required uint64 curSegmentTxId = 3;
+ required StorageInfoProto storageInfo = 4;
+}
+
+/**
+ * Command sent from one namenode to another namenode.
+ */
+message NamenodeCommandProto {
+ enum Type {
+ NamenodeCommand = 0; // Base command
+ CheckPointCommand = 1; // Check point command
+ }
+ required uint32 action = 1;
+ required Type type = 2;
+ optional CheckpointCommandProto checkpointCmd = 3;
+}
+
+/**
+ * Command returned from primary to checkpointing namenode.
+ * This command has checkpoint signature that identifies
+ * checkpoint transaction and is needed for further
+ * communication related to checkpointing.
+ */
+message CheckpointCommandProto {
+ // Unique signature to identify checkpoint transation
+ required CheckpointSignatureProto signature = 1;
+
+ // If true, return transfer image to primary upon the completion of checkpoint
+ required bool needToReturnImage = 2;
+}
+
+/**
+ * Block information
+ */
+message BlockProto {
+ required uint64 blockId = 1;
+ required uint64 genStamp = 2;
+ optional uint64 numBytes = 3;
+}
+
+/**
+ * Block and datanodes where is it located
+ */
+message BlockWithLocationsProto {
+ required BlockProto block = 1; // Block
+ repeated DatanodeIDProto datanodeIDs = 2; // Datanodes with replicas of the block
+}
+
+/**
+ * List of block with locations
+ */
+message BlocksWithLocationsProto {
+ repeated BlockWithLocationsProto blocks = 1;
+}
+
+/**
+ * Editlog information with available transactions
+ */
+message RemoteEditLogProto {
+ required uint64 startTxId = 1; // Starting available edit log transaction
+ required uint64 endTxId = 2; // Ending available edit log transaction
+}
+
+/**
+ * Enumeration of editlogs available on a remote namenode
+ */
+message RemoteEditLogManifestProto {
+ repeated RemoteEditLogProto logs = 1;
+}
+
+/**
+ * Namespace information that describes namespace on a namenode
+ */
+message NamespaceInfoProto {
+ required string buildVersion = 1; // Software build version
+ required uint32 distUpgradeVersion = 2; // Distributed upgrade version
+ required string blockPoolID = 3; // block pool used by the namespace
+ required StorageInfoProto storageInfo = 4;// Noe information
+}
+
+/**
+ * Block access token information
+ */
+message BlockKeyProto {
+ required uint32 keyId = 1; // Key identifier
+ required uint64 expiryDate = 2; // Expiry time in milliseconds
+ required bytes keyBytes = 3; // Key secret
+}
+
+/**
+ * Current key and set of block keys at the namenode.
+ */
+message ExportedBlockKeysProto {
+ required bool isBlockTokenEnabled = 1;
+ required uint64 keyUpdateInterval = 2;
+ required uint64 tokenLifeTime = 3;
+ required BlockKeyProto currentKey = 4;
+ repeated BlockKeyProto allKeys = 5;
+}
+
+/**
+ * State of a block replica at a datanode
+ */
+enum ReplicaState {
+ FINALIZED = 0; // State of a replica when it is not modified
+ RBW = 1; // State of replica that is being written to
+ RWR = 2; // State of replica that is waiting to be recovered
+ RUR = 3; // State of replica that is under recovery
+ TEMPORARY = 4; // State of replica that is created for replication
+}
+
+/**
+ * Block that needs to be recovered with at a given location
+ */
+message RecoveringBlockProto {
+ required uint64 newGenStamp = 1; // New genstamp post recovery
+ required LocatedBlockProto block = 2; // Block to be recovered
+}
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj
index c762e32385..e69de29bb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataXceiverAspects.aj
@@ -1,41 +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.hdfs.server.datanode;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * This aspect takes care about faults injected into datanode.DataXceiver
- * class
- */
-privileged public aspect DataXceiverAspects {
- public static final Log LOG = LogFactory.getLog(DataXceiverAspects.class);
-
- pointcut runXceiverThread(DataXceiver xceiver) :
- execution (* run(..)) && target(xceiver);
-
- void around (DataXceiver xceiver) : runXceiverThread(xceiver) {
- if ("true".equals(System.getProperty("fi.enabledOOM"))) {
- LOG.info("fi.enabledOOM is enabled");
- throw new OutOfMemoryError("Pretend there's no more memory");
- } else {
- proceed(xceiver);
- }
- }
-}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java
index b7e6277501..fcad32e0b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java
@@ -29,6 +29,7 @@
import org.apache.hadoop.fi.DataTransferTestUtil.VerificationAction;
import org.apache.hadoop.fi.FiTestUtil;
import org.apache.hadoop.fi.FiTestUtil.Action;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -56,8 +57,9 @@ public class TestFiDataTransferProtocol {
static private FSDataOutputStream createFile(FileSystem fs, Path p
) throws IOException {
- return fs.create(p, true, fs.getConf().getInt("io.file.buffer.size", 4096),
- REPLICATION, BLOCKSIZE);
+ return fs.create(p, true,
+ fs.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY,
+ 4096), REPLICATION, BLOCKSIZE);
}
{
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
index dcfdcf9e26..5832bf0491 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
@@ -30,6 +30,7 @@
import org.apache.hadoop.fi.DataTransferTestUtil.SleepAction;
import org.apache.hadoop.fi.DataTransferTestUtil.VerificationAction;
import org.apache.hadoop.fi.FiTestUtil;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -65,8 +66,8 @@ public class TestFiDataTransferProtocol2 {
static private FSDataOutputStream createFile(FileSystem fs, Path p
) throws IOException {
- return fs.create(p, true, fs.getConf().getInt("io.file.buffer.size", 4096),
- REPLICATION, BLOCKSIZE);
+ return fs.create(p, true, fs.getConf()
+ .getInt(IO_FILE_BUFFER_SIZE_KEY, 4096), REPLICATION, BLOCKSIZE);
}
{
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java
index 2f92fcf6ec..e69de29bb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataXceiverServer.java
@@ -1,97 +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.hdfs.server.datanode;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.util.concurrent.CountDownLatch;
-
-import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-/**
- * This is a test for DataXceiverServer when DataXceiver thread spawning is
- * failed due to OutOfMemoryError. Expected behavior is that DataXceiverServer
- * should not be exited. It should retry again after 30 seconds
- */
-public class TestFiDataXceiverServer {
-
- @Test(timeout = 30000)
- public void testOutOfMemoryErrorInDataXceiverServerRun() throws Exception {
- final CountDownLatch latch = new CountDownLatch(1);
- ServerSocket sock = new ServerSocket() {
- @Override
- public Socket accept() throws IOException {
- return new Socket() {
- @Override
- public InetAddress getInetAddress() {
- return super.getLocalAddress();
- }
-
- @Override
- public SocketAddress getRemoteSocketAddress() {
- return new InetSocketAddress(8080);
- }
-
- @Override
- public SocketAddress getLocalSocketAddress() {
- return new InetSocketAddress(0);
- }
-
- @Override
- public synchronized void close() throws IOException {
- latch.countDown();
- super.close();
- }
-
- @Override
- public InputStream getInputStream() throws IOException {
- return null;
- }
- };
- }
- };
- Thread thread = null;
- System.setProperty("fi.enabledOOM", "true");
- DataNode dn = Mockito.mock(DataNode.class);
- try {
- Configuration conf = new Configuration();
- Mockito.doReturn(conf).when(dn).getConf();
- dn.shouldRun = true;
- DataXceiverServer server = new DataXceiverServer(sock, conf, dn);
- thread = new Thread(server);
- thread.start();
- latch.await();
- assertTrue("Not running the thread", thread.isAlive());
- } finally {
- System.setProperty("fi.enabledOOM", "false");
- dn.shouldRun = false;
- if (null != thread)
- thread.interrupt();
- sock.close();
- }
- }
-}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestHelperDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestHelperDFS.java
index fc3567ea5d..1dfa2952cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestHelperDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestHelperDFS.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.cli;
import org.apache.hadoop.cli.util.CLICommandDFSAdmin;
-import org.apache.hadoop.cli.util.CLITestCmd;
import org.xml.sax.SAXException;
public class CLITestHelperDFS extends CLITestHelper {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestHDFSCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestHDFSCLI.java
index eb62f4063d..4bf16d7ea4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestHDFSCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestHDFSCLI.java
@@ -29,7 +29,6 @@
import org.junit.After;
import static org.junit.Assert.assertTrue;
import org.junit.Before;
-import org.junit.Test;
public class TestHDFSCLI extends CLITestHelperDFS {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSymlink.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSymlink.java
index 617b90026c..3f74789ae9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSymlink.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSymlink.java
@@ -20,6 +20,9 @@
import java.io.*;
import java.net.URI;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.log4j.Level;
+
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileContext;
@@ -28,9 +31,11 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import static org.apache.hadoop.fs.FileContextTestHelper.*;
import org.apache.hadoop.ipc.RemoteException;
+
import static org.junit.Assert.*;
import org.junit.Test;
import org.junit.BeforeClass;
@@ -41,6 +46,10 @@
*/
public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
+ {
+ ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+ }
+
private static MiniDFSCluster cluster;
protected String getScheme() {
@@ -250,8 +259,8 @@ public void testLinkOwner() throws IOException {
Path link = new Path(testBaseDir1(), "symlinkToFile");
createAndWriteFile(file);
fc.createSymlink(file, link, false);
- FileStatus stat_file = fc.getFileStatus(file);
- FileStatus stat_link = fc.getFileStatus(link);
- assertEquals(stat_link.getOwner(), stat_file.getOwner());
+ FileStatus statFile = fc.getFileStatus(file);
+ FileStatus statLink = fc.getFileStatus(link);
+ assertEquals(statLink.getOwner(), statFile.getOwner());
}
}
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 fb562d14b2..593134350d 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,7 @@ public void testGeneralSBBehavior() throws IOException, InterruptedException {
try {
Configuration conf = new HdfsConfiguration();
conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
- conf.setBoolean("dfs.support.append", 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/AppendTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
index 5369a9e2e4..384cfe75b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
@@ -143,8 +143,8 @@ static byte[] initBuffer(int size) {
public static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
throws IOException {
return fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short) repl, (long) BLOCK_SIZE);
+ fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short) repl, BLOCK_SIZE);
}
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
index 3c338e56f5..9d4f4a2e19 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
@@ -30,6 +30,7 @@
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.net.NetUtils;
@@ -148,7 +149,7 @@ public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToR
sock, targetAddr.toString()+ ":" + block.getBlockId(), block,
testBlock.getBlockToken(),
offset, lenToRead,
- conf.getInt("io.file.buffer.size", 4096),
+ conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
true, "");
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index c7566d2c62..409dd37525 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -84,6 +84,7 @@ public class DFSTestUtil {
private int maxLevels;// = 3;
private int maxSize;// = 8*1024;
+ private int minSize = 1;
private int nFiles;
private MyFile[] files;
@@ -139,7 +140,7 @@ private class MyFile {
long fidx = -1;
while (fidx < 0) { fidx = gen.nextLong(); }
name = name + Long.toString(fidx);
- size = gen.nextInt(maxSize);
+ size = minSize + gen.nextInt(maxSize - minSize);
seed = gen.nextLong();
}
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 7f2c1aecd6..d837c0f71a 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
@@ -20,6 +20,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.junit.AfterClass;
@@ -66,7 +67,7 @@ public static void startUp () throws IOException {
}
@AfterClass
- public static void tearDown() throws IOException {
+ public static void tearDown() {
cluster.shutdown();
}
@@ -91,7 +92,7 @@ public void testAppend() throws IOException {
new Path("foo"+ oldFileLen +"_"+ flushedBytes1 +"_"+ flushedBytes2);
LOG.info("Creating file " + p);
FSDataOutputStream out = fs.create(p, false,
- conf.getInt("io.file.buffer.size", 4096),
+ conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
REPLICATION, BLOCK_SIZE);
out.write(contents, 0, oldFileLen);
out.close();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 8f5f9f8fda..df913b37d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -36,21 +36,22 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.commons.math.stat.descriptive.rank.Min;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ClientDatanodeWireProtocol;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
@@ -323,8 +324,8 @@ public MiniDFSCluster() {
* Servers will be started on free ports.
*
* The caller must manage the creation of NameNode and DataNode directories
- * and have already set {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and
- * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} in the given conf.
+ * and have already set {@link #DFS_NAMENODE_NAME_DIR_KEY} and
+ * {@link #DFS_DATANODE_DATA_DIR_KEY} in the given conf.
*
* @param conf the base configuration to use in starting the servers. This
* will be modified as necessary.
@@ -398,8 +399,8 @@ public MiniDFSCluster(Configuration conf,
* @param format if true, format the NameNode and DataNodes before starting
* up
* @param manageDfsDirs if true, the data directories for servers will be
- * created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and
- * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in
+ * created and {@link #DFS_NAMENODE_NAME_DIR_KEY} and
+ * {@link #DFS_DATANODE_DATA_DIR_KEY} will be set in
* the conf
* @param operation the operation with which to start the servers. If null
* or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -430,8 +431,8 @@ public MiniDFSCluster(int nameNodePort,
* @param numDataNodes Number of DataNodes to start; may be zero
* @param format if true, format the NameNode and DataNodes before starting up
* @param manageDfsDirs if true, the data directories for servers will be
- * created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and
- * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in
+ * created and {@link #DFS_NAMENODE_NAME_DIR_KEY} and
+ * {@link #DFS_DATANODE_DATA_DIR_KEY} will be set in
* the conf
* @param operation the operation with which to start the servers. If null
* or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -464,11 +465,11 @@ public MiniDFSCluster(int nameNodePort,
* @param numDataNodes Number of DataNodes to start; may be zero
* @param format if true, format the NameNode and DataNodes before starting up
* @param manageNameDfsDirs if true, the data directories for servers will be
- * created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and
- * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in
+ * created and {@link #DFS_NAMENODE_NAME_DIR_KEY} and
+ * {@link #DFS_DATANODE_DATA_DIR_KEY} will be set in
* the conf
* @param manageDataDfsDirs if true, the data directories for datanodes will
- * be created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY}
+ * be created and {@link #DFS_DATANODE_DATA_DIR_KEY}
* set to same in the conf
* @param operation the operation with which to start the servers. If null
* or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -513,6 +514,8 @@ private void initMiniDFSCluster(int nameNodePort, int nameNodeHttpPort,
try {
Class> rpcEngine = conf.getClassByName(rpcEngineName);
setRpcEngine(conf, NamenodeProtocols.class, rpcEngine);
+ setRpcEngine(conf, ClientNamenodeWireProtocol.class, rpcEngine);
+ setRpcEngine(conf, ClientDatanodeWireProtocol.class, rpcEngine);
setRpcEngine(conf, NamenodeProtocol.class, rpcEngine);
setRpcEngine(conf, ClientProtocol.class, rpcEngine);
setRpcEngine(conf, DatanodeProtocol.class, rpcEngine);
@@ -524,15 +527,15 @@ private void initMiniDFSCluster(int nameNodePort, int nameNodeHttpPort,
}
// disable service authorization, as it does not work with tunnelled RPC
- conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
+ conf.setBoolean(HADOOP_SECURITY_AUTHORIZATION,
false);
}
- int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
- conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
- conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
- conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
- conf.setClass(DFSConfigKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+ int replication = conf.getInt(DFS_REPLICATION_KEY, 3);
+ conf.setInt(DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
+ conf.setInt(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
+ conf.setInt(DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
+ conf.setClass(NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
StaticMapping.class, DNSToSwitchMapping.class);
Collection nameserviceIds = DFSUtil.getNameServiceIds(conf);
@@ -540,8 +543,8 @@ private void initMiniDFSCluster(int nameNodePort, int nameNodeHttpPort,
federation = true;
if (!federation) {
- conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "127.0.0.1:" + nameNodePort);
- conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:"
+ conf.set(FS_DEFAULT_NAME_KEY, "127.0.0.1:" + nameNodePort);
+ conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:"
+ nameNodeHttpPort);
NameNode nn = createNameNode(0, conf, numDataNodes, manageNameDfsDirs,
format, operation, clusterId);
@@ -585,7 +588,7 @@ private static void initFederationConf(Configuration conf,
initFederatedNamenodeAddress(conf, nameserviceId, nnPort);
nnPort = nnPort == 0 ? 0 : nnPort + 2;
}
- conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nameserviceIdList);
+ conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIdList);
}
/* For federated namenode initialize the address:port */
@@ -593,11 +596,11 @@ private static void initFederatedNamenodeAddress(Configuration conf,
String nameserviceId, int nnPort) {
// Set nameserviceId specific key
String key = DFSUtil.addKeySuffixes(
- DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId);
+ DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId);
conf.set(key, "127.0.0.1:0");
key = DFSUtil.addKeySuffixes(
- DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId);
+ DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId);
conf.set(key, "127.0.0.1:" + nnPort);
}
@@ -618,10 +621,10 @@ private NameNode createNameNode(int nnIndex, Configuration conf,
StartupOption operation, String clusterId)
throws IOException {
if (manageNameDfsDirs) {
- conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+ conf.set(DFS_NAMENODE_NAME_DIR_KEY,
fileAsURI(new File(base_dir, "name" + (2*nnIndex + 1)))+","+
fileAsURI(new File(base_dir, "name" + (2*nnIndex + 2))));
- conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
+ conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,
fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 1)))+","+
fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 2))));
}
@@ -646,17 +649,17 @@ private void createFederatedNameNode(int nnIndex, Configuration conf,
int numDataNodes, boolean manageNameDfsDirs, boolean format,
StartupOption operation, String clusterId, String nameserviceId)
throws IOException {
- conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
+ conf.set(DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
NameNode nn = createNameNode(nnIndex, conf, numDataNodes, manageNameDfsDirs,
format, operation, clusterId);
conf.set(DFSUtil.addKeySuffixes(
- DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId), NameNode
+ DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId), NameNode
.getHostPortString(nn.getNameNodeAddress()));
conf.set(DFSUtil.addKeySuffixes(
- DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId), NameNode
+ DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId), NameNode
.getHostPortString(nn.getHttpAddress()));
DFSUtil.setGenericConf(conf, nameserviceId,
- DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+ DFS_NAMENODE_HTTP_ADDRESS_KEY);
nameNodes[nnIndex] = new NameNodeInfo(nn, new Configuration(conf));
}
@@ -736,7 +739,7 @@ public void waitClusterUp() {
* will be modified as necessary.
* @param numDataNodes Number of DataNodes to start; may be zero
* @param manageDfsDirs if true, the data directories for DataNodes will be
- * created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set
+ * created and {@link #DFS_DATANODE_DATA_DIR_KEY} will be set
* in the conf
* @param operation the operation with which to start the DataNodes. If null
* or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -768,7 +771,7 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes,
* will be modified as necessary.
* @param numDataNodes Number of DataNodes to start; may be zero
* @param manageDfsDirs if true, the data directories for DataNodes will be
- * created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be
+ * created and {@link #DFS_DATANODE_DATA_DIR_KEY} will be
* set in the conf
* @param operation the operation with which to start the DataNodes. If null
* or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -802,7 +805,7 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes,
* will be modified as necessary.
* @param numDataNodes Number of DataNodes to start; may be zero
* @param manageDfsDirs if true, the data directories for DataNodes will be
- * created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be
+ * created and {@link #DFS_DATANODE_DATA_DIR_KEY} will be
* set in the conf
* @param operation the operation with which to start the DataNodes. If null
* or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
@@ -820,12 +823,12 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes,
long[] simulatedCapacities,
boolean setupHostsFile,
boolean checkDataNodeAddrConfig) throws IOException {
- conf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
+ conf.set(DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
int curDatanodesNum = dataNodes.size();
// for mincluster's the default initialDelay for BRs is 0
- if (conf.get(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
- conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, 0);
+ if (conf.get(DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
+ conf.setLong(DFS_BLOCKREPORT_INITIAL_DELAY_KEY, 0);
}
// If minicluster's name node is null assume that the conf has been
// set with the right address:port of the name node.
@@ -872,8 +875,8 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes,
+ i + ": " + dir1 + " or " + dir2);
}
String dirs = fileAsURI(dir1) + "," + fileAsURI(dir2);
- dnConf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs);
- conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs);
+ dnConf.set(DFS_DATANODE_DATA_DIR_KEY, dirs);
+ conf.set(DFS_DATANODE_DATA_DIR_KEY, dirs);
}
if (simulatedCapacities != null) {
dnConf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
@@ -902,7 +905,7 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes,
DataNode dn = DataNode.instantiateDataNode(dnArgs, dnConf);
if(dn == null)
throw new IOException("Cannot start DataNode in "
- + dnConf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
+ + dnConf.get(DFS_DATANODE_DATA_DIR_KEY));
//since the HDFS does things based on IP:port, we need to add the mapping
//for IP:port to rackId
String ipAddr = dn.getSelfAddr().getAddress().getHostAddress();
@@ -1318,7 +1321,7 @@ public synchronized boolean restartDataNode(DataNodeProperties dnprop,
Configuration newconf = new HdfsConfiguration(conf); // save cloned config
if (keepPort) {
InetSocketAddress addr = dnprop.datanode.getSelfAddr();
- conf.set("dfs.datanode.address", addr.getAddress().getHostAddress() + ":"
+ conf.set(DFS_DATANODE_ADDRESS_KEY, addr.getAddress().getHostAddress() + ":"
+ addr.getPort());
}
dataNodes.add(new DataNodeProperties(DataNode.createDataNode(args, conf),
@@ -1445,10 +1448,10 @@ public FileSystem getNewFileSystemInstance(int nnIndex) throws IOException {
/**
* @return a http URL
*/
- public String getHttpUri(int nnIndex) throws IOException {
+ public String getHttpUri(int nnIndex) {
return "http://"
+ nameNodes[nnIndex].conf
- .get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+ .get(DFS_NAMENODE_HTTP_ADDRESS_KEY);
}
/**
@@ -1457,7 +1460,7 @@ public String getHttpUri(int nnIndex) throws IOException {
public HftpFileSystem getHftpFileSystem(int nnIndex) throws IOException {
String uri = "hftp://"
+ nameNodes[nnIndex].conf
- .get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+ .get(DFS_NAMENODE_HTTP_ADDRESS_KEY);
try {
return (HftpFileSystem)FileSystem.get(new URI(uri), conf);
} catch (URISyntaxException e) {
@@ -1907,9 +1910,9 @@ public NameNode addNameNode(Configuration conf, int namenodePort)
nameNodes = newlist;
String nameserviceId = NAMESERVICE_ID_PREFIX + (nnIndex + 1);
- String nameserviceIds = conf.get(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES);
+ String nameserviceIds = conf.get(DFS_FEDERATION_NAMESERVICES);
nameserviceIds += "," + nameserviceId;
- conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nameserviceIds);
+ conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIds);
initFederatedNamenodeAddress(conf, nameserviceId, namenodePort);
createFederatedNameNode(nnIndex, conf, numDataNodes, true, true, null,
@@ -1942,28 +1945,28 @@ private int getFreeSocketPort() {
private void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,
boolean checkDataNodeAddrConfig) throws IOException {
if (setupHostsFile) {
- String hostsFile = conf.get(DFSConfigKeys.DFS_HOSTS, "").trim();
+ String hostsFile = conf.get(DFS_HOSTS, "").trim();
if (hostsFile.length() == 0) {
throw new IOException("Parameter dfs.hosts is not setup in conf");
}
// Setup datanode in the include file, if it is defined in the conf
String address = "127.0.0.1:" + getFreeSocketPort();
if (checkDataNodeAddrConfig) {
- conf.setIfUnset("dfs.datanode.address", address);
+ conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, address);
} else {
- conf.set("dfs.datanode.address", address);
+ conf.set(DFS_DATANODE_ADDRESS_KEY, address);
}
addToFile(hostsFile, address);
LOG.info("Adding datanode " + address + " to hosts file " + hostsFile);
} else {
if (checkDataNodeAddrConfig) {
- conf.setIfUnset("dfs.datanode.address", "127.0.0.1:0");
- conf.setIfUnset("dfs.datanode.http.address", "127.0.0.1:0");
- conf.setIfUnset("dfs.datanode.ipc.address", "127.0.0.1:0");
+ conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
+ conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+ conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
} else {
- conf.set("dfs.datanode.address", "127.0.0.1:0");
- conf.set("dfs.datanode.http.address", "127.0.0.1:0");
- conf.set("dfs.datanode.ipc.address", "127.0.0.1:0");
+ conf.set(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
+ conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+ conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
}
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java
index afb58c4555..117952a7e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java
@@ -25,7 +25,6 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.DFSConfigKeys;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java
index 64e8588790..be6e741a0a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java
@@ -24,6 +24,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -80,9 +81,9 @@ public void testBlockMissingException() throws Exception {
//
private void createOldFile(FileSystem fileSys, Path name, int repl, int numBlocks, long blocksize)
throws IOException {
- FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short)repl, blocksize);
+ FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short) repl, blocksize);
// fill data into file
final byte[] b = new byte[(int)blocksize];
for (int i = 0; i < numBlocks; i++) {
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 e7988f99bb..259d26a304 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
@@ -25,7 +25,6 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.io.IOUtils;
@@ -41,7 +40,7 @@ public class TestClientProtocolForPipelineRecovery {
@Test public void testGetNewStamp() throws IOException {
int numDataNodes = 1;
Configuration conf = new HdfsConfiguration();
- conf.setBoolean("dfs.support.append", true);
+ 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/TestClientReportBadBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
index 97387ccfc1..33ee5c95a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
@@ -28,6 +28,7 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -80,7 +81,7 @@ public void startUpCluster() throws IOException {
.build();
cluster.waitActive();
dfs = (DistributedFileSystem) cluster.getFileSystem();
- buffersize = conf.getInt("io.file.buffer.size", 4096);
+ buffersize = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
}
@After
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java
index 136a72205c..6673bf547b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java
@@ -20,8 +20,6 @@
import java.net.InetSocketAddress;
import java.net.Socket;
import java.io.IOException;
-import java.util.List;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -31,11 +29,11 @@
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSInputStream;
import org.apache.hadoop.hdfs.SocketCache;
-import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.security.token.Token;
import org.junit.Test;
@@ -212,6 +210,7 @@ public void testReadFromOneDN() throws IOException {
MockGetBlockReader answer = new MockGetBlockReader();
Mockito.doAnswer(answer).when(in).getBlockReader(
(InetSocketAddress) Matchers.anyObject(),
+ (DatanodeInfo) Matchers.anyObject(),
Matchers.anyString(),
(ExtendedBlock) Matchers.anyObject(),
(Token) Matchers.anyObject(),
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java
index 1407fd46a0..0f0caa673b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java
@@ -26,11 +26,11 @@
import java.io.IOException;
import java.util.ArrayList;
-import junit.framework.Assert;
import junit.framework.TestCase;
-import java.net.InetSocketAddress;
-
import org.apache.hadoop.conf.Configuration;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
@@ -65,9 +65,9 @@ public void testDFSAddressConfig() throws IOException {
assertNotNull("Should have been able to stop simulated datanode", dnp);
}
- conf.unset("dfs.datanode.address");
- conf.unset("dfs.datanode.http.address");
- conf.unset("dfs.datanode.ipc.address");
+ conf.unset(DFS_DATANODE_ADDRESS_KEY);
+ conf.unset(DFS_DATANODE_HTTP_ADDRESS_KEY);
+ conf.unset(DFS_DATANODE_IPC_ADDRESS_KEY);
cluster.startDataNodes(conf, 1, true, StartupOption.REGULAR,
null, null, null, false, true);
@@ -90,9 +90,9 @@ public void testDFSAddressConfig() throws IOException {
assertNotNull("Should have been able to stop simulated datanode", dnp);
}
- conf.set("dfs.datanode.address","0.0.0.0:0");
- conf.set("dfs.datanode.http.address","0.0.0.0:0");
- conf.set("dfs.datanode.ipc.address","0.0.0.0:0");
+ conf.set(DFS_DATANODE_ADDRESS_KEY, "0.0.0.0:0");
+ conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+ conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
cluster.startDataNodes(conf, 1, true, StartupOption.REGULAR,
null, null, null, false, true);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index 9cc1b2999c..af12badddb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -43,6 +43,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileSystem;
@@ -58,10 +59,7 @@
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils;
@@ -144,7 +142,7 @@ public void testWriteTimeoutAtDataNode() throws IOException,
conf.setInt(DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, 1);
// set a small buffer size
final int bufferSize = 4096;
- conf.setInt("io.file.buffer.size", bufferSize);
+ conf.setInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, bufferSize);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
index 495e8e191a..38a837247a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
@@ -22,14 +22,13 @@
import java.util.Map;
import java.util.Random;
-import javax.security.auth.login.LoginException;
-
import junit.framework.AssertionFailedError;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
@@ -202,7 +201,7 @@ private void create(OpType op, Path name, short umask,
switch (op) {
case CREATE:
FSDataOutputStream out = fs.create(name, permission, true,
- conf.getInt("io.file.buffer.size", 4096),
+ conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
out.close();
break;
@@ -520,8 +519,7 @@ protected void set(Path path, short ancestorPermission,
}
/* Perform an operation and verify if the permission checking is correct */
- void verifyPermission(UserGroupInformation ugi) throws LoginException,
- IOException {
+ void verifyPermission(UserGroupInformation ugi) throws IOException {
if (this.ugi != ugi) {
setRequiredPermissions(ugi);
this.ugi = ugi;
@@ -564,8 +562,7 @@ protected boolean expectPermissionDeny() {
}
/* Set the permissions required to pass the permission checking */
- protected void setRequiredPermissions(UserGroupInformation ugi)
- throws IOException {
+ protected void setRequiredPermissions(UserGroupInformation ugi) {
if (SUPERUSER.equals(ugi)) {
requiredAncestorPermission = SUPER_MASK;
requiredParentPermission = SUPER_MASK;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
index a8f814b652..b07bad252e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
@@ -30,7 +30,6 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
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;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
index 6ad08cd2aa..567fbabddd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
@@ -33,7 +33,6 @@
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
-import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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 4333f1c0f9..089ab4d837 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
@@ -31,7 +31,6 @@
import junit.framework.TestCase;
-import org.apache.commons.digester.SetRootRule;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -209,7 +208,7 @@ private void testWrite(ExtendedBlock block, BlockConstructionStage stage, long n
@Test public void testOpWrite() throws IOException {
int numDataNodes = 1;
Configuration conf = new HdfsConfiguration();
- conf.setBoolean("dfs.support.append", true);
+ 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/TestDatanodeDeath.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java
index b061f26783..e271bb95a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java
@@ -25,6 +25,7 @@
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -123,9 +124,9 @@ public synchronized long getStamp() {
static private FSDataOutputStream createFile(FileSystem fileSys, Path name, short repl)
throws IOException {
// create and write a file that contains three blocks of data
- FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- repl, (long)blockSize);
+ FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
+ blockSize);
return stm;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index 3069727a48..faf7efd536 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -30,6 +30,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -115,9 +116,9 @@ private void writeConfigFile(Path name, ArrayList nodes)
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
// create and write a file that contains three blocks of data
- FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short)repl, (long)blockSize);
+ FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short) repl, blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
@@ -246,7 +247,7 @@ private void recomissionNode(DatanodeInfo decommissionedNode) throws IOException
* Wait till node is fully decommissioned.
*/
private void waitNodeState(DatanodeInfo node,
- AdminStates state) throws IOException {
+ AdminStates state) {
boolean done = state == node.getAdminState();
while (!done) {
LOG.info("Waiting for node " + node + " to change state to "
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeprecatedKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeprecatedKeys.java
index 474b2f5086..518adddf10 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeprecatedKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeprecatedKeys.java
@@ -20,9 +20,6 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.conf.Configuration;
-import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.log4j.Level;
-
import junit.framework.TestCase;
public class TestDeprecatedKeys extends TestCase {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index aaa085f1e7..4055cd8d3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -31,6 +31,7 @@
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileChecksum;
@@ -88,17 +89,17 @@ public void testFileSystemCloseAll() throws Exception {
@Test
public void testDFSClose() throws Exception {
Configuration conf = getTestConfiguration();
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
- FileSystem fileSys = cluster.getFileSystem();
-
+ MiniDFSCluster cluster = null;
try {
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+ FileSystem fileSys = cluster.getFileSystem();
+
// create two files
fileSys.create(new Path("/test/dfsclose/file-0"));
fileSys.create(new Path("/test/dfsclose/file-1"));
fileSys.close();
- }
- finally {
+ } finally {
if (cluster != null) {cluster.shutdown();}
}
}
@@ -106,10 +107,10 @@ public void testDFSClose() throws Exception {
@Test
public void testDFSSeekExceptions() throws IOException {
Configuration conf = getTestConfiguration();
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
- FileSystem fileSys = cluster.getFileSystem();
-
+ MiniDFSCluster cluster = null;
try {
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+ FileSystem fileSys = cluster.getFileSystem();
String file = "/test/fileclosethenseek/file-0";
Path path = new Path(file);
// create file
@@ -455,7 +456,7 @@ public FileSystem run() throws Exception {
final Path dir = new Path("/filechecksum");
final int block_size = 1024;
- final int buffer_size = conf.getInt("io.file.buffer.size", 4096);
+ final int buffer_size = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
//try different number of blocks
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java
index 335d8a58ac..fea024c2c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java
@@ -21,14 +21,13 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.RandomAccessFile;
-import java.util.EnumSet;
import java.util.Random;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -56,7 +55,7 @@ public class TestFSInputChecker extends TestCase {
private void writeFile(FileSystem fileSys, Path name) throws IOException {
// create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, new FsPermission((short)0777),
- true, fileSys.getConf().getInt("io.file.buffer.size", 4096),
+ true, fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
NUM_OF_DATANODES, BLOCK_SIZE, null);
stm.write(expected);
stm.close();
@@ -327,8 +326,10 @@ private void testSeekAndRead(FileSystem fileSys)
throws IOException {
Path file = new Path("try.dat");
writeFile(fileSys, file);
- stm = fileSys.open(file,
- fileSys.getConf().getInt("io.file.buffer.size", 4096));
+ stm = fileSys.open(
+ file,
+ fileSys.getConf().getInt(
+ CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096));
checkSeekAndRead();
stm.close();
cleanupFile(fileSys, file);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSOutputSummer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSOutputSummer.java
index 8a18420aad..da18bbe0cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSOutputSummer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSOutputSummer.java
@@ -21,6 +21,8 @@
import java.io.*;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY;
+
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -43,7 +45,7 @@ public class TestFSOutputSummer extends TestCase {
/* create a file, write all data at once */
private void writeFile1(Path name) throws Exception {
FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
+ fileSys.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, 4096),
NUM_OF_DATANODES, BLOCK_SIZE);
stm.write(expected);
stm.close();
@@ -54,7 +56,7 @@ private void writeFile1(Path name) throws Exception {
/* create a file, write data chunk by chunk */
private void writeFile2(Path name) throws Exception {
FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
+ fileSys.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, 4096),
NUM_OF_DATANODES, BLOCK_SIZE);
int i=0;
for( ;i token = lb.get(0).getBlockToken();
+ final DatanodeInfo dnInfo = lb.get(0).getLocations()[0];
+ ClientDatanodeProtocol proxy = aUgi
+ .doAs(new PrivilegedExceptionAction() {
+ @Override
+ public ClientDatanodeProtocol run() throws Exception {
+ return DFSUtil.createClientDatanodeProtocolProxy(dnInfo, conf,
+ 60000);
+ }
+ });
+
+ //This should succeed
+ BlockLocalPathInfo blpi = proxy.getBlockLocalPathInfo(blk, token);
+ Assert.assertEquals(dn.data.getBlockLocalPathInfo(blk).getBlockPath(),
+ blpi.getBlockPath());
+
+ // Now try with a not allowed user.
+ UserGroupInformation bUgi = UserGroupInformation
+ .createRemoteUser("notalloweduser");
+ proxy = bUgi
+ .doAs(new PrivilegedExceptionAction() {
+ @Override
+ public ClientDatanodeProtocol run() throws Exception {
+ return DFSUtil.createClientDatanodeProtocolProxy(dnInfo, conf,
+ 60000);
+ }
+ });
+ try {
+ proxy.getBlockLocalPathInfo(blk, token);
+ Assert.fail("The call should have failed as " + bUgi.getShortUserName()
+ + " is not allowed to call getBlockLocalPathInfo");
+ } catch (IOException ex) {
+ Assert.assertTrue(ex.getMessage().contains(
+ "not allowed to call getBlockLocalPathInfo"));
+ }
+ } finally {
+ fs.close();
+ cluster.shutdown();
+ }
+ }
+
+ /**
+ * Test to run benchmarks between shortcircuit read vs regular read with
+ * specified number of threads simultaneously reading.
+ *
+ * Run this using the following command:
+ * bin/hadoop --config confdir \
+ * org.apache.hadoop.hdfs.TestShortCircuitLocalRead \
+ *
+ */
+ public static void main(String[] args) throws Exception {
+ if (args.length != 3) {
+ System.out.println("Usage: test shortcircuit checksum threadCount");
+ System.exit(1);
+ }
+ boolean shortcircuit = Boolean.valueOf(args[0]);
+ boolean checksum = Boolean.valueOf(args[1]);
+ int threadCount = Integer.valueOf(args[2]);
+
+ // Setup create a file
+ Configuration conf = new Configuration();
+ conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, shortcircuit);
+ conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
+ checksum);
+
+ //Override fileSize and DATA_TO_WRITE to much larger values for benchmark test
+ int fileSize = 1000 * blockSize + 100; // File with 1000 blocks
+ final byte [] dataToWrite = AppendTestUtil.randomBytes(seed, fileSize);
+
+ // create a new file in home directory. Do not close it.
+ final Path file1 = new Path("filelocal.dat");
+ final FileSystem fs = FileSystem.get(conf);
+ FSDataOutputStream stm = createFile(fs, file1, 1);
+
+ stm.write(dataToWrite);
+ stm.close();
+
+ long start = System.currentTimeMillis();
+ final int iteration = 20;
+ Thread[] threads = new Thread[threadCount];
+ for (int i = 0; i < threadCount; i++) {
+ threads[i] = new Thread() {
+ public void run() {
+ for (int i = 0; i < iteration; i++) {
+ try {
+ checkFileContent(fs, file1, dataToWrite, 0);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ };
+ }
+ for (int i = 0; i < threadCount; i++) {
+ threads[i].start();
+ }
+ for (int i = 0; i < threadCount; i++) {
+ threads[i].join();
+ }
+ long end = System.currentTimeMillis();
+ System.out.println("Iteration " + iteration + " took " + (end - start));
+ fs.delete(file1, false);
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
index 8fb2b7a38c..77c4007e4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
@@ -21,6 +21,7 @@
import java.io.*;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -40,9 +41,9 @@ public class TestSmallBlock extends TestCase {
private void writeFile(FileSystem fileSys, Path name) throws IOException {
// create and write a file that contains three blocks of data
- FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short)1, (long)blockSize);
+ FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short) 1, blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
@@ -92,7 +93,7 @@ private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
public void testSmallBlock() throws IOException {
Configuration conf = new HdfsConfiguration();
if (simulatedStorage) {
- conf.setBoolean("dfs.datanode.simulateddatastorage", true);
+ conf.setBoolean(DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY, true);
}
conf.set(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, "1");
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
index 45f41dc7ef..6ca0ffe7b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
@@ -19,7 +19,6 @@
import java.io.IOException;
import java.net.InetSocketAddress;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Random;
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 ddef17ba0f..0a25ef7983 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
@@ -31,6 +31,7 @@
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -123,7 +124,7 @@ private boolean checkFile(byte[] fileToCheck) {
private static FSDataOutputStream writeFile(FileSystem fileSys, Path name,
short repl, long blockSize) throws IOException {
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
- .getInt("io.file.buffer.size", 4096), repl, blockSize);
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl, blockSize);
return stm;
}
@@ -173,7 +174,7 @@ private static void tryRead(Configuration conf, LocatedBlock lblock,
}
// get a conf for testing
- private static Configuration getConf(int numDataNodes) throws IOException {
+ private static Configuration getConf(int numDataNodes) {
Configuration conf = new Configuration();
conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -181,7 +182,8 @@ private static Configuration getConf(int numDataNodes) throws IOException {
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("dfs.support.append", true);
+ 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/blockmanagement/TestBlocksWithNotEnoughRacks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
index 01018e8c05..df00728764 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
@@ -391,7 +391,7 @@ public void testNodeDecomissionRespectsRackPolicy() throws Exception {
Path excludeFile = new Path(dir, "exclude");
assertTrue(localFileSys.mkdirs(dir));
DFSTestUtil.writeFile(localFileSys, excludeFile, "");
- conf.set("dfs.hosts.exclude", excludeFile.toUri().getPath());
+ conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
// Two blocks and four racks
String racks[] = {"/rack1", "/rack1", "/rack2", "/rack2"};
@@ -441,7 +441,7 @@ public void testNodeDecomissionWithOverreplicationRespectsRackPolicy()
Path excludeFile = new Path(dir, "exclude");
assertTrue(localFileSys.mkdirs(dir));
DFSTestUtil.writeFile(localFileSys, excludeFile, "");
- conf.set("dfs.hosts.exclude", excludeFile.toUri().getPath());
+ conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
// All hosts are on two racks, only one host on /rack2
String racks[] = {"/rack1", "/rack2", "/rack1", "/rack1", "/rack1"};
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
index 359fa0245b..2bfcbdefd5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
@@ -21,8 +21,6 @@
import java.io.IOException;
-import junit.framework.TestCase;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -38,9 +36,12 @@
import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
/**
*/
-public class TestDistributedUpgrade extends TestCase {
+public class TestDistributedUpgrade {
private static final Log LOG = LogFactory.getLog(TestDistributedUpgrade.class);
private Configuration conf;
private int testCounter = 0;
@@ -95,6 +96,7 @@ void startDataNodeShouldFail(StartupOption operation) {
/**
*/
+ @Test(timeout=120000)
public void testDistributedUpgrade() throws Exception {
int numDirs = 1;
TestDFSUpgradeFromImage testImg = new TestDFSUpgradeFromImage();
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 a3d47b623e..2e3bd92373 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
@@ -21,10 +21,7 @@
import java.io.IOException;
-import org.apache.hadoop.hdfs.server.datanode.DataNode.BPOfferService;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
/**
* Utility class for accessing package-private DataNode information during tests.
@@ -41,27 +38,4 @@ public class DataNodeTestUtils {
return dn.getDNRegistrationForBP(bpid);
}
- /**
- * manually setup datanode to testing
- * @param dn - datanode
- * @param nsifno - namenode info
- * @param bpid - block pool id
- * @param nn - namenode object
- * @throws IOException
- */
- public static void setBPNamenodeByIndex(DataNode dn,
- NamespaceInfo nsifno, String bpid, DatanodeProtocol nn)
- throws IOException {
- // setup the right BPOS..
- BPOfferService [] bposs = dn.getAllBpOs();
- if(bposs.length<0) {
- throw new IOException("Datanode wasn't initializes with at least one NN");
- }
- for(BPOfferService bpos : bposs) {
- bpos.setNamespaceInfo(nsifno);
-
- dn.setBPNamenode(bpid, nn);
- dn.initBlockPool(bpos, nsifno);
- }
- }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index ac82322283..9dbcc2f736 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -35,8 +35,8 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
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.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.datanode.FSDataset.BlockPoolSlice;
import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolumeSet;
@@ -66,9 +66,9 @@
public class SimulatedFSDataset implements FSDatasetInterface, Configurable{
public static final String CONFIG_PROPERTY_SIMULATED =
- "dfs.datanode.simulateddatastorage";
+ DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY;
public static final String CONFIG_PROPERTY_CAPACITY =
- "dfs.datanode.simulateddatastorage.capacity";
+ DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_CAPACITY_KEY;
public static final long DEFAULT_CAPACITY = 2L<<40; // 1 terabyte
public static final byte DEFAULT_DATABYTE = 9; // 1 terabyte
@@ -136,7 +136,7 @@ synchronized public void setNumBytes(long length) {
}
}
- synchronized SimulatedInputStream getIStream() throws IOException {
+ synchronized SimulatedInputStream getIStream() {
if (!finalized) {
// throw new IOException("Trying to read an unfinalized block");
return new SimulatedInputStream(oStream.getLength(), DEFAULT_DATABYTE);
@@ -363,7 +363,7 @@ private SimulatedBPStorage getBPStorage(String bpid) throws IOException {
private SimulatedStorage storage = null;
private String storageId;
- public SimulatedFSDataset(Configuration conf) throws IOException {
+ public SimulatedFSDataset(Configuration conf) {
setConf(conf);
}
@@ -992,4 +992,10 @@ public ReplicaInPipelineInterface convertTemporaryToRbw(ExtendedBlock temporary)
}
return r;
}
+
+ @Override
+ public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b)
+ throws IOException {
+ throw new IOException("getBlockLocalPathInfo not supported.");
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java
index 20d5dc6a70..b7a10177c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java
@@ -28,7 +28,6 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.datanode.DataNode.BPOfferService;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
index 04fa0ac87f..7b26f4e805 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
@@ -31,7 +31,6 @@
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.datanode.DataNode.BPOfferService;
import org.apache.hadoop.hdfs.server.datanode.FSDataset.VolumeInfo;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
index b27f2efdc4..47bfa703db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
@@ -38,7 +38,6 @@
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.log4j.Level;
import org.junit.After;
import org.junit.Before;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
index 11e5f994fa..97554e7a80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
@@ -18,39 +18,41 @@
package org.apache.hadoop.hdfs.server.datanode;
-import java.io.File;
-import java.io.IOException;
-import java.util.AbstractList;
-
+import java.net.InetSocketAddress;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.junit.Test;
+import org.mockito.Mockito;
public class TestDatanodeRegister {
public static final Log LOG = LogFactory.getLog(TestDatanodeRegister.class);
+
+ // Invalid address
+ static final InetSocketAddress INVALID_ADDR =
+ new InetSocketAddress("127.0.0.1", 1);
+
@Test
public void testDataNodeRegister() throws Exception {
- DataNode.BPOfferService myMockBPOS = mock(DataNode.BPOfferService.class);
- doCallRealMethod().when(myMockBPOS).register();
- myMockBPOS.bpRegistration = mock(DatanodeRegistration.class);
- when(myMockBPOS.bpRegistration.getStorageID()).thenReturn("myTestStorageID");
+ DataNode mockDN = mock(DataNode.class);
+ Mockito.doReturn(true).when(mockDN).shouldRun();
+ BPOfferService bpos = new BPOfferService(INVALID_ADDR, mockDN);
+
NamespaceInfo fakeNSInfo = mock(NamespaceInfo.class);
when(fakeNSInfo.getBuildVersion()).thenReturn("NSBuildVersion");
DatanodeProtocol fakeDNProt = mock(DatanodeProtocol.class);
when(fakeDNProt.versionRequest()).thenReturn(fakeNSInfo);
- doCallRealMethod().when(myMockBPOS).setNameNode(fakeDNProt);
- myMockBPOS.setNameNode( fakeDNProt );
+
+ bpos.setNameNode( fakeDNProt );
+ bpos.bpNSInfo = fakeNSInfo;
try {
- myMockBPOS.register();
+ bpos.retrieveNamespaceInfo();
fail("register() did not throw exception! " +
"Expected: IncorrectVersionException");
} catch (IncorrectVersionException ie) {
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/TestDatanodeRestart.java
index d0beaa2698..e6bd1ea8b1 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/TestDatanodeRestart.java
@@ -73,7 +73,7 @@ public void testRbwReplicas() throws IOException {
Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
- conf.setBoolean("dfs.support.append", true);
+ conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
cluster.waitActive();
try {
@@ -137,7 +137,7 @@ private void testRbwReplicas(MiniDFSCluster cluster, boolean isCorrupt)
Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
- conf.setBoolean("dfs.support.append", true);
+ conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
cluster.waitActive();
try {
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/TestInterDatanodeProtocol.java
index 487adfe5bb..d50376aaff 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/TestInterDatanodeProtocol.java
@@ -25,14 +25,8 @@
import java.net.InetSocketAddress;
import java.net.SocketTimeoutException;
-import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.StringUtils;
-
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
index f84a7a3c6a..150f117840 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
@@ -26,7 +26,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.datanode.DataNode.BPOfferService;
import org.junit.Test;
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
index 6eef234bbf..65169fa011 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
@@ -34,6 +34,7 @@
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileContext;
@@ -99,9 +100,9 @@ public void setUp() throws IOException {
static void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
- FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short)repl, (long)blockSize);
+ FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short) repl, blockSize);
byte[] buffer = new byte[TestCheckpoint.fileSize];
Random rand = new Random(TestCheckpoint.seed);
rand.nextBytes(buffer);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
index 220bfd6a39..aed6787446 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
@@ -20,8 +20,6 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
import java.io.File;
import java.io.IOException;
import java.net.URI;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java
index b75f2ed35a..dacd03bbfc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.hdfs.server.namenode;
import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-
import java.net.URL;
import java.util.Collection;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
index 0d2ea934e2..9934a6f534 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
@@ -28,6 +28,7 @@
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -117,7 +118,8 @@ private void writeFile(FileSystem fileSys, Path name, short repl)
throws IOException {
// create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
- .getInt("io.file.buffer.size", 4096), repl, (long) blockSize);
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
+ blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
@@ -129,7 +131,8 @@ private FSDataOutputStream writeIncompleteFile(FileSystem fileSys, Path name,
short repl) throws IOException {
// create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
- .getInt("io.file.buffer.size", 4096), repl, (long) blockSize);
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
+ blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 5c14ab3061..0d2479319e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -46,7 +46,6 @@
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java
index 649c415287..500c5c3c69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java
@@ -19,13 +19,8 @@
import static org.junit.Assert.*;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.spy;
-
import java.io.File;
import java.io.IOException;
-import java.util.List;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -34,10 +29,8 @@
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
-import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
import org.junit.Test;
-import org.mockito.Mockito;
public class TestFSImageStorageInspector {
private static final Log LOG = LogFactory.getLog(
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
index d2f9781bed..e4ff4bb732 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
@@ -21,21 +21,13 @@
import java.net.URI;
import java.util.Collections;
-import java.util.Arrays;
import java.util.List;
-import java.util.ArrayList;
import java.util.Iterator;
import java.io.RandomAccessFile;
import java.io.File;
import java.io.FilenameFilter;
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.security.SecurityUtil;
import org.junit.Test;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -48,9 +40,6 @@
import com.google.common.collect.ImmutableList;
import com.google.common.base.Joiner;
-import java.util.zip.CheckedInputStream;
-import java.util.zip.Checksum;
-
public class TestFileJournalManager {
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
index 48ab6ce18e..0dfb5e31d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
@@ -23,6 +23,7 @@
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -44,9 +45,9 @@ public class TestFileLimit extends TestCase {
// creates a zero file.
private void createFile(FileSystem fileSys, Path name)
throws IOException {
- FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short)1, (long)blockSize);
+ FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short) 1, blockSize);
byte[] buffer = new byte[1024];
Random rand = new Random(seed);
rand.nextBytes(buffer);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
index a334de66b4..9ec5d95ba6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
@@ -29,6 +29,7 @@
import java.util.Random;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -50,7 +51,8 @@ public class TestMetaSave {
private void createFile(FileSystem fileSys, Path name) throws IOException {
FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
- .getInt("io.file.buffer.size", 4096), (short) 2, (long) blockSize);
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short) 2, blockSize);
byte[] buffer = new byte[1024];
Random rand = new Random(seed);
rand.nextBytes(buffer);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
index b024bab1d7..5d0bd62247 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
@@ -24,6 +24,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
@@ -59,9 +60,9 @@ protected void setUp() throws java.lang.Exception {
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
- FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short)repl, (long)BLOCK_SIZE);
+ FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short) repl, BLOCK_SIZE);
byte[] buffer = new byte[FILE_SIZE];
Random rand = new Random(SEED);
rand.nextBytes(buffer);
@@ -96,7 +97,7 @@ private void checkFile(FileSystem fileSys, Path name, int repl)
int replication = fileSys.getFileStatus(name).getReplication();
assertEquals("replication for " + name, repl, replication);
long size = fileSys.getContentSummary(name).getLength();
- assertEquals("file size for " + name, size, (long)FILE_SIZE);
+ assertEquals("file size for " + name, size, FILE_SIZE);
}
private void cleanupFile(FileSystem fileSys, Path name)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
index 13e256d78c..25a458b0c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
@@ -22,7 +22,6 @@
import static org.junit.Assert.*;
import static org.mockito.Matchers.anyObject;
-import static org.mockito.Matchers.anyLong;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.spy;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
index c4e6377857..d4fd72d3b0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
@@ -30,7 +30,6 @@
import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
-import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
index 8948f7843e..ceb6261db4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
@@ -34,6 +34,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
@@ -76,9 +77,9 @@ public class TestStartup extends TestCase {
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
- FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short)repl, (long)blockSize);
+ FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+ .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short) repl, blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
@@ -233,11 +234,13 @@ private void verifyDifferentDirs(FSImage img, long expectedImgSize, long expecte
sd = it.next();
if(sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
- File imf = img.getStorage().getStorageFile(sd, NameNodeFile.IMAGE, 0);
+ img.getStorage();
+ File imf = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, 0);
LOG.info("--image file " + imf.getAbsolutePath() + "; len = " + imf.length() + "; expected = " + expectedImgSize);
assertEquals(expectedImgSize, imf.length());
} else if(sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
- File edf = img.getStorage().getStorageFile(sd, NameNodeFile.EDITS, 0);
+ img.getStorage();
+ File edf = NNStorage.getStorageFile(sd, NameNodeFile.EDITS, 0);
LOG.info("-- edits file " + edf.getAbsolutePath() + "; len = " + edf.length() + "; expected = " + expectedEditsSize);
assertEquals(expectedEditsSize, edf.length());
} else {
@@ -342,8 +345,10 @@ public void testSNNStartup() throws IOException{
FSImage image = nn.getFSImage();
StorageDirectory sd = image.getStorage().getStorageDir(0); //only one
assertEquals(sd.getStorageDirType(), NameNodeDirType.IMAGE_AND_EDITS);
- File imf = image.getStorage().getStorageFile(sd, NameNodeFile.IMAGE, 0);
- File edf = image.getStorage().getStorageFile(sd, NameNodeFile.EDITS, 0);
+ image.getStorage();
+ File imf = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, 0);
+ image.getStorage();
+ File edf = NNStorage.getStorageFile(sd, NameNodeFile.EDITS, 0);
LOG.info("--image file " + imf.getAbsolutePath() + "; len = " + imf.length());
LOG.info("--edits file " + edf.getAbsolutePath() + "; len = " + edf.length());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
index e65b9009d5..3823822d0c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.security;
import java.io.IOException;
-import java.util.EnumSet;
import java.util.Random;
import org.apache.commons.logging.Log;
@@ -125,7 +124,7 @@ public void testCreate() throws Exception {
FsPermission filePerm = new FsPermission((short)0444);
FSDataOutputStream out = fs.create(new Path("/b1/b2/b3.txt"), filePerm,
- true, conf.getInt("io.file.buffer.size", 4096),
+ true, conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
out.write(123);
out.close();
@@ -224,7 +223,7 @@ public void testFilePermision() throws Exception {
userfs.mkdirs(RENAME_PATH);
assertTrue(canRename(userfs, RENAME_PATH, CHILD_DIR1));
} finally {
- if(cluster != null) cluster.shutdown();
+ cluster.shutdown();
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
index f3ab700e70..2b18efc71a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
@@ -25,6 +25,7 @@
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
@@ -51,8 +52,8 @@ public class TestJMXGet extends TestCase {
private void writeFile(FileSystem fileSys, Path name, int repl)
throws IOException {
FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short)repl, (long)blockSize);
+ fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+ (short)repl, blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 8e4a8b3b45..2ee9c405ea 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -153,6 +153,12 @@ Release 0.23.1 - Unreleased
MAPREDUCE-3407. Fixed pom files to refer to the correct MR app-jar needed
by the integration tests. (Hitesh Shah via vinodkv)
+ MAPREDUCE-3434. Nightly build broken (Hitesh Shah via mahadev)
+
+ MAPREDUCE-3447. mapreduce examples not working (mahadev)
+
+ MAPREDUCE-3444. trunk/0.23 builds broken (Hitesh Shah via mahadev)
+
Release 0.23.0 - 2011-11-01
INCOMPATIBLE CHANGES
diff --git a/hadoop-mapreduce-project/build.xml b/hadoop-mapreduce-project/build.xml
index 16f02a3742..a8b4618268 100644
--- a/hadoop-mapreduce-project/build.xml
+++ b/hadoop-mapreduce-project/build.xml
@@ -867,7 +867,6 @@
-
@@ -897,12 +896,10 @@
-
-
@@ -962,7 +959,6 @@
-
@@ -987,7 +983,6 @@
-
@@ -1026,7 +1021,6 @@
-
@@ -1052,7 +1046,6 @@
-
@@ -1792,10 +1785,6 @@
output="${build.dir.eclipse-contrib-classes}/raid/main" />
-
-
test-jar