Skip to content

Commit

Permalink
Fixed JobManagerITCase to properly wait for task managers to deregist…
Browse files Browse the repository at this point in the history
…er their tasks. Replaced the scheduler's execution service with akka's futures. Introduced TestStreamEnvironment to use ForkableFlinkMiniCluster for test execution.
  • Loading branch information
tillrohrmann committed Dec 18, 2014
1 parent bd4ee47 commit c175ebe
Show file tree
Hide file tree
Showing 53 changed files with 878 additions and 562 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
import org.apache.flink.test.util.ForkableFlinkMiniCluster;
import org.junit.Assert;
import org.junit.Test;

Expand All @@ -39,12 +39,12 @@ public class AvroExternalJarProgramITCase {
@Test
public void testExternalProgram() {

LocalFlinkMiniCluster testMiniCluster = null;
ForkableFlinkMiniCluster testMiniCluster = null;

try {
Configuration config = new Configuration();
config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
testMiniCluster = new LocalFlinkMiniCluster(config);
testMiniCluster = new ForkableFlinkMiniCluster(config);

String jarFile = JAR_FILE;
String testData = getClass().getResource(TEST_DATA_FILE).toString();
Expand Down
7 changes: 7 additions & 0 deletions flink-addons/flink-streaming/flink-streaming-core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,13 @@ under the License.
<artifactId>commons-math</artifactId>
<version>2.2</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,4 @@ public void execute(String jobName) throws Exception {
ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(jobName),
getDegreeOfParallelism());
}

public void executeTest(long memorySize) throws Exception {
ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(), getDegreeOfParallelism(),
memorySize);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ public ExecutionConfig getConfig() {
/**
* Gets the degree of parallelism with which operation are executed by
* default. Operations can individually override this value to use a
* specific degree of parallelism via {@link DataStream#setParallelism}.
* specific degree of parallelism.
*
* @return The degree of parallelism used by operations, unless they
* override that value.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,5 +81,4 @@ public static void runOnMiniCluster(JobGraph jobGraph, int degreeOfParallelism,
public static void runOnMiniCluster(JobGraph jobGraph, int numOfSlots) throws Exception {
runOnMiniCluster(jobGraph, numOfSlots, -1);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.TestStreamEnvironment;
import org.apache.flink.util.Collector;
import org.junit.Test;

Expand Down Expand Up @@ -73,7 +74,7 @@ public void invoke(Boolean tuple) {

@Test
public void test() throws Exception {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE);

env.setBufferTimeout(10);

Expand All @@ -86,7 +87,7 @@ public void test() throws Exception {

iteration.closeWith(increment).addSink(new MySink());

env.executeTest(MEMORYSIZE);
env.execute();

assertTrue(iterated);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@

import org.apache.flink.api.common.functions.FilterFunction;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.TestStreamEnvironment;
import org.junit.Test;

public class PrintTest implements Serializable {
Expand Down Expand Up @@ -50,9 +50,8 @@ public boolean filter(Long value) throws Exception {

@Test
public void test() throws Exception {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE);
env.generateSequence(1, 10).map(new IdentityMap()).filter(new FilterAll()).print();
env.executeTest(MEMORYSIZE);

env.execute();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,10 @@
import org.apache.flink.api.common.functions.CrossFunction;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.invokable.util.TimeStamp;
import org.apache.flink.streaming.util.TestStreamEnvironment;
import org.junit.Test;

public class WindowCrossJoinTest implements Serializable {
Expand All @@ -45,7 +45,7 @@ public class WindowCrossJoinTest implements Serializable {

@Test
public void test() throws Exception {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE);
env.setBufferTimeout(1);

ArrayList<Tuple2<Integer, String>> in1 = new ArrayList<Tuple2<Integer, String>>();
Expand Down Expand Up @@ -111,7 +111,7 @@ public Tuple2<Tuple2<Integer, String>, Integer> cross(
})
.addSink(new CrossResultSink());

env.executeTest(MEMORYSIZE);
env.execute();

assertEquals(joinExpectedResults, joinResults);
assertEquals(crossExpectedResults, crossResults);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,9 @@

import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.TestStreamEnvironment;
import org.apache.flink.util.Collector;
import org.junit.BeforeClass;
import org.junit.Test;
Expand Down Expand Up @@ -132,7 +132,7 @@ public void run() {

@Test
public void test() throws Exception {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE);

@SuppressWarnings("unused")
DataStream<Tuple1<Integer>> dataStream1 = env.addSource(new MySource1()).writeAsCsv(PREFIX + "test1.txt");
Expand All @@ -159,7 +159,7 @@ public void test() throws Exception {

fillExpected5();

env.executeTest(MEMORYSIZE);
env.execute();

readFile(PREFIX + "test1.txt", result1);
readFile(PREFIX + "test2.txt", result2);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,9 @@

import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.TestStreamEnvironment;
import org.apache.flink.util.Collector;
import org.junit.BeforeClass;
import org.junit.Test;
Expand Down Expand Up @@ -132,7 +132,7 @@ public void run() {

@Test
public void test() throws Exception {
LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE);

@SuppressWarnings("unused")
DataStream<Tuple1<Integer>> dataStream1 = env.addSource(new MySource1()).writeAsText(PREFIX + "test1.txt");
Expand All @@ -159,7 +159,7 @@ public void test() throws Exception {

fillExpected5();

env.executeTest(MEMORYSIZE);
env.execute();

readFile(PREFIX + "test1.txt", result1);
readFile(PREFIX + "test2.txt", result2);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.util.TestStreamEnvironment;
import org.junit.Test;

public class DirectedOutputTest {
Expand Down Expand Up @@ -104,16 +105,15 @@ private void readObject(java.io.ObjectInputStream in) throws IOException,

@Test
public void outputSelectorTest() throws Exception {

LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
StreamExecutionEnvironment env = new TestStreamEnvironment(1, 128);

SplitDataStream<Long> source = env.generateSequence(1, 11).split(new MyOutputSelector());
source.select(EVEN).addSink(new ListSink(EVEN));
source.select(ODD, TEN).addSink(new ListSink(ODD_AND_TEN));
source.select(EVEN, ODD).addSink(new ListSink(EVEN_AND_ODD));
source.selectAll().addSink(new ListSink(ALL));

env.executeTest(128);
env.execute();
assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), outputs.get(EVEN));
assertEquals(Arrays.asList(1L, 3L, 5L, 7L, 9L, 10L, 11L), outputs.get(ODD_AND_TEN));
assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.function.sink.SinkFunction;
import org.apache.flink.streaming.api.function.source.SourceFunction;
import org.apache.flink.streaming.util.TestStreamEnvironment;
import org.apache.flink.util.Collector;
import org.junit.Test;

Expand Down Expand Up @@ -145,16 +146,15 @@ public void invoke(String value) {

@Test
public void coTest() throws Exception {
LocalStreamEnvironment env = StreamExecutionEnvironment
.createLocalEnvironment(SOURCE_PARALELISM);
StreamExecutionEnvironment env = new TestStreamEnvironment(SOURCE_PARALELISM, MEMORYSIZE);

DataStream<String> fromStringElements = env.fromElements("aa", "bb", "cc");
DataStream<Long> generatedSequence = env.generateSequence(0, 3);

fromStringElements.connect(generatedSequence).map(new CoMap()).addSink(new SetSink());

resultSet = new HashSet<String>();
env.executeTest(MEMORYSIZE);
env.execute();

HashSet<String> expectedSet = new HashSet<String>(Arrays.asList("aa", "bb", "cc", "0", "1",
"2", "3"));
Expand All @@ -163,12 +163,11 @@ public void coTest() throws Exception {

@Test
public void runStream() throws Exception {
LocalStreamEnvironment env = StreamExecutionEnvironment
.createLocalEnvironment(SOURCE_PARALELISM);
StreamExecutionEnvironment env = new TestStreamEnvironment(SOURCE_PARALELISM, MEMORYSIZE);

env.addSource(new MySource()).setParallelism(SOURCE_PARALELISM).map(new MyTask()).addSink(new MySink());

env.executeTest(MEMORYSIZE);
env.execute();
assertEquals(10, data.keySet().size());

for (Integer k : data.keySet()) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.streaming.util;

import akka.actor.ActorRef;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.client.JobClient;
import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.test.util.ForkableFlinkMiniCluster;

public class TestStreamEnvironment extends StreamExecutionEnvironment {
private static final String DEFAULT_JOBNAME = "TestStreamingJob";
private static final String CANNOT_EXECUTE_EMPTY_JOB = "Cannot execute empty job";

private long memorySize;

public TestStreamEnvironment(int degreeOfParallelism, long memorySize){
this.setDegreeOfParallelism(degreeOfParallelism);

this.memorySize = memorySize;
}

@Override
public void execute() throws Exception {
execute(DEFAULT_JOBNAME);
}

@Override
public void execute(String jobName) throws Exception {
JobGraph jobGraph = jobGraphBuilder.getJobGraph(jobName);

Configuration configuration = jobGraph.getJobConfiguration();

configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
getDegreeOfParallelism());
configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize);

ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration);

try{
ActorRef client = cluster.getJobClient();
JobClient.submitJobAndWait(jobGraph, false, client);
}catch(JobExecutionException e){
if(e.getMessage().contains("GraphConversionException")){
throw new Exception(CANNOT_EXECUTE_EMPTY_JOB, e);
}else{
throw e;
}
}finally{
cluster.stop();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ public class Client {

private final PactCompiler compiler; // the compiler to compile the jobs

private boolean printStatusDuringExecution;
private boolean printStatusDuringExecution = false;

// ------------------------------------------------------------------------
// Construction
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -598,7 +598,7 @@ public final class ConfigConstants {

public static String DEFAULT_AKKA_FRAMESIZE = "10485760b";

public static String DEFAULT_AKKA_LOG_LEVEL = "OFF";
public static String DEFAULT_AKKA_LOG_LEVEL = "ERROR";

public static int DEFAULT_AKKA_ASK_TIMEOUT = 100;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
Expand Down
Loading

0 comments on commit c175ebe

Please sign in to comment.