|
@@ -1,26 +1,27 @@
|
|
|
/**
|
|
|
- * 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
|
|
|
- * <p>
|
|
|
- * http://www.apache.org/licenses/LICENSE-2.0
|
|
|
- * <p>
|
|
|
+ * 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.
|
|
|
+ * 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.ozone.scm.node;
|
|
|
|
|
|
-import org.apache.commons.lang.RandomStringUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
|
|
+import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.hamcrest.CoreMatchers;
|
|
|
import org.junit.Assert;
|
|
@@ -76,8 +77,9 @@ public class TestNodeManager {
|
|
|
*
|
|
|
* @return DatanodeID
|
|
|
*/
|
|
|
- DatanodeID getDatanodeID() {
|
|
|
- return getDatanodeID(UUID.randomUUID().toString());
|
|
|
+ DatanodeID getDatanodeID(SCMNodeManager nodeManager) {
|
|
|
+
|
|
|
+ return getDatanodeID(nodeManager, UUID.randomUUID().toString());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -86,16 +88,19 @@ public class TestNodeManager {
|
|
|
* @param uuid - node ID, it is generally UUID.
|
|
|
* @return DatanodeID.
|
|
|
*/
|
|
|
- DatanodeID getDatanodeID(String uuid) {
|
|
|
+ DatanodeID getDatanodeID(SCMNodeManager nodeManager, String uuid) {
|
|
|
Random random = new Random();
|
|
|
String ipAddress = random.nextInt(256) + "."
|
|
|
+ random.nextInt(256) + "."
|
|
|
+ random.nextInt(256) + "."
|
|
|
+ random.nextInt(256);
|
|
|
|
|
|
- String hostName = RandomStringUtils.randomAscii(8);
|
|
|
- return new DatanodeID(ipAddress, hostName, uuid,
|
|
|
- 0, 0, 0, 0);
|
|
|
+ String hostName = uuid;
|
|
|
+ DatanodeID tempDataNode = new DatanodeID(ipAddress,
|
|
|
+ hostName, uuid, 0, 0, 0, 0);
|
|
|
+ RegisteredCommand command =
|
|
|
+ (RegisteredCommand) nodeManager.register(tempDataNode);
|
|
|
+ return new DatanodeID(command.getDatanodeUUID(), tempDataNode);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -107,7 +112,8 @@ public class TestNodeManager {
|
|
|
*/
|
|
|
|
|
|
SCMNodeManager createNodeManager(Configuration config) throws IOException {
|
|
|
- SCMNodeManager nodeManager = new SCMNodeManager(config);
|
|
|
+ SCMNodeManager nodeManager = new SCMNodeManager(config,
|
|
|
+ UUID.randomUUID().toString());
|
|
|
assertFalse("Node manager should be in chill mode",
|
|
|
nodeManager.isOutOfNodeChillMode());
|
|
|
return nodeManager;
|
|
@@ -126,10 +132,10 @@ public class TestNodeManager {
|
|
|
InterruptedException, TimeoutException {
|
|
|
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
|
|
|
-
|
|
|
// Send some heartbeats from different nodes.
|
|
|
for (int x = 0; x < nodeManager.getMinimumChillModeNodes(); x++) {
|
|
|
- nodeManager.updateHeartbeat(getDatanodeID());
|
|
|
+ DatanodeID datanodeID = getDatanodeID(nodeManager);
|
|
|
+ nodeManager.sendHeartbeat(datanodeID);
|
|
|
}
|
|
|
|
|
|
// Wait for 4 seconds max.
|
|
@@ -175,7 +181,7 @@ public class TestNodeManager {
|
|
|
|
|
|
// Need 100 nodes to come out of chill mode, only one node is sending HB.
|
|
|
nodeManager.setMinimumChillModeNodes(100);
|
|
|
- nodeManager.updateHeartbeat(getDatanodeID());
|
|
|
+ nodeManager.sendHeartbeat(getDatanodeID(nodeManager));
|
|
|
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatThead(), 100,
|
|
|
4 * 1000);
|
|
|
assertFalse("Not enough heartbeat, Node manager should have been in " +
|
|
@@ -197,11 +203,11 @@ public class TestNodeManager {
|
|
|
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
|
|
|
nodeManager.setMinimumChillModeNodes(3);
|
|
|
- DatanodeID datanodeID = getDatanodeID();
|
|
|
+ DatanodeID datanodeID = getDatanodeID(nodeManager);
|
|
|
|
|
|
// Send 10 heartbeat from same node, and assert we never leave chill mode.
|
|
|
for (int x = 0; x < 10; x++) {
|
|
|
- nodeManager.updateHeartbeat(datanodeID);
|
|
|
+ nodeManager.sendHeartbeat(datanodeID);
|
|
|
}
|
|
|
|
|
|
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatThead(), 100,
|
|
@@ -226,17 +232,15 @@ public class TestNodeManager {
|
|
|
Configuration conf = getConf();
|
|
|
conf.setInt(OzoneConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS, 100);
|
|
|
SCMNodeManager nodeManager = createNodeManager(conf);
|
|
|
+ DatanodeID datanodeID = getDatanodeID(nodeManager);
|
|
|
nodeManager.close();
|
|
|
|
|
|
// These should never be processed.
|
|
|
- nodeManager.updateHeartbeat(getDatanodeID());
|
|
|
+ nodeManager.sendHeartbeat(datanodeID);
|
|
|
|
|
|
// Let us just wait for 2 seconds to prove that HBs are not processed.
|
|
|
Thread.sleep(2 * 1000);
|
|
|
|
|
|
- assertFalse("Node manager executor service is shutdown, should never exit" +
|
|
|
- " chill mode", nodeManager.isOutOfNodeChillMode());
|
|
|
-
|
|
|
assertEquals("Assert new HBs were never processed", 0,
|
|
|
nodeManager.getLastHBProcessedCount());
|
|
|
}
|
|
@@ -256,8 +260,10 @@ public class TestNodeManager {
|
|
|
final int count = 10;
|
|
|
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
|
|
+
|
|
|
for (int x = 0; x < count; x++) {
|
|
|
- nodeManager.updateHeartbeat(getDatanodeID());
|
|
|
+ DatanodeID datanodeID = getDatanodeID(nodeManager);
|
|
|
+ nodeManager.sendHeartbeat(datanodeID);
|
|
|
}
|
|
|
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatThead(), 100,
|
|
|
4 * 1000);
|
|
@@ -338,20 +344,19 @@ public class TestNodeManager {
|
|
|
conf.setInt(OZONE_SCM_STALENODE_INTERVAL_MS, 3 * 1000);
|
|
|
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
|
|
- List<DatanodeID> nodeList = new LinkedList<>();
|
|
|
- DatanodeID staleNode = getDatanodeID();
|
|
|
- for (int x = 0; x < nodeCount; x++) {
|
|
|
- nodeList.add(getDatanodeID());
|
|
|
- }
|
|
|
+ List<DatanodeID> nodeList = createNodeSet(nodeManager, nodeCount,
|
|
|
+ "staleNode");
|
|
|
+ DatanodeID staleNode = getDatanodeID(nodeManager);
|
|
|
+
|
|
|
// Heartbeat once
|
|
|
- nodeManager.updateHeartbeat(staleNode);
|
|
|
+ nodeManager.sendHeartbeat(staleNode);
|
|
|
|
|
|
// Heartbeat all other nodes.
|
|
|
- nodeList.forEach(nodeManager::updateHeartbeat);
|
|
|
+ nodeList.forEach(nodeManager::sendHeartbeat);
|
|
|
|
|
|
// Wait for 2 seconds .. and heartbeat good nodes again.
|
|
|
Thread.sleep(2 * 1000);
|
|
|
- nodeList.forEach(nodeManager::updateHeartbeat);
|
|
|
+ nodeList.forEach(nodeManager::sendHeartbeat);
|
|
|
|
|
|
// Wait for 2 more seconds, 3 seconds is the stale window for this test
|
|
|
Thread.sleep(2 * 1000);
|
|
@@ -388,36 +393,34 @@ public class TestNodeManager {
|
|
|
|
|
|
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
|
|
- List<DatanodeID> nodeList = new LinkedList<>();
|
|
|
+ List<DatanodeID> nodeList = createNodeSet(nodeManager, nodeCount,
|
|
|
+ "Node");
|
|
|
+
|
|
|
+ DatanodeID deadNode = getDatanodeID(nodeManager);
|
|
|
|
|
|
- DatanodeID deadNode = getDatanodeID();
|
|
|
- for (int x = 0; x < nodeCount; x++) {
|
|
|
- nodeList.add(getDatanodeID());
|
|
|
- }
|
|
|
// Heartbeat once
|
|
|
- nodeManager.updateHeartbeat(deadNode);
|
|
|
+ nodeManager.sendHeartbeat(deadNode);
|
|
|
|
|
|
// Heartbeat all other nodes.
|
|
|
- nodeList.forEach(nodeManager::updateHeartbeat);
|
|
|
+ nodeList.forEach(nodeManager::sendHeartbeat);
|
|
|
|
|
|
// Wait for 2 seconds .. and heartbeat good nodes again.
|
|
|
Thread.sleep(2 * 1000);
|
|
|
|
|
|
- nodeList.forEach(nodeManager::updateHeartbeat);
|
|
|
+ nodeList.forEach(nodeManager::sendHeartbeat);
|
|
|
Thread.sleep(3 * 1000);
|
|
|
|
|
|
// heartbeat good nodes again.
|
|
|
- nodeList.forEach(nodeManager::updateHeartbeat);
|
|
|
+ nodeList.forEach(nodeManager::sendHeartbeat);
|
|
|
|
|
|
// 6 seconds is the dead window for this test , so we wait a total of
|
|
|
// 7 seconds to make sure that the node moves into dead state.
|
|
|
Thread.sleep(2 * 1000);
|
|
|
|
|
|
// Check for the dead node now.
|
|
|
- List<DatanodeID> deadNodeList = nodeManager
|
|
|
- .getNodes(DEAD);
|
|
|
- assertEquals("Expected to find 1 dead node", 1, nodeManager
|
|
|
- .getNodeCount(DEAD));
|
|
|
+ List<DatanodeID> deadNodeList = nodeManager.getNodes(DEAD);
|
|
|
+ assertEquals("Expected to find 1 dead node", 1,
|
|
|
+ nodeManager.getNodeCount(DEAD));
|
|
|
assertEquals("Expected to find 1 dead node", 1, deadNodeList.size());
|
|
|
assertEquals("Dead node is not the expected ID", deadNode
|
|
|
.getDatanodeUuid(), deadNodeList.get(0).getDatanodeUuid());
|
|
@@ -438,9 +441,8 @@ public class TestNodeManager {
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
|
|
|
GenericTestUtils.LogCapturer logCapturer =
|
|
|
GenericTestUtils.LogCapturer.captureLogs(SCMNodeManager.LOG);
|
|
|
- DatanodeID duplicateNodeID = getDatanodeID();
|
|
|
- nodeManager.registerNode(duplicateNodeID);
|
|
|
- nodeManager.registerNode(duplicateNodeID);
|
|
|
+ DatanodeID duplicateNodeID = getDatanodeID(nodeManager);
|
|
|
+ nodeManager.register(duplicateNodeID);
|
|
|
logCapturer.stopCapturing();
|
|
|
assertThat(logCapturer.getOutput(), containsString("Datanode is already" +
|
|
|
" registered."));
|
|
@@ -460,7 +462,7 @@ public class TestNodeManager {
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
|
|
|
GenericTestUtils.LogCapturer logCapturer =
|
|
|
GenericTestUtils.LogCapturer.captureLogs(SCMNodeManager.LOG);
|
|
|
- nodeManager.updateHeartbeat(null);
|
|
|
+ nodeManager.sendHeartbeat(null);
|
|
|
logCapturer.stopCapturing();
|
|
|
assertThat(logCapturer.getOutput(), containsString("Datanode ID in " +
|
|
|
"heartbeat is null"));
|
|
@@ -486,11 +488,6 @@ public class TestNodeManager {
|
|
|
@Test
|
|
|
public void testScmClusterIsInExpectedState1() throws IOException,
|
|
|
InterruptedException, TimeoutException {
|
|
|
-
|
|
|
- DatanodeID healthyNode = getDatanodeID("HealthyNode");
|
|
|
- DatanodeID staleNode = getDatanodeID("StaleNode");
|
|
|
- DatanodeID deadNode = getDatanodeID("DeadNode");
|
|
|
-
|
|
|
/**
|
|
|
* These values are very important. Here is what it means so you don't
|
|
|
* have to look it up while reading this code.
|
|
@@ -535,9 +532,12 @@ public class TestNodeManager {
|
|
|
* Cluster state: Healthy: All nodes are heartbeat-ing like normal.
|
|
|
*/
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
|
|
- nodeManager.updateHeartbeat(healthyNode);
|
|
|
- nodeManager.updateHeartbeat(staleNode);
|
|
|
- nodeManager.updateHeartbeat(deadNode);
|
|
|
+ DatanodeID healthyNode = getDatanodeID(nodeManager, "HealthyNode");
|
|
|
+ DatanodeID staleNode = getDatanodeID(nodeManager, "StaleNode");
|
|
|
+ DatanodeID deadNode = getDatanodeID(nodeManager, "DeadNode");
|
|
|
+ nodeManager.sendHeartbeat(healthyNode);
|
|
|
+ nodeManager.sendHeartbeat(staleNode);
|
|
|
+ nodeManager.sendHeartbeat(deadNode);
|
|
|
|
|
|
// Sleep so that heartbeat processing thread gets to run.
|
|
|
Thread.sleep(500);
|
|
@@ -563,12 +563,12 @@ public class TestNodeManager {
|
|
|
* the 3 second windows.
|
|
|
*/
|
|
|
|
|
|
- nodeManager.updateHeartbeat(healthyNode);
|
|
|
- nodeManager.updateHeartbeat(staleNode);
|
|
|
- nodeManager.updateHeartbeat(deadNode);
|
|
|
+ nodeManager.sendHeartbeat(healthyNode);
|
|
|
+ nodeManager.sendHeartbeat(staleNode);
|
|
|
+ nodeManager.sendHeartbeat(deadNode);
|
|
|
|
|
|
Thread.sleep(1500);
|
|
|
- nodeManager.updateHeartbeat(healthyNode);
|
|
|
+ nodeManager.sendHeartbeat(healthyNode);
|
|
|
Thread.sleep(2 * 1000);
|
|
|
assertEquals(1, nodeManager.getNodeCount(HEALTHY));
|
|
|
|
|
@@ -588,10 +588,10 @@ public class TestNodeManager {
|
|
|
* staleNode to move to stale state and deadNode to move to dead state.
|
|
|
*/
|
|
|
|
|
|
- nodeManager.updateHeartbeat(healthyNode);
|
|
|
- nodeManager.updateHeartbeat(staleNode);
|
|
|
+ nodeManager.sendHeartbeat(healthyNode);
|
|
|
+ nodeManager.sendHeartbeat(staleNode);
|
|
|
Thread.sleep(1500);
|
|
|
- nodeManager.updateHeartbeat(healthyNode);
|
|
|
+ nodeManager.sendHeartbeat(healthyNode);
|
|
|
Thread.sleep(2 * 1000);
|
|
|
|
|
|
// 3.5 seconds have elapsed for stale node, so it moves into Stale.
|
|
@@ -617,14 +617,13 @@ public class TestNodeManager {
|
|
|
assertEquals("Expected one dead node", 1, deadList.size());
|
|
|
assertEquals("Dead node is not the expected ID", deadNode
|
|
|
.getDatanodeUuid(), deadList.get(0).getDatanodeUuid());
|
|
|
-
|
|
|
/**
|
|
|
* Cluster State : let us heartbeat all the nodes and verify that we get
|
|
|
* back all the nodes in healthy state.
|
|
|
*/
|
|
|
- nodeManager.updateHeartbeat(healthyNode);
|
|
|
- nodeManager.updateHeartbeat(staleNode);
|
|
|
- nodeManager.updateHeartbeat(deadNode);
|
|
|
+ nodeManager.sendHeartbeat(healthyNode);
|
|
|
+ nodeManager.sendHeartbeat(staleNode);
|
|
|
+ nodeManager.sendHeartbeat(deadNode);
|
|
|
Thread.sleep(500);
|
|
|
//Assert all nodes are healthy.
|
|
|
assertEquals(3, nodeManager.getAllNodes().size());
|
|
@@ -640,10 +639,10 @@ public class TestNodeManager {
|
|
|
* @param sleepDuration - Duration to sleep between heartbeats.
|
|
|
* @throws InterruptedException
|
|
|
*/
|
|
|
- private void heartbeatNodeSet(NodeManager manager, List<DatanodeID> list,
|
|
|
+ private void heartbeatNodeSet(SCMNodeManager manager, List<DatanodeID> list,
|
|
|
int sleepDuration) throws InterruptedException {
|
|
|
while (!Thread.currentThread().isInterrupted()) {
|
|
|
- list.forEach(manager::updateHeartbeat);
|
|
|
+ list.forEach(manager::sendHeartbeat);
|
|
|
Thread.sleep(sleepDuration);
|
|
|
}
|
|
|
}
|
|
@@ -655,10 +654,12 @@ public class TestNodeManager {
|
|
|
* @param prefix - A prefix string that can be used in verification.
|
|
|
* @return List of Nodes.
|
|
|
*/
|
|
|
- private List<DatanodeID> createNodeSet(int count, String prefix) {
|
|
|
+ private List<DatanodeID> createNodeSet(SCMNodeManager nodeManager, int
|
|
|
+ count, String
|
|
|
+ prefix) {
|
|
|
List<DatanodeID> list = new LinkedList<>();
|
|
|
for (int x = 0; x < count; x++) {
|
|
|
- list.add(getDatanodeID(prefix + x));
|
|
|
+ list.add(getDatanodeID(nodeManager, prefix + x));
|
|
|
}
|
|
|
return list;
|
|
|
}
|
|
@@ -696,12 +697,16 @@ public class TestNodeManager {
|
|
|
conf.setInt(OZONE_SCM_DEADNODE_INTERVAL_MS, 6 * 1000);
|
|
|
conf.setInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS, 7000);
|
|
|
|
|
|
- List<DatanodeID> healthyNodeList = createNodeSet(healthyCount, "Healthy");
|
|
|
- List<DatanodeID> staleNodeList = createNodeSet(staleCount, "Stale");
|
|
|
- List<DatanodeID> deadNodeList = createNodeSet(deadCount, "Dead");
|
|
|
|
|
|
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
|
|
+ List<DatanodeID> healthyNodeList = createNodeSet(nodeManager,
|
|
|
+ healthyCount, "Healthy");
|
|
|
+ List<DatanodeID> staleNodeList = createNodeSet(nodeManager, staleCount,
|
|
|
+ "Stale");
|
|
|
+ List<DatanodeID> deadNodeList = createNodeSet(nodeManager, deadCount,
|
|
|
+ "Dead");
|
|
|
+
|
|
|
Runnable healthyNodeTask = () -> {
|
|
|
try {
|
|
|
// 2 second heartbeat makes these nodes stay healthy.
|
|
@@ -722,7 +727,7 @@ public class TestNodeManager {
|
|
|
|
|
|
// No Thread just one time HBs the node manager, so that these will be
|
|
|
// marked as dead nodes eventually.
|
|
|
- deadNodeList.forEach(nodeManager::updateHeartbeat);
|
|
|
+ deadNodeList.forEach(nodeManager::sendHeartbeat);
|
|
|
|
|
|
Thread thread1 = new Thread(healthyNodeTask);
|
|
|
thread1.setDaemon(true);
|
|
@@ -745,7 +750,7 @@ public class TestNodeManager {
|
|
|
List<DatanodeID> deadList = nodeManager.getNodes(DEAD);
|
|
|
|
|
|
for (DatanodeID node : deadList) {
|
|
|
- assertThat(node.getDatanodeUuid(), CoreMatchers.startsWith("Dead"));
|
|
|
+ assertThat(node.getHostName(), CoreMatchers.startsWith("Dead"));
|
|
|
}
|
|
|
|
|
|
// Checking stale nodes is tricky since they have to move between
|
|
@@ -772,8 +777,6 @@ public class TestNodeManager {
|
|
|
InterruptedException, TimeoutException {
|
|
|
final int healthyCount = 3000;
|
|
|
final int staleCount = 3000;
|
|
|
- List<DatanodeID> healthyList = createNodeSet(healthyCount, "h");
|
|
|
- List<DatanodeID> staleList = createNodeSet(staleCount, "s");
|
|
|
Configuration conf = getConf();
|
|
|
conf.setInt(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS, 100);
|
|
|
conf.setInt(OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS, 1);
|
|
@@ -781,6 +784,10 @@ public class TestNodeManager {
|
|
|
conf.setInt(OZONE_SCM_DEADNODE_INTERVAL_MS, 6 * 1000);
|
|
|
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
|
|
+ List<DatanodeID> healthyList = createNodeSet(nodeManager,
|
|
|
+ healthyCount, "h");
|
|
|
+ List<DatanodeID> staleList = createNodeSet(nodeManager, staleCount, "s");
|
|
|
+
|
|
|
Runnable healthyNodeTask = () -> {
|
|
|
try {
|
|
|
heartbeatNodeSet(nodeManager, healthyList, 2 * 1000);
|
|
@@ -800,7 +807,6 @@ public class TestNodeManager {
|
|
|
thread1.setDaemon(true);
|
|
|
thread1.start();
|
|
|
|
|
|
-
|
|
|
Thread thread2 = new Thread(staleNodeTask);
|
|
|
thread2.setDaemon(true);
|
|
|
thread2.start();
|
|
@@ -829,7 +835,6 @@ public class TestNodeManager {
|
|
|
public void testScmLogsHeartbeatFlooding() throws IOException,
|
|
|
InterruptedException {
|
|
|
final int healthyCount = 3000;
|
|
|
- List<DatanodeID> healthyList = createNodeSet(healthyCount, "h");
|
|
|
|
|
|
// Make the HB process thread run slower.
|
|
|
Configuration conf = getConf();
|
|
@@ -838,6 +843,8 @@ public class TestNodeManager {
|
|
|
conf.setInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS, 500);
|
|
|
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
|
|
+ List<DatanodeID> healthyList = createNodeSet(nodeManager, healthyCount,
|
|
|
+ "h");
|
|
|
GenericTestUtils.LogCapturer logCapturer =
|
|
|
GenericTestUtils.LogCapturer.captureLogs(SCMNodeManager.LOG);
|
|
|
Runnable healthyNodeTask = () -> {
|
|
@@ -871,7 +878,8 @@ public class TestNodeManager {
|
|
|
|
|
|
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
|
|
nodeManager.setMinimumChillModeNodes(10);
|
|
|
- nodeManager.updateHeartbeat(getDatanodeID());
|
|
|
+ DatanodeID datanodeID = getDatanodeID(nodeManager);
|
|
|
+ nodeManager.sendHeartbeat(datanodeID);
|
|
|
String status = nodeManager.getChillModeStatus();
|
|
|
Assert.assertThat(status, CoreMatchers.containsString("Still in chill " +
|
|
|
"mode. Waiting on nodes to report in."));
|
|
@@ -900,7 +908,8 @@ public class TestNodeManager {
|
|
|
|
|
|
// Assert that node manager force enter cannot be overridden by nodes HBs.
|
|
|
for(int x= 0; x < 20; x++) {
|
|
|
- nodeManager.updateHeartbeat(getDatanodeID());
|
|
|
+ DatanodeID datanode = getDatanodeID(nodeManager);
|
|
|
+ nodeManager.sendHeartbeat(datanode);
|
|
|
}
|
|
|
|
|
|
Thread.sleep(500);
|