Skip to content

Commit

Permalink
[Java] Use test groups to filter tests of different run modes (#9703)
Browse files Browse the repository at this point in the history
  • Loading branch information
kfstorm committed Jul 29, 2020
1 parent 9fbfee2 commit bdc005a
Show file tree
Hide file tree
Showing 22 changed files with 54 additions and 92 deletions.
3 changes: 3 additions & 0 deletions java/test/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,9 @@
<artifactId>maven-surefire-plugin</artifactId>
<version>2.21.0</version>
<configuration>
<suiteXmlFiles>
<suiteXmlFile>../testng.xml</suiteXmlFile>
</suiteXmlFiles>
<trimStackTrace>false</trimStackTrace>
<testSourceDirectory>${basedir}/src/main/java/</testSourceDirectory>
<testClassesDirectory>${project.build.directory}/classes/</testClassesDirectory>
Expand Down
3 changes: 3 additions & 0 deletions java/test/pom_template.xml
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,9 @@
<artifactId>maven-surefire-plugin</artifactId>
<version>2.21.0</version>
<configuration>
<suiteXmlFiles>
<suiteXmlFile>../testng.xml</suiteXmlFile>
</suiteXmlFiles>
<trimStackTrace>false</trimStackTrace>
<testSourceDirectory>${basedir}/src/main/java/</testSourceDirectory>
<testClassesDirectory>${project.build.directory}/classes/</testClassesDirectory>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@
import org.testng.Assert;
import org.testng.annotations.Test;

@Test
@Test(groups = {"cluster"})
public class ActorConcurrentCallTest extends BaseTest {

public static class ConcurrentActor {
Expand All @@ -27,8 +27,6 @@ public String countDown() {
}

public void testConcurrentCall() {
TestUtils.skipTestUnderSingleProcess();

ActorHandle<ConcurrentActor> actor =
Ray.actor(ConcurrentActor::new)
.setMaxConcurrency(3)
Expand Down
4 changes: 1 addition & 3 deletions java/test/src/main/java/io/ray/test/ActorRestartTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@
import org.testng.Assert;
import org.testng.annotations.Test;

@Test
@Test(groups = {"cluster"})
public class ActorRestartTest extends BaseTest {

public static class Counter {
Expand Down Expand Up @@ -41,7 +41,6 @@ public int getPid() {
}

public void testActorRestart() throws InterruptedException, IOException {
TestUtils.skipTestUnderSingleProcess();
ActorHandle<Counter> actor = Ray.actor(Counter::new).setMaxRestarts(1).remote();
// Call increase 3 times.
for (int i = 0; i < 3; i++) {
Expand Down Expand Up @@ -118,7 +117,6 @@ public void checkpointExpired(ActorId actorId, UniqueId checkpointId) {
}

public void testActorCheckpointing() throws IOException, InterruptedException {
TestUtils.skipTestUnderSingleProcess();
ActorHandle<CheckpointableCounter> actor = Ray.actor(CheckpointableCounter::new)
.setMaxRestarts(1).remote();
// Call increase 3 times.
Expand Down
4 changes: 1 addition & 3 deletions java/test/src/main/java/io/ray/test/ActorTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -118,10 +118,8 @@ public void testPassActorAsParameter() {
}

// TODO(qwang): Will re-enable this test case once ref counting is supported in Java.
@Test(enabled = false)
@Test(enabled = false, groups = {"cluster"})
public void testUnreconstructableActorObject() throws InterruptedException {
TestUtils.skipTestUnderSingleProcess();

// The UnreconstructableException is created by raylet.
ActorHandle<Counter> counter = Ray.actor(Counter::new, 100).remote();
// Call an actor method.
Expand Down
15 changes: 2 additions & 13 deletions java/test/src/main/java/io/ray/test/BaseMultiLanguageTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
import org.testng.SkipException;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

@Test(groups = {"cluster", "multiLanguage"})
public abstract class BaseMultiLanguageTest {

private static final Logger LOGGER = LoggerFactory.getLogger(BaseMultiLanguageTest.class);
Expand Down Expand Up @@ -50,18 +51,8 @@ private boolean executeCommand(List<String> command, int waitTimeoutSeconds,
}
}

private void checkMultiLanguageTestFlag() {
if (!"1".equals(System.getenv("ENABLE_MULTI_LANGUAGE_TESTS"))) {
LOGGER.info("Skip Multi-language tests because environment variable "
+ "ENABLE_MULTI_LANGUAGE_TESTS isn't set");
throw new SkipException("Skip test.");
}
}

@BeforeClass(alwaysRun = true)
public void setUp() {
checkMultiLanguageTestFlag();

// Delete existing socket files.
for (String socket : ImmutableList.of(RAYLET_SOCKET_NAME, PLASMA_STORE_SOCKET_NAME)) {
File file = new File(socket);
Expand Down Expand Up @@ -117,8 +108,6 @@ protected Map<String, String> getRayStartEnv() {

@AfterClass(alwaysRun = true)
public void tearDown() {
checkMultiLanguageTestFlag();

// Disconnect to the cluster.
Ray.shutdown();
System.clearProperty("ray.redis.address");
Expand Down
3 changes: 1 addition & 2 deletions java/test/src/main/java/io/ray/test/ClassLoaderTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,8 @@ public void tearDown() {
System.clearProperty("ray.job.resource-path");
}

@Test
@Test(groups = {"cluster"})
public void testClassLoaderInMultiThreading() throws Exception {
TestUtils.skipTestUnderSingleProcess();
Assert.assertTrue(TestUtils.getRuntime().getRayConfig().numWorkersPerProcess > 1);

final String jobResourcePath = resourcePath + "/" + Ray.getRuntimeContext().getCurrentJobId();
Expand Down
4 changes: 1 addition & 3 deletions java/test/src/main/java/io/ray/test/DynamicResourceTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,8 @@ public static String sayHi() {
return "hi";
}

@Test
@Test(groups = {"cluster"})
public void testSetResource() {
TestUtils.skipTestUnderSingleProcess();

// Call a task in advance to warm up the cluster to avoid being too slow to start workers.
TestUtils.warmUpCluster();

Expand Down
13 changes: 1 addition & 12 deletions java/test/src/main/java/io/ray/test/FailureTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

@Test(groups = {"cluster"})
public class FailureTest extends BaseTest {

private static final String EXCEPTION_MESSAGE = "Oops";
Expand Down Expand Up @@ -84,29 +85,21 @@ private static void assertTaskFailedWithRayTaskException(ObjectRef<?> objectRef)
}
}

@Test
public void testNormalTaskFailure() {
TestUtils.skipTestUnderSingleProcess();
assertTaskFailedWithRayTaskException(Ray.task(FailureTest::badFunc).remote());
}

@Test
public void testActorCreationFailure() {
TestUtils.skipTestUnderSingleProcess();
ActorHandle<BadActor> actor = Ray.actor(BadActor::new, true).remote();
assertTaskFailedWithRayTaskException(actor.task(BadActor::badMethod).remote());
}

@Test
public void testActorTaskFailure() {
TestUtils.skipTestUnderSingleProcess();
ActorHandle<BadActor> actor = Ray.actor(BadActor::new, false).remote();
assertTaskFailedWithRayTaskException(actor.task(BadActor::badMethod).remote());
}

@Test
public void testWorkerProcessDying() {
TestUtils.skipTestUnderSingleProcess();
try {
Ray.task(FailureTest::badFunc2).remote().get();
Assert.fail("This line shouldn't be reached.");
Expand All @@ -116,9 +109,7 @@ public void testWorkerProcessDying() {
}
}

@Test
public void testActorProcessDying() {
TestUtils.skipTestUnderSingleProcess();
ActorHandle<BadActor> actor = Ray.actor(BadActor::new, false).remote();
try {
actor.task(BadActor::badMethod2).remote().get();
Expand All @@ -136,9 +127,7 @@ public void testActorProcessDying() {
}
}

@Test
public void testGetThrowsQuicklyWhenFoundException() {
TestUtils.skipTestUnderSingleProcess();
List<RayFunc0<Integer>> badFunctions = Arrays.asList(FailureTest::badFunc,
FailureTest::badFunc2);
TestUtils.warmUpCluster();
Expand Down
4 changes: 1 addition & 3 deletions java/test/src/main/java/io/ray/test/GcsClientTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

@Test(groups = {"cluster"})
public class GcsClientTest extends BaseTest {

@BeforeClass
Expand All @@ -23,9 +24,7 @@ public void tearDown() {
System.clearProperty("ray.resources");
}

@Test
public void testGetAllNodeInfo() {
TestUtils.skipTestUnderSingleProcess();
RayConfig config = TestUtils.getRuntime().getRayConfig();

Preconditions.checkNotNull(config);
Expand All @@ -39,7 +38,6 @@ public void testGetAllNodeInfo() {

@Test
public void testNextJob() {
TestUtils.skipTestUnderSingleProcess();
RayConfig config = TestUtils.getRuntime().getRayConfig();
// The value of job id of this driver in cluster should be 1.
Assert.assertEquals(config.getJobId(), JobId.fromInt(1));
Expand Down
4 changes: 1 addition & 3 deletions java/test/src/main/java/io/ray/test/KillActorTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

@Test
@Test(groups = {"cluster"})
public class KillActorTest extends BaseTest {

@BeforeClass
Expand Down Expand Up @@ -54,8 +54,6 @@ private static void remoteKill(ActorHandle<?> actor, boolean noRestart) {
}

private void testKillActor(BiConsumer<ActorHandle<?>, Boolean> kill, boolean noRestart) {
TestUtils.skipTestUnderSingleProcess();

ActorHandle<HangActor> actor = Ray.actor(HangActor::new)
.setMaxRestarts(1)
.remote();
Expand Down
9 changes: 1 addition & 8 deletions java/test/src/main/java/io/ray/test/MetricTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import org.testng.annotations.AfterMethod;
import org.testng.annotations.Test;

@Test(groups = {"cluster"})
public class MetricTest extends BaseTest {

boolean doubleEqual(double value, double other) {
Expand Down Expand Up @@ -77,9 +78,7 @@ public void maybeShutdownMetrics() {
Metrics.shutdown();
}

@Test
public void testAddGauge() {
TestUtils.skipTestUnderSingleProcess();
Map<TagKey, String> tags = new HashMap<>();
tags.put(new TagKey("tag1"), "value1");

Expand All @@ -90,9 +89,7 @@ public void testAddGauge() {
gauge.unregister();
}

@Test
public void testAddCount() {
TestUtils.skipTestUnderSingleProcess();
Map<TagKey, String> tags = new HashMap<>();
tags.put(new TagKey("tag1"), "value1");
tags.put(new TagKey("count_tag"), "default");
Expand All @@ -104,9 +101,7 @@ public void testAddCount() {
Assert.assertTrue(doubleEqual(count.getCount(), 30.0));
}

@Test
public void testAddSum() {
TestUtils.skipTestUnderSingleProcess();
Map<TagKey, String> tags = new HashMap<>();
tags.put(new TagKey("tag1"), "value1");
tags.put(new TagKey("sum_tag"), "default");
Expand All @@ -118,9 +113,7 @@ public void testAddSum() {
Assert.assertTrue(doubleEqual(sum.getSum(), 30.0));
}

@Test
public void testAddHistogram() {
TestUtils.skipTestUnderSingleProcess();
Map<TagKey, String> tags = new HashMap<>();
tags.put(new TagKey("tag1"), "value1");
tags.put(new TagKey("histogram_tag"), "default");
Expand Down
6 changes: 3 additions & 3 deletions java/test/src/main/java/io/ray/test/MultiThreadingTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -121,15 +121,15 @@ public void testInDriver() {
testMultiThreading();
}

// Single-process mode doesn't have real workers.
@Test(groups = {"cluster"})
public void testInWorker() {
// Single-process mode doesn't have real workers.
TestUtils.skipTestUnderSingleProcess();
ObjectRef<String> obj = Ray.task(MultiThreadingTest::testMultiThreading).remote();
Assert.assertEquals("ok", obj.get());
}

@Test(groups = {"cluster"})
public void testGetCurrentActorId() {
TestUtils.skipTestUnderSingleProcess();
ActorHandle<ActorIdTester> actorIdTester = Ray.actor(ActorIdTester::new).remote();
ActorId actorId = actorIdTester.task(ActorIdTester::getCurrentActorId).remote().get();
Assert.assertEquals(actorId, actorIdTester.getId());
Expand Down
3 changes: 1 addition & 2 deletions java/test/src/main/java/io/ray/test/PlasmaFreeTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,8 @@ public void testDeleteObjects() {
}
}

@Test
@Test(groups = {"cluster"})
public void testDeleteCreatingTasks() {
TestUtils.skipTestUnderSingleProcess();
ObjectRef<String> helloId = Ray.task(PlasmaFreeTest::hello).remote();
Assert.assertEquals("hello", helloId.get());
Ray.internal().free(ImmutableList.of(helloId.getId()), true, true);
Expand Down
3 changes: 1 addition & 2 deletions java/test/src/main/java/io/ray/test/PlasmaStoreTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -8,9 +8,8 @@

public class PlasmaStoreTest extends BaseTest {

@Test
@Test(groups = {"cluster"})
public void testPutWithDuplicateId() {
TestUtils.skipTestUnderSingleProcess();
ObjectId objectId = ObjectId.fromRandom();
ObjectStore objectStore = TestUtils.getRuntime().getObjectStore();
objectStore.put("1", objectId);
Expand Down
27 changes: 27 additions & 0 deletions java/test/src/main/java/io/ray/test/RayAlterSuiteListener.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
package io.ray.test;

import io.ray.runtime.config.RayConfig;
import io.ray.runtime.config.RunMode;
import java.util.List;
import org.testng.IAlterSuiteListener;
import org.testng.xml.XmlGroups;
import org.testng.xml.XmlRun;
import org.testng.xml.XmlSuite;

public class RayAlterSuiteListener implements IAlterSuiteListener {

@Override
public void alter(List<XmlSuite> suites) {
XmlSuite suite = suites.get(0);
String excludedGroup =
RayConfig.create().runMode == RunMode.SINGLE_PROCESS ? "cluster" : "singleProcess";
XmlGroups groups = new XmlGroups();
XmlRun run = new XmlRun();
run.onExclude(excludedGroup);
if (!"1".equals(System.getenv("ENABLE_MULTI_LANGUAGE_TESTS"))) {
run.onExclude("multiLanguage");
}
groups.setRun(run);
suite.setGroups(groups);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
/**
* Resources Management Test.
*/
@Test(groups = {"cluster"})
public class ResourcesManagementTest extends BaseTest {

@BeforeClass
Expand All @@ -38,10 +39,7 @@ public Integer echo(Integer number) {
}
}

@Test
public void testMethods() {
TestUtils.skipTestUnderSingleProcess();

// This is a case that can satisfy required resources.
// The static resources for test are "CPU:4,RES-A:4".
ObjectRef<Integer> result1 = Ray.task(ResourcesManagementTest::echo, 100)
Expand Down Expand Up @@ -69,10 +67,7 @@ public void testMethods() {
}
}

@Test
public void testActors() {
TestUtils.skipTestUnderSingleProcess();

// This is a case that can satisfy required resources.
// The static resources for test are "CPU:4,RES-A:4".
ActorHandle<Echo> echo1 = Ray.actor(Echo::new)
Expand Down
Loading

0 comments on commit bdc005a

Please sign in to comment.