Skip to content

Commit

Permalink
[FLINK-8994] [tests] Let general purpose DataStream job include Avro …
Browse files Browse the repository at this point in the history
…as state

This closes apache#6435.
  • Loading branch information
tzulitai committed Jul 30, 2018
1 parent 0d6040b commit 6ac0145
Show file tree
Hide file tree
Showing 5 changed files with 126 additions and 8 deletions.
1 change: 1 addition & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ tmp
*.log
.DS_Store
build-target
flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/avro/
flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/generated/
flink-runtime-web/web-dashboard/assets/fonts/
flink-runtime-web/web-dashboard/node_modules/
Expand Down
34 changes: 34 additions & 0 deletions flink-end-to-end-tests/flink-datastream-allround-test/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,40 @@ under the License.
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>${avro.version}</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
<fieldVisibility>PRIVATE</fieldVisibility>
<includes>
<include>**/*.avsc</include>
</includes>
<!--
This forces Avro to use Java Strings instead of Avro's Utf8.
This is required since the job relies on equals checks on some String fields
to verify that state restore was successful.
-->
<stringType>String</stringType>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-checkstyle-plugin</artifactId>
<configuration>
<excludes>**/org/apache/flink/streaming/tests/avro/*</excludes>
</configuration>
</plugin>
</plugins>
</build>

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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.
*/

{"namespace": "org.apache.flink.streaming.tests.avro",
"type": "record",
"name": "ComplexPayloadAvro",
"fields": [
{
"name": "eventTime",
"type": "long",
"default": ""
},
{
"name": "stringList",
"type": {
"type": "array",
"items": {
"type": "string"
}
}
},
{
"name": "strPayload",
"type": "string",
"default": ""
},
{
"name": "innerPayLoad",
"type": {
"name": "InnerPayLoadAvro",
"type": "record",
"fields": [
{"name": "sequenceNumber", "type": "long"}
]
}
}
]
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,18 @@
import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.formats.avro.typeutils.AvroSerializer;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.tests.artificialstate.ComplexPayload;
import org.apache.flink.streaming.tests.avro.ComplexPayloadAvro;
import org.apache.flink.streaming.tests.avro.InnerPayLoadAvro;
import org.apache.flink.util.Collector;

import java.util.Arrays;
import java.util.Collections;

import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.applyTumblingWindows;
Expand Down Expand Up @@ -72,26 +76,53 @@ public static void main(String[] args) throws Exception {

setupEnvironment(env, pt);

// add a keyed stateful map operator, which uses Kryo for state serialization
DataStream<Event> eventStream = env.addSource(createEventSource(pt))
.assignTimestampsAndWatermarks(createTimestampExtractor(pt))
.keyBy(Event::getKey)
.map(createArtificialKeyedStateMapper(
// map function simply forwards the inputs
(MapFunction<Event, Event>) in -> in,
// state is verified and updated per event as a wrapped ComplexPayload state object
(Event first, ComplexPayload second) -> {
if (second != null && !second.getStrPayload().equals(KEYED_STATE_OPER_NAME)) {
(Event event, ComplexPayload lastState) -> {
if (lastState != null && !lastState.getStrPayload().equals(KEYED_STATE_OPER_NAME)
&& lastState.getInnerPayLoad().getSequenceNumber() == (event.getSequenceNumber() - 1)) {
System.out.println("State is set or restored incorrectly");
}
return new ComplexPayload(first, KEYED_STATE_OPER_NAME);
return new ComplexPayload(event, KEYED_STATE_OPER_NAME);
},
Collections.singletonList(
new KryoSerializer<>(ComplexPayload.class, env.getConfig())), // custom KryoSerializer
Collections.singletonList(ComplexPayload.class) // KryoSerializer via type extraction
)
)
.name(KEYED_STATE_OPER_NAME)
.returns(Event.class);
).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Kryo");

// add a keyed stateful map operator, which uses Avro for state serialization
eventStream = eventStream
.keyBy(Event::getKey)
.map(createArtificialKeyedStateMapper(
// map function simply forwards the inputs
(MapFunction<Event, Event>) in -> in,
// state is verified and updated per event as a wrapped ComplexPayloadAvro state object
(Event event, ComplexPayloadAvro lastState) -> {
if (lastState != null && !lastState.getStrPayload().equals(KEYED_STATE_OPER_NAME)
&& lastState.getInnerPayLoad().getSequenceNumber() == (event.getSequenceNumber() - 1)) {
System.out.println("State is set or restored incorrectly");
}

ComplexPayloadAvro payload = new ComplexPayloadAvro();
payload.setEventTime(event.getEventTime());
payload.setInnerPayLoad(new InnerPayLoadAvro(event.getSequenceNumber()));
payload.setStrPayload(KEYED_STATE_OPER_NAME);
payload.setStringList(Arrays.asList(String.valueOf(event.getKey()), event.getPayload()));

return payload;
},
Collections.singletonList(
new AvroSerializer<>(ComplexPayloadAvro.class)), // custom AvroSerializer
Collections.singletonList(ComplexPayloadAvro.class) // AvroSerializer via type extraction
)
).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Avro");

DataStream<Event> eventStream2 = eventStream
.map(createArtificialOperatorStateMapper((MapFunction<Event, Event>) in -> in))
Expand Down
4 changes: 2 additions & 2 deletions flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ DATASTREAM_JOB=$($FLINK_DIR/bin/flink run -d -p $ORIGINAL_DOP $TEST_PROGRAM_JAR

wait_job_running $DATASTREAM_JOB

wait_oper_metric_num_in_records ArtificalKeyedStateMapper.0 200
wait_oper_metric_num_in_records SemanticsCheckMapper.0 200

# take a savepoint of the state machine job
SAVEPOINT_PATH=$(take_savepoint $DATASTREAM_JOB $TEST_DATA_DIR \
Expand All @@ -120,7 +120,7 @@ DATASTREAM_JOB=$($FLINK_DIR/bin/flink run -s $SAVEPOINT_PATH -p $NEW_DOP -d $TES

wait_job_running $DATASTREAM_JOB

wait_oper_metric_num_in_records ArtificalKeyedStateMapper.0 200
wait_oper_metric_num_in_records SemanticsCheckMapper.0 200

# if state is errorneous and the state machine job produces alerting state transitions,
# output would be non-empty and the test will not pass

0 comments on commit 6ac0145

Please sign in to comment.