Skip to content

Commit

Permalink
Add a test for single node rack case.
Browse files Browse the repository at this point in the history
  • Loading branch information
timmylicheng committed Mar 27, 2020
1 parent 1fe913f commit 8857bca
Show file tree
Hide file tree
Showing 2 changed files with 36 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -408,7 +408,7 @@ private boolean checkAllNodesAreEqual(NetworkTopology topology) {
@VisibleForTesting
protected DatanodeDetails chooseNodeFromNetworkTopology(
NetworkTopology networkTopology, DatanodeDetails anchor,
List<DatanodeDetails> excludedNodes) throws SCMException {
List<DatanodeDetails> excludedNodes) {
Preconditions.checkArgument(networkTopology != null);

Collection<Node> excluded = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,35 @@ public void testChooseNodeBasedOnNetworkTopology() throws SCMException {
nextNode.getNetworkLocation());
}

@Test
public void testChooseNodeWithSingleNodeRack() throws SCMException {
// There is only one node on 3 racks altogether.
List<DatanodeDetails> datanodes = new ArrayList<>();
for (Node node : SINGLE_NODE_RACK) {
DatanodeDetails datanode = overwriteLocationInNode(
MockDatanodeDetails.randomDatanodeDetails(), node);
datanodes.add(datanode);
}
MockNodeManager localNodeManager = new MockNodeManager(initTopology(),
datanodes, false, datanodes.size());
PipelinePlacementPolicy localPlacementPolicy = new PipelinePlacementPolicy(
localNodeManager, new PipelineStateManager(), conf);
int nodesRequired = HddsProtos.ReplicationFactor.THREE.getNumber();
List<DatanodeDetails> results = localPlacementPolicy.chooseDatanodes(
new ArrayList<>(datanodes.size()),
new ArrayList<>(datanodes.size()),
nodesRequired, 0);

Assert.assertEquals(nodesRequired, results.size());
// 3 nodes should be on different racks.
Assert.assertNotEquals(results.get(0).getNetworkLocation(),
results.get(1).getNetworkLocation());
Assert.assertNotEquals(results.get(0).getNetworkLocation(),
results.get(2).getNetworkLocation());
Assert.assertNotEquals(results.get(1).getNetworkLocation(),
results.get(2).getNetworkLocation());
}

@Test
public void testChooseNodeBasedOnRackAwareness() {
List<DatanodeDetails> healthyNodes = overWriteLocationInNodes(
Expand Down Expand Up @@ -192,6 +221,12 @@ public void testRackAwarenessNotEnabledWithFallBack() throws SCMException{
new NodeImpl("h8", "/r4", NetConstants.NODE_COST_DEFAULT),
};

// 3 racks with single node.
private final static Node[] SINGLE_NODE_RACK = new NodeImpl[] {
new NodeImpl("h1", "/r1", NetConstants.NODE_COST_DEFAULT),
new NodeImpl("h2", "/r2", NetConstants.NODE_COST_DEFAULT),
new NodeImpl("h3", "/r3", NetConstants.NODE_COST_DEFAULT)
};

private NetworkTopology createNetworkTopologyOnDifRacks() {
NetworkTopology topology = new NetworkTopologyImpl(new Configuration());
Expand Down

0 comments on commit 8857bca

Please sign in to comment.