Skip to content

Commit

Permalink
[FLINK-3745] [runtime] Fix early stopping of stream sources
Browse files Browse the repository at this point in the history
  • Loading branch information
StephanEwen committed Apr 13, 2016
1 parent 2728f92 commit 8570b6d
Show file tree
Hide file tree
Showing 5 changed files with 123 additions and 45 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,9 +31,20 @@
public class StoppableSourceStreamTask<OUT, SRC extends SourceFunction<OUT> & StoppableFunction>
extends SourceStreamTask<OUT, SRC, StoppableStreamSource<OUT, SRC>> implements StoppableTask {

private volatile boolean stopped;

@Override
public void stop() {
this.headOperator.stop();
protected void run() throws Exception {
if (!stopped) {
super.run();
}
}

@Override
public void stop() {
stopped = true;
if (this.headOperator != null) {
this.headOperator.stop();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
/**
* 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.runtime.tasks;

import org.apache.flink.api.common.functions.StoppableFunction;
import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
import org.apache.flink.streaming.api.operators.StoppableStreamSource;

import org.junit.Test;

import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;

/**
* These tests verify that the RichFunction methods are called (in correct order). And that
* checkpointing/element emission don't occur concurrently.
*/
public class SourceStreamTaskStoppingTest {


// test flag for testStop()
static boolean stopped = false;

@Test
public void testStop() {
final StoppableSourceStreamTask<Object, StoppableSource> sourceTask = new StoppableSourceStreamTask<>();
sourceTask.headOperator = new StoppableStreamSource<>(new StoppableSource());

sourceTask.stop();

assertTrue(stopped);
}

@Test
public void testStopBeforeInitialization() throws Exception {

final StoppableSourceStreamTask<Object, StoppableFailingSource> sourceTask = new StoppableSourceStreamTask<>();
sourceTask.stop();

sourceTask.headOperator = new StoppableStreamSource<>(new StoppableFailingSource());
sourceTask.run();
}

// ------------------------------------------------------------------------

private static class StoppableSource extends RichSourceFunction<Object> implements StoppableFunction {
private static final long serialVersionUID = 728864804042338806L;

@Override
public void run(org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext<Object> ctx)
throws Exception {
}

@Override
public void cancel() {}

@Override
public void stop() {
stopped = true;
}
}

private static class StoppableFailingSource extends RichSourceFunction<Object> implements StoppableFunction {
private static final long serialVersionUID = 728864804042338806L;

@Override
public void run(SourceContext<Object> ctx) throws Exception {
fail("should not be called");
}

@Override
public void cancel() {}

@Override
public void stop() {}
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package org.apache.flink.streaming.runtime.tasks;

import org.apache.flink.api.common.functions.StoppableFunction;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
Expand All @@ -28,12 +27,13 @@
import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.StoppableStreamSource;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.apache.flink.streaming.util.TestHarnessUtil;

import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;

import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;

Expand Down Expand Up @@ -77,19 +77,6 @@ public void testOpenClose() throws Exception {
Assert.assertEquals(10, resultElements.size());
}

// test flag for testStop()
static boolean stopped = false;

@Test
public void testStop() {
final StoppableSourceStreamTask<Object, StoppableSource> sourceTask = new StoppableSourceStreamTask<>();
sourceTask.headOperator = new StoppableStreamSource<>(new StoppableSource());

sourceTask.stop();

Assert.assertTrue(stopped);
}

/**
* This test ensures that the SourceStreamTask properly serializes checkpointing
* and element emission. This also verifies that there are no concurrent invocations
Expand Down Expand Up @@ -155,24 +142,7 @@ public void testCheckpointing() throws Exception {
}
}

private static class StoppableSource extends RichSourceFunction<Object> implements StoppableFunction {
private static final long serialVersionUID = 728864804042338806L;

@Override
public void run(org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext<Object> ctx)
throws Exception {
}

@Override
public void cancel() {}

@Override
public void stop() {
stopped = true;
}
}

private static class MockSource implements SourceFunction<Tuple2<Long, Integer>>, Checkpointed {
private static class MockSource implements SourceFunction<Tuple2<Long, Integer>>, Checkpointed<Serializable> {
private static final long serialVersionUID = 1;

private int maxElements;
Expand Down Expand Up @@ -240,9 +210,7 @@ public Serializable snapshotState(long checkpointId, long checkpointTimestamp) t
}

@Override
public void restoreState(Serializable state) {

}
public void restoreState(Serializable state) {}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,7 @@ public void run() {
// try until we get the running jobs
List<JobID> running;
while ((running = cluster.getCurrentlyRunningJobsJava()).isEmpty()) {
Thread.sleep(100);
Thread.sleep(50);
}

JobID id = running.get(0);
Expand All @@ -223,22 +223,26 @@ public void run() {
env.execute();

// verify that all the watermarks arrived at the final custom operator
for (int i = 0; i < PARALLELISM; i++) {
for (List<Watermark> subtaskWatermarks : CustomOperator.finalWatermarks) {

// we are only guaranteed to see NUM_WATERMARKS / 2 watermarks because the
// other source stops emitting after that
for (int j = 0; j < NUM_WATERMARKS / 2; j++) {
if (!CustomOperator.finalWatermarks[i].get(j).equals(new Watermark(initialTime + j))) {
for (int j = 0; j < subtaskWatermarks.size(); j++) {
if (subtaskWatermarks.get(j).getTimestamp() != initialTime + j) {
System.err.println("All Watermarks: ");
for (int k = 0; k <= NUM_WATERMARKS / 2; k++) {
System.err.println(CustomOperator.finalWatermarks[i].get(k));
System.err.println(subtaskWatermarks.get(k));
}

fail("Wrong watermark.");
}
}

assertNotEquals(Watermark.MAX_WATERMARK,
CustomOperator.finalWatermarks[i].get(CustomOperator.finalWatermarks[i].size()-1));
// if there are watermarks, the final one must not be the MAX watermark
if (subtaskWatermarks.size() > 0) {
assertNotEquals(Watermark.MAX_WATERMARK,
subtaskWatermarks.get(subtaskWatermarks.size()-1));
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ public static void main(String[] args) throws Exception {
int port = Integer.parseInt(args[2]);

ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
env.getConfig().disableSysoutLogging();

DataSet<Integer> input = env.fromElements(1,2,3,4,5);

Expand Down

0 comments on commit 8570b6d

Please sign in to comment.