|
@@ -25,7 +25,6 @@ import java.io.DataOutputStream;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
|
-import java.lang.management.ManagementFactory;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.ServerSocket;
|
|
|
import java.net.Socket;
|
|
@@ -50,9 +49,6 @@ import java.util.Random;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
-import javax.management.MBeanServer;
|
|
|
-import javax.management.ObjectName;
|
|
|
-
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
@@ -69,7 +65,6 @@ 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.DataTransferProtocol;
|
|
|
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
@@ -77,16 +72,17 @@ import org.apache.hadoop.hdfs.protocol.FSConstants;
|
|
|
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.DataTransferProtocol.BlockConstructionStage;
|
|
|
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.ExportedBlockKeys;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
|
|
|
-import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
|
|
|
-import org.apache.hadoop.hdfs.server.common.HdfsConstants.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.Util;
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
|
|
|
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;
|
|
@@ -96,7 +92,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
|
|
|
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;
|
|
@@ -106,6 +101,7 @@ import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
|
import org.apache.hadoop.http.HttpServer;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.ipc.RPC;
|
|
@@ -121,15 +117,20 @@ import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
import org.apache.hadoop.util.DiskChecker;
|
|
|
-import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
-import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
|
import org.apache.hadoop.util.GenericOptionsParser;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.apache.hadoop.util.ServicePlugin;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.VersionInfo;
|
|
|
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
|
import org.mortbay.util.ajax.JSON;
|
|
|
|
|
|
+import java.lang.management.ManagementFactory;
|
|
|
+
|
|
|
+import javax.management.MBeanServer;
|
|
|
+import javax.management.ObjectName;
|
|
|
+
|
|
|
/**********************************************************
|
|
|
* DataNode is a class (and program) that stores a set of
|
|
|
* blocks for a DFS deployment. A single deployment can
|
|
@@ -363,7 +364,6 @@ public class DataNode extends Configured
|
|
|
} else { // real storage
|
|
|
// read storage info, lock data dirs and transition fs state if necessary
|
|
|
storage.recoverTransitionRead(nsInfo, dataDirs, startOpt);
|
|
|
-
|
|
|
// adjust
|
|
|
this.dnRegistration.setStorageInfo(storage);
|
|
|
// initialize data node internal structure
|
|
@@ -1143,10 +1143,11 @@ public class DataNode extends Configured
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- private void transferBlock( Block block,
|
|
|
+ private void transferBlock( ExtendedBlock block,
|
|
|
DatanodeInfo xferTargets[]
|
|
|
) throws IOException {
|
|
|
- if (!data.isValidBlock(block)) {
|
|
|
+ // TODO:FEDERATION use ExtendedBlock
|
|
|
+ if (!data.isValidBlock(block.getLocalBlock())) {
|
|
|
// block does not exist or is under-construction
|
|
|
String errStr = "Can't send invalid block " + block;
|
|
|
LOG.info(errStr);
|
|
@@ -1157,7 +1158,8 @@ public class DataNode extends Configured
|
|
|
}
|
|
|
|
|
|
// Check if NN recorded length matches on-disk length
|
|
|
- long onDiskLength = data.getLength(block);
|
|
|
+ // TODO:FEDERATION use ExtendedBlock
|
|
|
+ long onDiskLength = data.getLength(block.getLocalBlock());
|
|
|
if (block.getNumBytes() > onDiskLength) {
|
|
|
// Shorter on-disk len indicates corruption so report NN the corrupt block
|
|
|
namenode.reportBadBlocks(new LocatedBlock[]{
|
|
@@ -1190,7 +1192,8 @@ public class DataNode extends Configured
|
|
|
) {
|
|
|
for (int i = 0; i < blocks.length; i++) {
|
|
|
try {
|
|
|
- transferBlock(blocks[i], xferTargets[i]);
|
|
|
+ // TODO:FEDERATION cleanup
|
|
|
+ transferBlock(new ExtendedBlock(blocks[i]), xferTargets[i]);
|
|
|
} catch (IOException ie) {
|
|
|
LOG.warn("Failed to transfer block " + blocks[i], ie);
|
|
|
}
|
|
@@ -1306,14 +1309,15 @@ public class DataNode extends Configured
|
|
|
*/
|
|
|
class DataTransfer implements Runnable {
|
|
|
DatanodeInfo targets[];
|
|
|
- Block b;
|
|
|
+ ExtendedBlock b;
|
|
|
DataNode datanode;
|
|
|
|
|
|
/**
|
|
|
* Connect to the first item in the target list. Pass along the
|
|
|
* entire target list, the block, and the data.
|
|
|
*/
|
|
|
- public DataTransfer(DatanodeInfo targets[], Block b, DataNode datanode) throws IOException {
|
|
|
+ public DataTransfer(DatanodeInfo targets[], ExtendedBlock b,
|
|
|
+ DataNode datanode) throws IOException {
|
|
|
this.targets = targets;
|
|
|
this.b = b;
|
|
|
this.datanode = datanode;
|
|
@@ -1350,8 +1354,8 @@ public class DataNode extends Configured
|
|
|
//
|
|
|
Token<BlockTokenIdentifier> accessToken = BlockTokenSecretManager.DUMMY_TOKEN;
|
|
|
if (isBlockTokenEnabled) {
|
|
|
- accessToken = blockTokenSecretManager.generateToken(null, b,
|
|
|
- EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE));
|
|
|
+ accessToken = blockTokenSecretManager.generateToken(b,
|
|
|
+ EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE));
|
|
|
}
|
|
|
DataTransferProtocol.Sender.opWriteBlock(out,
|
|
|
b, 0, BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, "",
|