Skip to content
Closed
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 @@ -85,6 +85,7 @@
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.not;
import static org.junit.Assert.*;
import static org.junit.Assume.assumeTrue;
import static org.hamcrest.core.StringContains.containsString;

/**
Expand Down Expand Up @@ -576,26 +577,33 @@ public void run() {
//use SecurityManager to pause the copying of f1 and begin copying f2
SecurityManager sm = System.getSecurityManager();
System.out.println("SecurityManager = " + sm);
System.setSecurityManager(new SecurityManager() {
private boolean firstTime = true;
try {
System.setSecurityManager(new SecurityManager() {
private boolean firstTime = true;

@Override
public void checkPermission(Permission perm) {
if (firstTime) {
Thread t = Thread.currentThread();
if (!t.toString().contains("DataNode")) {
String s = "" + Arrays.asList(t.getStackTrace());
if (s.contains("FileUtil.copyContent")) {
//pause at FileUtil.copyContent

firstTime = false;
copy2ndFileThread.start();
try {Thread.sleep(5000);} catch (InterruptedException e) {}
@Override
public void checkPermission(Permission perm) {
if (firstTime) {
Thread t = Thread.currentThread();
if (!t.toString().contains("DataNode")) {
String s = "" + Arrays.asList(t.getStackTrace());
if (s.contains("FileUtil.copyContent")) {
//pause at FileUtil.copyContent

firstTime = false;
copy2ndFileThread.start();
try {
Thread.sleep(5000);
} catch (InterruptedException e) {
}
}
}
}
}
}
});
});
} catch (UnsupportedOperationException e) {
assumeTrue("Test is skipped because SecurityManager cannot be set (JEP 411)", false);
}
show("copy local " + f1 + " to remote " + dst);
dfs.copyFromLocalFile(false, false, new Path(f1.getPath()), dst);
show("done");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import java.util.concurrent.TimeoutException;

import static org.junit.Assert.*;
import static org.junit.Assume.assumeTrue;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;

Expand Down Expand Up @@ -79,8 +80,12 @@ public void checkConnect(String host, int port) {
@Test
public void testDNSLookups() throws Exception {
MonitorDNS sm = new MonitorDNS();
System.setSecurityManager(sm);

try {
System.setSecurityManager(sm);
} catch (UnsupportedOperationException e) {
assumeTrue("Test is skipped because SecurityManager cannot be set (JEP 411)", false);
}

MiniDFSCluster cluster = null;
try {
HdfsConfiguration conf = new HdfsConfiguration();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -272,8 +272,6 @@ public void testSubmitter() throws Exception {
Submitter.setJavaPartitioner(conf, partitioner.getClass());

assertEquals(PipesPartitioner.class, (Submitter.getJavaPartitioner(conf)));
// test going to call main method with System.exit(). Change Security
SecurityManager securityManager = System.getSecurityManager();
// store System.out
PrintStream oldps = System.out;
ByteArrayOutputStream out = new ByteArrayOutputStream();
Expand Down Expand Up @@ -330,8 +328,6 @@ public void testSubmitter() throws Exception {
+ "archives to be unarchived on the compute machines"));
} finally {
System.setOut(oldps);
// restore
System.setSecurityManager(securityManager);
if (psw != null) {
// remove password files
for (File file : psw) {
Expand Down Expand Up @@ -381,7 +377,6 @@ public void testSubmitter() throws Exception {

} finally {
System.setOut(oldps);
System.setSecurityManager(securityManager);
}

}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.junit.jupiter.api.Assumptions.assumeTrue;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyBoolean;
import static org.mockito.Mockito.doReturn;
Expand Down Expand Up @@ -66,7 +67,11 @@ private static Configuration getConf() {
public void setup() {

securityManager = System.getSecurityManager();
System.setSecurityManager(new NoExitSecurityManager());
try {
System.setSecurityManager(new NoExitSecurityManager());
} catch (UnsupportedOperationException e) {
assumeTrue(false, "Test is skipped because SecurityManager cannot be set (JEP 411)");
}
try {
fs = FileSystem.get(getConf());
root = new Path("target/tmp").makeQualified(fs.getUri(),
Expand All @@ -79,10 +84,16 @@ public void setup() {

@AfterEach
public void tearDown() {
System.setSecurityManager(securityManager);
try {
System.setSecurityManager(securityManager);
} catch (UnsupportedOperationException e) {
// JUnit 5 calls @AfterEach even if @BeforeEach has thrown TestAbortedException
// The test has already been already skipped
}
}

/**
* test methods run end execute of DistCp class. silple copy file
* test methods run end execute of DistCp class. simple copy file
* @throws Exception
*/
@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,8 +174,6 @@ public void testStressSubmit() throws Exception {
@Test (timeout=100000)
public void testMain() throws Exception {

SecurityManager securityManager = System.getSecurityManager();

final ByteArrayOutputStream bytes = new ByteArrayOutputStream();
final PrintStream out = new PrintStream(bytes);
final PrintStream oldOut = System.out;
Expand All @@ -190,7 +188,6 @@ public void testMain() throws Exception {
ExitUtil.resetFirstExitException();
} finally {
System.setErr(oldOut);
System.setSecurityManager(securityManager);
}
String print = bytes.toString();
// should be printed tip in std error stream
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
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 static org.junit.jupiter.api.Assumptions.assumeTrue;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
Expand Down Expand Up @@ -371,7 +372,11 @@ private void testPreMountedControllerInitialization(String myHierarchy)
assertFalse(cpuCgroupMountDir.delete(), "Could not delete cgroups");
assertFalse(cpuCgroupMountDir.exists(), "Directory should be deleted");
SecurityManager manager = System.getSecurityManager();
System.setSecurityManager(new MockSecurityManagerDenyWrite());
try {
System.setSecurityManager(new MockSecurityManagerDenyWrite());
} catch (UnsupportedOperationException e) {
assumeTrue(false, "Test is skipped because SecurityManager cannot be set (JEP 411)");
}
try {
cGroupsHandler.initializeCGroupController(
CGroupsHandler.CGroupController.CPU);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimpleCandidateNodeSet;
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.jupiter.api.Assumptions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
Expand Down Expand Up @@ -1103,17 +1104,23 @@ public Boolean answer(InvocationOnMock invocation) throws Exception {
@Test
@Timeout(value = 30)
public void testAsyncScheduleThreadExit() throws Exception {
// init RM & NM
final MockRM rm = new MockRM(conf);
rm.start();
rm.registerNode("192.168.0.1:1234", 8 * GB);
rm.drainEvents();

// Set no exit security manager to catch System.exit
SecurityManager originalSecurityManager = System.getSecurityManager();
NoExitSecurityManager noExitSecurityManager =
new NoExitSecurityManager(originalSecurityManager);
System.setSecurityManager(noExitSecurityManager);
try {
System.setSecurityManager(noExitSecurityManager);
} catch (UnsupportedOperationException e) {
Assumptions.assumeTrue(false,
"Test is skipped because SecurityManager cannot be set (JEP411)");
}

// init RM & NM
final MockRM rm = new MockRM(conf);
rm.start();
rm.registerNode("192.168.0.1:1234", 8 * GB);
rm.drainEvents();

// test async-scheduling thread exit
try{
Expand Down