Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BEAM-6263] Fix error-prone test setup for FlinkJobServerDriver #7309

Merged
merged 4 commits into from
Dec 19, 2018
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 @@ -49,8 +49,8 @@ public class FlinkJobServerDriver implements Runnable {
@VisibleForTesting ServerConfiguration configuration;
private final ServerFactory jobServerFactory;
private final ServerFactory artifactServerFactory;
private GrpcFnServer<InMemoryJobService> jobServer;
private GrpcFnServer<BeamFileSystemArtifactStagingService> artifactStagingServer;
private volatile GrpcFnServer<InMemoryJobService> jobServer;
private volatile GrpcFnServer<BeamFileSystemArtifactStagingService> artifactStagingServer;

/** Configuration for the jobServer. */
public static class ServerConfiguration {
Expand Down Expand Up @@ -184,12 +184,16 @@ public void run() {
}
}

// This method is executed by TestPortableRunner via Reflection
public String start() throws IOException {
jobServer = createJobServer();
return jobServer.getApiServiceDescriptor().getUrl();
}

public void stop() {
// This method is executed by TestPortableRunner via Reflection
// Needs to be synchronized to prevent concurrency issues in testing shutdown
@SuppressWarnings("WeakerAccess")
public synchronized void stop() {
if (jobServer != null) {
try {
jobServer.close();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,7 @@

import com.google.common.base.Charsets;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.PrintStream;
import java.net.ServerSocket;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -84,33 +82,30 @@ public void testConfigurationFromConfig() {
public void testJobServerDriver() throws Exception {
FlinkJobServerDriver driver = null;
Thread driverThread = null;
final PrintStream oldOut = System.out;
final PrintStream oldOut = System.err;
ByteArrayOutputStream baos = new ByteArrayOutputStream();
PrintStream newOut = new PrintStream(baos);
try {
System.setErr(newOut);
int freePort = getFreePort();
int freePort2 = getFreePort();
driver =
FlinkJobServerDriver.fromParams(
new String[] {
"--job-port", String.valueOf(freePort),
"--artifact-port", String.valueOf(freePort2)
});
driver = FlinkJobServerDriver.fromParams(new String[] {"--job-port=0", "--artifact-port=0"});
driverThread = new Thread(driver);
driverThread.start();
boolean success = false;
while (!success) {
newOut.flush();
String output = baos.toString(Charsets.UTF_8.name());
if (output.contains("JobService started on localhost:" + freePort)
&& output.contains("ArtifactStagingService started on localhost:" + freePort2)) {
if (output.contains("JobService started on localhost:")
&& output.contains("ArtifactStagingService started on localhost:")) {
success = true;
} else {
Thread.sleep(100);
}
}
assertThat(driverThread.isAlive(), is(true));
} catch (Throwable t) {
// restore to print exception
System.setErr(oldOut);
throw t;
} finally {
System.setErr(oldOut);
if (driver != null) {
Expand All @@ -122,10 +117,4 @@ public void testJobServerDriver() throws Exception {
}
}
}

private static int getFreePort() throws IOException {
try (ServerSocket socket = new ServerSocket(0)) {
return socket.getLocalPort();
}
}
}