HDFS-3164. Move DatanodeInfo#hostName to DatanodeID. Contributed by Eli Collins
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1307890 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
eeb687daa7
commit
8bd825bb6f
@ -121,6 +121,8 @@ Release 2.0.0 - UNRELEASED
|
||||
|
||||
HDFS-3138. Move DatanodeInfo#ipcPort to DatanodeID. (eli)
|
||||
|
||||
HDFS-3164. Move DatanodeInfo#hostName to DatanodeID. (eli)
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
HDFS-2978. The NameNode should expose name dir statuses via JMX. (atm)
|
||||
|
@ -240,7 +240,7 @@ private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
|
||||
private static BlockLocalPathInfo getBlockPathInfo(ExtendedBlock blk,
|
||||
DatanodeInfo node, Configuration conf, int timeout,
|
||||
Token<BlockTokenIdentifier> token) throws IOException {
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.ipcPort);
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
|
||||
BlockLocalPathInfo pathinfo = null;
|
||||
ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(node,
|
||||
conf, timeout);
|
||||
|
@ -24,7 +24,7 @@
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.DeprecatedUTF8;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
||||
/**
|
||||
@ -38,16 +38,17 @@
|
||||
public class DatanodeID implements WritableComparable<DatanodeID> {
|
||||
public static final DatanodeID[] EMPTY_ARRAY = {};
|
||||
|
||||
public String name; // hostname:port (data transfer port)
|
||||
public String storageID; // unique per cluster storageID
|
||||
protected int infoPort; // info server port
|
||||
public int ipcPort; // ipc server port
|
||||
protected String name; // IP:port (data transfer port)
|
||||
protected String hostName; // hostname
|
||||
protected String storageID; // unique per cluster storageID
|
||||
protected int infoPort; // info server port
|
||||
protected int ipcPort; // IPC server port
|
||||
|
||||
/** Equivalent to DatanodeID(""). */
|
||||
public DatanodeID() {this("");}
|
||||
|
||||
/** Equivalent to DatanodeID(nodeName, "", -1, -1). */
|
||||
public DatanodeID(String nodeName) {this(nodeName, "", -1, -1);}
|
||||
public DatanodeID(String nodeName) {this(nodeName, "", "", -1, -1);}
|
||||
|
||||
/**
|
||||
* DatanodeID copy constructor
|
||||
@ -56,6 +57,7 @@ public class DatanodeID implements WritableComparable<DatanodeID> {
|
||||
*/
|
||||
public DatanodeID(DatanodeID from) {
|
||||
this(from.getName(),
|
||||
from.getHostName(),
|
||||
from.getStorageID(),
|
||||
from.getInfoPort(),
|
||||
from.getIpcPort());
|
||||
@ -63,14 +65,16 @@ public DatanodeID(DatanodeID from) {
|
||||
|
||||
/**
|
||||
* Create DatanodeID
|
||||
* @param nodeName (hostname:portNumber)
|
||||
* @param node IP:port
|
||||
* @param hostName hostname
|
||||
* @param storageID data storage ID
|
||||
* @param infoPort info server port
|
||||
* @param ipcPort ipc server port
|
||||
*/
|
||||
public DatanodeID(String nodeName, String storageID,
|
||||
int infoPort, int ipcPort) {
|
||||
this.name = nodeName;
|
||||
public DatanodeID(String name, String hostName,
|
||||
String storageID, int infoPort, int ipcPort) {
|
||||
this.name = name;
|
||||
this.hostName = hostName;
|
||||
this.storageID = storageID;
|
||||
this.infoPort = infoPort;
|
||||
this.ipcPort = ipcPort;
|
||||
@ -80,6 +84,10 @@ public void setName(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
public void setHostName(String hostName) {
|
||||
this.hostName = hostName;
|
||||
}
|
||||
|
||||
public void setInfoPort(int infoPort) {
|
||||
this.infoPort = infoPort;
|
||||
}
|
||||
@ -94,7 +102,14 @@ public void setIpcPort(int ipcPort) {
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @return hostname
|
||||
*/
|
||||
public String getHostName() {
|
||||
return (hostName == null || hostName.length() == 0) ? getHost() : hostName;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return data storage ID.
|
||||
*/
|
||||
@ -186,17 +201,19 @@ public int compareTo(DatanodeID that) {
|
||||
/////////////////////////////////////////////////
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
DeprecatedUTF8.writeString(out, name);
|
||||
DeprecatedUTF8.writeString(out, storageID);
|
||||
Text.writeString(out, name);
|
||||
Text.writeString(out, hostName);
|
||||
Text.writeString(out, storageID);
|
||||
out.writeShort(infoPort);
|
||||
out.writeShort(ipcPort);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
name = DeprecatedUTF8.readString(in);
|
||||
storageID = DeprecatedUTF8.readString(in);
|
||||
// the infoPort read could be negative, if the port is a large number (more
|
||||
name = Text.readString(in);
|
||||
hostName = Text.readString(in);
|
||||
storageID = Text.readString(in);
|
||||
// The port read could be negative, if the port is a large number (more
|
||||
// than 15 bits in storage size (but less than 16 bits).
|
||||
// So chop off the first two bytes (and hence the signed bits) before
|
||||
// setting the field.
|
||||
|
@ -51,9 +51,6 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
||||
protected long lastUpdate;
|
||||
protected int xceiverCount;
|
||||
protected String location = NetworkTopology.DEFAULT_RACK;
|
||||
|
||||
// The FQDN of the IP associated with the Datanode's hostname
|
||||
protected String hostName = null;
|
||||
|
||||
// Datanode administrative states
|
||||
public enum AdminStates {
|
||||
@ -110,30 +107,27 @@ public DatanodeInfo(DatanodeID nodeID) {
|
||||
this.adminState = null;
|
||||
}
|
||||
|
||||
public DatanodeInfo(DatanodeID nodeID, String location, String hostName) {
|
||||
public DatanodeInfo(DatanodeID nodeID, String location) {
|
||||
this(nodeID);
|
||||
this.location = location;
|
||||
this.hostName = hostName;
|
||||
}
|
||||
|
||||
public DatanodeInfo(DatanodeID nodeID, String location, String hostName,
|
||||
public DatanodeInfo(DatanodeID nodeID, String location,
|
||||
final long capacity, final long dfsUsed, final long remaining,
|
||||
final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
|
||||
final AdminStates adminState) {
|
||||
this(nodeID.getName(), nodeID.getStorageID(), nodeID.getInfoPort(), nodeID
|
||||
this(nodeID.getName(), nodeID.getHostName(), nodeID.getStorageID(), nodeID.getInfoPort(), nodeID
|
||||
.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed, lastUpdate,
|
||||
xceiverCount, location, hostName, adminState);
|
||||
xceiverCount, location, adminState);
|
||||
}
|
||||
|
||||
/** Constructor */
|
||||
public DatanodeInfo(final String name, final String storageID,
|
||||
final int infoPort, final int ipcPort,
|
||||
public DatanodeInfo(final String name, final String hostName,
|
||||
final String storageID, final int infoPort, final int ipcPort,
|
||||
final long capacity, final long dfsUsed, final long remaining,
|
||||
final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
|
||||
final String networkLocation, final String hostName,
|
||||
final AdminStates adminState) {
|
||||
super(name, storageID, infoPort, ipcPort);
|
||||
|
||||
final String networkLocation, final AdminStates adminState) {
|
||||
super(name, hostName, storageID, infoPort, ipcPort);
|
||||
this.capacity = capacity;
|
||||
this.dfsUsed = dfsUsed;
|
||||
this.remaining = remaining;
|
||||
@ -141,7 +135,6 @@ public DatanodeInfo(final String name, final String storageID,
|
||||
this.lastUpdate = lastUpdate;
|
||||
this.xceiverCount = xceiverCount;
|
||||
this.location = networkLocation;
|
||||
this.hostName = hostName;
|
||||
this.adminState = adminState;
|
||||
}
|
||||
|
||||
@ -221,15 +214,7 @@ public void setXceiverCount(int xceiverCount) {
|
||||
public synchronized void setNetworkLocation(String location) {
|
||||
this.location = NodeBase.normalize(location);
|
||||
}
|
||||
|
||||
public String getHostName() {
|
||||
return (hostName == null || hostName.length()==0) ? getHost() : hostName;
|
||||
}
|
||||
|
||||
public void setHostName(String host) {
|
||||
hostName = host;
|
||||
}
|
||||
|
||||
|
||||
/** A formatted string for reporting the status of the DataNode. */
|
||||
public String getDatanodeReport() {
|
||||
StringBuilder buffer = new StringBuilder();
|
||||
@ -387,7 +372,6 @@ public void write(DataOutput out) throws IOException {
|
||||
out.writeLong(lastUpdate);
|
||||
out.writeInt(xceiverCount);
|
||||
Text.writeString(out, location);
|
||||
Text.writeString(out, hostName == null? "": hostName);
|
||||
WritableUtils.writeEnum(out, getAdminState());
|
||||
}
|
||||
|
||||
@ -401,7 +385,6 @@ public void readFields(DataInput in) throws IOException {
|
||||
this.lastUpdate = in.readLong();
|
||||
this.xceiverCount = in.readInt();
|
||||
this.location = Text.readString(in);
|
||||
this.hostName = Text.readString(in);
|
||||
setAdminState(WritableUtils.readEnum(in, AdminStates.class));
|
||||
}
|
||||
|
||||
|
@ -85,6 +85,7 @@ private static HdfsProtos.DatanodeIDProto toProto(
|
||||
DatanodeID dni) {
|
||||
return HdfsProtos.DatanodeIDProto.newBuilder()
|
||||
.setName(dni.getName())
|
||||
.setHostName(dni.getHostName())
|
||||
.setStorageID(dni.getStorageID())
|
||||
.setInfoPort(dni.getInfoPort())
|
||||
.setIpcPort(dni.getIpcPort())
|
||||
@ -94,6 +95,7 @@ private static HdfsProtos.DatanodeIDProto toProto(
|
||||
private static DatanodeID fromProto(HdfsProtos.DatanodeIDProto idProto) {
|
||||
return new DatanodeID(
|
||||
idProto.getName(),
|
||||
idProto.getHostName(),
|
||||
idProto.getStorageID(),
|
||||
idProto.getInfoPort(),
|
||||
idProto.getIpcPort());
|
||||
@ -111,7 +113,6 @@ public static HdfsProtos.DatanodeInfoProto toProto(DatanodeInfo dni) {
|
||||
.setLastUpdate(dni.getLastUpdate())
|
||||
.setXceiverCount(dni.getXceiverCount())
|
||||
.setLocation(dni.getNetworkLocation())
|
||||
.setHostName(dni.getHostName())
|
||||
.setAdminState(HdfsProtos.DatanodeInfoProto.AdminState.valueOf(
|
||||
dni.getAdminState().name()))
|
||||
.build();
|
||||
@ -119,7 +120,7 @@ public static HdfsProtos.DatanodeInfoProto toProto(DatanodeInfo dni) {
|
||||
|
||||
public static DatanodeInfo fromProto(HdfsProtos.DatanodeInfoProto dniProto) {
|
||||
DatanodeInfo dniObj = new DatanodeInfo(fromProto(dniProto.getId()),
|
||||
dniProto.getLocation(), dniProto.getHostName());
|
||||
dniProto.getLocation());
|
||||
|
||||
dniObj.setCapacity(dniProto.getCapacity());
|
||||
dniObj.setDfsUsed(dniProto.getDfsUsed());
|
||||
|
@ -204,12 +204,13 @@ public static NamenodeRegistration convert(NamenodeRegistrationProto reg) {
|
||||
|
||||
// DatanodeId
|
||||
public static DatanodeID convert(DatanodeIDProto dn) {
|
||||
return new DatanodeID(dn.getName(), dn.getStorageID(), dn.getInfoPort(),
|
||||
return new DatanodeID(dn.getName(), dn.getHostName(), dn.getStorageID(), dn.getInfoPort(),
|
||||
dn.getIpcPort());
|
||||
}
|
||||
|
||||
public static DatanodeIDProto convert(DatanodeID dn) {
|
||||
return DatanodeIDProto.newBuilder().setName(dn.getName())
|
||||
return DatanodeIDProto.newBuilder()
|
||||
.setName(dn.getName()).setHostName(dn.getHostName())
|
||||
.setInfoPort(dn.getInfoPort()).setIpcPort(dn.getIpcPort())
|
||||
.setStorageID(dn.getStorageID()).build();
|
||||
}
|
||||
@ -442,7 +443,6 @@ static public DatanodeInfo convert(DatanodeInfoProto di) {
|
||||
return new DatanodeInfo(
|
||||
PBHelper.convert(di.getId()),
|
||||
di.hasLocation() ? di.getLocation() : null ,
|
||||
di.hasHostName() ? di.getHostName() : null,
|
||||
di.getCapacity(), di.getDfsUsed(), di.getRemaining(),
|
||||
di.getBlockPoolUsed() , di.getLastUpdate() , di.getXceiverCount() ,
|
||||
PBHelper.convert(di.getAdminState()));
|
||||
@ -451,9 +451,6 @@ static public DatanodeInfo convert(DatanodeInfoProto di) {
|
||||
static public DatanodeInfoProto convertDatanodeInfo(DatanodeInfo di) {
|
||||
if (di == null) return null;
|
||||
DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
|
||||
if (di.getHostName() != null) {
|
||||
builder.setHostName(di.getHostName());
|
||||
}
|
||||
if (di.getNetworkLocation() != null) {
|
||||
builder.setLocation(di.getNetworkLocation());
|
||||
}
|
||||
@ -503,7 +500,6 @@ public static DatanodeInfoProto convert(DatanodeInfo info) {
|
||||
builder.setAdminState(PBHelper.convert(info.getAdminState()));
|
||||
builder.setCapacity(info.getCapacity())
|
||||
.setDfsUsed(info.getDfsUsed())
|
||||
.setHostName(info.getHostName())
|
||||
.setId(PBHelper.convert((DatanodeID)info))
|
||||
.setLastUpdate(info.getLastUpdate())
|
||||
.setLocation(info.getNetworkLocation())
|
||||
|
@ -2618,7 +2618,7 @@ private void logBlockReplicationInfo(Block block, DatanodeDescriptor srcNode,
|
||||
StringBuilder nodeList = new StringBuilder();
|
||||
while (nodeIter.hasNext()) {
|
||||
DatanodeDescriptor node = nodeIter.next();
|
||||
nodeList.append(node.name);
|
||||
nodeList.append(node.getName());
|
||||
nodeList.append(" ");
|
||||
}
|
||||
LOG.info("Block: " + block + ", Expected Replicas: "
|
||||
@ -2628,7 +2628,7 @@ private void logBlockReplicationInfo(Block block, DatanodeDescriptor srcNode,
|
||||
+ ", excess replicas: " + num.excessReplicas()
|
||||
+ ", Is Open File: " + fileINode.isUnderConstruction()
|
||||
+ ", Datanodes having this block: " + nodeList + ", Current Datanode: "
|
||||
+ srcNode.name + ", Is current datanode decommissioning: "
|
||||
+ srcNode.getName() + ", Is current datanode decommissioning: "
|
||||
+ srcNode.isDecommissionInProgress());
|
||||
}
|
||||
|
||||
|
@ -175,19 +175,7 @@ public DatanodeDescriptor(DatanodeID nodeID) {
|
||||
*/
|
||||
public DatanodeDescriptor(DatanodeID nodeID,
|
||||
String networkLocation) {
|
||||
this(nodeID, networkLocation, null);
|
||||
}
|
||||
|
||||
/** DatanodeDescriptor constructor
|
||||
*
|
||||
* @param nodeID id of the data node
|
||||
* @param networkLocation location of the data node in network
|
||||
* @param hostName it could be different from host specified for DatanodeID
|
||||
*/
|
||||
public DatanodeDescriptor(DatanodeID nodeID,
|
||||
String networkLocation,
|
||||
String hostName) {
|
||||
this(nodeID, networkLocation, hostName, 0L, 0L, 0L, 0L, 0, 0);
|
||||
this(nodeID, networkLocation, 0L, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
/** DatanodeDescriptor constructor
|
||||
@ -223,14 +211,13 @@ public DatanodeDescriptor(DatanodeID nodeID,
|
||||
*/
|
||||
public DatanodeDescriptor(DatanodeID nodeID,
|
||||
String networkLocation,
|
||||
String hostName,
|
||||
long capacity,
|
||||
long dfsUsed,
|
||||
long remaining,
|
||||
long bpused,
|
||||
int xceiverCount,
|
||||
int failedVolumes) {
|
||||
super(nodeID, networkLocation, hostName);
|
||||
super(nodeID, networkLocation);
|
||||
updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount,
|
||||
failedVolumes);
|
||||
}
|
||||
|
@ -564,6 +564,7 @@ public void registerDatanode(DatanodeRegistration nodeReg
|
||||
|
||||
// update the datanode's name with ip:port
|
||||
DatanodeID dnReg = new DatanodeID(dnAddress + ":" + nodeReg.getPort(),
|
||||
hostName,
|
||||
nodeReg.getStorageID(),
|
||||
nodeReg.getInfoPort(),
|
||||
nodeReg.getIpcPort());
|
||||
@ -630,10 +631,10 @@ nodes with its data cleared (or user can just remove the StorageID
|
||||
}
|
||||
|
||||
// this is a new datanode serving a new data storage
|
||||
if (nodeReg.getStorageID().equals("")) {
|
||||
if ("".equals(nodeReg.getStorageID())) {
|
||||
// this data storage has never been registered
|
||||
// it is either empty or was created by pre-storageID version of DFS
|
||||
nodeReg.storageID = newStorageID();
|
||||
nodeReg.setStorageID(newStorageID());
|
||||
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug(
|
||||
"BLOCK* NameSystem.registerDatanode: "
|
||||
@ -642,7 +643,7 @@ nodes with its data cleared (or user can just remove the StorageID
|
||||
}
|
||||
// register new datanode
|
||||
DatanodeDescriptor nodeDescr
|
||||
= new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK, hostName);
|
||||
= new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK);
|
||||
resolveNetworkLocation(nodeDescr);
|
||||
addDatanode(nodeDescr);
|
||||
checkDecommissioning(nodeDescr, dnAddress);
|
||||
|
@ -164,9 +164,9 @@
|
||||
import org.mortbay.util.ajax.JSON;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.BlockingService;
|
||||
|
||||
|
||||
/**********************************************************
|
||||
* DataNode is a class (and program) that stores a set of
|
||||
* blocks for a DFS deployment. A single deployment can
|
||||
@ -947,7 +947,7 @@ DataNodeMetrics getMetrics() {
|
||||
|
||||
public static void setNewStorageID(DatanodeID dnId) {
|
||||
LOG.info("Datanode is " + dnId);
|
||||
dnId.storageID = createNewStorageId(dnId.getPort());
|
||||
dnId.setStorageID(createNewStorageId(dnId.getPort()));
|
||||
}
|
||||
|
||||
static String createNewStorageId(int port) {
|
||||
@ -2204,9 +2204,9 @@ public boolean isDatanodeFullyStarted() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Methods used by fault injection tests */
|
||||
@VisibleForTesting
|
||||
public DatanodeID getDatanodeId() {
|
||||
return new DatanodeID(getMachineName(), getStorageId(),
|
||||
return new DatanodeID(getMachineName(), hostName, getStorageId(),
|
||||
infoServer.getPort(), getIpcPort());
|
||||
}
|
||||
|
||||
|
@ -306,6 +306,7 @@ private static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
|
||||
|
||||
return new DatanodeInfo(
|
||||
(String)m.get("name"),
|
||||
(String)m.get("hostName"),
|
||||
(String)m.get("storageID"),
|
||||
(int)(long)(Long)m.get("infoPort"),
|
||||
(int)(long)(Long)m.get("ipcPort"),
|
||||
@ -317,7 +318,6 @@ private static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
|
||||
(Long)m.get("lastUpdate"),
|
||||
(int)(long)(Long)m.get("xceiverCount"),
|
||||
(String)m.get("networkLocation"),
|
||||
(String)m.get("hostName"),
|
||||
AdminStates.valueOf((String)m.get("adminState")));
|
||||
}
|
||||
|
||||
|
@ -48,10 +48,11 @@ message BlockTokenIdentifierProto {
|
||||
* Identifies a Datanode
|
||||
*/
|
||||
message DatanodeIDProto {
|
||||
required string name = 1; // hostname:portNumber
|
||||
required string storageID = 2; // Unique storage id
|
||||
required uint32 infoPort = 3; // the port where the infoserver is running
|
||||
required uint32 ipcPort = 4; // the port where the ipc Server is running
|
||||
required string name = 1; // IP:port (data transfer port)
|
||||
required string hostName = 2; // hostname
|
||||
required string storageID = 3; // unique storage id
|
||||
required uint32 infoPort = 4; // info server port
|
||||
required uint32 ipcPort = 5; // ipc server port
|
||||
}
|
||||
|
||||
/**
|
||||
@ -73,7 +74,6 @@ message DatanodeInfoProto {
|
||||
optional uint64 lastUpdate = 6 [default = 0];
|
||||
optional uint32 xceiverCount = 7 [default = 0];
|
||||
optional string location = 8;
|
||||
optional string hostName = 9;
|
||||
enum AdminState {
|
||||
NORMAL = 0;
|
||||
DECOMMISSION_INPROGRESS = 1;
|
||||
|
@ -162,7 +162,7 @@ public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToR
|
||||
*/
|
||||
public DataNode getDataNode(LocatedBlock testBlock) {
|
||||
DatanodeInfo[] nodes = testBlock.getLocations();
|
||||
int ipcport = nodes[0].ipcPort;
|
||||
int ipcport = nodes[0].getIpcPort();
|
||||
return cluster.getDataNode(ipcport);
|
||||
}
|
||||
|
||||
|
@ -629,7 +629,7 @@ public void testClientDNProtocolTimeout() throws IOException {
|
||||
|
||||
final InetSocketAddress addr = NetUtils.getConnectAddress(server);
|
||||
DatanodeID fakeDnId = new DatanodeID(
|
||||
"localhost:" + addr.getPort(), "fake-storage", 0, addr.getPort());
|
||||
"localhost:" + addr.getPort(), "localhost", "fake-storage", 0, addr.getPort());
|
||||
|
||||
ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
|
||||
LocatedBlock fakeBlock = new LocatedBlock(b, new DatanodeInfo[0]);
|
||||
|
@ -844,7 +844,7 @@ public void testLeaseExpireHardLimit() throws Exception {
|
||||
LocatedBlock locatedblock = locations.getLocatedBlocks().get(0);
|
||||
int successcount = 0;
|
||||
for(DatanodeInfo datanodeinfo: locatedblock.getLocations()) {
|
||||
DataNode datanode = cluster.getDataNode(datanodeinfo.ipcPort);
|
||||
DataNode datanode = cluster.getDataNode(datanodeinfo.getIpcPort());
|
||||
ExtendedBlock blk = locatedblock.getBlock();
|
||||
Block b = DataNodeTestUtils.getFSDataset(datanode).getStoredBlock(
|
||||
blk.getBlockPoolId(), blk.getBlockId());
|
||||
|
@ -130,7 +130,7 @@ public void testConvertNamenodeRegistration() {
|
||||
|
||||
@Test
|
||||
public void testConvertDatanodeID() {
|
||||
DatanodeID dn = new DatanodeID("node", "sid", 1, 2);
|
||||
DatanodeID dn = new DatanodeID("node", "node", "sid", 1, 2);
|
||||
DatanodeIDProto dnProto = PBHelper.convert(dn);
|
||||
DatanodeID dn2 = PBHelper.convert(dnProto);
|
||||
compare(dn, dn2);
|
||||
@ -141,6 +141,7 @@ void compare(DatanodeID dn, DatanodeID dn2) {
|
||||
assertEquals(dn.getInfoPort(), dn2.getInfoPort());
|
||||
assertEquals(dn.getIpcPort(), dn2.getIpcPort());
|
||||
assertEquals(dn.getName(), dn2.getName());
|
||||
assertEquals(dn.getHostName(), dn2.getHostName());
|
||||
assertEquals(dn.getPort(), dn2.getPort());
|
||||
assertEquals(dn.getStorageID(), dn2.getStorageID());
|
||||
}
|
||||
@ -279,8 +280,8 @@ public ExtendedBlock getExtendedBlock(long blkid) {
|
||||
return new ExtendedBlock("bpid", blkid, 100, 2);
|
||||
}
|
||||
|
||||
public DatanodeInfo getDNInfo() {
|
||||
return new DatanodeInfo(new DatanodeID("node", "sid", 1, 2));
|
||||
private DatanodeInfo getDNInfo() {
|
||||
return new DatanodeInfo(new DatanodeID("node", "node", "sid", 1, 2));
|
||||
}
|
||||
|
||||
private void compare(DatanodeInfo dn1, DatanodeInfo dn2) {
|
||||
@ -400,12 +401,12 @@ private void compare(Token<BlockTokenIdentifier> expected,
|
||||
@Test
|
||||
public void testConvertLocatedBlock() {
|
||||
DatanodeInfo [] dnInfos = new DatanodeInfo[3];
|
||||
dnInfos[0] = new DatanodeInfo("host0", "0", 5000, 5001, 20000, 10001, 9999,
|
||||
59, 69, 32, "local", "host0", AdminStates.DECOMMISSION_INPROGRESS);
|
||||
dnInfos[1] = new DatanodeInfo("host1", "1", 5000, 5001, 20000, 10001, 9999,
|
||||
59, 69, 32, "local", "host1", AdminStates.DECOMMISSIONED);
|
||||
dnInfos[2] = new DatanodeInfo("host2", "2", 5000, 5001, 20000, 10001, 9999,
|
||||
59, 69, 32, "local", "host1", AdminStates.NORMAL);
|
||||
dnInfos[0] = new DatanodeInfo("host0", "host0", "0", 5000, 5001, 20000, 10001, 9999,
|
||||
59, 69, 32, "local", AdminStates.DECOMMISSION_INPROGRESS);
|
||||
dnInfos[1] = new DatanodeInfo("host1", "host1", "1", 5000, 5001, 20000, 10001, 9999,
|
||||
59, 69, 32, "local", AdminStates.DECOMMISSIONED);
|
||||
dnInfos[2] = new DatanodeInfo("host2", "host2", "2", 5000, 5001, 20000, 10001, 9999,
|
||||
59, 69, 32, "local", AdminStates.NORMAL);
|
||||
LocatedBlock lb = new LocatedBlock(
|
||||
new ExtendedBlock("bp12", 12345, 10, 53), dnInfos, 5, false);
|
||||
LocatedBlockProto lbProto = PBHelper.convert(lb);
|
||||
@ -423,7 +424,7 @@ public void testConvertLocatedBlock() {
|
||||
|
||||
@Test
|
||||
public void testConvertDatanodeRegistration() {
|
||||
DatanodeID dnId = new DatanodeID("host", "xyz", 1, 0);
|
||||
DatanodeID dnId = new DatanodeID("host", "host", "xyz", 1, 0);
|
||||
BlockKey[] keys = new BlockKey[] { getBlockKey(2), getBlockKey(3) };
|
||||
ExportedBlockKeys expKeys = new ExportedBlockKeys(true, 9, 10,
|
||||
getBlockKey(1), keys);
|
||||
|
@ -280,7 +280,7 @@ public void testBlockTokenRpcLeak() throws Exception {
|
||||
|
||||
final InetSocketAddress addr = NetUtils.getConnectAddress(server);
|
||||
DatanodeID fakeDnId = new DatanodeID("localhost:" + addr.getPort(),
|
||||
"fake-storage", 0, addr.getPort());
|
||||
"localhost", "fake-storage", 0, addr.getPort());
|
||||
|
||||
ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
|
||||
LocatedBlock fakeBlock = new LocatedBlock(b, new DatanodeInfo[0]);
|
||||
|
@ -197,9 +197,9 @@ private void testSyncReplicas(ReplicaRecoveryInfo replica1,
|
||||
locs, RECOVERY_ID);
|
||||
ArrayList<BlockRecord> syncList = new ArrayList<BlockRecord>(2);
|
||||
BlockRecord record1 = new BlockRecord(
|
||||
new DatanodeID("xx", "yy", 44, 55), dn1, replica1);
|
||||
new DatanodeID("xx", "yy", "zz", 44, 55), dn1, replica1);
|
||||
BlockRecord record2 = new BlockRecord(
|
||||
new DatanodeID("aa", "bb", 11, 22), dn2, replica2);
|
||||
new DatanodeID("aa", "bb", "cc", 11, 22), dn2, replica2);
|
||||
syncList.add(record1);
|
||||
syncList.add(record2);
|
||||
|
||||
@ -402,7 +402,7 @@ public void testRWRReplicas() throws IOException {
|
||||
private Collection<RecoveringBlock> initRecoveringBlocks() throws IOException {
|
||||
Collection<RecoveringBlock> blocks = new ArrayList<RecoveringBlock>(1);
|
||||
DatanodeInfo mockOtherDN = new DatanodeInfo(
|
||||
new DatanodeID("127.0.0.1", "storage-1234", 0, 0));
|
||||
new DatanodeID("127.0.0.1", "localhost", "storage-1234", 0, 0));
|
||||
DatanodeInfo[] locs = new DatanodeInfo[] {
|
||||
new DatanodeInfo(dn.getDNRegistrationForBP(block.getBlockPoolId())),
|
||||
mockOtherDN };
|
||||
|
@ -161,8 +161,8 @@ public void testFedSingleNN() throws IOException {
|
||||
assertEquals("number of volumes is wrong", 2, volInfos.size());
|
||||
|
||||
for (BPOfferService bpos : dn.getAllBpOs()) {
|
||||
LOG.info("reg: bpid=" + "; name=" + bpos.bpRegistration.name + "; sid="
|
||||
+ bpos.bpRegistration.storageID + "; nna=" +
|
||||
LOG.info("reg: bpid=" + "; name=" + bpos.bpRegistration.getName() + "; sid="
|
||||
+ bpos.bpRegistration.getStorageID() + "; nna=" +
|
||||
getNNSocketAddress(bpos));
|
||||
}
|
||||
|
||||
|
@ -348,7 +348,7 @@ public void testInterDNProtocolTimeout() throws Throwable {
|
||||
|
||||
final InetSocketAddress addr = NetUtils.getConnectAddress(server);
|
||||
DatanodeID fakeDnId = new DatanodeID(
|
||||
"localhost:" + addr.getPort(), "fake-storage", 0, addr.getPort());
|
||||
"localhost:" + addr.getPort(), "localhost", "fake-storage", 0, addr.getPort());
|
||||
DatanodeInfo dInfo = new DatanodeInfo(fakeDnId);
|
||||
InterDatanodeProtocol proxy = null;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user