Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,8 @@
* limitations under the License.
*/
package org.apache.hadoop.hdfs;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;

import java.io.DataOutputStream;
import java.io.IOException;
Expand All @@ -28,7 +28,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.junit.Test;
import org.junit.jupiter.api.Test;

public class TestDFSRemove {
final Path dir = new Path("/test/remove/");
Expand Down Expand Up @@ -80,8 +80,8 @@ public void testRemove() throws Exception {
Thread.sleep(3 * DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT * 1000);
// all blocks should be gone now.
long dfsUsedFinal = getTotalDfsUsed(cluster);
assertEquals("All blocks should be gone. start=" + dfsUsedStart
+ " max=" + dfsUsedMax + " final=" + dfsUsedFinal, dfsUsedStart, dfsUsedFinal);
assertEquals(dfsUsedStart, dfsUsedFinal, "All blocks should be gone. start=" + dfsUsedStart
+ " max=" + dfsUsedMax + " final=" + dfsUsedFinal);
}

fs.delete(dir, true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,9 @@
* limitations under the License.
*/
package org.apache.hadoop.hdfs;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;

import java.io.DataOutputStream;
import java.io.IOException;
Expand All @@ -34,7 +34,8 @@
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Test;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;

public class TestDFSRename {
static int countLease(MiniDFSCluster cluster) {
Expand Down Expand Up @@ -136,7 +137,8 @@ public void testRename() throws Exception {
* Check the blocks of dst file are cleaned after rename with overwrite
* Restart NN to check the rename successfully
*/
@Test(timeout = 120000)
@Test
@Timeout(value = 120)
public void testRenameWithOverwrite() throws Exception {
final short replFactor = 2;
final long blockSize = 512;
Expand Down Expand Up @@ -194,9 +196,9 @@ public void testRename2Options() throws Exception {
dfs.rename(path, new Path("/dir1"),
new Rename[] {Rename.OVERWRITE, Rename.TO_TRASH});
String auditOut = auditLog.getOutput();
assertTrue("Rename should have both OVERWRITE and TO_TRASH "
+ "flags at namenode but had only " + auditOut,
auditOut.contains("options=[OVERWRITE, TO_TRASH]"));
assertTrue(auditOut.contains("options=[OVERWRITE, TO_TRASH]"),
"Rename should have both OVERWRITE and TO_TRASH "
+ "flags at namenode but had only " + auditOut);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,10 @@

import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
import static org.junit.Assert.*;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;

import java.io.File;
import java.io.IOException;
Expand All @@ -39,8 +42,8 @@
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.util.StringUtils;
import org.junit.After;
import org.junit.Test;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;


/**
Expand Down Expand Up @@ -80,8 +83,7 @@ void checkResult(NodeType nodeType, String[] baseDirs) throws Exception {
FSImageTestUtil.assertReasonableNameCurrentDir(curDir);
break;
case DATA_NODE:
assertEquals(
UpgradeUtilities.checksumContents(nodeType, curDir, false),
assertEquals(UpgradeUtilities.checksumContents(nodeType, curDir, false),
UpgradeUtilities.checksumMasterDataNodeContents());
break;
}
Expand Down Expand Up @@ -127,8 +129,8 @@ void startNameNodeShouldFail(String searchString) {
void startBlockPoolShouldFail(StartupOption operation, String bpid)
throws IOException {
cluster.startDataNodes(conf, 1, false, operation, null); // should fail
assertFalse("Block pool " + bpid + " should have failed to start",
cluster.getDataNodes().get(0).isBPServiceAlive(bpid));
assertFalse(cluster.getDataNodes().get(0).isBPServiceAlive(bpid),
"Block pool " + bpid + " should have failed to start");
}

/**
Expand Down Expand Up @@ -344,7 +346,7 @@ private void deleteMatchingFiles(File[] baseDirs, String regex) {
}
}

@After
@AfterEach
public void tearDown() throws Exception {
LOG.info("Shutting down MiniDFSCluster");
if (cluster != null) {
Expand Down
Loading