Skip to content

Commit

Permalink
(TWILL-83) Expose testing utility to help writing twill tests
Browse files Browse the repository at this point in the history
- Replaced YarnTestUtils with TwillTester to provide better support for writing test
  - Use junit ExternalResource
- Refactor existing twill tests to use TwillTester

This closes apache#33 on GitHub.

Signed-off-by: Terence Yim <[email protected]>
  • Loading branch information
chtyim committed May 22, 2015
1 parent c4463ee commit 6896257
Show file tree
Hide file tree
Showing 18 changed files with 191 additions and 145 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import org.apache.twill.api.TwillRunner;
import org.apache.twill.api.logging.PrinterLogHandler;
import org.apache.twill.yarn.BaseYarnTest;
import org.apache.twill.yarn.YarnTestUtils;
import org.junit.Assert;
import org.junit.Test;
import org.slf4j.Logger;
Expand All @@ -43,7 +42,7 @@ public class Java8Test extends BaseYarnTest {

@Test
public void test() throws ExecutionException, InterruptedException, TimeoutException {
TwillRunner runner = YarnTestUtils.getTwillRunner();
TwillRunner runner = getTwillRunner();

// Start the TestRunnable and make sure it is executed with the log message emitted.
CountDownLatch logLatch = new CountDownLatch(1);
Expand Down
66 changes: 59 additions & 7 deletions twill-yarn/src/test/java/org/apache/twill/yarn/BaseYarnTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,19 @@
*/
package org.apache.twill.yarn;

import com.google.common.collect.Iterables;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.twill.api.TwillController;
import org.apache.twill.api.TwillRunner;
import org.junit.After;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.rules.TemporaryFolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;

/**
* Base class for all YARN tests.
Expand All @@ -38,15 +41,33 @@ public abstract class BaseYarnTest {
@ClassRule
public static TemporaryFolder tmpFolder = new TemporaryFolder();

@BeforeClass
public static void init() throws IOException {
YarnTestUtils.initOnce();
}
/**
* A singleton wrapper so that yarn cluster only bring up once across all tests in the YarnTestSuite.
*/
@ClassRule
public static final TwillTester TWILL_TESTER = new TwillTester() {
private final AtomicInteger instances = new AtomicInteger();

@Override
protected void before() throws Throwable {
if (instances.getAndIncrement() == 0) {
super.before();
}
}

@Override
protected void after() {
if (instances.decrementAndGet() == 0) {
super.after();
}
}
};


@After
public final void cleanupTest() {
// Make sure all applications are stopped after a test case is executed, even it failed.
TwillRunner twillRunner = YarnTestUtils.getTwillRunner();
TwillRunner twillRunner = TWILL_TESTER.getTwillRunner();
for (TwillRunner.LiveInfo liveInfo : twillRunner.lookupLive()) {
for (TwillController controller : liveInfo.getControllers()) {
try {
Expand All @@ -57,4 +78,35 @@ public final void cleanupTest() {
}
}
}

/**
* Poll the given {@link Iterable} until its size is the same as the given count,
* with a limited amount of polls. There is one second sleep between each poll.
*
* @param iterable the Iterable to poll
* @param count the expected size
* @param limit number of times to poll.
* @param <T> type of the element inside the Iterable
* @return true if the Iterable size is the same as the given count
*/
public <T> boolean waitForSize(Iterable<T> iterable, int count, int limit) throws InterruptedException {
int trial = 0;
int size = Iterables.size(iterable);
while (size != count && trial < limit) {
LOG.info("Waiting for {} size {} == {}", iterable, size, count);
TimeUnit.SECONDS.sleep(1);
trial++;
size = Iterables.size(iterable);
}
return trial < limit;
}

@SuppressWarnings("unchecked")
public <T extends TwillRunner> T getTwillRunner() {
return (T) TWILL_TESTER.getTwillRunner();
}

public List<NodeReport> getNodeReports() throws Exception {
return TWILL_TESTER.getNodeReports();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,14 +43,14 @@ public class ContainerSizeTestRun extends BaseYarnTest {

@Test
public void testContainerSize() throws InterruptedException, TimeoutException, ExecutionException {
TwillRunner runner = YarnTestUtils.getTwillRunner();
TwillRunner runner = getTwillRunner();
TwillController controller = runner.prepare(new SleepApp())
.addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
.start();

try {
ServiceDiscovered discovered = controller.discoverService("sleep");
Assert.assertTrue(YarnTestUtils.waitForSize(discovered, 2, 120));
Assert.assertTrue(waitForSize(discovered, 2, 120));
} finally {
controller.terminate().get(120, TimeUnit.SECONDS);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ private boolean waitForDebugPort(TwillController controller, String runnable, in

@Test
public void testDebugPortOneRunnable() throws Exception {
YarnTwillRunnerService runner = (YarnTwillRunnerService) YarnTestUtils.getTwillRunner();
YarnTwillRunnerService runner = getTwillRunner();
runner.start();

TwillController controller = runner.prepare(new DummyApplication())
Expand All @@ -126,7 +126,7 @@ public void run() {

@Test
public void testDebugPortAllRunnables() throws Exception {
YarnTwillRunnerService runner = (YarnTwillRunnerService) YarnTestUtils.getTwillRunner();
YarnTwillRunnerService runner = getTwillRunner();
runner.start();

TwillController controller = runner.prepare(new DummyApplication())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ public final class DistributeShellTestRun extends BaseYarnTest {
@Ignore
@Test
public void testDistributedShell() throws InterruptedException {
TwillRunner twillRunner = YarnTestUtils.getTwillRunner();
TwillRunner twillRunner = getTwillRunner();

TwillController controller = twillRunner.prepare(new DistributedShell("pwd", "ls -al"))
.addLogHandler(new PrinterLogHandler(new PrintWriter(System.out)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@

/**
* Using echo server to test various behavior of YarnTwillService.
* This test is executed by {@link YarnTestUtils}.
*/
public final class EchoServerTestRun extends BaseYarnTest {

Expand All @@ -53,7 +52,7 @@ public final class EchoServerTestRun extends BaseYarnTest {
@Test
public void testEchoServer() throws InterruptedException, ExecutionException, IOException,
URISyntaxException, TimeoutException {
TwillRunner runner = YarnTestUtils.getTwillRunner();
TwillRunner runner = getTwillRunner();

TwillController controller = runner.prepare(new EchoServer(),
ResourceSpecification.Builder.with()
Expand All @@ -77,7 +76,7 @@ public void run() {
Assert.assertTrue(running.await(120, TimeUnit.SECONDS));

Iterable<Discoverable> echoServices = controller.discoverService("echo");
Assert.assertTrue(YarnTestUtils.waitForSize(echoServices, 2, 120));
Assert.assertTrue(waitForSize(echoServices, 2, 120));

for (Discoverable discoverable : echoServices) {
String msg = "Hello: " + discoverable.getSocketAddress();
Expand All @@ -97,36 +96,36 @@ public void run() {

// Increase number of instances
controller.changeInstances("EchoServer", 3);
Assert.assertTrue(YarnTestUtils.waitForSize(echoServices, 3, 120));
Assert.assertTrue(waitForSize(echoServices, 3, 120));

echoServices = controller.discoverService("echo2");

// Decrease number of instances
controller.changeInstances("EchoServer", 1);
Assert.assertTrue(YarnTestUtils.waitForSize(echoServices, 1, 120));
Assert.assertTrue(waitForSize(echoServices, 1, 120));

// Increase number of instances again
controller.changeInstances("EchoServer", 2);
Assert.assertTrue(YarnTestUtils.waitForSize(echoServices, 2, 120));
Assert.assertTrue(waitForSize(echoServices, 2, 120));

// Make sure still only one app is running
Iterable<TwillRunner.LiveInfo> apps = runner.lookupLive();
Assert.assertTrue(YarnTestUtils.waitForSize(apps, 1, 120));
Assert.assertTrue(waitForSize(apps, 1, 120));

// Creates a new runner service to check it can regain control over running app.
TwillRunnerService runnerService = YarnTestUtils.createTwillRunnerService();
TwillRunnerService runnerService = TWILL_TESTER.createTwillRunnerService();
runnerService.start();

try {
Iterable <TwillController> controllers = runnerService.lookup("EchoServer");
Assert.assertTrue(YarnTestUtils.waitForSize(controllers, 1, 120));
Assert.assertTrue(waitForSize(controllers, 1, 120));

for (TwillController c : controllers) {
LOG.info("Stopping application: " + c.getRunId());
c.terminate().get(30, TimeUnit.SECONDS);
}

Assert.assertTrue(YarnTestUtils.waitForSize(apps, 0, 120));
Assert.assertTrue(waitForSize(apps, 0, 120));
} finally {
runnerService.stop();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public final class FailureRestartTestRun extends BaseYarnTest {

@Test
public void testFailureRestart() throws Exception {
TwillRunner runner = YarnTestUtils.getTwillRunner();
TwillRunner runner = getTwillRunner();

ResourceSpecification resource = ResourceSpecification.Builder.with()
.setVirtualCores(1)
Expand All @@ -60,7 +60,7 @@ public void testFailureRestart() throws Exception {
.start();

Iterable<Discoverable> discoverables = controller.discoverService("failure");
Assert.assertTrue(YarnTestUtils.waitForSize(discoverables, 2, 120));
Assert.assertTrue(waitForSize(discoverables, 2, 120));

// Make sure we see the right instance IDs
Assert.assertEquals(Sets.newHashSet(0, 1), getInstances(discoverables));
Expand All @@ -69,10 +69,10 @@ public void testFailureRestart() throws Exception {
controller.sendCommand(FailureRunnable.class.getSimpleName(), Command.Builder.of("kill0").build());

// Make sure the runnable is killed.
Assert.assertTrue(YarnTestUtils.waitForSize(discoverables, 1, 120));
Assert.assertTrue(waitForSize(discoverables, 1, 120));

// Wait for the restart
Assert.assertTrue(YarnTestUtils.waitForSize(discoverables, 2, 120));
Assert.assertTrue(waitForSize(discoverables, 2, 120));

// Make sure we see the right instance IDs
Assert.assertEquals(Sets.newHashSet(0, 1), getInstances(discoverables));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ public class InitializeFailTestRun extends BaseYarnTest {

@Test
public void testInitFail() throws InterruptedException, ExecutionException, TimeoutException {
TwillRunner runner = YarnTestUtils.getTwillRunner();
TwillRunner runner = getTwillRunner();
final CountDownLatch logLatch = new CountDownLatch(1);

// Verify that it receives the exception log entry that thrown when runnable initialize
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public void testLocalFile() throws Exception {
String header = Files.readFirstLine(new File(getClass().getClassLoader().getResource("header.txt").toURI()),
Charsets.UTF_8);

TwillRunner runner = YarnTestUtils.getTwillRunner();
TwillRunner runner = getTwillRunner();

TwillController controller = runner.prepare(new LocalFileApplication())
.addJVMOptions(" -verbose:gc -Xloggc:gc.log -XX:+PrintGCDetails")
Expand All @@ -66,7 +66,7 @@ public void testLocalFile() throws Exception {
.start();

Iterable<Discoverable> discoverables = controller.discoverService("local");
Assert.assertTrue(YarnTestUtils.waitForSize(discoverables, 1, 60));
Assert.assertTrue(waitForSize(discoverables, 1, 60));

InetSocketAddress socketAddress = discoverables.iterator().next().getSocketAddress();
Socket socket = new Socket(socketAddress.getAddress(), socketAddress.getPort());
Expand All @@ -84,7 +84,7 @@ public void testLocalFile() throws Exception {

controller.terminate().get(120, TimeUnit.SECONDS);

Assert.assertTrue(YarnTestUtils.waitForSize(discoverables, 0, 60));
Assert.assertTrue(waitForSize(discoverables, 0, 60));

TimeUnit.SECONDS.sleep(2);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ public void onLog(LogEntry logEntry) {
}
};

TwillRunner runner = YarnTestUtils.getTwillRunner();
TwillRunner runner = getTwillRunner();
TwillController controller = runner.prepare(new LogRunnable())
.addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
.addLogHandler(logHandler)
Expand Down
Loading

0 comments on commit 6896257

Please sign in to comment.