Skip to content

Commit 759635d

Browse files
authored
[FLINK-37268][example] Use async state in flink-examples-streaming (#26117)
1 parent 064bfe4 commit 759635d

File tree

9 files changed

+158
-95
lines changed

9 files changed

+158
-95
lines changed

flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java

+11-2
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@
3333
import org.apache.flink.core.fs.Path;
3434
import org.apache.flink.streaming.api.datastream.DataStream;
3535
import org.apache.flink.streaming.api.datastream.DataStreamSource;
36+
import org.apache.flink.streaming.api.datastream.KeyedStream;
3637
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
3738
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
3839
import org.apache.flink.streaming.api.functions.ProcessFunction;
@@ -95,6 +96,8 @@ public static void main(String[] args) throws Exception {
9596
text.assignTimestampsAndWatermarks(IngestionTimeWatermarkStrategy.create());
9697
}
9798

99+
final boolean asyncState = params.has("async-state");
100+
98101
SingleOutputStreamOperator<Tuple2<String, Integer>> tokenized =
99102
textWithTimestampAndWatermark.process(new Tokenizer());
100103

@@ -103,9 +106,15 @@ public static void main(String[] args) throws Exception {
103106
.getSideOutput(rejectedWordsTag)
104107
.map(value -> "rejected: " + value, Types.STRING);
105108

109+
KeyedStream<Tuple2<String, Integer>, String> keyedTokenized =
110+
tokenized.keyBy(value -> value.f0);
111+
112+
if (asyncState) {
113+
keyedTokenized.enableAsyncState();
114+
}
115+
106116
DataStream<Tuple2<String, Integer>> counts =
107-
tokenized
108-
.keyBy(value -> value.f0)
117+
keyedTokenized
109118
.window(TumblingEventTimeWindows.of(Duration.ofSeconds(5)))
110119
// group by the tuple field "0" and sum up tuple field "1"
111120
.sum(1);

flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java

+11-3
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import org.apache.flink.api.common.functions.FlatMapFunction;
2222
import org.apache.flink.api.common.typeinfo.Types;
2323
import org.apache.flink.streaming.api.datastream.DataStream;
24+
import org.apache.flink.streaming.api.datastream.KeyedStream;
2425
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
2526
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
2627
import org.apache.flink.util.ParameterTool;
@@ -46,14 +47,16 @@ public static void main(String[] args) throws Exception {
4647
// the host and the port to connect to
4748
final String hostname;
4849
final int port;
50+
final boolean asyncState;
4951
try {
5052
final ParameterTool params = ParameterTool.fromArgs(args);
5153
hostname = params.has("hostname") ? params.get("hostname") : "localhost";
5254
port = params.getInt("port");
55+
asyncState = params.has("async-state");
5356
} catch (Exception e) {
5457
System.err.println(
5558
"No port specified. Please run 'SocketWindowWordCount "
56-
+ "--hostname <hostname> --port <port>', where hostname (localhost by default) "
59+
+ "--hostname <hostname> --port <port> [--asyncState]', where hostname (localhost by default) "
5760
+ "and port is the address of the text server");
5861
System.err.println(
5962
"To start a simple text server, run 'netcat -l <port>' and "
@@ -68,7 +71,7 @@ public static void main(String[] args) throws Exception {
6871
DataStream<String> text = env.socketTextStream(hostname, port, "\n");
6972

7073
// parse the data, group it, window it, and aggregate the counts
71-
DataStream<WordWithCount> windowCounts =
74+
KeyedStream<WordWithCount, String> keyedStream =
7275
text.flatMap(
7376
(FlatMapFunction<String, WordWithCount>)
7477
(value, out) -> {
@@ -77,7 +80,12 @@ public static void main(String[] args) throws Exception {
7780
}
7881
},
7982
Types.POJO(WordWithCount.class))
80-
.keyBy(value -> value.word)
83+
.keyBy(value -> value.word);
84+
if (asyncState) {
85+
keyedStream = keyedStream.enableAsyncState();
86+
}
87+
DataStream<WordWithCount> windowCounts =
88+
keyedStream
8189
.window(TumblingProcessingTimeWindows.of(Duration.ofSeconds(5)))
8290
.reduce((a, b) -> new WordWithCount(a.word, a.count + b.count))
8391
.returns(WordWithCount.class);

flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java

+37-29
Original file line numberDiff line numberDiff line change
@@ -22,10 +22,11 @@
2222
import org.apache.flink.api.common.functions.OpenContext;
2323
import org.apache.flink.api.common.functions.RichFlatMapFunction;
2424
import org.apache.flink.api.common.serialization.SimpleStringEncoder;
25-
import org.apache.flink.api.common.state.ValueState;
26-
import org.apache.flink.api.common.state.ValueStateDescriptor;
25+
import org.apache.flink.api.common.state.v2.ValueState;
26+
import org.apache.flink.api.common.state.v2.ValueStateDescriptor;
2727
import org.apache.flink.api.common.typeinfo.TypeInformation;
2828
import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy;
29+
import org.apache.flink.api.java.typeutils.TypeExtractor;
2930
import org.apache.flink.configuration.CheckpointingOptions;
3031
import org.apache.flink.configuration.Configuration;
3132
import org.apache.flink.configuration.MemorySize;
@@ -71,7 +72,7 @@ public static void main(String[] args) throws Exception {
7172
System.out.println(
7273
"Usage with Kafka: StateMachineExample --kafka-topic <topic> [--brokers <brokers>]");
7374
System.out.println("Options for both the above setups: ");
74-
System.out.println("\t[--backend <hashmap|rocks>]");
75+
System.out.println("\t[--backend <hashmap|rocksdb|forst>]");
7576
System.out.println("\t[--checkpoint-dir <filepath>]");
7677
System.out.println("\t[--incremental-checkpoints <true|false>]");
7778
System.out.println("\t[--output <filepath> OR null for stdout]");
@@ -91,12 +92,10 @@ public static void main(String[] args) throws Exception {
9192
configuration.set(StateBackendOptions.STATE_BACKEND, "hashmap");
9293
configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem");
9394
configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir);
94-
} else if ("rocks".equals(stateBackend)) {
95+
} else if ("rocksdb".equals(stateBackend) || "forst".equals(stateBackend)) {
9596
final String checkpointDir = params.get("checkpoint-dir");
9697
boolean incrementalCheckpoints = params.getBoolean("incremental-checkpoints", false);
97-
configuration.set(
98-
StateBackendOptions.STATE_BACKEND,
99-
"org.apache.flink.state.rocksdb.EmbeddedRocksDBStateBackendFactory");
98+
configuration.set(StateBackendOptions.STATE_BACKEND, stateBackend);
10099
configuration.set(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, incrementalCheckpoints);
101100
configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem");
102101
configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir);
@@ -164,7 +163,7 @@ public static void main(String[] args) throws Exception {
164163
// partition on the address to make sure equal addresses
165164
// end up in the same state machine flatMap function
166165
.keyBy(Event::sourceAddress)
167-
166+
.enableAsyncState()
168167
// the function that evaluates the state machine over the sequence of events
169168
.flatMap(new StateMachineMapper());
170169

@@ -206,32 +205,41 @@ static class StateMachineMapper extends RichFlatMapFunction<Event, Alert> {
206205
public void open(OpenContext openContext) {
207206
// get access to the state object
208207
currentState =
209-
getRuntimeContext().getState(new ValueStateDescriptor<>("state", State.class));
208+
getRuntimeContext()
209+
.getState(
210+
new ValueStateDescriptor<>(
211+
"state", TypeExtractor.createTypeInfo(State.class)));
210212
}
211213

212214
@Override
213215
public void flatMap(Event evt, Collector<Alert> out) throws Exception {
214216
// get the current state for the key (source address)
215-
// if no state exists, yet, the state must be the state machine's initial state
216-
State state = currentState.value();
217-
if (state == null) {
218-
state = State.Initial;
219-
}
220-
221-
// ask the state machine what state we should go to based on the given event
222-
State nextState = state.transition(evt.type());
223-
224-
if (nextState == State.InvalidTransition) {
225-
// the current event resulted in an invalid transition
226-
// raise an alert!
227-
out.collect(new Alert(evt.sourceAddress(), state, evt.type()));
228-
} else if (nextState.isTerminal()) {
229-
// we reached a terminal state, clean up the current state
230-
currentState.clear();
231-
} else {
232-
// remember the new state
233-
currentState.update(nextState);
234-
}
217+
currentState
218+
.asyncValue()
219+
.thenAccept(
220+
state -> {
221+
// if no state exists, yet, the state must be the state machine's
222+
// initial state
223+
if (state == null) {
224+
state = State.Initial;
225+
}
226+
227+
// ask the state machine what state we should go to based on the
228+
// given event
229+
230+
State nextState = state.transition(evt.type());
231+
if (nextState == State.InvalidTransition) {
232+
// the current event resulted in an invalid transition
233+
// raise an alert!
234+
out.collect(new Alert(evt.sourceAddress(), state, evt.type()));
235+
} else if (nextState.isTerminal()) {
236+
// we reached a terminal state, clean up the current state
237+
currentState.asyncClear();
238+
} else {
239+
// remember the new state
240+
currentState.asyncUpdate(nextState);
241+
}
242+
});
235243
}
236244
}
237245

flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java

+9-1
Original file line numberDiff line numberDiff line change
@@ -27,19 +27,23 @@
2727
import org.apache.flink.connector.datagen.source.DataGeneratorSource;
2828
import org.apache.flink.connector.datagen.source.GeneratorFunction;
2929
import org.apache.flink.streaming.api.datastream.DataStream;
30+
import org.apache.flink.streaming.api.datastream.KeyedStream;
3031
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
3132
import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
3233
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
3334
import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
3435
import org.apache.flink.streaming.api.windowing.windows.Window;
3536
import org.apache.flink.util.Collector;
37+
import org.apache.flink.util.ParameterTool;
3638

3739
import java.time.Duration;
3840

3941
/** An example of grouped stream windowing into sliding time windows. */
4042
public class GroupedProcessingTimeWindowExample {
4143

4244
public static void main(String[] args) throws Exception {
45+
final ParameterTool params = ParameterTool.fromArgs(args);
46+
final boolean asyncState = params.has("async-state");
4347

4448
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
4549

@@ -57,8 +61,12 @@ public static void main(String[] args) throws Exception {
5761

5862
DataStream<Tuple2<Long, Long>> stream =
5963
env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator");
64+
KeyedStream<Tuple2<Long, Long>, Long> keyedStream = stream.keyBy(value -> value.f0);
65+
if (asyncState) {
66+
keyedStream = keyedStream.enableAsyncState();
67+
}
6068

61-
stream.keyBy(value -> value.f0)
69+
keyedStream
6270
.window(
6371
SlidingProcessingTimeWindows.of(
6472
Duration.ofMillis(2500), Duration.ofMillis(500)))

flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java

+9-3
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@
2828
import org.apache.flink.connector.file.sink.FileSink;
2929
import org.apache.flink.core.fs.Path;
3030
import org.apache.flink.streaming.api.datastream.DataStream;
31+
import org.apache.flink.streaming.api.datastream.KeyedStream;
3132
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
3233
import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy;
3334
import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows;
@@ -52,6 +53,7 @@ public static void main(String[] args) throws Exception {
5253
env.setParallelism(1);
5354

5455
final boolean fileOutput = params.has("output");
56+
final boolean asyncState = params.has("async-state");
5557

5658
final List<Tuple3<String, Long, Integer>> input = new ArrayList<>();
5759

@@ -81,11 +83,15 @@ public static void main(String[] args) throws Exception {
8183
.withTimestampAssigner((event, timestamp) -> event.f1),
8284
"Generated data source");
8385

86+
KeyedStream<Tuple3<String, Long, Integer>, String> keyedStream =
87+
source.keyBy(value -> value.f0);
88+
if (asyncState) {
89+
keyedStream = keyedStream.enableAsyncState();
90+
}
91+
8492
// We create sessions for each id with max timeout of 3 time units
8593
DataStream<Tuple3<String, Long, Integer>> aggregated =
86-
source.keyBy(value -> value.f0)
87-
.window(EventTimeSessionWindows.withGap(Duration.ofMillis(3L)))
88-
.sum(2);
94+
keyedStream.window(EventTimeSessionWindows.withGap(Duration.ofMillis(3L))).sum(2);
8995

9096
if (fileOutput) {
9197
aggregated

flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java

+8-2
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030
import org.apache.flink.connector.file.src.FileSource;
3131
import org.apache.flink.connector.file.src.reader.TextLineInputFormat;
3232
import org.apache.flink.streaming.api.datastream.DataStream;
33+
import org.apache.flink.streaming.api.datastream.KeyedStream;
3334
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
3435
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
3536
import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy;
@@ -120,13 +121,18 @@ public static void main(String[] args) throws Exception {
120121

121122
int evictionSec = 10;
122123
double triggerMeters = 50;
123-
DataStream<Tuple4<Integer, Integer, Double, Long>> topSpeeds =
124+
KeyedStream<Tuple4<Integer, Integer, Double, Long>, Integer> keyedStream =
124125
carData.assignTimestampsAndWatermarks(
125126
WatermarkStrategy
126127
.<Tuple4<Integer, Integer, Double, Long>>
127128
forMonotonousTimestamps()
128129
.withTimestampAssigner((car, ts) -> car.f3))
129-
.keyBy(value -> value.f0)
130+
.keyBy(value -> value.f0);
131+
if (params.isAsyncState()) {
132+
keyedStream = keyedStream.enableAsyncState();
133+
}
134+
DataStream<Tuple4<Integer, Integer, Double, Long>> topSpeeds =
135+
keyedStream
130136
.window(GlobalWindows.create())
131137
.evictor(TimeEvictor.of(Duration.ofSeconds(evictionSec)))
132138
.trigger(

flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java

+1
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,7 @@
5858
* </code>.
5959
* <li><code>--execution-mode &lt;mode&gt;</code>The execution mode (BATCH, STREAMING, or
6060
* AUTOMATIC) of this pipeline.
61+
* <li><code>--async-state</code> Whether enable async state.
6162
* </ul>
6263
*
6364
* <p>This example shows how to:

0 commit comments

Comments
 (0)