|
@@ -104,7 +104,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
|
|
@@ -200,7 +199,7 @@ public class DataNode extends Configured
|
|
|
BPOfferService[] nameNodeThreads;
|
|
|
private Map<String, BPOfferService> bpMapping =
|
|
|
new HashMap<String, BPOfferService>();
|
|
|
- public DatanodeProtocol namenode = null;
|
|
|
+ public DatanodeProtocol namenodeTODO_FED = null; //TODO:FEDERATION needs to be taken out.
|
|
|
public FSDatasetInterface data = null;
|
|
|
public DatanodeRegistration dnRegistration = null;
|
|
|
private String clusterId = null;
|
|
@@ -260,6 +259,7 @@ public class DataNode extends Configured
|
|
|
* where they are run with privileged ports and injected from a higher
|
|
|
* level of capability
|
|
|
*/
|
|
|
+ /*
|
|
|
DataNode(final Configuration conf,
|
|
|
final AbstractList<File> dataDirs, final SecureResources resources) throws IOException {
|
|
|
this(conf, dataDirs, (DatanodeProtocol)RPC.waitForProxy(DatanodeProtocol.class,
|
|
@@ -267,6 +267,7 @@ public class DataNode extends Configured
|
|
|
NameNode.getServiceAddress(conf, true),
|
|
|
conf), resources);
|
|
|
}
|
|
|
+ */
|
|
|
|
|
|
/**
|
|
|
* Create the DataNode given a configuration, an array of dataDirs,
|
|
@@ -274,13 +275,13 @@ public class DataNode extends Configured
|
|
|
*/
|
|
|
DataNode(final Configuration conf,
|
|
|
final AbstractList<File> dataDirs,
|
|
|
- final DatanodeProtocol namenode, final SecureResources resources) throws IOException {
|
|
|
+ final SecureResources resources) throws IOException {
|
|
|
super(conf);
|
|
|
|
|
|
DataNode.setDataNode(this);
|
|
|
|
|
|
try {
|
|
|
- startDataNode(conf, dataDirs, namenode, resources);
|
|
|
+ startDataNode(conf, dataDirs, resources);
|
|
|
} catch (IOException ie) {
|
|
|
shutdown();
|
|
|
throw ie;
|
|
@@ -451,7 +452,6 @@ public class DataNode extends Configured
|
|
|
+ block.getPoolId());
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
|
|
|
public void reportBadBlocks(ExtendedBlock block) throws IOException{
|
|
|
BPOfferService bpos = bpMapping.get(block.getPoolId());
|
|
@@ -1042,6 +1042,7 @@ public class DataNode extends Configured
|
|
|
break;
|
|
|
case UpgradeCommand.UC_ACTION_START_UPGRADE:
|
|
|
// start distributed upgrade here
|
|
|
+ datanodeObject.namenodeTODO_FED = bpNamenode; //TODO:FEDERATION - this needs to be converted.
|
|
|
processDistributedUpgradeCommand((UpgradeCommand)cmd);
|
|
|
break;
|
|
|
case DatanodeProtocol.DNA_RECOVERBLOCK:
|
|
@@ -1074,7 +1075,8 @@ public class DataNode extends Configured
|
|
|
*/
|
|
|
void startDataNode(Configuration conf,
|
|
|
AbstractList<File> dataDirs,
|
|
|
- DatanodeProtocol namenode, SecureResources resources
|
|
|
+ // DatanodeProtocol namenode,
|
|
|
+ SecureResources resources
|
|
|
) throws IOException {
|
|
|
if(UserGroupInformation.isSecurityEnabled() && resources == null)
|
|
|
throw new RuntimeException("Cannot start secure cluster without " +
|
|
@@ -1098,7 +1100,6 @@ public class DataNode extends Configured
|
|
|
|
|
|
// get all the NNs configured
|
|
|
nameNodeThreads = getAllNamenodes(conf);
|
|
|
- this.namenode = namenode;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1488,7 +1489,7 @@ public class DataNode extends Configured
|
|
|
private void transferBlock( ExtendedBlock block,
|
|
|
DatanodeInfo xferTargets[]
|
|
|
) throws IOException {
|
|
|
- DatanodeProtocol nn = getNamenode(block);
|
|
|
+ DatanodeProtocol nn = getBPNamenode(block.getPoolId());
|
|
|
|
|
|
if (!data.isValidBlock(block)) {
|
|
|
// block does not exist or is under-construction
|
|
@@ -1504,7 +1505,7 @@ public class DataNode extends Configured
|
|
|
long onDiskLength = data.getLength(block);
|
|
|
if (block.getNumBytes() > onDiskLength) {
|
|
|
// Shorter on-disk len indicates corruption so report NN the corrupt block
|
|
|
- namenode.reportBadBlocks(new LocatedBlock[]{
|
|
|
+ nn.reportBadBlocks(new LocatedBlock[]{
|
|
|
new LocatedBlock(block, new DatanodeInfo[] {
|
|
|
new DatanodeInfo(dnRegistration)})});
|
|
|
LOG.info("Can't replicate block " + block
|
|
@@ -2085,23 +2086,13 @@ public class DataNode extends Configured
|
|
|
syncBlock(rBlock, syncList);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- *
|
|
|
- * @param eblock
|
|
|
- * @return namenode corresponding to the Extended block
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- public DatanodeProtocol getNamenode(ExtendedBlock eblock) throws IOException {
|
|
|
- return getNamenodeForBP(eblock.getPoolId());
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
*
|
|
|
* @param bpid
|
|
|
* @return Namenode corresponding to the bpid
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public DatanodeProtocol getNamenodeForBP(String bpid) throws IOException {
|
|
|
+ public DatanodeProtocol getBPNamenode(String bpid) throws IOException {
|
|
|
BPOfferService bpos = bpMapping.get(bpid);
|
|
|
if(bpos == null || bpos.bpNamenode == null) {
|
|
|
throw new IOException("cannot find a namnode proxy for bpid=" + bpid);
|
|
@@ -2113,7 +2104,7 @@ public class DataNode extends Configured
|
|
|
void syncBlock(RecoveringBlock rBlock,
|
|
|
List<BlockRecord> syncList) throws IOException {
|
|
|
ExtendedBlock block = rBlock.getBlock();
|
|
|
- DatanodeProtocol nn = getNamenode(block);
|
|
|
+ DatanodeProtocol nn = getBPNamenode(block.getPoolId());
|
|
|
|
|
|
long recoveryId = rBlock.getNewGenerationStamp();
|
|
|
if (LOG.isDebugEnabled()) {
|