From c9d2ecf13c7b280a5997236042ca7650b7abc17c Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 15:28:12 +0530 Subject: [PATCH 01/23] Add additional stream samples --- java/Streaming/README.md | 1 + java/Streaming/pom.xml | 460 ++++++++++++++++++ .../streaming/examples/async/AsyncClient.java | 43 ++ .../examples/async/AsyncIOExample.java | 120 +++++ .../examples/datagen/DataGenerator.java | 51 ++ .../datagen/DataGeneratorPerCheckpoint.java | 56 +++ .../examples/gpu/MatrixVectorMul.java | 253 ++++++++++ .../examples/iteration/IterateExample.java | 242 +++++++++ .../iteration/util/IterateExampleData.java | 63 +++ .../streaming/examples/join/WindowJoin.java | 172 +++++++ .../examples/join/WindowJoinSampleData.java | 62 +++ .../sideoutput/SideOutputExample.java | 206 ++++++++ .../socket/SocketWindowWordCount.java | 112 +++++ .../statemachine/KafkaEventsGeneratorJob.java | 88 ++++ .../streaming/examples/statemachine/README.md | 52 ++ .../statemachine/StateMachineExample.java | 242 +++++++++ .../statemachine/dfa/EventTypeAndState.java | 34 ++ .../examples/statemachine/dfa/State.java | 144 ++++++ .../examples/statemachine/dfa/Transition.java | 99 ++++ .../examples/statemachine/event/Alert.java | 92 ++++ .../examples/statemachine/event/Event.java | 96 ++++ .../statemachine/event/EventType.java | 30 ++ .../generator/EventsGenerator.java | 156 ++++++ .../generator/EventsGeneratorFunction.java | 57 +++ .../StandaloneThreadedGenerator.java | 252 ++++++++++ .../kafka/EventDeSerializationSchema.java | 62 +++ .../kafka/KafkaStandaloneGenerator.java | 96 ++++ .../examples/utils/ThrottledIterator.java | 101 ++++ .../GroupedProcessingTimeWindowExample.java | 149 ++++++ .../examples/windowing/SessionWindowing.java | 110 +++++ .../examples/windowing/TopSpeedWindowing.java | 194 ++++++++ .../examples/windowing/WindowWordCount.java | 153 ++++++ .../windowing/util/CarGeneratorFunction.java | 84 ++++ .../windowing/util/SessionWindowingData.java | 27 + .../util/TopSpeedWindowingExampleData.java | 372 ++++++++++++++ .../examples/wordcount/WordCount.java | 187 +++++++ .../examples/wordcount/util/CLI.java | 149 ++++++ .../wordcount/util/WordCountData.java | 65 +++ .../src/main/resources/log4j2.properties | 25 + java/Streaming/src/main/resources/logback.xml | 29 ++ .../test/StreamingExamplesITCase.java | 177 +++++++ .../test/examples/join/WindowJoinData.java | 229 +++++++++ .../TopSpeedWindowingExampleITCase.java | 67 +++ .../socket/SocketWindowWordCountITCase.java | 122 +++++ .../src/test/resources/log4j2-test.properties | 28 ++ 45 files changed, 5609 insertions(+) create mode 100644 java/Streaming/README.md create mode 100644 java/Streaming/pom.xml create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncClient.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGenerator.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGeneratorPerCheckpoint.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoinSampleData.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/README.md create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/EventTypeAndState.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/State.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/Transition.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Alert.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Event.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/EventType.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGenerator.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorFunction.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/StandaloneThreadedGenerator.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/EventDeSerializationSchema.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/KafkaStandaloneGenerator.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/utils/ThrottledIterator.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java create mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/WordCountData.java create mode 100644 java/Streaming/src/main/resources/log4j2.properties create mode 100644 java/Streaming/src/main/resources/logback.xml create mode 100644 java/Streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java create mode 100644 java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/join/WindowJoinData.java create mode 100644 java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/windowing/TopSpeedWindowingExampleITCase.java create mode 100644 java/Streaming/src/test/java/org/apache/flink/streaming/test/socket/SocketWindowWordCountITCase.java create mode 100644 java/Streaming/src/test/resources/log4j2-test.properties diff --git a/java/Streaming/README.md b/java/Streaming/README.md new file mode 100644 index 0000000..5972c72 --- /dev/null +++ b/java/Streaming/README.md @@ -0,0 +1 @@ +Samples from https://github.com/apache/flink/tree/release-1.20.0/flink-examples/flink-examples-streaming diff --git a/java/Streaming/pom.xml b/java/Streaming/pom.xml new file mode 100644 index 0000000..205fbe7 --- /dev/null +++ b/java/Streaming/pom.xml @@ -0,0 +1,460 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-examples + 1.20.0 + + + flink-examples-streaming + Flink : Examples : Streaming + + jar + + + + 10.0.0 + + + + + + + + org.apache.flink + flink-streaming-java + ${project.version} + + + + org.apache.flink + flink-clients + ${project.version} + + + + org.apache.flink + flink-connector-files + ${project.version} + + + + org.apache.flink + flink-connector-kafka + 3.0.0-1.17 + + + + org.apache.flink + flink-connector-datagen + ${project.version} + + + + org.apache.flink + flink-shaded-jackson + + + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + + org.apache.flink + flink-statebackend-rocksdb + ${project.version} + + + + + org.jcuda + jcuda + ${jcuda.version} + + + org.jcuda + jcuda-natives + + + + + + org.jcuda + jcublas + ${jcuda.version} + + + org.jcuda + jcublas-natives + + + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + process-sources + + compile + + + -Xlint:deprecation + true + + + + org/apache/flink/streaming/examples/iteration/IterateExample.java + + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + default + package + + test-jar + + + + + + WindowJoin + package + + jar + + + WindowJoin + + + + org.apache.flink.streaming.examples.join.WindowJoin + + + + + org/apache/flink/streaming/examples/join/*.class + org/apache/flink/streaming/examples/utils/ThrottledIterator.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + WordCount + package + + jar + + + WordCount + + + + org.apache.flink.streaming.examples.wordcount.WordCount + + + + + org/apache/flink/streaming/examples/wordcount/WordCount.class + org/apache/flink/streaming/examples/wordcount/WordCount$*.class + org/apache/flink/streaming/examples/wordcount/util/WordCountData.class + org/apache/flink/streaming/examples/wordcount/util/CLI.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + SocketWindowWordCount + package + + jar + + + SocketWindowWordCount + + + + org.apache.flink.streaming.examples.socket.SocketWindowWordCount + + + + + org/apache/flink/streaming/examples/socket/SocketWindowWordCount.class + org/apache/flink/streaming/examples/socket/SocketWindowWordCount$*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + + + AsyncIO + package + + shade + + + false + AsyncIO + true + AsyncIO + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/async/*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.async.AsyncIOExample + + + + + + + + MatrixVectorMul + package + + shade + + + false + MatrixVectorMul + + + org.apache.flink:flink-connector-datagen + org.jcuda:* + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class + org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class + + + + + + org.apache.flink.streaming.examples.gpu.MatrixVectorMul + + + + + + + + Iteration + package + + shade + + + false + Iteration + true + Iteration + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/iteration/*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.iteration.IterateExample + + + + + + + + TopSpeedWindowing + package + + shade + + + false + TopSpeedWindowing + true + TopSpeedWindowing + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class + org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class + org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.class + org/apache/flink/streaming/examples/wordcount/util/CLI.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.windowing.TopSpeedWindowing + + + + + + + + SessionWindowing + package + + shade + + + false + SessionWindowing + true + SessionWindowing + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/windowing/SessionWindowing.class + org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.windowing.SessionWindowing + + + + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + rename + + + + + + + + + + + + + + diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncClient.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncClient.java new file mode 100644 index 0000000..3ff8d3a --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncClient.java @@ -0,0 +1,43 @@ +/* + * 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.examples.async; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ThreadLocalRandom; + +/** A simple asynchronous client that simulates interacting with an unreliable external service. */ +public class AsyncClient { + + public CompletableFuture query(int key) { + return CompletableFuture.supplyAsync( + () -> { + long sleep = (long) (ThreadLocalRandom.current().nextFloat() * 100); + try { + Thread.sleep(sleep); + } catch (InterruptedException e) { + throw new RuntimeException("AsyncClient was interrupted", e); + } + + if (ThreadLocalRandom.current().nextFloat() < 0.001f) { + throw new RuntimeException("wahahahaha..."); + } else { + return "key" + (key % 10); + } + }); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java new file mode 100644 index 0000000..2d96426 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java @@ -0,0 +1,120 @@ +/* + * 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.examples.async; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.streaming.api.datastream.AsyncDataStream; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; + +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +/** Example to illustrate how to use {@link AsyncFunction}. */ +public class AsyncIOExample { + + /** An example of {@link AsyncFunction} using an async client to query an external service. */ + private static class SampleAsyncFunction extends RichAsyncFunction { + private static final long serialVersionUID = 1L; + + private transient AsyncClient client; + + @Override + public void open(OpenContext openContext) { + client = new AsyncClient(); + } + + @Override + public void asyncInvoke(final Integer input, final ResultFuture resultFuture) { + client.query(input) + .whenComplete( + (response, error) -> { + if (response != null) { + resultFuture.complete(Collections.singletonList(response)); + } else { + resultFuture.completeExceptionally(error); + } + }); + } + } + + public static void main(String[] args) throws Exception { + final ParameterTool params = ParameterTool.fromArgs(args); + + final String mode; + final long timeout; + + try { + mode = params.get("waitMode", "ordered"); + timeout = params.getLong("timeout", 10000L); + } catch (Exception e) { + System.out.println( + "To customize example, use: AsyncIOExample [--waitMode ]"); + throw e; + } + + // obtain execution environment + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataGeneratorSource generatorSource = + new DataGeneratorSource<>( + Long::intValue, + Integer.MAX_VALUE, + RateLimiterStrategy.perSecond(100), + Types.INT); + + // create input stream of a single integer + DataStream inputStream = + env.fromSource( + generatorSource, + WatermarkStrategy.noWatermarks(), + "Integers-generating Source"); + + AsyncFunction function = new SampleAsyncFunction(); + + // add async operator to streaming job + DataStream result; + switch (mode.toUpperCase()) { + case "ORDERED": + result = + AsyncDataStream.orderedWait( + inputStream, function, timeout, TimeUnit.MILLISECONDS, 20); + break; + case "UNORDERED": + result = + AsyncDataStream.unorderedWait( + inputStream, function, timeout, TimeUnit.MILLISECONDS, 20); + break; + default: + throw new IllegalStateException("Unknown mode: " + mode); + } + + result.print(); + + // execute the program + env.execute("Async IO Example: " + mode); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGenerator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGenerator.java new file mode 100644 index 0000000..326e2a2 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGenerator.java @@ -0,0 +1,51 @@ +/* + * 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.examples.datagen; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +/** An example for generating data with a {@link DataGeneratorSource}. */ +public class DataGenerator { + + public static void main(String[] args) throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(4); + + GeneratorFunction generatorFunction = index -> "Number: " + index; + + DataGeneratorSource generatorSource = + new DataGeneratorSource<>( + generatorFunction, + Long.MAX_VALUE, + RateLimiterStrategy.perSecond(4), + Types.STRING); + + DataStreamSource streamSource = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); + streamSource.print(); + + env.execute("Data Generator Source Example"); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGeneratorPerCheckpoint.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGeneratorPerCheckpoint.java new file mode 100644 index 0000000..5eef737 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGeneratorPerCheckpoint.java @@ -0,0 +1,56 @@ +/* + * 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.examples.datagen; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +/** An example for generating specific data per checkpoint with a {@link DataGeneratorSource} . */ +public class DataGeneratorPerCheckpoint { + + public static void main(String[] args) throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(3000); + env.setParallelism(1); + + final String[] elements = new String[] {"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}; + final int size = elements.length; + final GeneratorFunction generatorFunction = + index -> elements[(int) (index % size)]; + + final DataGeneratorSource generatorSource = + new DataGeneratorSource<>( + generatorFunction, + Long.MAX_VALUE, + RateLimiterStrategy.perCheckpoint(size), + Types.STRING); + + final DataStreamSource streamSource = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); + streamSource.print(); + + env.execute("Data Generator Source Example"); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java new file mode 100644 index 0000000..c437c40 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java @@ -0,0 +1,253 @@ +/* + * 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.examples.gpu; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.externalresource.ExternalResourceInfo; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; + +import jcuda.Pointer; +import jcuda.Sizeof; +import jcuda.jcublas.JCublas; +import jcuda.runtime.JCuda; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; + +/** + * Implements the matrix-vector multiplication program that shows how to use GPU resources in Flink. + * + *

The input is a vector stream, which will generate random vectors with specified dimension. The + * data size of the vector stream could be specified by user. Each vector will be multiplied with a + * random dimension * dimension matrix in {@link Multiplier} and the result would be emitted to + * output. + * + *

Usage: MatrixVectorMul [--output <path>] [--dimension <dimension> --data-size + * <data_size>] + * + *

If no parameters are provided, the program is run with default vector dimension 10 and data + * size 100. + * + *

This example shows how to: + * + *

    + *
  • leverage external resource in operators, + *
  • accelerate complex calculation with GPU resources. + *
+ * + *

Notice that you need to add JCuda natives libraries in your Flink distribution by the + * following steps: + * + *

    + *
  • download the JCuda native libraries bundle for your CUDA version from + * http://www.jcuda.org/downloads/ + *
  • copy the native libraries jcuda-natives and jcublas-natives for your CUDA version, + * operating system and architecture to the "lib/" folder of your Flink distribution + *
+ */ +public class MatrixVectorMul { + + private static final int DEFAULT_DIM = 10; + private static final int DEFAULT_DATA_SIZE = 100; + private static final String DEFAULT_RESOURCE_NAME = "gpu"; + + public static void main(String[] args) throws Exception { + + // Checking input parameters + final ParameterTool params = ParameterTool.fromArgs(args); + System.out.println( + "Usage: MatrixVectorMul [--output ] [--dimension --data-size ] [--resource-name ]"); + + // Set up the execution environment + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // Make parameters available in the web interface + env.getConfig().setGlobalJobParameters(params); + + final int dimension = params.getInt("dimension", DEFAULT_DIM); + final int dataSize = params.getInt("data-size", DEFAULT_DATA_SIZE); + final String resourceName = params.get("resource-name", DEFAULT_RESOURCE_NAME); + + GeneratorFunction> generatorFunction = + index -> { + List randomRecord = new ArrayList<>(); + for (int i = 0; i < dimension; ++i) { + randomRecord.add((float) Math.random()); + } + return randomRecord; + }; + + // Generates random vectors with specified dimension + DataGeneratorSource> generatorSource = + new DataGeneratorSource<>(generatorFunction, dataSize, Types.LIST(Types.FLOAT)); + + DataStream> result = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Vectors Source") + .map(new Multiplier(dimension, resourceName)); + + // Emit result + if (params.has("output")) { + result.sinkTo( + FileSink.forRowFormat( + new Path(params.get("output")), + new SimpleStringEncoder>()) + .build()); + } else { + System.out.println("Printing result to stdout. Use --output to specify output path."); + result.print(); + } + // Execute program + env.execute("Matrix-Vector Multiplication"); + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** Matrix-Vector multiplier using CUBLAS library. */ + private static final class Multiplier extends RichMapFunction, List> { + private final int dimension; + private final String resourceName; + private Pointer matrixPointer; + + Multiplier(int dimension, String resourceName) { + this.dimension = dimension; + this.resourceName = resourceName; + } + + @Override + public void open(OpenContext openContext) { + // When multiple instances of this class and JCuda exist in different class loaders, + // then we will get UnsatisfiedLinkError. + // To avoid that, we need to temporarily override the java.io.tmpdir, where the JCuda + // store its native library, with a random path. + // For more details please refer to https://issues.apache.org/jira/browse/FLINK-5408 and + // the discussion in + // http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Classloader-and-removal-of-native-libraries-td14808.html + final String originTempDir = System.getProperty("java.io.tmpdir"); + final String newTempDir = originTempDir + "/jcuda-" + UUID.randomUUID(); + System.setProperty("java.io.tmpdir", newTempDir); + + final Set externalResourceInfos = + getRuntimeContext().getExternalResourceInfos(resourceName); + Preconditions.checkState( + !externalResourceInfos.isEmpty(), + "The MatrixVectorMul needs at least one GPU device while finding 0 GPU."); + final Optional firstIndexOptional = + externalResourceInfos.iterator().next().getProperty("index"); + Preconditions.checkState(firstIndexOptional.isPresent()); + + matrixPointer = new Pointer(); + final float[] matrix = new float[dimension * dimension]; + // Initialize a random matrix + for (int i = 0; i < dimension * dimension; ++i) { + matrix[i] = (float) Math.random(); + } + + // Set the CUDA device + JCuda.cudaSetDevice(Integer.parseInt(firstIndexOptional.get())); + + // Initialize JCublas + JCublas.cublasInit(); + + // Allocate device memory for the matrix + JCublas.cublasAlloc(dimension * dimension, Sizeof.FLOAT, matrixPointer); + JCublas.cublasSetVector( + dimension * dimension, Sizeof.FLOAT, Pointer.to(matrix), 1, matrixPointer, 1); + + // Change the java.io.tmpdir back to its original value. + System.setProperty("java.io.tmpdir", originTempDir); + } + + @Override + public List map(List value) { + final float[] input = new float[dimension]; + final float[] output = new float[dimension]; + final Pointer inputPointer = new Pointer(); + final Pointer outputPointer = new Pointer(); + + // Fill the input and output vector + for (int i = 0; i < dimension; i++) { + input[i] = value.get(i); + output[i] = 0; + } + + // Allocate device memory for the input and output + JCublas.cublasAlloc(dimension, Sizeof.FLOAT, inputPointer); + JCublas.cublasAlloc(dimension, Sizeof.FLOAT, outputPointer); + + // Initialize the device matrices + JCublas.cublasSetVector(dimension, Sizeof.FLOAT, Pointer.to(input), 1, inputPointer, 1); + JCublas.cublasSetVector( + dimension, Sizeof.FLOAT, Pointer.to(output), 1, outputPointer, 1); + + // Performs operation using JCublas + JCublas.cublasSgemv( + 'n', + dimension, + dimension, + 1.0f, + matrixPointer, + dimension, + inputPointer, + 1, + 0.0f, + outputPointer, + 1); + + // Read the result back + JCublas.cublasGetVector( + dimension, Sizeof.FLOAT, outputPointer, 1, Pointer.to(output), 1); + + // Memory clean up + JCublas.cublasFree(inputPointer); + JCublas.cublasFree(outputPointer); + + List outputList = new ArrayList<>(); + for (int i = 0; i < dimension; ++i) { + outputList.add(output[i]); + } + + return outputList; + } + + @Override + public void close() { + // Memory clean up + JCublas.cublasFree(matrixPointer); + + // Shutdown cublas + JCublas.cublasShutdown(); + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java new file mode 100644 index 0000000..d72933b --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java @@ -0,0 +1,242 @@ +/* + * 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.examples.iteration; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.TextLineInputFormat; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.IterativeStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; + +import java.time.Duration; +import java.util.Random; + +/** + * Example illustrating iterations in Flink streaming. + * + *

The program sums up random numbers and counts additions it performs to reach a specific + * threshold in an iterative streaming fashion. + * + *

This example shows how to use: + * + *

    + *
  • streaming iterations, + *
  • buffer timeout to enhance latency, + *
  • directed outputs. + *
+ */ +public class IterateExample { + + private static final int BOUND = 100; + + private static final OutputTag> + ITERATE_TAG = + new OutputTag>( + "iterate") {}; + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + + // Checking input parameters + final ParameterTool params = ParameterTool.fromArgs(args); + + // set up input for the stream of integer pairs + + // obtain execution environment and set setBufferTimeout to 1 to enable + // continuous flushing of the output buffers (lowest latency) + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment().setBufferTimeout(1); + + // make parameters available in the web interface + env.getConfig().setGlobalJobParameters(params); + + // create input stream of integer pairs + DataStream> inputStream; + if (params.has("input")) { + FileSource fileSource = + FileSource.forRecordStreamFormat( + new TextLineInputFormat(), new Path(params.get("input"))) + .build(); + inputStream = + env.fromSource(fileSource, WatermarkStrategy.noWatermarks(), "Tuples Source") + .map(new FibonacciInputMap()); + } else { + System.out.println("Executing Iterate example with default input data set."); + System.out.println("Use --input to specify file input."); + + GeneratorFunction> dataGenerator = + new RandomFibonacciGenerator(); + DataGeneratorSource> generatorSource = + new DataGeneratorSource<>( + dataGenerator, + BOUND, + RateLimiterStrategy.perSecond(20), + Types.TUPLE(Types.INT, Types.INT)); + + inputStream = + env.fromSource( + generatorSource, + WatermarkStrategy.noWatermarks(), + "Generated tuples Source"); + } + + // create an iterative data stream from the input with 5 second timeout + IterativeStream> it = + inputStream.map(new InputMap()).iterate(5000L); + + // apply the step function to get the next Fibonacci number + // increment the counter and split the output + SingleOutputStreamOperator> step = + it.process(new Step()); + + // close the iteration by selecting the tuples that were directed to the + // 'iterate' channel in the output selector + it.closeWith(step.getSideOutput(ITERATE_TAG)); + + // to produce the final get the input pairs that have the greatest iteration counter + // on a 1 second sliding window + DataStream, Integer>> numbers = step.map(new OutputMap()); + + // emit results + if (params.has("output")) { + numbers.sinkTo( + FileSink., Integer>>forRowFormat( + new Path(params.get("output")), new SimpleStringEncoder<>()) + .withRollingPolicy( + DefaultRollingPolicy.builder() + .withMaxPartSize(MemorySize.ofMebiBytes(1)) + .withRolloverInterval(Duration.ofSeconds(10)) + .build()) + .build()); + } else { + System.out.println("Printing result to stdout. Use --output to specify output path."); + numbers.print(); + } + + // execute the program + env.execute("Streaming Iteration Example"); + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** Generate BOUND number of random integer pairs from the range from 1 to BOUND/2. */ + private static class RandomFibonacciGenerator + implements GeneratorFunction> { + private static final long serialVersionUID = 1L; + + private final Random rnd = new Random(); + + @Override + public Tuple2 map(Long ignoredIndex) throws Exception { + int first = rnd.nextInt(BOUND / 2 - 1) + 1; + int second = rnd.nextInt(BOUND / 2 - 1) + 1; + return new Tuple2<>(first, second); + } + } + + /** Generate random integer pairs from the range from 0 to BOUND/2. */ + private static class FibonacciInputMap + implements MapFunction> { + private static final long serialVersionUID = 1L; + + @Override + public Tuple2 map(String value) throws Exception { + String record = value.substring(1, value.length() - 1); + String[] splitted = record.split(","); + return new Tuple2<>(Integer.parseInt(splitted[0]), Integer.parseInt(splitted[1])); + } + } + + /** + * Map the inputs so that the next Fibonacci numbers can be calculated while preserving the + * original input tuple. A counter is attached to the tuple and incremented in every iteration + * step. + */ + public static class InputMap + implements MapFunction< + Tuple2, Tuple5> { + private static final long serialVersionUID = 1L; + + @Override + public Tuple5 map( + Tuple2 value) throws Exception { + return new Tuple5<>(value.f0, value.f1, value.f0, value.f1, 0); + } + } + + /** Iteration step function that calculates the next Fibonacci number. */ + public static class Step + extends ProcessFunction< + Tuple5, + Tuple5> { + private static final long serialVersionUID = 1L; + + @Override + public void processElement( + Tuple5 value, + Context ctx, + Collector> out) + throws Exception { + Tuple5 element = + new Tuple5<>(value.f0, value.f1, value.f3, value.f2 + value.f3, ++value.f4); + + if (value.f2 < BOUND && value.f3 < BOUND) { + ctx.output(ITERATE_TAG, element); + } else { + out.collect(element); + } + } + } + + /** Giving back the input pair and the counter. */ + public static class OutputMap + implements MapFunction< + Tuple5, + Tuple2, Integer>> { + private static final long serialVersionUID = 1L; + + @Override + public Tuple2, Integer> map( + Tuple5 value) throws Exception { + return new Tuple2<>(new Tuple2<>(value.f0, value.f1), value.f4); + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java new file mode 100644 index 0000000..031fa69 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java @@ -0,0 +1,63 @@ +/* + * 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.examples.iteration.util; + +/** Data for IterateExampleITCase. */ +public class IterateExampleData { + public static final String INPUT_PAIRS = + "(1,40)\n" + + "(29,38)\n" + + "(11,15)\n" + + "(17,39)\n" + + "(24,41)\n" + + "(7,33)\n" + + "(20,2)\n" + + "(11,5)\n" + + "(3,16)\n" + + "(23,36)\n" + + "(15,23)\n" + + "(28,13)\n" + + "(1,1)\n" + + "(10,6)\n" + + "(21,5)\n" + + "(14,36)\n" + + "(17,15)\n" + + "(7,9)"; + + public static final String RESULTS = + "((1,40),3)\n" + + "((24,41),2)\n" + + "((3,16),5)\n" + + "((1,1),10)\n" + + "((17,15),4)\n" + + "((29,38),2)\n" + + "((7,33),3)\n" + + "((23,36),3)\n" + + "((10,6),6)\n" + + "((7,9),5)\n" + + "((11,15),4)\n" + + "((20,2),5)\n" + + "((15,23),4)\n" + + "((21,5),5)\n" + + "((17,39),3)\n" + + "((11,5),6)\n" + + "((28,13),4)\n" + + "((14,36),3)"; + + private IterateExampleData() {} +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java new file mode 100644 index 0000000..abd4439 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java @@ -0,0 +1,172 @@ +/* + * 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.examples.join; + +import org.apache.flink.api.common.eventtime.AscendingTimestampsWatermarks; +import org.apache.flink.api.common.eventtime.TimestampAssigner; +import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; + +import java.time.Duration; + +/** + * Example illustrating a windowed stream join between two data streams. + * + *

The example works on two input streams with pairs (name, grade) and (name, salary) + * respectively. It joins the streams based on "name" within a configurable window. + * + *

The example uses a built-in sample data generator that generates the streams of pairs at a + * configurable rate. + */ +public class WindowJoin { + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + // parse the parameters + final ParameterTool params = ParameterTool.fromArgs(args); + final long windowSize = params.getLong("windowSize", 2000); + final long rate = params.getLong("rate", 3L); + final boolean fileOutput = params.has("output"); + + System.out.println("Using windowSize=" + windowSize + ", data rate=" + rate); + System.out.println( + "To customize example, use: WindowJoin [--windowSize ] [--rate ]"); + + // obtain execution environment, run this example in "ingestion time" + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // make parameters available in the web interface + env.getConfig().setGlobalJobParameters(params); + + // create the data sources for both grades and salaries + DataStream> grades = + env.fromSource( + WindowJoinSampleData.getGradeGeneratorSource(rate), + IngestionTimeWatermarkStrategy.create(), + "Grades Data Generator") + .setParallelism(1); + + DataStream> salaries = + env.fromSource( + WindowJoinSampleData.getSalaryGeneratorSource(rate), + IngestionTimeWatermarkStrategy.create(), + "Grades Data Generator") + .setParallelism(1); + + // run the actual window join program + // for testability, this functionality is in a separate method. + DataStream> joinedStream = + runWindowJoin(grades, salaries, windowSize); + + if (fileOutput) { + joinedStream + .sinkTo( + FileSink.>forRowFormat( + new Path(params.get("output")), + new SimpleStringEncoder<>()) + .withRollingPolicy( + DefaultRollingPolicy.builder() + .withMaxPartSize(MemorySize.ofMebiBytes(1)) + .withRolloverInterval(Duration.ofSeconds(10)) + .build()) + .build()) + .name("output"); + } else { + // print the results with a single thread, rather than in parallel + joinedStream.print().setParallelism(1); + } + + // execute program + env.execute("Windowed Join Example"); + } + + public static DataStream> runWindowJoin( + DataStream> grades, + DataStream> salaries, + long windowSize) { + + return grades.join(salaries) + .where(new NameKeySelector()) + .equalTo(new NameKeySelector()) + .window(TumblingEventTimeWindows.of(Duration.ofMillis(windowSize))) + .apply( + new JoinFunction< + Tuple2, + Tuple2, + Tuple3>() { + + @Override + public Tuple3 join( + Tuple2 first, Tuple2 second) { + return new Tuple3( + first.f0, first.f1, second.f1); + } + }); + } + + private static class NameKeySelector implements KeySelector, String> { + @Override + public String getKey(Tuple2 value) { + return value.f0; + } + } + + /** + * This {@link WatermarkStrategy} assigns the current system time as the event-time timestamp. + * In a real use case you should use proper timestamps and an appropriate {@link + * WatermarkStrategy}. + */ + private static class IngestionTimeWatermarkStrategy implements WatermarkStrategy { + + private IngestionTimeWatermarkStrategy() {} + + public static IngestionTimeWatermarkStrategy create() { + return new IngestionTimeWatermarkStrategy<>(); + } + + @Override + public WatermarkGenerator createWatermarkGenerator( + WatermarkGeneratorSupplier.Context context) { + return new AscendingTimestampsWatermarks<>(); + } + + @Override + public TimestampAssigner createTimestampAssigner( + TimestampAssignerSupplier.Context context) { + return (event, timestamp) -> System.currentTimeMillis(); + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoinSampleData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoinSampleData.java new file mode 100644 index 0000000..55f59ed --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoinSampleData.java @@ -0,0 +1,62 @@ +/* + * 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.examples.join; + +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; + +import java.util.Random; + +/** Sample data for the {@link WindowJoin} example. */ +@SuppressWarnings("serial") +public class WindowJoinSampleData { + + static final String[] NAMES = {"tom", "jerry", "alice", "bob", "john", "grace"}; + static final int GRADE_COUNT = 5; + static final int SALARY_MAX = 10000; + + /** Continuously generates (name, grade). */ + public static DataGeneratorSource> getGradeGeneratorSource( + double elementsPerSecond) { + return getTupleGeneratorSource(GRADE_COUNT, elementsPerSecond); + } + + /** Continuously generates (name, salary). */ + public static DataGeneratorSource> getSalaryGeneratorSource( + double elementsPerSecond) { + return getTupleGeneratorSource(SALARY_MAX, elementsPerSecond); + } + + private static DataGeneratorSource> getTupleGeneratorSource( + int maxValue, double elementsPerSecond) { + final Random rnd = new Random(); + final GeneratorFunction> generatorFunction = + index -> new Tuple2<>(NAMES[rnd.nextInt(NAMES.length)], rnd.nextInt(maxValue) + 1); + + return new DataGeneratorSource<>( + generatorFunction, + Long.MAX_VALUE, + RateLimiterStrategy.perSecond(elementsPerSecond), + TypeInformation.of(new TypeHint>() {})); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java new file mode 100644 index 0000000..6e33f5d --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java @@ -0,0 +1,206 @@ +/* + * 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.examples.sideoutput; + +import org.apache.flink.api.common.eventtime.AscendingTimestampsWatermarks; +import org.apache.flink.api.common.eventtime.TimestampAssigner; +import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.TextLineInputFormat; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.examples.wordcount.util.WordCountData; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; + +import java.time.Duration; + +/** + * An example that illustrates the use of side output. + * + *

This is a modified version of {@link + * org.apache.flink.streaming.examples.windowing.WindowWordCount} that has a filter in the tokenizer + * and only emits some words for counting while emitting the other words to a side output. + */ +public class SideOutputExample { + + /** + * We need to create an {@link OutputTag} so that we can reference it when emitting data to a + * side output and also to retrieve the side output stream from an operation. + */ + private static final OutputTag rejectedWordsTag = new OutputTag("rejected") {}; + + public static void main(String[] args) throws Exception { + + // Checking input parameters + final ParameterTool params = ParameterTool.fromArgs(args); + + // set up the execution environment + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // make parameters available in the web interface + env.getConfig().setGlobalJobParameters(params); + + // get input data + DataStream textWithTimestampAndWatermark; + if (params.has("input")) { + // read the text file from given input path + FileSource fileSource = + FileSource.forRecordStreamFormat( + new TextLineInputFormat(), new Path(params.get("input"))) + .build(); + textWithTimestampAndWatermark = + env.fromSource( + fileSource, IngestionTimeWatermarkStrategy.create(), "Words Source"); + } else { + System.out.println("Executing WordCount example with default input data set."); + System.out.println("Use --input to specify file input."); + // get default test text data + DataStreamSource text = env.fromData(WordCountData.WORDS); + // We assign the WatermarkStrategy after creating the source because + // StreamExecutionEnvironment#fromElemenets() methods currently does not accept + // WatermarkStrategies. In a real-world job you should integrate the WatermarkStrategy + // in the source as shown above for the FileSource. + textWithTimestampAndWatermark = + text.assignTimestampsAndWatermarks(IngestionTimeWatermarkStrategy.create()); + } + + SingleOutputStreamOperator> tokenized = + textWithTimestampAndWatermark.process(new Tokenizer()); + + DataStream rejectedWords = + tokenized + .getSideOutput(rejectedWordsTag) + .map(value -> "rejected: " + value, Types.STRING); + + DataStream> counts = + tokenized + .keyBy(value -> value.f0) + .window(TumblingEventTimeWindows.of(Duration.ofSeconds(5))) + // group by the tuple field "0" and sum up tuple field "1" + .sum(1); + + // emit result + if (params.has("output")) { + counts.sinkTo( + FileSink.>forRowFormat( + new Path(params.get("output")), + new SimpleStringEncoder<>()) + .withRollingPolicy( + DefaultRollingPolicy.builder() + .withMaxPartSize(MemorySize.ofMebiBytes(1)) + .withRolloverInterval(Duration.ofSeconds(10)) + .build()) + .build()) + .name("output"); + + rejectedWords + .sinkTo( + FileSink.forRowFormat( + new Path(params.get("rejected-words-output")), + new SimpleStringEncoder<>()) + .withRollingPolicy( + DefaultRollingPolicy.builder() + .withMaxPartSize(MemorySize.ofMebiBytes(1)) + .withRolloverInterval(Duration.ofSeconds(10)) + .build()) + .build()) + .name("rejected-words-output"); + } else { + System.out.println("Printing result to stdout. Use --output to specify output path."); + counts.print(); + rejectedWords.print(); + } + + // execute program + env.execute("Streaming WordCount SideOutput"); + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** + * Implements the string tokenizer that splits sentences into words as a user-defined + * FlatMapFunction. The function takes a line (String) and splits it into multiple pairs in the + * form of "(word,1)" ({@code Tuple2}). + * + *

This rejects words that are longer than 5 characters long. + */ + public static final class Tokenizer extends ProcessFunction> { + private static final long serialVersionUID = 1L; + + @Override + public void processElement( + String value, Context ctx, Collector> out) + throws Exception { + // normalize and split the line + String[] tokens = value.toLowerCase().split("\\W+"); + + // emit the pairs + for (String token : tokens) { + if (token.length() > 5) { + ctx.output(rejectedWordsTag, token); + } else if (token.length() > 0) { + out.collect(new Tuple2<>(token, 1)); + } + } + } + } + + /** + * This {@link WatermarkStrategy} assigns the current system time as the event-time timestamp. + * In a real use case you should use proper timestamps and an appropriate {@link + * WatermarkStrategy}. + */ + private static class IngestionTimeWatermarkStrategy implements WatermarkStrategy { + + private IngestionTimeWatermarkStrategy() {} + + public static IngestionTimeWatermarkStrategy create() { + return new IngestionTimeWatermarkStrategy<>(); + } + + @Override + public WatermarkGenerator createWatermarkGenerator( + WatermarkGeneratorSupplier.Context context) { + return new AscendingTimestampsWatermarks<>(); + } + + @Override + public TimestampAssigner createTimestampAssigner( + TimestampAssignerSupplier.Context context) { + return (event, timestamp) -> System.currentTimeMillis(); + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java new file mode 100644 index 0000000..584183a --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java @@ -0,0 +1,112 @@ +/* + * 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.examples.socket; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows; + +import java.time.Duration; + +/** + * Implements a streaming windowed version of the "WordCount" program. + * + *

This program connects to a server socket and reads strings from the socket. The easiest way to + * try this out is to open a text server (at port 12345) using the netcat tool via + * + *

+ * nc -l 12345 on Linux or nc -l -p 12345 on Windows
+ * 
+ * + *

and run this example with the hostname and the port as arguments. + */ +public class SocketWindowWordCount { + + public static void main(String[] args) throws Exception { + + // the host and the port to connect to + final String hostname; + final int port; + try { + final ParameterTool params = ParameterTool.fromArgs(args); + hostname = params.has("hostname") ? params.get("hostname") : "localhost"; + port = params.getInt("port"); + } catch (Exception e) { + System.err.println( + "No port specified. Please run 'SocketWindowWordCount " + + "--hostname --port ', where hostname (localhost by default) " + + "and port is the address of the text server"); + System.err.println( + "To start a simple text server, run 'netcat -l ' and " + + "type the input text into the command line"); + return; + } + + // get the execution environment + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // get input data by connecting to the socket + DataStream text = env.socketTextStream(hostname, port, "\n"); + + // parse the data, group it, window it, and aggregate the counts + DataStream windowCounts = + text.flatMap( + (FlatMapFunction) + (value, out) -> { + for (String word : value.split("\\s")) { + out.collect(new WordWithCount(word, 1L)); + } + }, + Types.POJO(WordWithCount.class)) + .keyBy(value -> value.word) + .window(TumblingProcessingTimeWindows.of(Duration.ofSeconds(5))) + .reduce((a, b) -> new WordWithCount(a.word, a.count + b.count)) + .returns(WordWithCount.class); + + // print the results with a single thread, rather than in parallel + windowCounts.print().setParallelism(1); + + env.execute("Socket Window WordCount"); + } + + // ------------------------------------------------------------------------ + + /** Data type for words with count. */ + public static class WordWithCount { + + public String word; + public long count; + + @SuppressWarnings("unused") + public WordWithCount() {} + + public WordWithCount(String word, long count) { + this.word = word; + this.count = count; + } + + @Override + public String toString() { + return word + " : " + count; + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java new file mode 100644 index 0000000..776fcfd --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java @@ -0,0 +1,88 @@ +/* + * 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.examples.statemachine; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaSink; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.examples.statemachine.event.Event; +import org.apache.flink.streaming.examples.statemachine.generator.EventsGeneratorFunction; +import org.apache.flink.streaming.examples.statemachine.kafka.EventDeSerializationSchema; + +/** + * Job to generate input events that are written to Kafka, for the {@link StateMachineExample} job. + */ +public class KafkaEventsGeneratorJob { + + public static void main(String[] args) throws Exception { + + final ParameterTool params = ParameterTool.fromArgs(args); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final double errorRate = params.getDouble("error-rate", 0.0); + final int sleep = params.getInt("sleep", 1); + final double recordsPerSecond = + params.getDouble("rps", rpsFromSleep(sleep, env.getParallelism())); + System.out.printf( + "Generating events to Kafka with standalone source with error rate %f and %.1f records per second\n", + errorRate, recordsPerSecond); + System.out.println(); + + String kafkaTopic = params.get("kafka-topic"); + String brokers = params.get("brokers", "localhost:9092"); + + GeneratorFunction generatorFunction = new EventsGeneratorFunction(errorRate); + DataGeneratorSource eventGeneratorSource = + new DataGeneratorSource<>( + generatorFunction, + Long.MAX_VALUE, + RateLimiterStrategy.perSecond(recordsPerSecond), + TypeInformation.of(Event.class)); + + env.fromSource( + eventGeneratorSource, + WatermarkStrategy.noWatermarks(), + "Events Generator Source") + .sinkTo( + KafkaSink.builder() + .setBootstrapServers(brokers) + .setRecordSerializer( + KafkaRecordSerializationSchema.builder() + .setValueSerializationSchema( + new EventDeSerializationSchema()) + .setTopic(kafkaTopic) + .build()) + .build()); + + // trigger program execution + env.execute("State machine example Kafka events generator job"); + } + + // Used for backwards compatibility to convert legacy 'sleep' parameter to records per second. + private static double rpsFromSleep(int sleep, int parallelism) { + return (1000d / sleep) * parallelism; + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/README.md b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/README.md new file mode 100644 index 0000000..b32ffe6 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/README.md @@ -0,0 +1,52 @@ +Example: Running a state machine for pattern detection +====================================================== + +This example illustrates a minimal roll-your-own event pattern detection scenario, +using a simple state machine that is evaluated over the stream. + +While this example is much simpler and more manual than what the CEP library supports, +it illustrates the use of event processing and state management for a medium +complex scenario. + +**Scenario Description** + +Events in streams are expected to occur in certain patterns. Any deviation from +these patterns indicates an anomaly that the streaming system should recognize and that +should trigger an alert. + +You can, for example, think of events as being generated by network devices and services, +such as firewalls login-, and registration with an authentication service, etc. A deviation +from the expected pattern might indicate an intrusion detection. + +The event patterns are tracked per interacting party (here simplified per source IP address) +and are validated by a state machine. The state machine's states define what possible +events may occur next, and what new states these events will result in. + +The following diagram depicts the state machine used in this example. + +``` + +----> W ----> Y -----+ + | ^ | + INITIAL-+ | | + | | +--> (Z) --------> TERM + +----> X ------+ | + | | + +-----------------+ +``` + +**Example Program** + +The main class of this example program is `org.apache.flink.streaming.examples.statemachine.StateMachineExample`. +The core logic is in the `flatMap` function, which runs the state machines per IP address. + +The streaming data flow is as shown below, where the source stream may come from either +an embedded data generator, or from a from a Kafka topic: + +``` + [ stream partition 1] --> source --> partition -+---> flatMap(state machine) --> sink + \/ + /\ + [ stream partition 2] --> source --> partition -+---> flatMap(state machine) --> sink +``` + + diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java new file mode 100644 index 0000000..940b836 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java @@ -0,0 +1,242 @@ +/* + * 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.examples.statemachine; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.StateBackendOptions; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; +import org.apache.flink.streaming.examples.statemachine.dfa.State; +import org.apache.flink.streaming.examples.statemachine.event.Alert; +import org.apache.flink.streaming.examples.statemachine.event.Event; +import org.apache.flink.streaming.examples.statemachine.generator.EventsGeneratorFunction; +import org.apache.flink.streaming.examples.statemachine.kafka.EventDeSerializationSchema; +import org.apache.flink.util.Collector; + +import java.time.Duration; + +/** + * Main class of the state machine example. This class implements the streaming application that + * receives the stream of events and evaluates a state machine (per originating address) to validate + * that the events follow the state machine's rules. + */ +public class StateMachineExample { + + /** + * Main entry point for the program. + * + * @param args The command line arguments. + */ + public static void main(String[] args) throws Exception { + + // ---- print some usage help ---- + + System.out.println( + "Usage with built-in data generator: StateMachineExample [--error-rate ] [--sleep | --rps ]"); + System.out.println( + "Usage with Kafka: StateMachineExample --kafka-topic [--brokers ]"); + System.out.println("Options for both the above setups: "); + System.out.println("\t[--backend ]"); + System.out.println("\t[--checkpoint-dir ]"); + System.out.println("\t[--incremental-checkpoints ]"); + System.out.println("\t[--output OR null for stdout]"); + System.out.println(); + + // ---- determine whether to use the built-in source, or read from Kafka ---- + + final DataStream events; + final ParameterTool params = ParameterTool.fromArgs(args); + + // create the environment to create streams and configure execution + Configuration configuration = new Configuration(); + + final String stateBackend = params.get("backend", "memory"); + if ("hashmap".equals(stateBackend)) { + final String checkpointDir = params.get("checkpoint-dir"); + configuration.set(StateBackendOptions.STATE_BACKEND, "hashmap"); + configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir); + } else if ("rocks".equals(stateBackend)) { + final String checkpointDir = params.get("checkpoint-dir"); + boolean incrementalCheckpoints = params.getBoolean("incremental-checkpoints", false); + configuration.set( + StateBackendOptions.STATE_BACKEND, + "org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackendFactory"); + configuration.set(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, incrementalCheckpoints); + configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir); + } + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); + env.enableCheckpointing(2000L); + + if (params.has("kafka-topic")) { + // set up the Kafka reader + String kafkaTopic = params.get("kafka-topic"); + String brokers = params.get("brokers", "localhost:9092"); + + System.out.printf("Reading from kafka topic %s @ %s\n", kafkaTopic, brokers); + System.out.println(); + + KafkaSource source = + KafkaSource.builder() + .setBootstrapServers(brokers) + .setGroupId("stateMachineExample") + .setTopics(kafkaTopic) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + new EventDeSerializationSchema())) + .setStartingOffsets(OffsetsInitializer.latest()) + .build(); + events = + env.fromSource( + source, WatermarkStrategy.noWatermarks(), "StateMachineExampleSource"); + } else { + final double errorRate = params.getDouble("error-rate", 0.0); + final int sleep = params.getInt("sleep", 1); + final double recordsPerSecond = + params.getDouble("rps", rpsFromSleep(sleep, env.getParallelism())); + System.out.printf( + "Using standalone source with error rate %f and %.1f records per second\n", + errorRate, recordsPerSecond); + System.out.println(); + + GeneratorFunction generatorFunction = + new EventsGeneratorFunction(errorRate); + DataGeneratorSource eventGeneratorSource = + new DataGeneratorSource<>( + generatorFunction, + Long.MAX_VALUE, + RateLimiterStrategy.perSecond(recordsPerSecond), + TypeInformation.of(Event.class)); + + events = + env.fromSource( + eventGeneratorSource, + WatermarkStrategy.noWatermarks(), + "Events Generator Source"); + } + + // ---- main program ---- + + final String outputFile = params.get("output"); + + // make parameters available in the web interface + env.getConfig().setGlobalJobParameters(params); + + DataStream alerts = + events + // partition on the address to make sure equal addresses + // end up in the same state machine flatMap function + .keyBy(Event::sourceAddress) + + // the function that evaluates the state machine over the sequence of events + .flatMap(new StateMachineMapper()); + + // output the alerts to std-out + if (outputFile == null) { + alerts.print(); + } else { + alerts.sinkTo( + FileSink.forRowFormat( + new Path(outputFile), new SimpleStringEncoder<>()) + .withRollingPolicy( + DefaultRollingPolicy.builder() + .withMaxPartSize(MemorySize.ofMebiBytes(1)) + .withRolloverInterval(Duration.ofSeconds(10)) + .build()) + .build()) + .setParallelism(1) + .name("output"); + } + + // trigger program execution + env.execute("State machine job"); + } + + // ------------------------------------------------------------------------ + + /** + * The function that maintains the per-IP-address state machines and verifies that the events + * are consistent with the current state of the state machine. If the event is not consistent + * with the current state, the function produces an alert. + */ + @SuppressWarnings("serial") + static class StateMachineMapper extends RichFlatMapFunction { + + /** The state for the current key. */ + private ValueState currentState; + + @Override + public void open(OpenContext openContext) { + // get access to the state object + currentState = + getRuntimeContext().getState(new ValueStateDescriptor<>("state", State.class)); + } + + @Override + public void flatMap(Event evt, Collector out) throws Exception { + // get the current state for the key (source address) + // if no state exists, yet, the state must be the state machine's initial state + State state = currentState.value(); + if (state == null) { + state = State.Initial; + } + + // ask the state machine what state we should go to based on the given event + State nextState = state.transition(evt.type()); + + if (nextState == State.InvalidTransition) { + // the current event resulted in an invalid transition + // raise an alert! + out.collect(new Alert(evt.sourceAddress(), state, evt.type())); + } else if (nextState.isTerminal()) { + // we reached a terminal state, clean up the current state + currentState.clear(); + } else { + // remember the new state + currentState.update(nextState); + } + } + } + + // Used for backwards compatibility to convert legacy 'sleep' parameter to records per second. + private static double rpsFromSleep(int sleep, int parallelism) { + return (1000d / sleep) * parallelism; + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/EventTypeAndState.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/EventTypeAndState.java new file mode 100644 index 0000000..2ad77b4 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/EventTypeAndState.java @@ -0,0 +1,34 @@ +/* + * 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.examples.statemachine.dfa; + +import org.apache.flink.streaming.examples.statemachine.event.EventType; + +/** Simple combination of EventType and State. */ +public class EventTypeAndState { + + public final EventType eventType; + + public final State state; + + public EventTypeAndState(EventType eventType, State state) { + this.eventType = eventType; + this.state = state; + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/State.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/State.java new file mode 100644 index 0000000..eb79fb8 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/State.java @@ -0,0 +1,144 @@ +/* + * 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.examples.statemachine.dfa; + +import org.apache.flink.streaming.examples.statemachine.event.EventType; + +import java.util.Random; + +/** + * The State captures the main functionality of the state machine. It represents a specific state in + * the state machine, and holds all transitions possible from a specific state. + * + *

The state transition diagram is as follows: + * + *

+ *           +--[a]--> W --[b]--> Y --[e]---+
+ *           |                    ^         |
+ *   Initial-+                    |         |
+ *           |                    |         +--> (Z)-----[g]---> Terminal
+ *           +--[c]--> X --[b]----+         |
+ *                     |                    |
+ *                     +--------[d]---------+
+ * 
+ */ +public enum State { + + /** The terminal state in the state machine. */ + Terminal, + + /** + * Special state returned by the State.transition(...) function when attempting an illegal state + * transition. + */ + InvalidTransition, + + /** State 'Z'. */ + Z(new Transition(EventType.g, Terminal, 1.0f)), + + /** State 'Y'. */ + Y(new Transition(EventType.e, Z, 1.0f)), + + /** State 'X'. */ + X(new Transition(EventType.b, Y, 0.2f), new Transition(EventType.d, Z, 0.8f)), + + /** State 'W'. */ + W(new Transition(EventType.b, Y, 1.0f)), + + /** The initial state from which all state sequences start. */ + Initial(new Transition(EventType.a, W, 0.6f), new Transition(EventType.c, X, 0.4f)); + + // ------------------------------------------------------------------------ + + private final Transition[] transitions; + + State(Transition... transitions) { + this.transitions = transitions; + } + + /** Checks if this state is a terminal state. A terminal state has no outgoing transitions. */ + public boolean isTerminal() { + return transitions.length == 0; + } + + // ------------------------------------------------------------------------ + + /** + * Gets the state after transitioning from this state based on the given event. If the + * transition is valid, this returns the new state, and if this transition is illegal, it + * returns [[InvalidTransition]]. + * + * @param evt The event that defined the transition. + * @return The new state, or [[InvalidTransition]]. + */ + public State transition(EventType evt) { + for (Transition t : transitions) { + if (t.eventType() == evt) { + return t.targetState(); + } + } + + // no transition found + return InvalidTransition; + } + + /** + * Picks a random transition, based on the probabilities of the outgoing transitions of this + * state. + * + * @param rnd The random number generator to use. + * @return A pair of (transition event , new state). + */ + public EventTypeAndState randomTransition(Random rnd) { + if (isTerminal()) { + throw new RuntimeException("Cannot transition from state " + name()); + } else { + final float p = rnd.nextFloat(); + float mass = 0.0f; + Transition transition = null; + + for (Transition t : transitions) { + mass += t.prob(); + if (p <= mass) { + transition = t; + break; + } + } + + assert transition != null; + return new EventTypeAndState(transition.eventType(), transition.targetState()); + } + } + + /** + * Returns an event type that, if applied as a transition on this state, will result in an + * illegal state transition. + * + * @param rnd The random number generator to use. + * @return And event type for an illegal state transition. + */ + public EventType randomInvalidTransition(Random rnd) { + while (true) { + EventType candidate = EventType.values()[rnd.nextInt(EventType.values().length)]; + if (transition(candidate) == InvalidTransition) { + return candidate; + } + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/Transition.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/Transition.java new file mode 100644 index 0000000..4f4b55d --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/Transition.java @@ -0,0 +1,99 @@ +/* + * 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.examples.statemachine.dfa; + +import org.apache.flink.streaming.examples.statemachine.event.EventType; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A possible transition on a given event into a target state. The transition belongs to its + * originating state and has an associated probability that is used to generate random transition + * events. + */ +public class Transition implements Serializable { + + // this class is serializable to be able to interact cleanly with enums. + private static final long serialVersionUID = 1L; + + /** The event that triggers the transition. */ + private final EventType eventType; + + /** The target state after the transition. */ + private final State targetState; + + /** The probability of the transition. */ + private final float prob; + + /** + * Creates a new transition. + * + * @param eventType The event that triggers the transition. + * @param targetState The target state after the transition. + * @param prob The probability of the transition. + */ + public Transition(EventType eventType, State targetState, float prob) { + this.eventType = checkNotNull(eventType); + this.targetState = checkNotNull(targetState); + this.prob = prob; + } + + // ------------------------------------------------------------------------ + + public EventType eventType() { + return eventType; + } + + public State targetState() { + return targetState; + } + + public float prob() { + return prob; + } + + // ------------------------------------------------------------------------ + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } else if (obj == null || getClass() != obj.getClass()) { + return false; + } else { + final Transition that = (Transition) obj; + return this.eventType == that.eventType + && this.targetState == that.targetState + && Float.compare(this.prob, that.prob) == 0; + } + } + + @Override + public int hashCode() { + int code = 31 * eventType.hashCode() + targetState.hashCode(); + return 31 * code + (prob != +0.0f ? Float.floatToIntBits(prob) : 0); + } + + @Override + public String toString() { + return "--[" + eventType.name() + "]--> " + targetState.name() + " (" + prob + ')'; + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Alert.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Alert.java new file mode 100644 index 0000000..0dc204a --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Alert.java @@ -0,0 +1,92 @@ +/* + * 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.examples.statemachine.event; + +import org.apache.flink.streaming.examples.statemachine.dfa.State; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Data type for alerts. */ +public class Alert { + + private final int address; + + private final State state; + + private final EventType transition; + + /** + * Creates a new alert. + * + * @param address The originating address (think 32 bit IPv4 address). + * @param state The state that the event state machine found. + * @param transition The transition that was considered invalid. + */ + public Alert(int address, State state, EventType transition) { + this.address = address; + this.state = checkNotNull(state); + this.transition = checkNotNull(transition); + } + + // ------------------------------------------------------------------------ + + public int address() { + return address; + } + + public State state() { + return state; + } + + public EventType transition() { + return transition; + } + + // ------------------------------------------------------------------------ + + @Override + public int hashCode() { + int code = 31 * address + state.hashCode(); + return 31 * code + transition.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } else if (obj == null || getClass() != obj.getClass()) { + return false; + } else { + final Alert that = (Alert) obj; + return this.address == that.address + && this.transition == that.transition + && this.state == that.state; + } + } + + @Override + public String toString() { + return "ALERT " + + Event.formatAddress(address) + + " : " + + state.name() + + " -> " + + transition.name(); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Event.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Event.java new file mode 100644 index 0000000..ec15bd1 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Event.java @@ -0,0 +1,96 @@ +/* + * 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.examples.statemachine.event; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Data type for events, consisting of the originating IP address and an event type. */ +public class Event { + + private final EventType type; + + private final int sourceAddress; + + /** + * Creates a new event. + * + * @param type The event type. + * @param sourceAddress The originating address (think 32 bit IPv4 address). + */ + public Event(EventType type, int sourceAddress) { + this.type = checkNotNull(type); + this.sourceAddress = sourceAddress; + } + + /** Gets the event's type. */ + public EventType type() { + return type; + } + + /** Gets the event's source address. */ + public int sourceAddress() { + return sourceAddress; + } + + // ------------------------------------------------------------------------ + // Miscellaneous + // ------------------------------------------------------------------------ + + @Override + public int hashCode() { + return 31 * type.hashCode() + sourceAddress; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } else if (obj == null || getClass() != obj.getClass()) { + return false; + } else { + final Event that = (Event) obj; + return this.type == that.type && this.sourceAddress == that.sourceAddress; + } + } + + @Override + public String toString() { + return "Event " + formatAddress(sourceAddress) + " : " + type.name(); + } + + // ------------------------------------------------------------------------ + // Utils + // ------------------------------------------------------------------------ + + /** + * Util method to create a string representation of a 32 bit integer representing an IPv4 + * address. + * + * @param address The address, MSB first. + * @return The IP address string. + */ + public static String formatAddress(int address) { + int b1 = (address >>> 24) & 0xff; + int b2 = (address >>> 16) & 0xff; + int b3 = (address >>> 8) & 0xff; + int b4 = address & 0xff; + + return "" + b1 + '.' + b2 + '.' + b3 + '.' + b4; + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/EventType.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/EventType.java new file mode 100644 index 0000000..9b81ff0 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/EventType.java @@ -0,0 +1,30 @@ +/* + * 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.examples.statemachine.event; + +/** The type of the event processed by the state machine. */ +public enum EventType { + a, + b, + c, + d, + e, + f, + g; +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGenerator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGenerator.java new file mode 100644 index 0000000..6f78e73 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGenerator.java @@ -0,0 +1,156 @@ +/* + * 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.examples.statemachine.generator; + +import org.apache.flink.streaming.examples.statemachine.dfa.EventTypeAndState; +import org.apache.flink.streaming.examples.statemachine.dfa.State; +import org.apache.flink.streaming.examples.statemachine.event.Event; +import org.apache.flink.streaming.examples.statemachine.event.EventType; + +import javax.annotation.Nullable; + +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map.Entry; +import java.util.Random; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * A generator for events. The generator internally maintains a series of state machines (addresses + * and current associated state) and returns transition events from those state machines. Each time + * the next event is generators, this generator picks a random state machine and creates a random + * transition on that state machine. + * + *

The generator randomly adds new state machines, and removes state machines as soon as they + * reach the terminal state. This implementation maintains up to 1000 state machines concurrently. + */ +public class EventsGenerator { + + /** The random number generator. */ + private final Random rnd; + + /** The currently active state machines. */ + private final LinkedHashMap states; + + /** Probability with this generator generates an illegal state transition. */ + private final double errorProb; + + public EventsGenerator() { + this(0.0); + } + + public EventsGenerator(double errorProb) { + checkArgument(errorProb >= 0.0 && errorProb <= 1.0, "Invalid error probability"); + this.errorProb = errorProb; + + this.rnd = new Random(); + this.states = new LinkedHashMap<>(); + } + + // ------------------------------------------------------------------------ + + /** + * Creates a new random event. This method randomly pick either one of its currently running + * state machines, or start a new state machine for a random IP address. + * + *

With {@link #errorProb} probability, the generated event will be from an illegal state + * transition of one of the currently running state machines. + * + * @param minIp The lower bound for the range from which a new IP address may be picked. + * @param maxIp The upper bound for the range from which a new IP address may be picked. + * @return A next random event. + */ + public Event next(int minIp, int maxIp) { + final double p = rnd.nextDouble(); + + if (p * 1000 >= states.size()) { + // create a new state machine + final int nextIP = rnd.nextInt(maxIp - minIp) + minIp; + + if (!states.containsKey(nextIP)) { + EventTypeAndState eventAndState = State.Initial.randomTransition(rnd); + states.put(nextIP, eventAndState.state); + return new Event(eventAndState.eventType, nextIP); + } else { + // collision on IP address, try again + return next(minIp, maxIp); + } + } else { + // pick an existing state machine + + // skip over some elements in the linked map, then take the next + // update it, and insert it at the end + + int numToSkip = Math.min(20, rnd.nextInt(states.size())); + Iterator> iter = states.entrySet().iterator(); + + for (int i = numToSkip; i > 0; --i) { + iter.next(); + } + + Entry entry = iter.next(); + State currentState = entry.getValue(); + int address = entry.getKey(); + + iter.remove(); + + if (p < errorProb) { + EventType event = currentState.randomInvalidTransition(rnd); + return new Event(event, address); + } else { + EventTypeAndState eventAndState = currentState.randomTransition(rnd); + if (!eventAndState.state.isTerminal()) { + // reinsert + states.put(address, eventAndState.state); + } + + return new Event(eventAndState.eventType, address); + } + } + } + + /** + * Creates an event for an illegal state transition of one of the internal state machines. If + * the generator has not yet started any state machines (for example, because no call to {@link + * #next(int, int)} was made, yet), this will return null. + * + * @return An event for a illegal state transition, or null, if not possible. + */ + @Nullable + public Event nextInvalid() { + final Iterator> iter = states.entrySet().iterator(); + if (iter.hasNext()) { + final Entry entry = iter.next(); + + State currentState = entry.getValue(); + int address = entry.getKey(); + iter.remove(); + + EventType event = currentState.randomInvalidTransition(rnd); + return new Event(event, address); + } else { + return null; + } + } + + public int numActiveEntries() { + return states.size(); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorFunction.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorFunction.java new file mode 100644 index 0000000..70d6ec3 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorFunction.java @@ -0,0 +1,57 @@ +/* + * 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.examples.statemachine.generator; + +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.streaming.examples.statemachine.event.Event; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** A generator function that produces the events on the fly. Useful for self-contained demos. */ +@SuppressWarnings("serial") +public class EventsGeneratorFunction implements GeneratorFunction { + + private final double errorProbability; + + transient EventsGenerator generator; + private int min; + private int max; + + public EventsGeneratorFunction(double errorProbability) { + checkArgument( + errorProbability >= 0.0 && errorProbability <= 1.0, + "error probability must be in [0.0, 1.0]"); + + this.errorProbability = errorProbability; + } + + @Override + public void open(SourceReaderContext readerContext) throws Exception { + final int range = Integer.MAX_VALUE / readerContext.currentParallelism(); + min = range * readerContext.getIndexOfSubtask(); + max = min + range; + generator = new EventsGenerator(errorProbability); + } + + @Override + public Event map(Long value) throws Exception { + return generator.next(min, max); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/StandaloneThreadedGenerator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/StandaloneThreadedGenerator.java new file mode 100644 index 0000000..6ab8f38 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/StandaloneThreadedGenerator.java @@ -0,0 +1,252 @@ +/* + * 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.examples.statemachine.generator; + +import org.apache.flink.streaming.examples.statemachine.event.Event; +import org.apache.flink.util.Collector; + +import java.io.IOException; + +/** + * Base for standalone generators that use the state machine to create event sequences and push them + * for example into Kafka. + */ +public class StandaloneThreadedGenerator { + + public static void runGenerator(Collector[] collectors) throws IOException { + + final GeneratorThread[] threads = new GeneratorThread[collectors.length]; + final int range = Integer.MAX_VALUE / collectors.length; + + // create the generator threads + for (int i = 0; i < threads.length; i++) { + int min = range * i; + int max = min + range; + GeneratorThread thread = new GeneratorThread(collectors[i], min, max); + threads[i] = thread; + thread.setName("Generator " + i); + } + + long delay = 2L; + int nextErroneous = 0; + boolean running = true; + + for (GeneratorThread t : threads) { + t.setDelay(delay); + t.start(); + } + + final ThroughputLogger throughputLogger = new ThroughputLogger(threads); + throughputLogger.start(); + + System.out.println("Commands:"); + System.out.println(" -> q : Quit"); + System.out.println(" -> + : increase latency"); + System.out.println(" -> - : decrease latency"); + System.out.println(" -> e : inject invalid state transition"); + + // input loop + + while (running) { + final int next = System.in.read(); + + switch (next) { + case 'q': + System.out.println("Quitting..."); + running = false; + break; + + case 'e': + System.out.println("Injecting erroneous transition ..."); + threads[nextErroneous].sendInvalidStateTransition(); + nextErroneous = (nextErroneous + 1) % threads.length; + break; + + case '+': + delay = Math.max(delay * 2, 1); + System.out.println("Delay is " + delay); + for (GeneratorThread t : threads) { + t.setDelay(delay); + } + break; + + case '-': + delay /= 2; + System.out.println("Delay is " + delay); + for (GeneratorThread t : threads) { + t.setDelay(delay); + } + break; + + default: + // do nothing + } + } + + // shutdown + throughputLogger.shutdown(); + + for (GeneratorThread t : threads) { + t.shutdown(); + + try { + t.join(); + } catch (InterruptedException e) { + // restore interrupted status + Thread.currentThread().interrupt(); + } + } + } + + // ------------------------------------------------------------------------ + + /** + * A thread running a {@link EventsGenerator} and pushing generated events to the given + * collector (such as Kafka / Socket / ...). + */ + private static class GeneratorThread extends Thread { + + private final Collector out; + + private final int minAddress; + private final int maxAddress; + + private long delay; + + private long count; + + private volatile boolean running; + + private volatile boolean injectInvalidNext; + + /** + * Creates a new generator thread. + * + * @param out The collector to push the generated records to. + * @param minAddress The lower bound for the range from which a new IP address may be + * picked. + * @param maxAddress The upper bound for the range from which a new IP address may be + * picked. + */ + GeneratorThread(Collector out, int minAddress, int maxAddress) { + this.out = out; + this.minAddress = minAddress; + this.maxAddress = maxAddress; + this.running = true; + } + + @Override + public void run() { + final EventsGenerator generator = new EventsGenerator(); + + while (running) { + if (injectInvalidNext) { + injectInvalidNext = false; + Event next = generator.nextInvalid(); + if (next != null) { + out.collect(next); + } + } else { + out.collect(generator.next(minAddress, maxAddress)); + } + + count += 1; + + // sleep the delay to throttle + if (delay > 0) { + try { + Thread.sleep(delay); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + } + + public long currentCount() { + return count; + } + + public void shutdown() { + running = false; + interrupt(); + } + + public void setDelay(long delay) { + this.delay = delay; + } + + public void sendInvalidStateTransition() { + injectInvalidNext = true; + } + } + + // ------------------------------------------------------------------------ + + /** Thread that periodically print the number of elements generated per second. */ + private static class ThroughputLogger extends Thread { + + private final GeneratorThread[] generators; + + private volatile boolean running; + + /** + * Instantiates the throughput logger. + * + * @param generators The generator threads whose aggregate throughput should be logged. + */ + ThroughputLogger(GeneratorThread[] generators) { + this.generators = generators; + this.running = true; + } + + @Override + public void run() { + long lastCount = 0L; + long lastTimeStamp = System.currentTimeMillis(); + + while (running) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + break; + } + + long ts = System.currentTimeMillis(); + long currCount = 0L; + for (GeneratorThread generator : generators) { + currCount += generator.currentCount(); + } + + double factor = (ts - lastTimeStamp) / 1000.0; + double perSec = (currCount - lastCount) / factor; + + lastTimeStamp = ts; + lastCount = currCount; + + System.out.println(perSec + " / sec"); + } + } + + public void shutdown() { + running = false; + interrupt(); + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/EventDeSerializationSchema.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/EventDeSerializationSchema.java new file mode 100644 index 0000000..42bd675 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/EventDeSerializationSchema.java @@ -0,0 +1,62 @@ +/* + * 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.examples.statemachine.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.examples.statemachine.event.Event; +import org.apache.flink.streaming.examples.statemachine.event.EventType; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** A serializer and deserializer for the {@link Event} type. */ +public class EventDeSerializationSchema + implements DeserializationSchema, SerializationSchema { + + private static final long serialVersionUID = 1L; + + @Override + public byte[] serialize(Event evt) { + ByteBuffer byteBuffer = ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN); + byteBuffer.putInt(0, evt.sourceAddress()); + byteBuffer.putInt(4, evt.type().ordinal()); + return byteBuffer.array(); + } + + @Override + public Event deserialize(byte[] message) throws IOException { + ByteBuffer buffer = ByteBuffer.wrap(message).order(ByteOrder.LITTLE_ENDIAN); + int address = buffer.getInt(0); + int typeOrdinal = buffer.getInt(4); + return new Event(EventType.values()[typeOrdinal], address); + } + + @Override + public boolean isEndOfStream(Event nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(Event.class); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/KafkaStandaloneGenerator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/KafkaStandaloneGenerator.java new file mode 100644 index 0000000..781cb09 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/KafkaStandaloneGenerator.java @@ -0,0 +1,96 @@ +/* + * 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.examples.statemachine.kafka; + +import org.apache.flink.streaming.examples.statemachine.event.Event; +import org.apache.flink.streaming.examples.statemachine.generator.StandaloneThreadedGenerator; +import org.apache.flink.util.Collector; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; + +import java.util.Properties; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A generator that pushes the data into Kafka. */ +public class KafkaStandaloneGenerator extends StandaloneThreadedGenerator { + + public static final String BROKER_ADDRESS = "localhost:9092"; + + public static final String TOPIC = "flink-demo-topic-1"; + + public static final int NUM_PARTITIONS = 1; + + /** Entry point to the kafka data producer. */ + public static void main(String[] args) throws Exception { + + final KafkaCollector[] collectors = new KafkaCollector[NUM_PARTITIONS]; + + // create the generator threads + for (int i = 0; i < collectors.length; i++) { + collectors[i] = new KafkaCollector(BROKER_ADDRESS, TOPIC, i); + } + + StandaloneThreadedGenerator.runGenerator(collectors); + } + + // ------------------------------------------------------------------------ + + private static class KafkaCollector implements Collector, AutoCloseable { + + private final KafkaProducer producer; + + private final EventDeSerializationSchema serializer; + + private final String topic; + + private final int partition; + + KafkaCollector(String brokerAddress, String topic, int partition) { + this.topic = checkNotNull(topic); + this.partition = partition; + this.serializer = new EventDeSerializationSchema(); + + // create Kafka producer + Properties properties = new Properties(); + properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerAddress); + properties.put( + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + ByteArraySerializer.class.getCanonicalName()); + properties.put( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + ByteArraySerializer.class.getCanonicalName()); + this.producer = new KafkaProducer<>(properties); + } + + @Override + public void collect(Event evt) { + byte[] serialized = serializer.serialize(evt); + producer.send(new ProducerRecord<>(topic, partition, null, serialized)); + } + + @Override + public void close() { + producer.close(); + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/utils/ThrottledIterator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/utils/ThrottledIterator.java new file mode 100644 index 0000000..830f0d9 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/utils/ThrottledIterator.java @@ -0,0 +1,101 @@ +/* + * 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.examples.utils; + +import java.io.Serializable; +import java.util.Iterator; + +import static java.util.Objects.requireNonNull; + +/** + * A variant of the collection source (emits a sequence of elements as a stream) that supports + * throttling the emission rate. + * + * @param + */ +public class ThrottledIterator implements Iterator, Serializable { + + private static final long serialVersionUID = 1L; + + @SuppressWarnings("NonSerializableFieldInSerializableClass") + private final Iterator source; + + private final long sleepBatchSize; + private final long sleepBatchTime; + + private long lastBatchCheckTime; + private long num; + + public ThrottledIterator(Iterator source, long elementsPerSecond) { + this.source = requireNonNull(source); + + if (!(source instanceof Serializable)) { + throw new IllegalArgumentException("source must be java.io.Serializable"); + } + + if (elementsPerSecond >= 100) { + // how many elements would we emit per 50ms + this.sleepBatchSize = elementsPerSecond / 20; + this.sleepBatchTime = 50; + } else if (elementsPerSecond >= 1) { + // how long does element take + this.sleepBatchSize = 1; + this.sleepBatchTime = 1000 / elementsPerSecond; + } else { + throw new IllegalArgumentException( + "'elements per second' must be positive and not zero"); + } + } + + @Override + public boolean hasNext() { + return source.hasNext(); + } + + @Override + public T next() { + // delay if necessary + if (lastBatchCheckTime > 0) { + if (++num >= sleepBatchSize) { + num = 0; + + final long now = System.currentTimeMillis(); + final long elapsed = now - lastBatchCheckTime; + if (elapsed < sleepBatchTime) { + try { + Thread.sleep(sleepBatchTime - elapsed); + } catch (InterruptedException e) { + // restore interrupt flag and proceed + Thread.currentThread().interrupt(); + } + } + lastBatchCheckTime = now; + } + } else { + lastBatchCheckTime = System.currentTimeMillis(); + } + + return source.next(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java new file mode 100644 index 0000000..9db4e35 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java @@ -0,0 +1,149 @@ +/* + * 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.examples.windowing; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.apache.flink.streaming.api.functions.windowing.WindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows; +import org.apache.flink.streaming.api.windowing.windows.Window; +import org.apache.flink.util.Collector; + +import java.time.Duration; + +/** An example of grouped stream windowing into sliding time windows. */ +public class GroupedProcessingTimeWindowExample { + + public static void main(String[] args) throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final long numElementsPerParallel = 20000000; + final long numKeys = 10000; + + GeneratorFunction> generatorFunction = + new DataGeneratorFunction(numElementsPerParallel, numKeys); + + DataGeneratorSource> generatorSource = + new DataGeneratorSource<>( + generatorFunction, + numElementsPerParallel * env.getParallelism(), + Types.TUPLE(Types.LONG, Types.LONG)); + + DataStream> stream = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); + + stream.keyBy(value -> value.f0) + .window( + SlidingProcessingTimeWindows.of( + Duration.ofMillis(2500), Duration.ofMillis(500))) + .reduce(new SummingReducer()) + + // alternative: use a apply function which does not pre-aggregate + // .keyBy(new FirstFieldKeyExtractor, Long>()) + // .window(SlidingProcessingTimeWindows.of(Time.milliseconds(2500), + // Time.milliseconds(500))) + // .apply(new SummingWindowFunction()) + + .sinkTo(new DiscardingSink<>()); + + env.execute(); + } + + private static class FirstFieldKeyExtractor + implements KeySelector { + + @Override + @SuppressWarnings("unchecked") + public Key getKey(Type value) { + return (Key) value.getField(0); + } + } + + private static class SummingWindowFunction + implements WindowFunction, Tuple2, Long, Window> { + + @Override + public void apply( + Long key, + Window window, + Iterable> values, + Collector> out) { + long sum = 0L; + for (Tuple2 value : values) { + sum += value.f1; + } + + out.collect(new Tuple2<>(key, sum)); + } + } + + private static class SummingReducer implements ReduceFunction> { + + @Override + public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { + return new Tuple2<>(value1.f0, value1.f1 + value2.f1); + } + } + + /** + * This class represents a data generator function that generates a stream of tuples. Each tuple + * contains a key and a value. The function measures and prints the time it takes to generate + * numElements. The key space is limited to numKeys. The value is always 1. + */ + private static class DataGeneratorFunction + implements GeneratorFunction> { + + private final long numElements; + private final long numKeys; + private long startTime; + + public DataGeneratorFunction(long numElements, long numKeys) { + this.numElements = numElements; + this.numKeys = numKeys; + } + + @Override + public Tuple2 map(Long value) throws Exception { + if ((value % numElements) == 0) { + startTime = System.currentTimeMillis(); + } + if ((value % numElements + 1) == numElements) { + final long endTime = System.currentTimeMillis(); + System.out.println( + Thread.currentThread() + + ": Took " + + (endTime - startTime) + + " msecs for " + + numElements + + " values"); + } + return new Tuple2<>(value % numKeys, 1L); + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java new file mode 100644 index 0000000..5708a7a --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java @@ -0,0 +1,110 @@ +/* + * 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.examples.windowing; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; +import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +/** + * An example of session windowing that keys events by ID and groups and counts them in session with + * gaps of 3 milliseconds. + */ +public class SessionWindowing { + + public static void main(String[] args) throws Exception { + + final ParameterTool params = ParameterTool.fromArgs(args); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.getConfig().setGlobalJobParameters(params); + env.setParallelism(1); + + final boolean fileOutput = params.has("output"); + + final List> input = new ArrayList<>(); + + input.add(new Tuple3<>("a", 1L, 1)); + input.add(new Tuple3<>("b", 1L, 1)); + input.add(new Tuple3<>("b", 3L, 1)); + input.add(new Tuple3<>("b", 5L, 1)); + input.add(new Tuple3<>("c", 6L, 1)); + // We expect to detect the session "a" earlier than this point (the old + // functionality can only detect here when the next starts) + input.add(new Tuple3<>("a", 10L, 1)); + // We expect to detect session "b" and "c" at this point as well + input.add(new Tuple3<>("c", 11L, 1)); + + GeneratorFunction> dataGenerator = + index -> input.get(index.intValue()); + DataGeneratorSource> generatorSource = + new DataGeneratorSource<>( + dataGenerator, + input.size(), + TypeInformation.of(new TypeHint>() {})); + + DataStream> source = + env.fromSource( + generatorSource, + WatermarkStrategy.>forMonotonousTimestamps() + .withTimestampAssigner((event, timestamp) -> event.f1), + "Generated data source"); + + // We create sessions for each id with max timeout of 3 time units + DataStream> aggregated = + source.keyBy(value -> value.f0) + .window(EventTimeSessionWindows.withGap(Duration.ofMillis(3L))) + .sum(2); + + if (fileOutput) { + aggregated + .sinkTo( + FileSink.>forRowFormat( + new Path(params.get("output")), + new SimpleStringEncoder<>()) + .withRollingPolicy( + DefaultRollingPolicy.builder() + .withMaxPartSize(MemorySize.ofMebiBytes(1)) + .withRolloverInterval(Duration.ofSeconds(10)) + .build()) + .build()) + .name("output"); + } else { + System.out.println("Printing result to stdout. Use --output to specify output path."); + aggregated.print(); + } + + env.execute(); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java new file mode 100644 index 0000000..135e48f --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java @@ -0,0 +1,194 @@ +/* + * 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.examples.windowing; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.util.ratelimit.GuavaRateLimiter; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.TextLineInputFormat; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; +import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger; +import org.apache.flink.streaming.examples.windowing.util.CarGeneratorFunction; +import org.apache.flink.streaming.examples.wordcount.util.CLI; + +import java.time.Duration; + +/** + * An example of grouped stream windowing where different eviction and trigger policies can be used. + * A source fetches events from cars containing their id, their current speed (kmh), overall elapsed + * distance (m) and a timestamp. The streaming example triggers the top speed of each car every x + * meters elapsed for the last y seconds. + */ +public class TopSpeedWindowing { + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + final CLI params = CLI.fromArgs(args); + + // Create the execution environment. This is the main entrypoint + // to building a Flink application. + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // Apache Flink’s unified approach to stream and batch processing means that a DataStream + // application executed over bounded input will produce the same final results regardless + // of the configured execution mode. It is important to note what final means here: a job + // executing in STREAMING mode might produce incremental updates (think upserts in + // a database) while a BATCH job would only produce one final result at the end. The final + // result will be the same if interpreted correctly, but getting there can be different. + // + // The “classic” execution behavior of the DataStream API is called STREAMING execution + // mode. Applications should use streaming execution for unbounded jobs that require + // continuous incremental processing and are expected to stay online indefinitely. + // + // By enabling BATCH execution, we allow Flink to apply additional optimizations that we + // can only do when we know that our input is bounded. For example, different + // join/aggregation strategies can be used, in addition to a different shuffle + // implementation that allows more efficient task scheduling and failure recovery behavior. + // + // By setting the runtime mode to AUTOMATIC, Flink will choose BATCH if all sources + // are bounded and otherwise STREAMING. + env.setRuntimeMode(params.getExecutionMode()); + + // This optional step makes the input parameters + // available in the Flink UI. + env.getConfig().setGlobalJobParameters(params); + + SingleOutputStreamOperator> carData; + if (params.getInputs().isPresent()) { + // Create a new file source that will read files from a given set of directories. + // Each file will be processed as plain text and split based on newlines. + FileSource.FileSourceBuilder builder = + FileSource.forRecordStreamFormat( + new TextLineInputFormat(), params.getInputs().get()); + + // If a discovery interval is provided, the source will + // continuously watch the given directories for new files. + params.getDiscoveryInterval().ifPresent(builder::monitorContinuously); + + carData = + env.fromSource(builder.build(), WatermarkStrategy.noWatermarks(), "file-input") + .map(new ParseCarData()) + .name("parse-input"); + } else { + CarGeneratorFunction carGenerator = new CarGeneratorFunction(2); + DataGeneratorSource> carGeneratorSource = + new DataGeneratorSource<>( + carGenerator, + Long.MAX_VALUE, + parallelismIgnored -> new GuavaRateLimiter(10), + TypeInformation.of( + new TypeHint>() {})); + carData = + env.fromSource( + carGeneratorSource, + WatermarkStrategy.noWatermarks(), + "Car data generator source"); + carData.setParallelism(1); + } + + int evictionSec = 10; + double triggerMeters = 50; + DataStream> topSpeeds = + carData.assignTimestampsAndWatermarks( + WatermarkStrategy + .> + forMonotonousTimestamps() + .withTimestampAssigner((car, ts) -> car.f3)) + .keyBy(value -> value.f0) + .window(GlobalWindows.create()) + .evictor(TimeEvictor.of(Duration.ofSeconds(evictionSec))) + .trigger( + DeltaTrigger.of( + triggerMeters, + new DeltaFunction< + Tuple4>() { + private static final long serialVersionUID = 1L; + + @Override + public double getDelta( + Tuple4 + oldDataPoint, + Tuple4 + newDataPoint) { + return newDataPoint.f2 - oldDataPoint.f2; + } + }, + carData.getType() + .createSerializer( + env.getConfig().getSerializerConfig()))) + .maxBy(1); + + if (params.getOutput().isPresent()) { + // Given an output directory, Flink will write the results to a file + // using a simple string encoding. In a production environment, this might + // be something more structured like CSV, Avro, JSON, or Parquet. + topSpeeds + .sinkTo( + FileSink.>forRowFormat( + params.getOutput().get(), new SimpleStringEncoder<>()) + .withRollingPolicy( + DefaultRollingPolicy.builder() + .withMaxPartSize(MemorySize.ofMebiBytes(1)) + .withRolloverInterval(Duration.ofSeconds(10)) + .build()) + .build()) + .name("file-sink"); + } else { + topSpeeds.print(); + } + + env.execute("CarTopSpeedWindowingExample"); + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + private static class ParseCarData + extends RichMapFunction> { + private static final long serialVersionUID = 1L; + + @Override + public Tuple4 map(String record) { + String rawData = record.substring(1, record.length() - 1); + String[] data = rawData.split(","); + return new Tuple4<>( + Integer.valueOf(data[0]), + Integer.valueOf(data[1]), + Double.valueOf(data[2]), + Long.valueOf(data[3])); + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java new file mode 100644 index 0000000..6075edf --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java @@ -0,0 +1,153 @@ +/* + * 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.examples.windowing; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.TextLineInputFormat; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; +import org.apache.flink.streaming.examples.wordcount.WordCount; +import org.apache.flink.streaming.examples.wordcount.util.CLI; +import org.apache.flink.streaming.examples.wordcount.util.WordCountData; + +import java.time.Duration; + +/** + * Implements a windowed version of the streaming "WordCount" program. + * + *

The input is a plain text file with lines separated by newline characters. + * + *

Usage: + * WordCount --input <path> --output <path> --window <n> --slide <n> + *
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}. + * + *

This example shows how to: + * + *

    + *
  • write a simple Flink Streaming program, + *
  • use tuple data types, + *
  • use basic windowing abstractions. + *
+ */ +public class WindowWordCount { + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + final CLI params = CLI.fromArgs(args); + + // Create the execution environment. This is the main entrypoint + // to building a Flink application. + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // Apache Flink’s unified approach to stream and batch processing means that a DataStream + // application executed over bounded input will produce the same final results regardless + // of the configured execution mode. It is important to note what final means here: a job + // executing in STREAMING mode might produce incremental updates (think upserts in + // a database) while a BATCH job would only produce one final result at the end. The final + // result will be the same if interpreted correctly, but getting there can be different. + // + // The “classic” execution behavior of the DataStream API is called STREAMING execution + // mode. Applications should use streaming execution for unbounded jobs that require + // continuous incremental processing and are expected to stay online indefinitely. + // + // By enabling BATCH execution, we allow Flink to apply additional optimizations that we + // can only do when we know that our input is bounded. For example, different + // join/aggregation strategies can be used, in addition to a different shuffle + // implementation that allows more efficient task scheduling and failure recovery behavior. + // + // By setting the runtime mode to AUTOMATIC, Flink will choose BATCH if all sources + // are bounded and otherwise STREAMING. + env.setRuntimeMode(params.getExecutionMode()); + + // This optional step makes the input parameters + // available in the Flink UI. + env.getConfig().setGlobalJobParameters(params); + + DataStream text; + if (params.getInputs().isPresent()) { + // Create a new file source that will read files from a given set of directories. + // Each file will be processed as plain text and split based on newlines. + FileSource.FileSourceBuilder builder = + FileSource.forRecordStreamFormat( + new TextLineInputFormat(), params.getInputs().get()); + + // If a discovery interval is provided, the source will + // continuously watch the given directories for new files. + params.getDiscoveryInterval().ifPresent(builder::monitorContinuously); + + text = env.fromSource(builder.build(), WatermarkStrategy.noWatermarks(), "file-input"); + } else { + text = env.fromData(WordCountData.WORDS).name("in-memory-input"); + } + + int windowSize = params.getInt("window").orElse(250); + int slideSize = params.getInt("slide").orElse(150); + + DataStream> counts = + // The text lines read from the source are split into words + // using a user-defined function. The tokenizer, implemented below, + // will output each words as a (2-tuple) containing (word, 1) + text.flatMap(new WordCount.Tokenizer()) + .name("tokenizer") + // keyBy groups tuples based on the "0" field, the word. + // Using a keyBy allows performing aggregations and other + // stateful transformations over data on a per-key basis. + // This is similar to a GROUP BY clause in a SQL query. + .keyBy(value -> value.f0) + // create windows of windowSize records slided every slideSize records + .countWindow(windowSize, slideSize) + // For each key, we perform a simple sum of the "1" field, the count. + // If the input data set is bounded, sum will output a final count for + // each word. If it is unbounded, it will continuously output updates + // each time it sees a new instance of each word in the stream. + .sum(1) + .name("counter"); + + if (params.getOutput().isPresent()) { + // Given an output directory, Flink will write the results to a file + // using a simple string encoding. In a production environment, this might + // be something more structured like CSV, Avro, JSON, or Parquet. + counts.sinkTo( + FileSink.>forRowFormat( + params.getOutput().get(), new SimpleStringEncoder<>()) + .withRollingPolicy( + DefaultRollingPolicy.builder() + .withMaxPartSize(MemorySize.ofMebiBytes(1)) + .withRolloverInterval(Duration.ofSeconds(10)) + .build()) + .build()) + .name("file-sink"); + } else { + counts.print().name("print-sink"); + } + + // Apache Flink applications are composed lazily. Calling execute + // submits the Job and begins processing. + env.execute("WindowWordCount"); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.java new file mode 100644 index 0000000..eb7cecd --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.java @@ -0,0 +1,84 @@ +/* + * 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.examples.windowing.util; + +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.connector.datagen.source.GeneratorFunction; + +import java.util.Arrays; +import java.util.Random; + +/** + * A generator function for simulating car data. + * + *

This generator function generates a stream of car data in a form of a four-element tuple. The + * data includes the car's ID, its speed in kilometers per hour, the distance it has traveled in + * meters, and the timestamp of the data generation. The speed and distance of each car are randomly + * updated in each invocation of the {@link #map(Long)} method. + */ +public class CarGeneratorFunction + implements GeneratorFunction> { + + private static final long serialVersionUID = 1L; + // in kilometers per hour + private final int[] speeds; + // in meters + private final double[] distances; + // in milliseconds + private final long[] lastUpdate; + private int nextCar; + + private static final int MILLIS_IN_HOUR = 1000 * 60 * 60; + private static final double HOURS_IN_MILLI = 1d / MILLIS_IN_HOUR; + private static final int METERS_IN_KILOMETER = 1000; + + private final Random rand = new Random(); + + // Previous version (CarSource) was overestimating the speed. This factor is used to preserve + // the original behaviour of the example. + private static final int COMPAT_FACTOR = 10; + + public CarGeneratorFunction(int numOfCars) { + speeds = new int[numOfCars]; + distances = new double[numOfCars]; + lastUpdate = new long[numOfCars]; + Arrays.fill(speeds, 50); + Arrays.fill(distances, 0d); + Arrays.fill(lastUpdate, 0); + } + + @Override + public Tuple4 map(Long ignoredIndex) throws Exception { + if (rand.nextBoolean()) { + speeds[nextCar] = Math.min(100, speeds[nextCar] + 5); + } else { + speeds[nextCar] = Math.max(0, speeds[nextCar] - 5); + } + long now = System.currentTimeMillis(); + long timeDiffMillis = lastUpdate[nextCar] == 0 ? 0 : now - lastUpdate[nextCar]; + lastUpdate[nextCar] = now; + distances[nextCar] += + speeds[nextCar] + * (timeDiffMillis * HOURS_IN_MILLI) + * METERS_IN_KILOMETER + * COMPAT_FACTOR; + nextCar = (++nextCar) % speeds.length; + return new Tuple4<>(nextCar, speeds[nextCar], distances[nextCar], now); + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java new file mode 100644 index 0000000..955c6aa --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java @@ -0,0 +1,27 @@ +/* + * 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.examples.windowing.util; + +/** Data for SessionWindowingITCase. */ +public class SessionWindowingData { + + public static final String EXPECTED = + "(a,1,1)\n" + "(c,6,1)\n" + "(c,11,1)\n" + "(b,1,3)\n" + "(a,10,1)"; + + private SessionWindowingData() {} +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java new file mode 100644 index 0000000..ab02b3b --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java @@ -0,0 +1,372 @@ +/* + * 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.examples.windowing.util; + +/** Data for TopSpeedWindowingExampleITCase. */ +public class TopSpeedWindowingExampleData { + + public static final String CAR_DATA = + "(0,55,15.277777777777777,1424951918630)\n" + + "(1,45,12.5,1424951918632)\n" + + "(0,50,29.166666666666664,1424951919632)\n" + + "(1,50,26.38888888888889,1424951919632)\n" + + "(0,55,44.44444444444444,1424951920633)\n" + + "(1,45,38.888888888888886,1424951920633)\n" + + "(0,50,58.33333333333333,1424951921634)\n" + + "(1,40,50.0,1424951921634)\n" + + "(0,55,73.6111111111111,1424951922634)\n" + + "(1,35,59.72222222222222,1424951922634)\n" + + "(0,60,90.27777777777777,1424951923634)\n" + + "(1,40,70.83333333333333,1424951923634)\n" + + "(0,65,108.33333333333333,1424951924635)\n" + + "(1,35,80.55555555555554,1424951924635)\n" + + "(0,60,125.0,1424951925635)\n" + + "(1,40,91.66666666666666,1424951925635)\n" + + "(0,55,140.27777777777777,1424951926635)\n" + + "(1,45,104.16666666666666,1424951926636)\n" + + "(0,60,156.94444444444443,1424951927636)\n" + + "(1,50,118.05555555555554,1424951927636)\n" + + "(0,55,172.2222222222222,1424951928636)\n" + + "(1,45,130.55555555555554,1424951928636)\n" + + "(0,50,186.1111111111111,1424951929636)\n" + + "(1,50,144.44444444444443,1424951929637)\n" + + "(0,55,201.38888888888886,1424951930637)\n" + + "(1,55,159.7222222222222,1424951930637)\n" + + "(0,60,218.05555555555551,1424951931637)\n" + + "(1,60,176.38888888888886,1424951931637)\n" + + "(0,55,233.3333333333333,1424951932637)\n" + + "(1,65,194.4444444444444,1424951932638)\n" + + "(0,50,247.22222222222217,1424951933638)\n" + + "(1,70,213.88888888888886,1424951933638)\n" + + "(0,45,259.7222222222222,1424951934638)\n" + + "(1,65,231.9444444444444,1424951934638)\n" + + "(0,50,273.6111111111111,1424951935638)\n" + + "(1,70,251.38888888888886,1424951935639)\n" + + "(0,55,288.88888888888886,1424951936639)\n" + + "(1,75,272.2222222222222,1424951936639)\n" + + "(0,50,302.77777777777777,1424951937639)\n" + + "(1,70,291.66666666666663,1424951937639)\n" + + "(0,45,315.27777777777777,1424951938640)\n" + + "(1,65,309.7222222222222,1424951938640)\n" + + "(0,50,329.1666666666667,1424951939640)\n" + + "(1,70,329.16666666666663,1424951939640)\n" + + "(0,55,344.44444444444446,1424951940640)\n" + + "(1,65,347.2222222222222,1424951940640)\n" + + "(0,50,358.33333333333337,1424951941641)\n" + + "(1,70,366.66666666666663,1424951941641)\n" + + "(0,55,373.61111111111114,1424951942641)\n" + + "(1,65,384.7222222222222,1424951942641)\n" + + "(0,50,387.50000000000006,1424951943641)\n" + + "(1,70,404.16666666666663,1424951943641)\n" + + "(0,45,400.00000000000006,1424951944642)\n" + + "(1,65,422.2222222222222,1424951944642)\n" + + "(0,50,413.88888888888897,1424951945642)\n" + + "(1,60,438.88888888888886,1424951945642)\n" + + "(0,45,426.38888888888897,1424951946642)\n" + + "(1,65,456.9444444444444,1424951946642)\n" + + "(0,40,437.50000000000006,1424951947643)\n" + + "(1,70,476.38888888888886,1424951947643)\n" + + "(0,45,450.00000000000006,1424951948643)\n" + + "(1,75,497.2222222222222,1424951948643)\n" + + "(0,40,461.11111111111114,1424951949643)\n" + + "(1,80,519.4444444444443,1424951949644)\n" + + "(0,45,473.61111111111114,1424951950644)\n" + + "(1,75,540.2777777777777,1424951950644)\n" + + "(0,50,487.50000000000006,1424951951644)\n" + + "(1,80,562.4999999999999,1424951951644)\n" + + "(0,45,500.00000000000006,1424951952644)\n" + + "(1,85,586.111111111111,1424951952645)\n" + + "(0,40,511.11111111111114,1424951953645)\n" + + "(1,80,608.3333333333331,1424951953645)\n" + + "(0,35,520.8333333333334,1424951954645)\n" + + "(1,75,629.1666666666665,1424951954645)\n" + + "(0,40,531.9444444444445,1424951955645)\n" + + "(1,70,648.611111111111,1424951955646)\n" + + "(0,45,544.4444444444445,1424951956646)\n" + + "(1,75,669.4444444444443,1424951956646)\n" + + "(0,50,558.3333333333334,1424951957646)\n" + + "(1,80,691.6666666666665,1424951957646)\n" + + "(0,55,573.6111111111112,1424951958646)\n" + + "(1,85,715.2777777777776,1424951958647)\n" + + "(0,60,590.2777777777778,1424951959647)\n" + + "(1,80,737.4999999999998,1424951959647)\n" + + "(0,65,608.3333333333334,1424951960647)\n" + + "(1,85,761.1111111111109,1424951960647)\n" + + "(0,70,627.7777777777778,1424951961647)\n" + + "(1,80,783.333333333333,1424951961648)\n" + + "(0,75,648.6111111111112,1424951962648)\n" + + "(1,85,806.9444444444441,1424951962648)\n" + + "(0,80,670.8333333333334,1424951963648)\n" + + "(1,90,831.9444444444441,1424951963648)\n" + + "(0,75,691.6666666666667,1424951964649)\n" + + "(1,95,858.333333333333,1424951964649)\n" + + "(0,70,711.1111111111112,1424951965649)\n" + + "(1,90,883.333333333333,1424951965649)\n" + + "(0,75,731.9444444444446,1424951966649)\n" + + "(1,95,909.722222222222,1424951966649)\n" + + "(0,70,751.388888888889,1424951967649)\n" + + "(1,100,937.4999999999998,1424951967650)\n" + + "(0,75,772.2222222222224,1424951968650)\n" + + "(1,100,965.2777777777776,1424951968650)\n" + + "(0,80,794.4444444444446,1424951969650)\n" + + "(1,100,993.0555555555554,1424951969650)\n" + + "(0,75,815.2777777777779,1424951970651)\n" + + "(1,100,1020.8333333333333,1424951970651)\n" + + "(0,80,837.5000000000001,1424951971651)\n" + + "(1,100,1048.611111111111,1424951971651)\n" + + "(0,85,861.1111111111112,1424951972651)\n" + + "(1,100,1076.388888888889,1424951972651)\n" + + "(0,80,883.3333333333334,1424951973652)\n" + + "(1,95,1102.7777777777778,1424951973652)\n" + + "(0,75,904.1666666666667,1424951974652)\n" + + "(1,100,1130.5555555555557,1424951974652)\n" + + "(0,70,923.6111111111112,1424951975652)\n" + + "(1,100,1158.3333333333335,1424951975652)\n" + + "(0,75,944.4444444444446,1424951976653)\n" + + "(1,100,1186.1111111111113,1424951976653)\n" + + "(0,80,966.6666666666667,1424951977653)\n" + + "(1,95,1212.5000000000002,1424951977653)\n" + + "(0,75,987.5000000000001,1424951978653)\n" + + "(1,100,1240.277777777778,1424951978653)\n" + + "(0,80,1009.7222222222223,1424951979654)\n" + + "(1,100,1268.0555555555559,1424951979654)\n" + + "(0,85,1033.3333333333335,1424951980654)\n" + + "(1,100,1295.8333333333337,1424951980654)\n" + + "(0,90,1058.3333333333335,1424951981654)\n" + + "(1,100,1323.6111111111115,1424951981654)\n" + + "(0,85,1081.9444444444446,1424951982655)\n" + + "(1,100,1351.3888888888894,1424951982655)\n" + + "(0,90,1106.9444444444446,1424951983655)\n" + + "(1,100,1379.1666666666672,1424951983655)\n" + + "(0,95,1133.3333333333335,1424951984655)\n" + + "(1,100,1406.944444444445,1424951984656)\n" + + "(0,90,1158.3333333333335,1424951985656)\n" + + "(1,95,1433.333333333334,1424951985656)\n" + + "(0,95,1184.7222222222224,1424951986656)\n" + + "(1,90,1458.333333333334,1424951986656)\n" + + "(0,90,1209.7222222222224,1424951987656)\n" + + "(1,95,1484.7222222222229,1424951987657)\n" + + "(0,85,1233.3333333333335,1424951988657)\n" + + "(1,90,1509.7222222222229,1424951988657)\n" + + "(0,80,1255.5555555555557,1424951989657)\n" + + "(1,95,1536.1111111111118,1424951989657)\n" + + "(0,85,1279.1666666666667,1424951990657)\n" + + "(1,100,1563.8888888888896,1424951990658)\n" + + "(0,90,1304.1666666666667,1424951991658)\n" + + "(1,95,1590.2777777777785,1424951991658)\n" + + "(0,95,1330.5555555555557,1424951992658)\n" + + "(1,90,1615.2777777777785,1424951992658)\n" + + "(0,100,1358.3333333333335,1424951993659)\n" + + "(1,95,1641.6666666666674,1424951993659)\n" + + "(0,100,1386.1111111111113,1424951994659)\n" + + "(1,100,1669.4444444444453,1424951994659)\n" + + "(0,95,1412.5000000000002,1424951995659)\n" + + "(1,95,1695.8333333333342,1424951995660)\n" + + "(0,100,1440.277777777778,1424951996660)\n" + + "(1,90,1720.8333333333342,1424951996660)\n" + + "(0,100,1468.0555555555559,1424951997660)\n" + + "(1,85,1744.4444444444453,1424951997660)\n" + + "(0,95,1494.4444444444448,1424951998660)\n" + + "(1,80,1766.6666666666674,1424951998661)\n" + + "(0,100,1522.2222222222226,1424951999661)\n" + + "(1,75,1787.5000000000007,1424951999661)\n" + + "(0,95,1548.6111111111115,1424952000661)\n" + + "(1,80,1809.7222222222229,1424952000661)\n" + + "(0,90,1573.6111111111115,1424952001662)\n" + + "(1,75,1830.555555555556,1424952001662)\n" + + "(0,95,1600.0000000000005,1424952002662)\n" + + "(1,80,1852.7777777777783,1424952002662)\n" + + "(0,100,1627.7777777777783,1424952003662)\n" + + "(1,85,1876.3888888888894,1424952003662)\n" + + "(0,100,1655.555555555556,1424952004663)\n" + + "(1,80,1898.6111111111115,1424952004663)\n" + + "(0,95,1681.944444444445,1424952005663)\n" + + "(1,85,1922.2222222222226,1424952005663)\n" + + "(0,100,1709.7222222222229,1424952006663)\n" + + "(1,90,1947.2222222222226,1424952006664)\n" + + "(0,100,1737.5000000000007,1424952007664)\n" + + "(1,95,1973.6111111111115,1424952007664)\n" + + "(0,95,1763.8888888888896,1424952008664)\n" + + "(1,90,1998.6111111111115,1424952008664)\n" + + "(0,100,1791.6666666666674,1424952009664)\n" + + "(1,85,2022.2222222222226,1424952009665)\n" + + "(0,95,1818.0555555555563,1424952010665)\n" + + "(1,80,2044.4444444444448,1424952010665)\n" + + "(0,90,1843.0555555555563,1424952011665)\n" + + "(1,75,2065.2777777777783,1424952011665)\n" + + "(0,95,1869.4444444444453,1424952012666)\n" + + "(1,80,2087.5000000000005,1424952012666)\n" + + "(0,100,1897.222222222223,1424952013666)\n" + + "(1,85,2111.1111111111118,1424952013666)\n" + + "(0,95,1923.611111111112,1424952014666)\n" + + "(1,90,2136.1111111111118,1424952014666)\n" + + "(0,100,1951.3888888888898,1424952015667)\n" + + "(1,85,2159.722222222223,1424952015667)\n" + + "(0,95,1977.7777777777787,1424952016667)\n" + + "(1,90,2184.722222222223,1424952016667)\n" + + "(0,100,2005.5555555555566,1424952017667)\n" + + "(1,95,2211.1111111111118,1424952017668)"; + + public static final String TOP_SPEEDS = + "(0,55,15.277777777777777,1424951918630)\n" + + "(1,50,26.38888888888889,1424951919632)\n" + + "(0,65,108.33333333333333,1424951924635)\n" + + "(1,50,26.38888888888889,1424951919632)\n" + + "(0,65,108.33333333333333,1424951924635)\n" + + "(1,65,194.4444444444444,1424951932638)\n" + + "(0,65,108.33333333333333,1424951924635)\n" + + "(1,70,213.88888888888886,1424951933638)\n" + + "(0,60,218.05555555555551,1424951931637)\n" + + "(1,75,272.2222222222222,1424951936639)\n" + + "(0,55,233.3333333333333,1424951932637)\n" + + "(1,75,272.2222222222222,1424951936639)\n" + + "(1,75,272.2222222222222,1424951936639)\n" + + "(0,55,288.88888888888886,1424951936639)\n" + + "(1,70,329.16666666666663,1424951939640)\n" + + "(0,55,373.61111111111114,1424951942641)\n" + + "(1,80,519.4444444444443,1424951949644)\n" + + "(1,85,586.111111111111,1424951952645)\n" + + "(0,50,487.50000000000006,1424951951644)\n" + + "(1,85,586.111111111111,1424951952645)\n" + + "(0,60,590.2777777777778,1424951959647)\n" + + "(1,85,586.111111111111,1424951952645)\n" + + "(0,75,648.6111111111112,1424951962648)\n" + + "(1,85,715.2777777777776,1424951958647)\n" + + "(1,95,858.333333333333,1424951964649)\n" + + "(0,80,670.8333333333334,1424951963648)\n" + + "(1,95,858.333333333333,1424951964649)\n" + + "(0,80,670.8333333333334,1424951963648)\n" + + "(1,100,937.4999999999998,1424951967650)\n" + + "(1,100,937.4999999999998,1424951967650)\n" + + "(0,80,670.8333333333334,1424951963648)\n" + + "(1,100,937.4999999999998,1424951967650)\n" + + "(0,85,861.1111111111112,1424951972651)\n" + + "(1,100,937.4999999999998,1424951967650)\n" + + "(1,100,937.4999999999998,1424951967650)\n" + + "(0,85,861.1111111111112,1424951972651)\n" + + "(1,100,993.0555555555554,1424951969650)\n" + + "(0,85,861.1111111111112,1424951972651)\n" + + "(1,100,1048.611111111111,1424951971651)\n" + + "(1,100,1130.5555555555557,1424951974652)\n" + + "(0,90,1058.3333333333335,1424951981654)\n" + + "(1,100,1158.3333333333335,1424951975652)\n" + + "(0,95,1133.3333333333335,1424951984655)\n" + + "(1,100,1240.277777777778,1424951978653)\n" + + "(0,95,1133.3333333333335,1424951984655)\n" + + "(1,100,1268.0555555555559,1424951979654)\n" + + "(0,95,1133.3333333333335,1424951984655)\n" + + "(1,100,1323.6111111111115,1424951981654)\n" + + "(0,95,1133.3333333333335,1424951984655)\n" + + "(1,100,1379.1666666666672,1424951983655)\n" + + "(0,100,1358.3333333333335,1424951993659)\n" + + "(1,100,1563.8888888888896,1424951990658)\n" + + "(0,100,1358.3333333333335,1424951993659)\n" + + "(1,100,1563.8888888888896,1424951990658)\n" + + "(0,100,1358.3333333333335,1424951993659)\n" + + "(1,100,1563.8888888888896,1424951990658)\n" + + "(0,100,1358.3333333333335,1424951993659)\n" + + "(0,100,1358.3333333333335,1424951993659)\n" + + "(1,100,1669.4444444444453,1424951994659)\n" + + "(0,100,1440.277777777778,1424951996660)\n" + + "(1,90,1720.8333333333342,1424951996660)\n" + + "(0,100,1468.0555555555559,1424951997660)\n" + + "(1,95,1973.6111111111115,1424952007664)\n" + + "(0,100,1522.2222222222226,1424951999661)\n" + + "(0,100,1627.7777777777783,1424952003662)\n" + + "(1,95,1973.6111111111115,1424952007664)\n" + + "(0,100,1627.7777777777783,1424952003662)\n" + + "(1,95,1973.6111111111115,1424952007664)\n" + + "(0,100,1709.7222222222229,1424952006663)\n" + + "(0,100,1737.5000000000007,1424952007664)\n" + + "(1,95,1973.6111111111115,1424952007664)\n"; + + public static final String TOP_CASE_CLASS_SPEEDS = + "CarEvent(0,55,15.277777777777777,1424951918630)\n" + + "CarEvent(1,50,26.38888888888889,1424951919632)\n" + + "CarEvent(0,65,108.33333333333333,1424951924635)\n" + + "CarEvent(1,50,26.38888888888889,1424951919632)\n" + + "CarEvent(0,65,108.33333333333333,1424951924635)\n" + + "CarEvent(1,65,194.4444444444444,1424951932638)\n" + + "CarEvent(0,65,108.33333333333333,1424951924635)\n" + + "CarEvent(1,70,213.88888888888886,1424951933638)\n" + + "CarEvent(0,60,218.05555555555551,1424951931637)\n" + + "CarEvent(1,75,272.2222222222222,1424951936639)\n" + + "CarEvent(0,55,233.3333333333333,1424951932637)\n" + + "CarEvent(1,75,272.2222222222222,1424951936639)\n" + + "CarEvent(1,75,272.2222222222222,1424951936639)\n" + + "CarEvent(0,55,288.88888888888886,1424951936639)\n" + + "CarEvent(1,70,329.16666666666663,1424951939640)\n" + + "CarEvent(0,55,373.61111111111114,1424951942641)\n" + + "CarEvent(1,80,519.4444444444443,1424951949644)\n" + + "CarEvent(1,85,586.111111111111,1424951952645)\n" + + "CarEvent(0,50,487.50000000000006,1424951951644)\n" + + "CarEvent(1,85,586.111111111111,1424951952645)\n" + + "CarEvent(0,60,590.2777777777778,1424951959647)\n" + + "CarEvent(1,85,586.111111111111,1424951952645)\n" + + "CarEvent(0,75,648.6111111111112,1424951962648)\n" + + "CarEvent(1,85,715.2777777777776,1424951958647)\n" + + "CarEvent(1,95,858.333333333333,1424951964649)\n" + + "CarEvent(0,80,670.8333333333334,1424951963648)\n" + + "CarEvent(1,95,858.333333333333,1424951964649)\n" + + "CarEvent(0,80,670.8333333333334,1424951963648)\n" + + "CarEvent(1,100,937.4999999999998,1424951967650)\n" + + "CarEvent(1,100,937.4999999999998,1424951967650)\n" + + "CarEvent(0,80,670.8333333333334,1424951963648)\n" + + "CarEvent(1,100,937.4999999999998,1424951967650)\n" + + "CarEvent(0,85,861.1111111111112,1424951972651)\n" + + "CarEvent(1,100,937.4999999999998,1424951967650)\n" + + "CarEvent(1,100,937.4999999999998,1424951967650)\n" + + "CarEvent(0,85,861.1111111111112,1424951972651)\n" + + "CarEvent(1,100,993.0555555555554,1424951969650)\n" + + "CarEvent(0,85,861.1111111111112,1424951972651)\n" + + "CarEvent(1,100,1048.611111111111,1424951971651)\n" + + "CarEvent(1,100,1130.5555555555557,1424951974652)\n" + + "CarEvent(0,90,1058.3333333333335,1424951981654)\n" + + "CarEvent(1,100,1158.3333333333335,1424951975652)\n" + + "CarEvent(0,95,1133.3333333333335,1424951984655)\n" + + "CarEvent(1,100,1240.277777777778,1424951978653)\n" + + "CarEvent(0,95,1133.3333333333335,1424951984655)\n" + + "CarEvent(1,100,1268.0555555555559,1424951979654)\n" + + "CarEvent(0,95,1133.3333333333335,1424951984655)\n" + + "CarEvent(1,100,1323.6111111111115,1424951981654)\n" + + "CarEvent(0,95,1133.3333333333335,1424951984655)\n" + + "CarEvent(1,100,1379.1666666666672,1424951983655)\n" + + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" + + "CarEvent(1,100,1563.8888888888896,1424951990658)\n" + + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" + + "CarEvent(1,100,1563.8888888888896,1424951990658)\n" + + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" + + "CarEvent(1,100,1563.8888888888896,1424951990658)\n" + + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" + + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" + + "CarEvent(1,100,1669.4444444444453,1424951994659)\n" + + "CarEvent(0,100,1440.277777777778,1424951996660)\n" + + "CarEvent(1,90,1720.8333333333342,1424951996660)\n" + + "CarEvent(0,100,1468.0555555555559,1424951997660)\n" + + "CarEvent(1,95,1973.6111111111115,1424952007664)\n" + + "CarEvent(0,100,1522.2222222222226,1424951999661)\n" + + "CarEvent(0,100,1627.7777777777783,1424952003662)\n" + + "CarEvent(1,95,1973.6111111111115,1424952007664)\n" + + "CarEvent(0,100,1627.7777777777783,1424952003662)\n" + + "CarEvent(1,95,1973.6111111111115,1424952007664)\n" + + "CarEvent(0,100,1709.7222222222229,1424952006663)\n" + + "CarEvent(0,100,1737.5000000000007,1424952007664)\n" + + "CarEvent(1,95,1973.6111111111115,1424952007664)\n"; + + private TopSpeedWindowingExampleData() {} +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java new file mode 100644 index 0000000..1c545b6 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java @@ -0,0 +1,187 @@ +/* + * 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.examples.wordcount; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.TextLineInputFormat; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; +import org.apache.flink.streaming.examples.wordcount.util.CLI; +import org.apache.flink.streaming.examples.wordcount.util.WordCountData; +import org.apache.flink.util.Collector; + +import java.time.Duration; + +/** + * Implements the "WordCount" program that computes a simple word occurrence histogram over text + * files. This Job can be executed in both streaming and batch execution modes. + * + *

The input is a [list of] plain text file[s] with lines separated by a newline character. + * + *

Usage: + * + *

    + *
  • --input <path>A list of input files and / or directories to read. If no + * input is provided, the program is run with default data from {@link WordCountData}. + *
  • --discovery-interval <duration>Turns the file reader into a continuous + * source that will monitor the provided input directories every interval and read any new + * files. + *
  • --output <path>The output directory where the Job will write the + * results. If no output path is provided, the Job will print the results to stdout + * . + *
  • --execution-mode <mode>The execution mode (BATCH, STREAMING, or + * AUTOMATIC) of this pipeline. + *
+ * + *

This example shows how to: + * + *

    + *
  • Write a simple Flink DataStream program + *
  • Use tuple data types + *
  • Write and use a user-defined function + *
+ */ +public class WordCount { + + // ************************************************************************* + // PROGRAM + // ************************************************************************* + + public static void main(String[] args) throws Exception { + final CLI params = CLI.fromArgs(args); + + // Create the execution environment. This is the main entrypoint + // to building a Flink application. + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // Apache Flink’s unified approach to stream and batch processing means that a DataStream + // application executed over bounded input will produce the same final results regardless + // of the configured execution mode. It is important to note what final means here: a job + // executing in STREAMING mode might produce incremental updates (think upserts in + // a database) while in BATCH mode, it would only produce one final result at the end. The + // final result will be the same if interpreted correctly, but getting there can be + // different. + // + // The “classic” execution behavior of the DataStream API is called STREAMING execution + // mode. Applications should use streaming execution for unbounded jobs that require + // continuous incremental processing and are expected to stay online indefinitely. + // + // By enabling BATCH execution, we allow Flink to apply additional optimizations that we + // can only do when we know that our input is bounded. For example, different + // join/aggregation strategies can be used, in addition to a different shuffle + // implementation that allows more efficient task scheduling and failure recovery behavior. + // + // By setting the runtime mode to AUTOMATIC, Flink will choose BATCH if all sources + // are bounded and otherwise STREAMING. + env.setRuntimeMode(params.getExecutionMode()); + + // This optional step makes the input parameters + // available in the Flink UI. + env.getConfig().setGlobalJobParameters(params); + + DataStream text; + if (params.getInputs().isPresent()) { + // Create a new file source that will read files from a given set of directories. + // Each file will be processed as plain text and split based on newlines. + FileSource.FileSourceBuilder builder = + FileSource.forRecordStreamFormat( + new TextLineInputFormat(), params.getInputs().get()); + + // If a discovery interval is provided, the source will + // continuously watch the given directories for new files. + params.getDiscoveryInterval().ifPresent(builder::monitorContinuously); + + text = env.fromSource(builder.build(), WatermarkStrategy.noWatermarks(), "file-input"); + } else { + text = env.fromData(WordCountData.WORDS).name("in-memory-input"); + } + + DataStream> counts = + // The text lines read from the source are split into words + // using a user-defined function. The tokenizer, implemented below, + // will output each word as a (2-tuple) containing (word, 1) + text.flatMap(new Tokenizer()) + .name("tokenizer") + // keyBy groups tuples based on the "0" field, the word. + // Using a keyBy allows performing aggregations and other + // stateful transformations over data on a per-key basis. + // This is similar to a GROUP BY clause in a SQL query. + .keyBy(value -> value.f0) + // For each key, we perform a simple sum of the "1" field, the count. + // If the input data stream is bounded, sum will output a final count for + // each word. If it is unbounded, it will continuously output updates + // each time it sees a new instance of each word in the stream. + .sum(1) + .name("counter"); + + if (params.getOutput().isPresent()) { + // Given an output directory, Flink will write the results to a file + // using a simple string encoding. In a production environment, this might + // be something more structured like CSV, Avro, JSON, or Parquet. + counts.sinkTo( + FileSink.>forRowFormat( + params.getOutput().get(), new SimpleStringEncoder<>()) + .withRollingPolicy( + DefaultRollingPolicy.builder() + .withMaxPartSize(MemorySize.ofMebiBytes(1)) + .withRolloverInterval(Duration.ofSeconds(10)) + .build()) + .build()) + .name("file-sink"); + } else { + counts.print().name("print-sink"); + } + + // Apache Flink applications are composed lazily. Calling execute + // submits the Job and begins processing. + env.execute("WordCount"); + } + + // ************************************************************************* + // USER FUNCTIONS + // ************************************************************************* + + /** + * Implements the string tokenizer that splits sentences into words as a user-defined + * FlatMapFunction. The function takes a line (String) and splits it into multiple pairs in the + * form of "(word,1)" ({@code Tuple2}). + */ + public static final class Tokenizer + implements FlatMapFunction> { + + @Override + public void flatMap(String value, Collector> out) { + // normalize and split the line + String[] tokens = value.toLowerCase().split("\\W+"); + + // emit the pairs + for (String token : tokens) { + if (token.length() > 0) { + out.collect(new Tuple2<>(token, 1)); + } + } + } + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java new file mode 100644 index 0000000..ddf1111 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java @@ -0,0 +1,149 @@ +/* + * 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.examples.wordcount.util; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.java.utils.MultipleParameterTool; +import org.apache.flink.configuration.ExecutionOptions; +import org.apache.flink.core.fs.Path; +import org.apache.flink.util.TimeUtils; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.OptionalInt; + +/** + * A simple CLI parser for the {@link org.apache.flink.streaming.examples.wordcount.WordCount} + * example application. + */ +public class CLI extends ExecutionConfig.GlobalJobParameters { + + public static final String INPUT_KEY = "input"; + public static final String OUTPUT_KEY = "output"; + public static final String DISCOVERY_INTERVAL = "discovery-interval"; + public static final String EXECUTION_MODE = "execution-mode"; + + public static CLI fromArgs(String[] args) throws Exception { + MultipleParameterTool params = MultipleParameterTool.fromArgs(args); + Path[] inputs = null; + if (params.has(INPUT_KEY)) { + inputs = + params.getMultiParameterRequired(INPUT_KEY).stream() + .map(Path::new) + .toArray(Path[]::new); + } else { + System.out.println("Executing example with default input data."); + System.out.println("Use --input to specify file input."); + } + + Path output = null; + if (params.has(OUTPUT_KEY)) { + output = new Path(params.get(OUTPUT_KEY)); + } else { + System.out.println("Printing result to stdout. Use --output to specify output path."); + } + + Duration watchInterval = null; + if (params.has(DISCOVERY_INTERVAL)) { + watchInterval = TimeUtils.parseDuration(params.get(DISCOVERY_INTERVAL)); + } + + RuntimeExecutionMode executionMode = ExecutionOptions.RUNTIME_MODE.defaultValue(); + if (params.has(EXECUTION_MODE)) { + executionMode = RuntimeExecutionMode.valueOf(params.get(EXECUTION_MODE).toUpperCase()); + } + + return new CLI(inputs, output, watchInterval, executionMode, params); + } + + private final Path[] inputs; + private final Path output; + private final Duration discoveryInterval; + private final RuntimeExecutionMode executionMode; + private final MultipleParameterTool params; + + private CLI( + Path[] inputs, + Path output, + Duration discoveryInterval, + RuntimeExecutionMode executionMode, + MultipleParameterTool params) { + this.inputs = inputs; + this.output = output; + this.discoveryInterval = discoveryInterval; + this.executionMode = executionMode; + this.params = params; + } + + public Optional getInputs() { + return Optional.ofNullable(inputs); + } + + public Optional getDiscoveryInterval() { + return Optional.ofNullable(discoveryInterval); + } + + public Optional getOutput() { + return Optional.ofNullable(output); + } + + public RuntimeExecutionMode getExecutionMode() { + return executionMode; + } + + public OptionalInt getInt(String key) { + if (params.has(key)) { + return OptionalInt.of(params.getInt(key)); + } + + return OptionalInt.empty(); + } + + @Override + public Map toMap() { + return params.toMap(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + CLI cli = (CLI) o; + return Arrays.equals(inputs, cli.inputs) + && Objects.equals(output, cli.output) + && Objects.equals(discoveryInterval, cli.discoveryInterval); + } + + @Override + public int hashCode() { + int result = Objects.hash(output, discoveryInterval); + result = 31 * result + Arrays.hashCode(inputs); + return result; + } +} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/WordCountData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/WordCountData.java new file mode 100644 index 0000000..6a08872 --- /dev/null +++ b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/WordCountData.java @@ -0,0 +1,65 @@ +/* + * 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.examples.wordcount.util; + +/** + * Provides the default data sets used for the WordCount example program. The default data sets are + * used, if no parameters are given to the program. + */ +public class WordCountData { + + public static final String[] WORDS = + new String[] { + "To be, or not to be,--that is the question:--", + "Whether 'tis nobler in the mind to suffer", + "The slings and arrows of outrageous fortune", + "Or to take arms against a sea of troubles,", + "And by opposing end them?--To die,--to sleep,--", + "No more; and by a sleep to say we end", + "The heartache, and the thousand natural shocks", + "That flesh is heir to,--'tis a consummation", + "Devoutly to be wish'd. To die,--to sleep;--", + "To sleep! perchance to dream:--ay, there's the rub;", + "For in that sleep of death what dreams may come,", + "When we have shuffled off this mortal coil,", + "Must give us pause: there's the respect", + "That makes calamity of so long life;", + "For who would bear the whips and scorns of time,", + "The oppressor's wrong, the proud man's contumely,", + "The pangs of despis'd love, the law's delay,", + "The insolence of office, and the spurns", + "That patient merit of the unworthy takes,", + "When he himself might his quietus make", + "With a bare bodkin? who would these fardels bear,", + "To grunt and sweat under a weary life,", + "But that the dread of something after death,--", + "The undiscover'd country, from whose bourn", + "No traveller returns,--puzzles the will,", + "And makes us rather bear those ills we have", + "Than fly to others that we know not of?", + "Thus conscience does make cowards of us all;", + "And thus the native hue of resolution", + "Is sicklied o'er with the pale cast of thought;", + "And enterprises of great pith and moment,", + "With this regard, their currents turn awry,", + "And lose the name of action.--Soft you now!", + "The fair Ophelia!--Nymph, in thy orisons", + "Be all my sins remember'd." + }; +} diff --git a/java/Streaming/src/main/resources/log4j2.properties b/java/Streaming/src/main/resources/log4j2.properties new file mode 100644 index 0000000..9206863 --- /dev/null +++ b/java/Streaming/src/main/resources/log4j2.properties @@ -0,0 +1,25 @@ +################################################################################ +# 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. +################################################################################ + +rootLogger.level = INFO +rootLogger.appenderRef.console.ref = ConsoleAppender + +appender.console.name = ConsoleAppender +appender.console.type = CONSOLE +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/java/Streaming/src/main/resources/logback.xml b/java/Streaming/src/main/resources/logback.xml new file mode 100644 index 0000000..95f2d04 --- /dev/null +++ b/java/Streaming/src/main/resources/logback.xml @@ -0,0 +1,29 @@ + + + + + + %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n + + + + + + + \ No newline at end of file diff --git a/java/Streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java b/java/Streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java new file mode 100644 index 0000000..a4b4685 --- /dev/null +++ b/java/Streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java @@ -0,0 +1,177 @@ +/* + * 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.test; + +import org.apache.flink.api.common.eventtime.AscendingTimestampsWatermarks; +import org.apache.flink.api.common.eventtime.TimestampAssigner; +import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.examples.iteration.util.IterateExampleData; +import org.apache.flink.streaming.test.examples.join.WindowJoinData; +import org.apache.flink.test.testdata.WordCountData; +import org.apache.flink.test.util.AbstractTestBaseJUnit4; + +import org.apache.commons.io.FileUtils; +import org.junit.Test; + +import java.io.File; + +import static org.apache.flink.test.util.TestBaseUtils.checkLinesAgainstRegexp; +import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; + +/** Integration test for streaming programs in Java examples. */ +public class StreamingExamplesITCase extends AbstractTestBaseJUnit4 { + + @Test + public void testIterateExample() throws Exception { + final String inputPath = + createTempFile("fibonacciInput.txt", IterateExampleData.INPUT_PAIRS); + final String resultPath = getTempDirPath("result"); + + // the example is inherently non-deterministic. The iteration timeout of 5000 ms + // is frequently not enough to make the test run stable on CI infrastructure + // with very small containers, so we cannot do a validation here + org.apache.flink.streaming.examples.iteration.IterateExample.main( + new String[] { + "--input", inputPath, + "--output", resultPath + }); + } + + @Test + public void testWindowJoin() throws Exception { + + final String resultPath = File.createTempFile("result-path", "dir").toURI().toString(); + + final class Parser implements MapFunction> { + + @Override + public Tuple2 map(String value) throws Exception { + String[] fields = value.split(","); + return new Tuple2<>(fields[1], Integer.parseInt(fields[2])); + } + } + + try { + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream> grades = + env.fromData(WindowJoinData.GRADES_INPUT.split("\n")) + .assignTimestampsAndWatermarks(IngestionTimeWatermarkStrategy.create()) + .map(new Parser()); + + DataStream> salaries = + env.fromData(WindowJoinData.SALARIES_INPUT.split("\n")) + .assignTimestampsAndWatermarks(IngestionTimeWatermarkStrategy.create()) + .map(new Parser()); + + org.apache.flink.streaming.examples.join.WindowJoin.runWindowJoin(grades, salaries, 100) + .writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE); + + env.execute(); + + // since the two sides of the join might have different speed + // the exact output can not be checked just whether it is well-formed + // checks that the result lines look like e.g. (bob, 2, 2015) + checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d),(\\d)+\\)"); + } finally { + try { + FileUtils.deleteDirectory(new File(resultPath)); + } catch (Throwable ignored) { + } + } + } + + @Test + public void testSessionWindowing() throws Exception { + final String resultPath = getTempDirPath("result"); + org.apache.flink.streaming.examples.windowing.SessionWindowing.main( + new String[] {"--output", resultPath}); + } + + @Test + public void testWindowWordCount() throws Exception { + final String windowSize = "25"; + final String slideSize = "15"; + final String textPath = createTempFile("text.txt", WordCountData.TEXT); + final String resultPath = getTempDirPath("result"); + + org.apache.flink.streaming.examples.windowing.WindowWordCount.main( + new String[] { + "--input", textPath, + "--output", resultPath, + "--window", windowSize, + "--slide", slideSize + }); + + // since the parallel tokenizers might have different speed + // the exact output can not be checked just whether it is well-formed + // checks that the result lines look like e.g. (faust, 2) + checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d)+\\)"); + } + + @Test + public void testWordCount() throws Exception { + final String textPath = createTempFile("text.txt", WordCountData.TEXT); + final String resultPath = getTempDirPath("result"); + + org.apache.flink.streaming.examples.wordcount.WordCount.main( + new String[] { + "--input", textPath, + "--output", resultPath, + "--execution-mode", "automatic" + }); + + compareResultsByLinesInMemory(WordCountData.COUNTS_AS_TUPLES, resultPath); + } + + /** + * This {@link WatermarkStrategy} assigns the current system time as the event-time timestamp. + * In a real use case you should use proper timestamps and an appropriate {@link + * WatermarkStrategy}. + */ + private static class IngestionTimeWatermarkStrategy implements WatermarkStrategy { + + private IngestionTimeWatermarkStrategy() {} + + public static IngestionTimeWatermarkStrategy create() { + return new IngestionTimeWatermarkStrategy<>(); + } + + @Override + public WatermarkGenerator createWatermarkGenerator( + WatermarkGeneratorSupplier.Context context) { + return new AscendingTimestampsWatermarks<>(); + } + + @Override + public TimestampAssigner createTimestampAssigner( + TimestampAssignerSupplier.Context context) { + return (event, timestamp) -> System.currentTimeMillis(); + } + } +} diff --git a/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/join/WindowJoinData.java b/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/join/WindowJoinData.java new file mode 100644 index 0000000..d15c512 --- /dev/null +++ b/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/join/WindowJoinData.java @@ -0,0 +1,229 @@ +/* + * 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.test.examples.join; + +/** Class with sample data for window join examples. */ +public class WindowJoinData { + + public static final String GRADES_INPUT = + "0,john,5\n" + + "0,tom,3\n" + + "0,alice,1\n" + + "0,grace,5\n" + + "1,john,4\n" + + "1,bob,1\n" + + "1,alice,2\n" + + "1,alice,3\n" + + "1,bob,5\n" + + "1,alice,3\n" + + "1,tom,5\n" + + "2,john,2\n" + + "2,john,1\n" + + "2,grace,2\n" + + "2,jerry,2\n" + + "2,tom,4\n" + + "2,bob,4\n" + + "2,bob,2\n" + + "3, tom,2\n" + + "3,alice,5\n" + + "3,grace,5\n" + + "3,grace,1\n" + + "3,alice,1\n" + + "3,grace,3\n" + + "3,tom,1\n" + + "4,jerry,5\n" + + "4,john,3\n" + + "4,john,4\n" + + "4,john,1\n" + + "4,jerry,3\n" + + "4,grace,3\n" + + "4,bob,3\n" + + "5,john,3\n" + + "5,jerry,4\n" + + "5,tom,5\n" + + "5,tom,4\n" + + "5,john,2\n" + + "5,jerry,1\n" + + "5,bob,1\n" + + "6,john,5\n" + + "6,grace,4\n" + + "6,tom,5\n" + + "6,john,4\n" + + "6,tom,1\n" + + "6,grace,1\n" + + "6,john,2\n" + + "7,jerry,3\n" + + "7,jerry,5\n" + + "7,tom,2\n" + + "7,tom,2\n" + + "7,alice,4\n" + + "7,tom,4\n" + + "7,jerry,4\n" + + "8,john,3\n" + + "8,grace,4\n" + + "8,tom,3\n" + + "8,jerry,4\n" + + "8,john,5\n" + + "8,john,4\n" + + "8,jerry,1\n" + + "9,john,5\n" + + "9,alice,2\n" + + "9,tom,1\n" + + "9,alice,5\n" + + "9,grace,4\n" + + "9,bob,4\n" + + "9,jerry,1\n" + + "10,john,5\n" + + "10,tom,4\n" + + "10,tom,5\n" + + "10,jerry,5\n" + + "10,tom,1\n" + + "10,grace,3\n" + + "10,bob,5\n" + + "11,john,1\n" + + "11,alice,1\n" + + "11,grace,3\n" + + "11,grace,1\n" + + "11,jerry,1\n" + + "11,jerry,4\n" + + "12,bob,4\n" + + "12,alice,3\n" + + "12,tom,5\n" + + "12,alice,4\n" + + "12,alice,4\n" + + "12,grace,4\n" + + "12,john,5\n" + + "13,john,5\n" + + "13,grace,4\n" + + "13,tom,4\n" + + "13,john,4\n" + + "13,john,5\n" + + "13,alice,5\n" + + "13,jerry,5\n" + + "14,john,3\n" + + "14,tom,5\n" + + "14,jerry,4\n" + + "14,grace,4\n" + + "14,john,3\n" + + "14,bob,2"; + + public static final String SALARIES_INPUT = + "0,john,6469\n" + + "0,jerry,6760\n" + + "0,jerry,8069\n" + + "1,tom,3662\n" + + "1,grace,8427\n" + + "1,john,9425\n" + + "1,bob,9018\n" + + "1,john,352\n" + + "1,tom,3770\n" + + "2,grace,7622\n" + + "2,jerry,7441\n" + + "2,alice,1468\n" + + "2,bob,5472\n" + + "2,grace,898\n" + + "3,tom,3849\n" + + "3,grace,1865\n" + + "3,alice,5582\n" + + "3,john,9511\n" + + "3,alice,1541\n" + + "4,john,2477\n" + + "4,grace,3561\n" + + "4,john,1670\n" + + "4,grace,7290\n" + + "4,grace,6565\n" + + "5,tom,6179\n" + + "5,tom,1601\n" + + "5,john,2940\n" + + "5,bob,4685\n" + + "5,bob,710\n" + + "5,bob,5936\n" + + "6,jerry,1412\n" + + "6,grace,6515\n" + + "6,grace,3321\n" + + "6,tom,8088\n" + + "6,john,2876\n" + + "7,bob,9896\n" + + "7,grace,7368\n" + + "7,grace,9749\n" + + "7,bob,2048\n" + + "7,alice,4782\n" + + "8,alice,3375\n" + + "8,tom,5841\n" + + "8,bob,958\n" + + "8,bob,5258\n" + + "8,tom,3935\n" + + "8,jerry,4394\n" + + "9,alice,102\n" + + "9,alice,4931\n" + + "9,alice,5240\n" + + "9,jerry,7951\n" + + "9,john,5675\n" + + "10,bob,609\n" + + "10,alice,5997\n" + + "10,jerry,9651\n" + + "10,alice,1328\n" + + "10,bob,1022\n" + + "11,grace,2578\n" + + "11,jerry,9704\n" + + "11,tom,4476\n" + + "11,grace,3784\n" + + "11,alice,6144\n" + + "12,bob,6213\n" + + "12,alice,7525\n" + + "12,jerry,2908\n" + + "12,grace,8464\n" + + "12,jerry,9920\n" + + "13,bob,3720\n" + + "13,bob,7612\n" + + "13,alice,7211\n" + + "13,jerry,6484\n" + + "13,alice,1711\n" + + "14,jerry,5994\n" + + "14,grace,928\n" + + "14,jerry,2492\n" + + "14,grace,9080\n" + + "14,tom,4330\n" + + "15,bob,8302\n" + + "15,john,4981\n" + + "15,tom,1781\n" + + "15,grace,1379\n" + + "15,jerry,3700\n" + + "16,jerry,3584\n" + + "16,jerry,2038\n" + + "16,jerry,3902\n" + + "16,tom,1336\n" + + "16,jerry,7500\n" + + "17,tom,3648\n" + + "17,alice,2533\n" + + "17,tom,8685\n" + + "17,bob,3968\n" + + "17,tom,3241\n" + + "17,bob,7461\n" + + "18,jerry,2138\n" + + "18,alice,7503\n" + + "18,alice,6424\n" + + "18,tom,140\n" + + "18,john,9802\n" + + "19,grace,2977\n" + + "19,grace,889\n" + + "19,john,1338"; + + /** Utility class, should not be instantiated. */ + private WindowJoinData() {} +} diff --git a/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/windowing/TopSpeedWindowingExampleITCase.java b/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/windowing/TopSpeedWindowingExampleITCase.java new file mode 100644 index 0000000..72dd508 --- /dev/null +++ b/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/windowing/TopSpeedWindowingExampleITCase.java @@ -0,0 +1,67 @@ +/* + * 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.test.examples.windowing; + +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.examples.windowing.TopSpeedWindowing; +import org.apache.flink.streaming.examples.windowing.util.TopSpeedWindowingExampleData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; + +import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; + +/** Tests for {@link TopSpeedWindowing}. */ +public class TopSpeedWindowingExampleITCase extends TestLogger { + + @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @ClassRule + public static MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(1) + .build()); + + @Test + public void testTopSpeedWindowingExampleITCase() throws Exception { + File inputFile = temporaryFolder.newFile(); + FileUtils.writeFileUtf8(inputFile, TopSpeedWindowingExampleData.CAR_DATA); + + final String resultPath = temporaryFolder.newFolder().toURI().toString(); + + TopSpeedWindowing.main( + new String[] { + "--input", + inputFile.getAbsolutePath(), + "--output", + resultPath, + "--execution-mode", + "AUTOMATIC" + }); + + compareResultsByLinesInMemory(TopSpeedWindowingExampleData.TOP_SPEEDS, resultPath); + } +} diff --git a/java/Streaming/src/test/java/org/apache/flink/streaming/test/socket/SocketWindowWordCountITCase.java b/java/Streaming/src/test/java/org/apache/flink/streaming/test/socket/SocketWindowWordCountITCase.java new file mode 100644 index 0000000..f8dbca2 --- /dev/null +++ b/java/Streaming/src/test/java/org/apache/flink/streaming/test/socket/SocketWindowWordCountITCase.java @@ -0,0 +1,122 @@ +/* + * 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.test.socket; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.streaming.examples.socket.SocketWindowWordCount; +import org.apache.flink.test.testdata.WordCountData; +import org.apache.flink.test.util.AbstractTestBaseJUnit4; +import org.apache.flink.util.NetUtils; + +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.io.PrintStream; +import java.io.PrintWriter; +import java.net.InetAddress; +import java.net.ServerSocket; +import java.net.Socket; + +import static org.junit.Assert.fail; + +/** Tests for {@link SocketWindowWordCount}. */ +public class SocketWindowWordCountITCase extends AbstractTestBaseJUnit4 { + + @Test + public void testJavaProgram() throws Exception { + InetAddress localhost = InetAddress.getByName("localhost"); + + // suppress sysout messages from this example + final PrintStream originalSysout = System.out; + final PrintStream originalSyserr = System.err; + + final ByteArrayOutputStream errorMessages = new ByteArrayOutputStream(); + + System.setOut(new PrintStream(new NullStream())); + System.setErr(new PrintStream(errorMessages)); + + try { + try (ServerSocket server = new ServerSocket(0, 10, localhost)) { + + final ServerThread serverThread = new ServerThread(server); + serverThread.setDaemon(true); + serverThread.start(); + + final int serverPort = server.getLocalPort(); + + SocketWindowWordCount.main(new String[] {"--port", String.valueOf(serverPort)}); + + if (errorMessages.size() != 0) { + fail( + "Found error message: " + + new String( + errorMessages.toByteArray(), + ConfigConstants.DEFAULT_CHARSET)); + } + + serverThread.join(); + serverThread.checkError(); + } + } finally { + System.setOut(originalSysout); + System.setErr(originalSyserr); + } + } + + // ------------------------------------------------------------------------ + + private static class ServerThread extends Thread { + + private final ServerSocket serverSocket; + + private volatile Throwable error; + + public ServerThread(ServerSocket serverSocket) { + super("Socket Server Thread"); + + this.serverSocket = serverSocket; + } + + @Override + public void run() { + try { + try (Socket socket = NetUtils.acceptWithoutTimeout(serverSocket); + PrintWriter writer = new PrintWriter(socket.getOutputStream(), true)) { + + writer.println(WordCountData.TEXT); + } + } catch (Throwable t) { + this.error = t; + } + } + + public void checkError() throws IOException { + if (error != null) { + throw new IOException("Error in server thread: " + error.getMessage(), error); + } + } + } + + private static final class NullStream extends OutputStream { + + @Override + public void write(int b) {} + } +} diff --git a/java/Streaming/src/test/resources/log4j2-test.properties b/java/Streaming/src/test/resources/log4j2-test.properties new file mode 100644 index 0000000..835c2ec --- /dev/null +++ b/java/Streaming/src/test/resources/log4j2-test.properties @@ -0,0 +1,28 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n From f54551370e86d16fdca3ff2fc22d5063c158027c Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 15:41:08 +0530 Subject: [PATCH 02/23] Add GitHub workflow --- .github/workflows/maven_packages.yml | 45 ++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 .github/workflows/maven_packages.yml diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml new file mode 100644 index 0000000..0d8ad3b --- /dev/null +++ b/.github/workflows/maven_packages.yml @@ -0,0 +1,45 @@ +name: Maven Packages + +on: + push: + paths: + - java/** + branches: + - master + pull_request: + paths: + - java/** + branches: + - master + workflow_dispatch: + +jobs: + build-publish: + strategy: + matrix: + include: + - path: "java/Streaming" + - path: "java/S3Sink" + runs-on: ubuntu-latest + permissions: + contents: read + packages: write + steps: + - name: Checkout + uses: actions/checkout@v4 + + - name: Set up JDK + uses: actions/setup-java@v4 + with: + java-version: '21' + distribution: 'temurin' + + - name: Build with Maven + working-directory: ${{ matrix.path }} + run: mvn package -Denforcer.skip=true + + - name: Publish to GitHub Packages Apache Maven + run: mvn deploy -s $GITHUB_WORKSPACE/settings.xml + env: + GITHUB_TOKEN: ${{ github.token }} + From 14c83bd1e26f51f2245c0402c01aebf91d4f6da8 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 15:49:18 +0530 Subject: [PATCH 03/23] Trigger when workflow changes --- .github/workflows/maven_packages.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index 0d8ad3b..0b50bd6 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -4,13 +4,16 @@ on: push: paths: - java/** + - .github/workflows/maven_packages.yml branches: - master + pull_request: paths: - java/** branches: - master + workflow_dispatch: jobs: From a368d5bf9476b7b519c3814955551c6b91397589 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 15:50:16 +0530 Subject: [PATCH 04/23] Omit branch filter --- .github/workflows/maven_packages.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index 0b50bd6..c79113f 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -5,14 +5,14 @@ on: paths: - java/** - .github/workflows/maven_packages.yml - branches: - - master +# branches: +# - master pull_request: paths: - java/** - branches: - - master +# branches: +# - master workflow_dispatch: From ca109eabf52d7f823383a07739a649afd6be29df Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 15:54:14 +0530 Subject: [PATCH 05/23] Make Java action create the settings.xml --- .github/workflows/maven_packages.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index c79113f..cd83b4b 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -36,6 +36,7 @@ jobs: with: java-version: '21' distribution: 'temurin' + settings-path: ${{ github.workspace }} - name: Build with Maven working-directory: ${{ matrix.path }} From 68b3a354d4e3064ebe64849e8997f9eef3941935 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 16:01:52 +0530 Subject: [PATCH 06/23] Try disabling compiler plugins --- .github/workflows/maven_packages.yml | 2 +- java/Streaming/pom.xml | 25 ------------------------- 2 files changed, 1 insertion(+), 26 deletions(-) diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index cd83b4b..8179055 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -40,7 +40,7 @@ jobs: - name: Build with Maven working-directory: ${{ matrix.path }} - run: mvn package -Denforcer.skip=true + run: mvn package - name: Publish to GitHub Packages Apache Maven run: mvn deploy -s $GITHUB_WORKSPACE/settings.xml diff --git a/java/Streaming/pom.xml b/java/Streaming/pom.xml index 205fbe7..d6e15aa 100644 --- a/java/Streaming/pom.xml +++ b/java/Streaming/pom.xml @@ -123,31 +123,6 @@ under the License. - - - org.apache.maven.plugins - maven-compiler-plugin - - - compile - process-sources - - compile - - - -Xlint:deprecation - true - - - - org/apache/flink/streaming/examples/iteration/IterateExample.java - - - - - - - org.apache.maven.plugins From 3ab8eaf9b9e82b6c9dc4f60d40d39e09f6f4d0ac Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 16:05:31 +0530 Subject: [PATCH 07/23] Fix wrong directory when publishing --- .github/workflows/maven_packages.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index 8179055..09713d9 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -43,6 +43,7 @@ jobs: run: mvn package - name: Publish to GitHub Packages Apache Maven + working-directory: ${{ matrix.path }} run: mvn deploy -s $GITHUB_WORKSPACE/settings.xml env: GITHUB_TOKEN: ${{ github.token }} From 2b199e64e3be19307ff4d7798f94c199b8a99910 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 16:10:01 +0530 Subject: [PATCH 08/23] Try batch mode and disable colour output --- .github/workflows/maven_packages.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index 09713d9..8930a14 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -36,15 +36,16 @@ jobs: with: java-version: '21' distribution: 'temurin' + server-id: github # Value of the distributionManagement/repository/id field of the pom.xml settings-path: ${{ github.workspace }} - name: Build with Maven working-directory: ${{ matrix.path }} - run: mvn package + run: mvn --batch-mode package - name: Publish to GitHub Packages Apache Maven working-directory: ${{ matrix.path }} - run: mvn deploy -s $GITHUB_WORKSPACE/settings.xml + run: mvn --batch-mode deploy -s $GITHUB_WORKSPACE/settings.xml env: GITHUB_TOKEN: ${{ github.token }} From c72329a08dc9f09d4f0ce6dba3f4db1a5743537a Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 16:10:15 +0530 Subject: [PATCH 09/23] Add distribution ID to pom.xmls --- java/S3Sink/pom.xml | 8 +++++++- java/Streaming/pom.xml | 6 ++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/java/S3Sink/pom.xml b/java/S3Sink/pom.xml index 74b75a1..10aacf1 100644 --- a/java/S3Sink/pom.xml +++ b/java/S3Sink/pom.xml @@ -8,6 +8,12 @@ flink-kds-s3 1.0 + + + github + + + UTF-8 ${project.basedir}/target @@ -173,4 +179,4 @@ - \ No newline at end of file + diff --git a/java/Streaming/pom.xml b/java/Streaming/pom.xml index d6e15aa..550cc4e 100644 --- a/java/Streaming/pom.xml +++ b/java/Streaming/pom.xml @@ -33,6 +33,12 @@ under the License. jar + + + github + + + 10.0.0 From 078c80592772eee8be33f6955c864753555bf971 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 16:22:27 +0530 Subject: [PATCH 10/23] Try reduced pom.xml --- java/Streaming/full_pom.xml | 441 +++++++++++++++ java/Streaming/pom.xml | 1010 ++++++++++++++++++++--------------- 2 files changed, 1010 insertions(+), 441 deletions(-) create mode 100644 java/Streaming/full_pom.xml diff --git a/java/Streaming/full_pom.xml b/java/Streaming/full_pom.xml new file mode 100644 index 0000000..550cc4e --- /dev/null +++ b/java/Streaming/full_pom.xml @@ -0,0 +1,441 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-examples + 1.20.0 + + + flink-examples-streaming + Flink : Examples : Streaming + + jar + + + + github + + + + + + 10.0.0 + + + + + + + + org.apache.flink + flink-streaming-java + ${project.version} + + + + org.apache.flink + flink-clients + ${project.version} + + + + org.apache.flink + flink-connector-files + ${project.version} + + + + org.apache.flink + flink-connector-kafka + 3.0.0-1.17 + + + + org.apache.flink + flink-connector-datagen + ${project.version} + + + + org.apache.flink + flink-shaded-jackson + + + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + + org.apache.flink + flink-statebackend-rocksdb + ${project.version} + + + + + org.jcuda + jcuda + ${jcuda.version} + + + org.jcuda + jcuda-natives + + + + + + org.jcuda + jcublas + ${jcuda.version} + + + org.jcuda + jcublas-natives + + + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + default + package + + test-jar + + + + + + WindowJoin + package + + jar + + + WindowJoin + + + + org.apache.flink.streaming.examples.join.WindowJoin + + + + + org/apache/flink/streaming/examples/join/*.class + org/apache/flink/streaming/examples/utils/ThrottledIterator.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + WordCount + package + + jar + + + WordCount + + + + org.apache.flink.streaming.examples.wordcount.WordCount + + + + + org/apache/flink/streaming/examples/wordcount/WordCount.class + org/apache/flink/streaming/examples/wordcount/WordCount$*.class + org/apache/flink/streaming/examples/wordcount/util/WordCountData.class + org/apache/flink/streaming/examples/wordcount/util/CLI.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + SocketWindowWordCount + package + + jar + + + SocketWindowWordCount + + + + org.apache.flink.streaming.examples.socket.SocketWindowWordCount + + + + + org/apache/flink/streaming/examples/socket/SocketWindowWordCount.class + org/apache/flink/streaming/examples/socket/SocketWindowWordCount$*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + + + AsyncIO + package + + shade + + + false + AsyncIO + true + AsyncIO + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/async/*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.async.AsyncIOExample + + + + + + + + MatrixVectorMul + package + + shade + + + false + MatrixVectorMul + + + org.apache.flink:flink-connector-datagen + org.jcuda:* + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class + org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class + + + + + + org.apache.flink.streaming.examples.gpu.MatrixVectorMul + + + + + + + + Iteration + package + + shade + + + false + Iteration + true + Iteration + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/iteration/*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.iteration.IterateExample + + + + + + + + TopSpeedWindowing + package + + shade + + + false + TopSpeedWindowing + true + TopSpeedWindowing + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class + org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class + org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.class + org/apache/flink/streaming/examples/wordcount/util/CLI.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.windowing.TopSpeedWindowing + + + + + + + + SessionWindowing + package + + shade + + + false + SessionWindowing + true + SessionWindowing + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/windowing/SessionWindowing.class + org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.windowing.SessionWindowing + + + + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + rename + + + + + + + + + + + + + + diff --git a/java/Streaming/pom.xml b/java/Streaming/pom.xml index 550cc4e..6c3501e 100644 --- a/java/Streaming/pom.xml +++ b/java/Streaming/pom.xml @@ -1,441 +1,569 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-examples - 1.20.0 - - - flink-examples-streaming - Flink : Examples : Streaming - - jar - - - - github - - - - - - 10.0.0 - - - - - - - - org.apache.flink - flink-streaming-java - ${project.version} - - - - org.apache.flink - flink-clients - ${project.version} - - - - org.apache.flink - flink-connector-files - ${project.version} - - - - org.apache.flink - flink-connector-kafka - 3.0.0-1.17 - - - - org.apache.flink - flink-connector-datagen - ${project.version} - - - - org.apache.flink - flink-shaded-jackson - - - - - - org.apache.flink - flink-test-utils - ${project.version} - test - - - - org.apache.flink - flink-statebackend-rocksdb - ${project.version} - - - - - org.jcuda - jcuda - ${jcuda.version} - - - org.jcuda - jcuda-natives - - - - - - org.jcuda - jcublas - ${jcuda.version} - - - org.jcuda - jcublas-natives - - - - - - - - - - - org.apache.maven.plugins - maven-jar-plugin - 2.4 - - - - default - package - - test-jar - - - - - - WindowJoin - package - - jar - - - WindowJoin - - - - org.apache.flink.streaming.examples.join.WindowJoin - - - - - org/apache/flink/streaming/examples/join/*.class - org/apache/flink/streaming/examples/utils/ThrottledIterator.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - WordCount - package - - jar - - - WordCount - - - - org.apache.flink.streaming.examples.wordcount.WordCount - - - - - org/apache/flink/streaming/examples/wordcount/WordCount.class - org/apache/flink/streaming/examples/wordcount/WordCount$*.class - org/apache/flink/streaming/examples/wordcount/util/WordCountData.class - org/apache/flink/streaming/examples/wordcount/util/CLI.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - SocketWindowWordCount - package - - jar - - - SocketWindowWordCount - - - - org.apache.flink.streaming.examples.socket.SocketWindowWordCount - - - - - org/apache/flink/streaming/examples/socket/SocketWindowWordCount.class - org/apache/flink/streaming/examples/socket/SocketWindowWordCount$*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - - - AsyncIO - package - - shade - - - false - AsyncIO - true - AsyncIO - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/async/*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.async.AsyncIOExample - - - - - - - - MatrixVectorMul - package - - shade - - - false - MatrixVectorMul - - - org.apache.flink:flink-connector-datagen - org.jcuda:* - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class - org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class - - - - - - org.apache.flink.streaming.examples.gpu.MatrixVectorMul - - - - - - - - Iteration - package - - shade - - - false - Iteration - true - Iteration - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/iteration/*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.iteration.IterateExample - - - - - - - - TopSpeedWindowing - package - - shade - - - false - TopSpeedWindowing - true - TopSpeedWindowing - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class - org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class - org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.class - org/apache/flink/streaming/examples/wordcount/util/CLI.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.windowing.TopSpeedWindowing - - - - - - - - SessionWindowing - package - - shade - - - false - SessionWindowing - true - SessionWindowing - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/windowing/SessionWindowing.class - org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.windowing.SessionWindowing - - - - - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - rename - - - - - - - - - - - - - - + + + + flink-examples + org.apache.flink + 1.20.0 + + 4.0.0 + flink-examples-streaming + Flink : Examples : Streaming + + + + maven-compiler-plugin + + + compile + process-sources + + compile + + + -Xlint:deprecation + true + + org/apache/flink/streaming/examples/iteration/IterateExample.java + + + + + + + maven-jar-plugin + 2.4 + + + package + + test-jar + + + + WindowJoin + package + + jar + + + WindowJoin + + + org.apache.flink.streaming.examples.join.WindowJoin + + + + org/apache/flink/streaming/examples/join/*.class + org/apache/flink/streaming/examples/utils/ThrottledIterator.class + META-INF/LICENSE + META-INF/NOTICE + + + + + WordCount + package + + jar + + + WordCount + + + org.apache.flink.streaming.examples.wordcount.WordCount + + + + org/apache/flink/streaming/examples/wordcount/WordCount.class + org/apache/flink/streaming/examples/wordcount/WordCount$*.class + org/apache/flink/streaming/examples/wordcount/util/WordCountData.class + org/apache/flink/streaming/examples/wordcount/util/CLI.class + META-INF/LICENSE + META-INF/NOTICE + + + + + SocketWindowWordCount + package + + jar + + + SocketWindowWordCount + + + org.apache.flink.streaming.examples.socket.SocketWindowWordCount + + + + org/apache/flink/streaming/examples/socket/SocketWindowWordCount.class + org/apache/flink/streaming/examples/socket/SocketWindowWordCount$*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + maven-shade-plugin + + + AsyncIO + package + + shade + + + false + AsyncIO + true + AsyncIO + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/async/*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.async.AsyncIOExample + + + + + + MatrixVectorMul + package + + shade + + + false + MatrixVectorMul + + + org.apache.flink:flink-connector-datagen + org.jcuda:* + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class + org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class + + + + + + org.apache.flink.streaming.examples.gpu.MatrixVectorMul + + + + + + Iteration + package + + shade + + + false + Iteration + true + Iteration + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/iteration/*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.iteration.IterateExample + + + + + + TopSpeedWindowing + package + + shade + + + false + TopSpeedWindowing + true + TopSpeedWindowing + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class + org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class + org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.class + org/apache/flink/streaming/examples/wordcount/util/CLI.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.windowing.TopSpeedWindowing + + + + + + SessionWindowing + package + + shade + + + false + SessionWindowing + true + SessionWindowing + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/windowing/SessionWindowing.class + org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.windowing.SessionWindowing + + + + + + + false + + + + maven-antrun-plugin + + + rename + + + + + + + + + + + + + + + org.apache.flink + flink-streaming-java + 1.20.0 + compile + + + org.apache.flink + flink-clients + 1.20.0 + compile + + + org.apache.flink + flink-connector-files + 1.20.0 + compile + + + org.apache.flink + flink-connector-kafka + 3.0.0-1.17 + compile + + + org.apache.flink + flink-connector-datagen + 1.20.0 + compile + + + org.apache.flink + flink-shaded-jackson + 2.14.2-17.0 + compile + + + org.apache.flink + flink-test-utils + 1.20.0 + test + + + flink-table-common + org.apache.flink + + + flink-runtime + org.apache.flink + + + flink-core + org.apache.flink + + + flink-rpc-akka-loader + org.apache.flink + + + flink-streaming-java + org.apache.flink + + + flink-statebackend-changelog + org.apache.flink + + + flink-dstl-dfs + org.apache.flink + + + curator-test + org.apache.curator + + + + + org.apache.flink + flink-statebackend-rocksdb + 1.20.0 + compile + + + org.jcuda + jcuda + 10.0.0 + compile + + + jcuda-natives + org.jcuda + + + + + org.jcuda + jcublas + 10.0.0 + compile + + + jcublas-natives + org.jcuda + + + + + org.apache.flink + flink-core + 1.20.0 + compile + + + org.apache.logging.log4j + log4j-slf4j-impl + 2.17.1 + compile + + + org.apache.logging.log4j + log4j-api + 2.17.1 + compile + + + org.apache.logging.log4j + log4j-core + 2.17.1 + compile + + + org.apache.flink + flink-test-utils-junit + 1.20.0 + test + + + testcontainers + org.testcontainers + + + + + org.slf4j + slf4j-api + 1.7.36 + compile + + + com.google.code.findbugs + jsr305 + 1.3.9 + compile + + + org.junit.jupiter + junit-jupiter + 5.10.1 + test + + + junit-jupiter-api + org.junit.jupiter + + + junit-jupiter-params + org.junit.jupiter + + + junit-jupiter-engine + org.junit.jupiter + + + + + org.junit.vintage + junit-vintage-engine + 5.10.1 + test + + + junit-platform-engine + org.junit.platform + + + junit + junit + + + apiguardian-api + org.apiguardian + + + + + org.assertj + assertj-core + 3.23.1 + test + + + byte-buddy + net.bytebuddy + + + + + org.mockito + mockito-core + 3.4.6 + test + + + byte-buddy-agent + net.bytebuddy + + + byte-buddy + net.bytebuddy + + + + + org.mockito + mockito-junit-jupiter + 3.4.6 + test + + + junit-jupiter-api + org.junit.jupiter + + + + + org.hamcrest + hamcrest-all + 1.3 + test + + + org.testcontainers + junit-jupiter + 1.19.1 + test + + + testcontainers + org.testcontainers + + + + + org.apache.logging.log4j + log4j-1.2-api + 2.17.1 + test + + + + 10.0.0 + + From c1d947559a056851638e4572b42163a83e167139 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 16:25:55 +0530 Subject: [PATCH 11/23] Revert "Try reduced pom.xml" This reverts commit 078c80592772eee8be33f6955c864753555bf971. --- java/Streaming/full_pom.xml | 441 --------------- java/Streaming/pom.xml | 1010 +++++++++++++++-------------------- 2 files changed, 441 insertions(+), 1010 deletions(-) delete mode 100644 java/Streaming/full_pom.xml diff --git a/java/Streaming/full_pom.xml b/java/Streaming/full_pom.xml deleted file mode 100644 index 550cc4e..0000000 --- a/java/Streaming/full_pom.xml +++ /dev/null @@ -1,441 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-examples - 1.20.0 - - - flink-examples-streaming - Flink : Examples : Streaming - - jar - - - - github - - - - - - 10.0.0 - - - - - - - - org.apache.flink - flink-streaming-java - ${project.version} - - - - org.apache.flink - flink-clients - ${project.version} - - - - org.apache.flink - flink-connector-files - ${project.version} - - - - org.apache.flink - flink-connector-kafka - 3.0.0-1.17 - - - - org.apache.flink - flink-connector-datagen - ${project.version} - - - - org.apache.flink - flink-shaded-jackson - - - - - - org.apache.flink - flink-test-utils - ${project.version} - test - - - - org.apache.flink - flink-statebackend-rocksdb - ${project.version} - - - - - org.jcuda - jcuda - ${jcuda.version} - - - org.jcuda - jcuda-natives - - - - - - org.jcuda - jcublas - ${jcuda.version} - - - org.jcuda - jcublas-natives - - - - - - - - - - - org.apache.maven.plugins - maven-jar-plugin - 2.4 - - - - default - package - - test-jar - - - - - - WindowJoin - package - - jar - - - WindowJoin - - - - org.apache.flink.streaming.examples.join.WindowJoin - - - - - org/apache/flink/streaming/examples/join/*.class - org/apache/flink/streaming/examples/utils/ThrottledIterator.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - WordCount - package - - jar - - - WordCount - - - - org.apache.flink.streaming.examples.wordcount.WordCount - - - - - org/apache/flink/streaming/examples/wordcount/WordCount.class - org/apache/flink/streaming/examples/wordcount/WordCount$*.class - org/apache/flink/streaming/examples/wordcount/util/WordCountData.class - org/apache/flink/streaming/examples/wordcount/util/CLI.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - SocketWindowWordCount - package - - jar - - - SocketWindowWordCount - - - - org.apache.flink.streaming.examples.socket.SocketWindowWordCount - - - - - org/apache/flink/streaming/examples/socket/SocketWindowWordCount.class - org/apache/flink/streaming/examples/socket/SocketWindowWordCount$*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - - - AsyncIO - package - - shade - - - false - AsyncIO - true - AsyncIO - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/async/*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.async.AsyncIOExample - - - - - - - - MatrixVectorMul - package - - shade - - - false - MatrixVectorMul - - - org.apache.flink:flink-connector-datagen - org.jcuda:* - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class - org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class - - - - - - org.apache.flink.streaming.examples.gpu.MatrixVectorMul - - - - - - - - Iteration - package - - shade - - - false - Iteration - true - Iteration - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/iteration/*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.iteration.IterateExample - - - - - - - - TopSpeedWindowing - package - - shade - - - false - TopSpeedWindowing - true - TopSpeedWindowing - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class - org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class - org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.class - org/apache/flink/streaming/examples/wordcount/util/CLI.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.windowing.TopSpeedWindowing - - - - - - - - SessionWindowing - package - - shade - - - false - SessionWindowing - true - SessionWindowing - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/windowing/SessionWindowing.class - org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.windowing.SessionWindowing - - - - - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - rename - - - - - - - - - - - - - - diff --git a/java/Streaming/pom.xml b/java/Streaming/pom.xml index 6c3501e..550cc4e 100644 --- a/java/Streaming/pom.xml +++ b/java/Streaming/pom.xml @@ -1,569 +1,441 @@ - - - - flink-examples - org.apache.flink - 1.20.0 - - 4.0.0 - flink-examples-streaming - Flink : Examples : Streaming - - - - maven-compiler-plugin - - - compile - process-sources - - compile - - - -Xlint:deprecation - true - - org/apache/flink/streaming/examples/iteration/IterateExample.java - - - - - - - maven-jar-plugin - 2.4 - - - package - - test-jar - - - - WindowJoin - package - - jar - - - WindowJoin - - - org.apache.flink.streaming.examples.join.WindowJoin - - - - org/apache/flink/streaming/examples/join/*.class - org/apache/flink/streaming/examples/utils/ThrottledIterator.class - META-INF/LICENSE - META-INF/NOTICE - - - - - WordCount - package - - jar - - - WordCount - - - org.apache.flink.streaming.examples.wordcount.WordCount - - - - org/apache/flink/streaming/examples/wordcount/WordCount.class - org/apache/flink/streaming/examples/wordcount/WordCount$*.class - org/apache/flink/streaming/examples/wordcount/util/WordCountData.class - org/apache/flink/streaming/examples/wordcount/util/CLI.class - META-INF/LICENSE - META-INF/NOTICE - - - - - SocketWindowWordCount - package - - jar - - - SocketWindowWordCount - - - org.apache.flink.streaming.examples.socket.SocketWindowWordCount - - - - org/apache/flink/streaming/examples/socket/SocketWindowWordCount.class - org/apache/flink/streaming/examples/socket/SocketWindowWordCount$*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - maven-shade-plugin - - - AsyncIO - package - - shade - - - false - AsyncIO - true - AsyncIO - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/async/*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.async.AsyncIOExample - - - - - - MatrixVectorMul - package - - shade - - - false - MatrixVectorMul - - - org.apache.flink:flink-connector-datagen - org.jcuda:* - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class - org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class - - - - - - org.apache.flink.streaming.examples.gpu.MatrixVectorMul - - - - - - Iteration - package - - shade - - - false - Iteration - true - Iteration - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/iteration/*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.iteration.IterateExample - - - - - - TopSpeedWindowing - package - - shade - - - false - TopSpeedWindowing - true - TopSpeedWindowing - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class - org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class - org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.class - org/apache/flink/streaming/examples/wordcount/util/CLI.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.windowing.TopSpeedWindowing - - - - - - SessionWindowing - package - - shade - - - false - SessionWindowing - true - SessionWindowing - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/windowing/SessionWindowing.class - org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.windowing.SessionWindowing - - - - - - - false - - - - maven-antrun-plugin - - - rename - - - - - - - - - - - - - - - org.apache.flink - flink-streaming-java - 1.20.0 - compile - - - org.apache.flink - flink-clients - 1.20.0 - compile - - - org.apache.flink - flink-connector-files - 1.20.0 - compile - - - org.apache.flink - flink-connector-kafka - 3.0.0-1.17 - compile - - - org.apache.flink - flink-connector-datagen - 1.20.0 - compile - - - org.apache.flink - flink-shaded-jackson - 2.14.2-17.0 - compile - - - org.apache.flink - flink-test-utils - 1.20.0 - test - - - flink-table-common - org.apache.flink - - - flink-runtime - org.apache.flink - - - flink-core - org.apache.flink - - - flink-rpc-akka-loader - org.apache.flink - - - flink-streaming-java - org.apache.flink - - - flink-statebackend-changelog - org.apache.flink - - - flink-dstl-dfs - org.apache.flink - - - curator-test - org.apache.curator - - - - - org.apache.flink - flink-statebackend-rocksdb - 1.20.0 - compile - - - org.jcuda - jcuda - 10.0.0 - compile - - - jcuda-natives - org.jcuda - - - - - org.jcuda - jcublas - 10.0.0 - compile - - - jcublas-natives - org.jcuda - - - - - org.apache.flink - flink-core - 1.20.0 - compile - - - org.apache.logging.log4j - log4j-slf4j-impl - 2.17.1 - compile - - - org.apache.logging.log4j - log4j-api - 2.17.1 - compile - - - org.apache.logging.log4j - log4j-core - 2.17.1 - compile - - - org.apache.flink - flink-test-utils-junit - 1.20.0 - test - - - testcontainers - org.testcontainers - - - - - org.slf4j - slf4j-api - 1.7.36 - compile - - - com.google.code.findbugs - jsr305 - 1.3.9 - compile - - - org.junit.jupiter - junit-jupiter - 5.10.1 - test - - - junit-jupiter-api - org.junit.jupiter - - - junit-jupiter-params - org.junit.jupiter - - - junit-jupiter-engine - org.junit.jupiter - - - - - org.junit.vintage - junit-vintage-engine - 5.10.1 - test - - - junit-platform-engine - org.junit.platform - - - junit - junit - - - apiguardian-api - org.apiguardian - - - - - org.assertj - assertj-core - 3.23.1 - test - - - byte-buddy - net.bytebuddy - - - - - org.mockito - mockito-core - 3.4.6 - test - - - byte-buddy-agent - net.bytebuddy - - - byte-buddy - net.bytebuddy - - - - - org.mockito - mockito-junit-jupiter - 3.4.6 - test - - - junit-jupiter-api - org.junit.jupiter - - - - - org.hamcrest - hamcrest-all - 1.3 - test - - - org.testcontainers - junit-jupiter - 1.19.1 - test - - - testcontainers - org.testcontainers - - - - - org.apache.logging.log4j - log4j-1.2-api - 2.17.1 - test - - - - 10.0.0 - - + + + + + 4.0.0 + + + org.apache.flink + flink-examples + 1.20.0 + + + flink-examples-streaming + Flink : Examples : Streaming + + jar + + + + github + + + + + + 10.0.0 + + + + + + + + org.apache.flink + flink-streaming-java + ${project.version} + + + + org.apache.flink + flink-clients + ${project.version} + + + + org.apache.flink + flink-connector-files + ${project.version} + + + + org.apache.flink + flink-connector-kafka + 3.0.0-1.17 + + + + org.apache.flink + flink-connector-datagen + ${project.version} + + + + org.apache.flink + flink-shaded-jackson + + + + + + org.apache.flink + flink-test-utils + ${project.version} + test + + + + org.apache.flink + flink-statebackend-rocksdb + ${project.version} + + + + + org.jcuda + jcuda + ${jcuda.version} + + + org.jcuda + jcuda-natives + + + + + + org.jcuda + jcublas + ${jcuda.version} + + + org.jcuda + jcublas-natives + + + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + default + package + + test-jar + + + + + + WindowJoin + package + + jar + + + WindowJoin + + + + org.apache.flink.streaming.examples.join.WindowJoin + + + + + org/apache/flink/streaming/examples/join/*.class + org/apache/flink/streaming/examples/utils/ThrottledIterator.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + WordCount + package + + jar + + + WordCount + + + + org.apache.flink.streaming.examples.wordcount.WordCount + + + + + org/apache/flink/streaming/examples/wordcount/WordCount.class + org/apache/flink/streaming/examples/wordcount/WordCount$*.class + org/apache/flink/streaming/examples/wordcount/util/WordCountData.class + org/apache/flink/streaming/examples/wordcount/util/CLI.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + SocketWindowWordCount + package + + jar + + + SocketWindowWordCount + + + + org.apache.flink.streaming.examples.socket.SocketWindowWordCount + + + + + org/apache/flink/streaming/examples/socket/SocketWindowWordCount.class + org/apache/flink/streaming/examples/socket/SocketWindowWordCount$*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + + + AsyncIO + package + + shade + + + false + AsyncIO + true + AsyncIO + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/async/*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.async.AsyncIOExample + + + + + + + + MatrixVectorMul + package + + shade + + + false + MatrixVectorMul + + + org.apache.flink:flink-connector-datagen + org.jcuda:* + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class + org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class + + + + + + org.apache.flink.streaming.examples.gpu.MatrixVectorMul + + + + + + + + Iteration + package + + shade + + + false + Iteration + true + Iteration + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/iteration/*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.iteration.IterateExample + + + + + + + + TopSpeedWindowing + package + + shade + + + false + TopSpeedWindowing + true + TopSpeedWindowing + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class + org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class + org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.class + org/apache/flink/streaming/examples/wordcount/util/CLI.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.windowing.TopSpeedWindowing + + + + + + + + SessionWindowing + package + + shade + + + false + SessionWindowing + true + SessionWindowing + + + org.apache.flink:flink-connector-datagen + + + + + org.apache.flink:* + + org/apache/flink/connector/datagen/** + org/apache/flink/streaming/examples/windowing/SessionWindowing.class + org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class + META-INF/LICENSE + META-INF/NOTICE + + + + + + org.apache.flink.streaming.examples.windowing.SessionWindowing + + + + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + rename + + + + + + + + + + + + + + From 887090181de7822032e021ebe7dfaf6cd2efddc2 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 16:26:48 +0530 Subject: [PATCH 12/23] Add distributionManagement.repository.url --- java/S3Sink/pom.xml | 2 ++ java/Streaming/pom.xml | 2 ++ 2 files changed, 4 insertions(+) diff --git a/java/S3Sink/pom.xml b/java/S3Sink/pom.xml index 10aacf1..182b6c6 100644 --- a/java/S3Sink/pom.xml +++ b/java/S3Sink/pom.xml @@ -11,6 +11,8 @@ github + GitHub Packages + https://maven.pkg.github.com/localstack-samples/amazon-managed-service-for-apache-flink-examples diff --git a/java/Streaming/pom.xml b/java/Streaming/pom.xml index 550cc4e..d592a78 100644 --- a/java/Streaming/pom.xml +++ b/java/Streaming/pom.xml @@ -36,6 +36,8 @@ under the License. github + GitHub Packages + https://maven.pkg.github.com/localstack-samples/amazon-managed-service-for-apache-flink-examples From a33760e92a5966500d6d90ed113d2b6c7c137b21 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 16:29:03 +0530 Subject: [PATCH 13/23] Try adding an empty suppressions file --- java/Streaming/suppressions.xml | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 java/Streaming/suppressions.xml diff --git a/java/Streaming/suppressions.xml b/java/Streaming/suppressions.xml new file mode 100644 index 0000000..8305483 --- /dev/null +++ b/java/Streaming/suppressions.xml @@ -0,0 +1,8 @@ + + + + + + From d92466d0feb9f2cea30f65e39f7101bf5a798f99 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 17:30:54 +0530 Subject: [PATCH 14/23] Revert "Try adding an empty suppressions file" This reverts commit a33760e92a5966500d6d90ed113d2b6c7c137b21. --- java/Streaming/suppressions.xml | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 java/Streaming/suppressions.xml diff --git a/java/Streaming/suppressions.xml b/java/Streaming/suppressions.xml deleted file mode 100644 index 8305483..0000000 --- a/java/Streaming/suppressions.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - - - From efc66ffd75cec42c122329d4534bab84b77d0b6f Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 17:33:06 +0530 Subject: [PATCH 15/23] Skip the checkstyle plugin --- .github/workflows/maven_packages.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index 8930a14..ff799be 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -41,11 +41,10 @@ jobs: - name: Build with Maven working-directory: ${{ matrix.path }} - run: mvn --batch-mode package + run: mvn --batch-mode package -Denforcer.skip=true -Dcheckstyle.skip=true - name: Publish to GitHub Packages Apache Maven working-directory: ${{ matrix.path }} run: mvn --batch-mode deploy -s $GITHUB_WORKSPACE/settings.xml env: GITHUB_TOKEN: ${{ github.token }} - From 757433c1f95e2f7bb75dfd399ae82b9b6a2ee62b Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 17:34:15 +0530 Subject: [PATCH 16/23] Revert "Try disabling compiler plugins" This reverts commit 68b3a354d4e3064ebe64849e8997f9eef3941935. --- java/Streaming/pom.xml | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/java/Streaming/pom.xml b/java/Streaming/pom.xml index d592a78..8e4fa79 100644 --- a/java/Streaming/pom.xml +++ b/java/Streaming/pom.xml @@ -131,6 +131,31 @@ under the License. + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + process-sources + + compile + + + -Xlint:deprecation + true + + + + org/apache/flink/streaming/examples/iteration/IterateExample.java + + + + + + + org.apache.maven.plugins From 5d2f7c24665830b2dcc056cb76f0e95f65f4b539 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 17:36:17 +0530 Subject: [PATCH 17/23] Skip spotless check --- .github/workflows/maven_packages.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index ff799be..1c3a5ca 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -41,7 +41,7 @@ jobs: - name: Build with Maven working-directory: ${{ matrix.path }} - run: mvn --batch-mode package -Denforcer.skip=true -Dcheckstyle.skip=true + run: mvn --batch-mode package -Denforcer.skip=true -Dcheckstyle.skip=true -Dspotless.check.skip=true - name: Publish to GitHub Packages Apache Maven working-directory: ${{ matrix.path }} From 246044c69ffd11de88b9ff01258631d2b35060a7 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 2 Dec 2024 17:38:20 +0530 Subject: [PATCH 18/23] Add SNAPSHOT to versions to allow overwrites --- java/S3Sink/pom.xml | 2 +- java/Streaming/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/java/S3Sink/pom.xml b/java/S3Sink/pom.xml index 182b6c6..0069290 100644 --- a/java/S3Sink/pom.xml +++ b/java/S3Sink/pom.xml @@ -6,7 +6,7 @@ com.amazonaws flink-kds-s3 - 1.0 + 1.0-SNAPSHOT diff --git a/java/Streaming/pom.xml b/java/Streaming/pom.xml index 8e4fa79..3d902ad 100644 --- a/java/Streaming/pom.xml +++ b/java/Streaming/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-examples - 1.20.0 + 1.20.0-SNAPSHOT flink-examples-streaming From d826b46d7e05cadf6e9d8c0a5adc6b66396b3b15 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Tue, 3 Dec 2024 19:02:20 +0530 Subject: [PATCH 19/23] Remove Streaming example --- java/Streaming/README.md | 1 - java/Streaming/pom.xml | 468 ------------------ .../streaming/examples/async/AsyncClient.java | 43 -- .../examples/async/AsyncIOExample.java | 120 ----- .../examples/datagen/DataGenerator.java | 51 -- .../datagen/DataGeneratorPerCheckpoint.java | 56 --- .../examples/gpu/MatrixVectorMul.java | 253 ---------- .../examples/iteration/IterateExample.java | 242 --------- .../iteration/util/IterateExampleData.java | 63 --- .../streaming/examples/join/WindowJoin.java | 172 ------- .../examples/join/WindowJoinSampleData.java | 62 --- .../sideoutput/SideOutputExample.java | 206 -------- .../socket/SocketWindowWordCount.java | 112 ----- .../statemachine/KafkaEventsGeneratorJob.java | 88 ---- .../streaming/examples/statemachine/README.md | 52 -- .../statemachine/StateMachineExample.java | 242 --------- .../statemachine/dfa/EventTypeAndState.java | 34 -- .../examples/statemachine/dfa/State.java | 144 ------ .../examples/statemachine/dfa/Transition.java | 99 ---- .../examples/statemachine/event/Alert.java | 92 ---- .../examples/statemachine/event/Event.java | 96 ---- .../statemachine/event/EventType.java | 30 -- .../generator/EventsGenerator.java | 156 ------ .../generator/EventsGeneratorFunction.java | 57 --- .../StandaloneThreadedGenerator.java | 252 ---------- .../kafka/EventDeSerializationSchema.java | 62 --- .../kafka/KafkaStandaloneGenerator.java | 96 ---- .../examples/utils/ThrottledIterator.java | 101 ---- .../GroupedProcessingTimeWindowExample.java | 149 ------ .../examples/windowing/SessionWindowing.java | 110 ---- .../examples/windowing/TopSpeedWindowing.java | 194 -------- .../examples/windowing/WindowWordCount.java | 153 ------ .../windowing/util/CarGeneratorFunction.java | 84 ---- .../windowing/util/SessionWindowingData.java | 27 - .../util/TopSpeedWindowingExampleData.java | 372 -------------- .../examples/wordcount/WordCount.java | 187 ------- .../examples/wordcount/util/CLI.java | 149 ------ .../wordcount/util/WordCountData.java | 65 --- .../src/main/resources/log4j2.properties | 25 - java/Streaming/src/main/resources/logback.xml | 29 -- .../test/StreamingExamplesITCase.java | 177 ------- .../test/examples/join/WindowJoinData.java | 229 --------- .../TopSpeedWindowingExampleITCase.java | 67 --- .../socket/SocketWindowWordCountITCase.java | 122 ----- .../src/test/resources/log4j2-test.properties | 28 -- 45 files changed, 5617 deletions(-) delete mode 100644 java/Streaming/README.md delete mode 100644 java/Streaming/pom.xml delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncClient.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGenerator.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGeneratorPerCheckpoint.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoinSampleData.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/README.md delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/EventTypeAndState.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/State.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/Transition.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Alert.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Event.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/EventType.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGenerator.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorFunction.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/StandaloneThreadedGenerator.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/EventDeSerializationSchema.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/KafkaStandaloneGenerator.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/utils/ThrottledIterator.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java delete mode 100644 java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/WordCountData.java delete mode 100644 java/Streaming/src/main/resources/log4j2.properties delete mode 100644 java/Streaming/src/main/resources/logback.xml delete mode 100644 java/Streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java delete mode 100644 java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/join/WindowJoinData.java delete mode 100644 java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/windowing/TopSpeedWindowingExampleITCase.java delete mode 100644 java/Streaming/src/test/java/org/apache/flink/streaming/test/socket/SocketWindowWordCountITCase.java delete mode 100644 java/Streaming/src/test/resources/log4j2-test.properties diff --git a/java/Streaming/README.md b/java/Streaming/README.md deleted file mode 100644 index 5972c72..0000000 --- a/java/Streaming/README.md +++ /dev/null @@ -1 +0,0 @@ -Samples from https://github.com/apache/flink/tree/release-1.20.0/flink-examples/flink-examples-streaming diff --git a/java/Streaming/pom.xml b/java/Streaming/pom.xml deleted file mode 100644 index 3d902ad..0000000 --- a/java/Streaming/pom.xml +++ /dev/null @@ -1,468 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-examples - 1.20.0-SNAPSHOT - - - flink-examples-streaming - Flink : Examples : Streaming - - jar - - - - github - GitHub Packages - https://maven.pkg.github.com/localstack-samples/amazon-managed-service-for-apache-flink-examples - - - - - - 10.0.0 - - - - - - - - org.apache.flink - flink-streaming-java - ${project.version} - - - - org.apache.flink - flink-clients - ${project.version} - - - - org.apache.flink - flink-connector-files - ${project.version} - - - - org.apache.flink - flink-connector-kafka - 3.0.0-1.17 - - - - org.apache.flink - flink-connector-datagen - ${project.version} - - - - org.apache.flink - flink-shaded-jackson - - - - - - org.apache.flink - flink-test-utils - ${project.version} - test - - - - org.apache.flink - flink-statebackend-rocksdb - ${project.version} - - - - - org.jcuda - jcuda - ${jcuda.version} - - - org.jcuda - jcuda-natives - - - - - - org.jcuda - jcublas - ${jcuda.version} - - - org.jcuda - jcublas-natives - - - - - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - compile - process-sources - - compile - - - -Xlint:deprecation - true - - - - org/apache/flink/streaming/examples/iteration/IterateExample.java - - - - - - - - - - org.apache.maven.plugins - maven-jar-plugin - 2.4 - - - - default - package - - test-jar - - - - - - WindowJoin - package - - jar - - - WindowJoin - - - - org.apache.flink.streaming.examples.join.WindowJoin - - - - - org/apache/flink/streaming/examples/join/*.class - org/apache/flink/streaming/examples/utils/ThrottledIterator.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - WordCount - package - - jar - - - WordCount - - - - org.apache.flink.streaming.examples.wordcount.WordCount - - - - - org/apache/flink/streaming/examples/wordcount/WordCount.class - org/apache/flink/streaming/examples/wordcount/WordCount$*.class - org/apache/flink/streaming/examples/wordcount/util/WordCountData.class - org/apache/flink/streaming/examples/wordcount/util/CLI.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - SocketWindowWordCount - package - - jar - - - SocketWindowWordCount - - - - org.apache.flink.streaming.examples.socket.SocketWindowWordCount - - - - - org/apache/flink/streaming/examples/socket/SocketWindowWordCount.class - org/apache/flink/streaming/examples/socket/SocketWindowWordCount$*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - - - AsyncIO - package - - shade - - - false - AsyncIO - true - AsyncIO - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/async/*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.async.AsyncIOExample - - - - - - - - MatrixVectorMul - package - - shade - - - false - MatrixVectorMul - - - org.apache.flink:flink-connector-datagen - org.jcuda:* - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class - org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class - - - - - - org.apache.flink.streaming.examples.gpu.MatrixVectorMul - - - - - - - - Iteration - package - - shade - - - false - Iteration - true - Iteration - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/iteration/*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.iteration.IterateExample - - - - - - - - TopSpeedWindowing - package - - shade - - - false - TopSpeedWindowing - true - TopSpeedWindowing - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class - org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class - org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.class - org/apache/flink/streaming/examples/wordcount/util/CLI.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.windowing.TopSpeedWindowing - - - - - - - - SessionWindowing - package - - shade - - - false - SessionWindowing - true - SessionWindowing - - - org.apache.flink:flink-connector-datagen - - - - - org.apache.flink:* - - org/apache/flink/connector/datagen/** - org/apache/flink/streaming/examples/windowing/SessionWindowing.class - org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class - META-INF/LICENSE - META-INF/NOTICE - - - - - - org.apache.flink.streaming.examples.windowing.SessionWindowing - - - - - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - rename - - - - - - - - - - - - - - diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncClient.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncClient.java deleted file mode 100644 index 3ff8d3a..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncClient.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.examples.async; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ThreadLocalRandom; - -/** A simple asynchronous client that simulates interacting with an unreliable external service. */ -public class AsyncClient { - - public CompletableFuture query(int key) { - return CompletableFuture.supplyAsync( - () -> { - long sleep = (long) (ThreadLocalRandom.current().nextFloat() * 100); - try { - Thread.sleep(sleep); - } catch (InterruptedException e) { - throw new RuntimeException("AsyncClient was interrupted", e); - } - - if (ThreadLocalRandom.current().nextFloat() < 0.001f) { - throw new RuntimeException("wahahahaha..."); - } else { - return "key" + (key % 10); - } - }); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java deleted file mode 100644 index 2d96426..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.examples.async; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.streaming.api.datastream.AsyncDataStream; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.async.AsyncFunction; -import org.apache.flink.streaming.api.functions.async.ResultFuture; -import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; - -import java.util.Collections; -import java.util.concurrent.TimeUnit; - -/** Example to illustrate how to use {@link AsyncFunction}. */ -public class AsyncIOExample { - - /** An example of {@link AsyncFunction} using an async client to query an external service. */ - private static class SampleAsyncFunction extends RichAsyncFunction { - private static final long serialVersionUID = 1L; - - private transient AsyncClient client; - - @Override - public void open(OpenContext openContext) { - client = new AsyncClient(); - } - - @Override - public void asyncInvoke(final Integer input, final ResultFuture resultFuture) { - client.query(input) - .whenComplete( - (response, error) -> { - if (response != null) { - resultFuture.complete(Collections.singletonList(response)); - } else { - resultFuture.completeExceptionally(error); - } - }); - } - } - - public static void main(String[] args) throws Exception { - final ParameterTool params = ParameterTool.fromArgs(args); - - final String mode; - final long timeout; - - try { - mode = params.get("waitMode", "ordered"); - timeout = params.getLong("timeout", 10000L); - } catch (Exception e) { - System.out.println( - "To customize example, use: AsyncIOExample [--waitMode ]"); - throw e; - } - - // obtain execution environment - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataGeneratorSource generatorSource = - new DataGeneratorSource<>( - Long::intValue, - Integer.MAX_VALUE, - RateLimiterStrategy.perSecond(100), - Types.INT); - - // create input stream of a single integer - DataStream inputStream = - env.fromSource( - generatorSource, - WatermarkStrategy.noWatermarks(), - "Integers-generating Source"); - - AsyncFunction function = new SampleAsyncFunction(); - - // add async operator to streaming job - DataStream result; - switch (mode.toUpperCase()) { - case "ORDERED": - result = - AsyncDataStream.orderedWait( - inputStream, function, timeout, TimeUnit.MILLISECONDS, 20); - break; - case "UNORDERED": - result = - AsyncDataStream.unorderedWait( - inputStream, function, timeout, TimeUnit.MILLISECONDS, 20); - break; - default: - throw new IllegalStateException("Unknown mode: " + mode); - } - - result.print(); - - // execute the program - env.execute("Async IO Example: " + mode); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGenerator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGenerator.java deleted file mode 100644 index 326e2a2..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGenerator.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.examples.datagen; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.datagen.source.GeneratorFunction; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - -/** An example for generating data with a {@link DataGeneratorSource}. */ -public class DataGenerator { - - public static void main(String[] args) throws Exception { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - GeneratorFunction generatorFunction = index -> "Number: " + index; - - DataGeneratorSource generatorSource = - new DataGeneratorSource<>( - generatorFunction, - Long.MAX_VALUE, - RateLimiterStrategy.perSecond(4), - Types.STRING); - - DataStreamSource streamSource = - env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); - streamSource.print(); - - env.execute("Data Generator Source Example"); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGeneratorPerCheckpoint.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGeneratorPerCheckpoint.java deleted file mode 100644 index 5eef737..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/datagen/DataGeneratorPerCheckpoint.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.examples.datagen; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.datagen.source.GeneratorFunction; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - -/** An example for generating specific data per checkpoint with a {@link DataGeneratorSource} . */ -public class DataGeneratorPerCheckpoint { - - public static void main(String[] args) throws Exception { - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(3000); - env.setParallelism(1); - - final String[] elements = new String[] {"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}; - final int size = elements.length; - final GeneratorFunction generatorFunction = - index -> elements[(int) (index % size)]; - - final DataGeneratorSource generatorSource = - new DataGeneratorSource<>( - generatorFunction, - Long.MAX_VALUE, - RateLimiterStrategy.perCheckpoint(size), - Types.STRING); - - final DataStreamSource streamSource = - env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); - streamSource.print(); - - env.execute("Data Generator Source Example"); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java deleted file mode 100644 index c437c40..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java +++ /dev/null @@ -1,253 +0,0 @@ -/* - * 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.examples.gpu; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.externalresource.ExternalResourceInfo; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.datagen.source.GeneratorFunction; -import org.apache.flink.connector.file.sink.FileSink; -import org.apache.flink.core.fs.Path; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.util.Preconditions; - -import jcuda.Pointer; -import jcuda.Sizeof; -import jcuda.jcublas.JCublas; -import jcuda.runtime.JCuda; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; - -/** - * Implements the matrix-vector multiplication program that shows how to use GPU resources in Flink. - * - *

The input is a vector stream, which will generate random vectors with specified dimension. The - * data size of the vector stream could be specified by user. Each vector will be multiplied with a - * random dimension * dimension matrix in {@link Multiplier} and the result would be emitted to - * output. - * - *

Usage: MatrixVectorMul [--output <path>] [--dimension <dimension> --data-size - * <data_size>] - * - *

If no parameters are provided, the program is run with default vector dimension 10 and data - * size 100. - * - *

This example shows how to: - * - *

    - *
  • leverage external resource in operators, - *
  • accelerate complex calculation with GPU resources. - *
- * - *

Notice that you need to add JCuda natives libraries in your Flink distribution by the - * following steps: - * - *

    - *
  • download the JCuda native libraries bundle for your CUDA version from - * http://www.jcuda.org/downloads/ - *
  • copy the native libraries jcuda-natives and jcublas-natives for your CUDA version, - * operating system and architecture to the "lib/" folder of your Flink distribution - *
- */ -public class MatrixVectorMul { - - private static final int DEFAULT_DIM = 10; - private static final int DEFAULT_DATA_SIZE = 100; - private static final String DEFAULT_RESOURCE_NAME = "gpu"; - - public static void main(String[] args) throws Exception { - - // Checking input parameters - final ParameterTool params = ParameterTool.fromArgs(args); - System.out.println( - "Usage: MatrixVectorMul [--output ] [--dimension --data-size ] [--resource-name ]"); - - // Set up the execution environment - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // Make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - final int dimension = params.getInt("dimension", DEFAULT_DIM); - final int dataSize = params.getInt("data-size", DEFAULT_DATA_SIZE); - final String resourceName = params.get("resource-name", DEFAULT_RESOURCE_NAME); - - GeneratorFunction> generatorFunction = - index -> { - List randomRecord = new ArrayList<>(); - for (int i = 0; i < dimension; ++i) { - randomRecord.add((float) Math.random()); - } - return randomRecord; - }; - - // Generates random vectors with specified dimension - DataGeneratorSource> generatorSource = - new DataGeneratorSource<>(generatorFunction, dataSize, Types.LIST(Types.FLOAT)); - - DataStream> result = - env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Vectors Source") - .map(new Multiplier(dimension, resourceName)); - - // Emit result - if (params.has("output")) { - result.sinkTo( - FileSink.forRowFormat( - new Path(params.get("output")), - new SimpleStringEncoder>()) - .build()); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - result.print(); - } - // Execute program - env.execute("Matrix-Vector Multiplication"); - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** Matrix-Vector multiplier using CUBLAS library. */ - private static final class Multiplier extends RichMapFunction, List> { - private final int dimension; - private final String resourceName; - private Pointer matrixPointer; - - Multiplier(int dimension, String resourceName) { - this.dimension = dimension; - this.resourceName = resourceName; - } - - @Override - public void open(OpenContext openContext) { - // When multiple instances of this class and JCuda exist in different class loaders, - // then we will get UnsatisfiedLinkError. - // To avoid that, we need to temporarily override the java.io.tmpdir, where the JCuda - // store its native library, with a random path. - // For more details please refer to https://issues.apache.org/jira/browse/FLINK-5408 and - // the discussion in - // http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Classloader-and-removal-of-native-libraries-td14808.html - final String originTempDir = System.getProperty("java.io.tmpdir"); - final String newTempDir = originTempDir + "/jcuda-" + UUID.randomUUID(); - System.setProperty("java.io.tmpdir", newTempDir); - - final Set externalResourceInfos = - getRuntimeContext().getExternalResourceInfos(resourceName); - Preconditions.checkState( - !externalResourceInfos.isEmpty(), - "The MatrixVectorMul needs at least one GPU device while finding 0 GPU."); - final Optional firstIndexOptional = - externalResourceInfos.iterator().next().getProperty("index"); - Preconditions.checkState(firstIndexOptional.isPresent()); - - matrixPointer = new Pointer(); - final float[] matrix = new float[dimension * dimension]; - // Initialize a random matrix - for (int i = 0; i < dimension * dimension; ++i) { - matrix[i] = (float) Math.random(); - } - - // Set the CUDA device - JCuda.cudaSetDevice(Integer.parseInt(firstIndexOptional.get())); - - // Initialize JCublas - JCublas.cublasInit(); - - // Allocate device memory for the matrix - JCublas.cublasAlloc(dimension * dimension, Sizeof.FLOAT, matrixPointer); - JCublas.cublasSetVector( - dimension * dimension, Sizeof.FLOAT, Pointer.to(matrix), 1, matrixPointer, 1); - - // Change the java.io.tmpdir back to its original value. - System.setProperty("java.io.tmpdir", originTempDir); - } - - @Override - public List map(List value) { - final float[] input = new float[dimension]; - final float[] output = new float[dimension]; - final Pointer inputPointer = new Pointer(); - final Pointer outputPointer = new Pointer(); - - // Fill the input and output vector - for (int i = 0; i < dimension; i++) { - input[i] = value.get(i); - output[i] = 0; - } - - // Allocate device memory for the input and output - JCublas.cublasAlloc(dimension, Sizeof.FLOAT, inputPointer); - JCublas.cublasAlloc(dimension, Sizeof.FLOAT, outputPointer); - - // Initialize the device matrices - JCublas.cublasSetVector(dimension, Sizeof.FLOAT, Pointer.to(input), 1, inputPointer, 1); - JCublas.cublasSetVector( - dimension, Sizeof.FLOAT, Pointer.to(output), 1, outputPointer, 1); - - // Performs operation using JCublas - JCublas.cublasSgemv( - 'n', - dimension, - dimension, - 1.0f, - matrixPointer, - dimension, - inputPointer, - 1, - 0.0f, - outputPointer, - 1); - - // Read the result back - JCublas.cublasGetVector( - dimension, Sizeof.FLOAT, outputPointer, 1, Pointer.to(output), 1); - - // Memory clean up - JCublas.cublasFree(inputPointer); - JCublas.cublasFree(outputPointer); - - List outputList = new ArrayList<>(); - for (int i = 0; i < dimension; ++i) { - outputList.add(output[i]); - } - - return outputList; - } - - @Override - public void close() { - // Memory clean up - JCublas.cublasFree(matrixPointer); - - // Shutdown cublas - JCublas.cublasShutdown(); - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java deleted file mode 100644 index d72933b..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java +++ /dev/null @@ -1,242 +0,0 @@ -/* - * 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.examples.iteration; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.datagen.source.GeneratorFunction; -import org.apache.flink.connector.file.sink.FileSink; -import org.apache.flink.connector.file.src.FileSource; -import org.apache.flink.connector.file.src.reader.TextLineInputFormat; -import org.apache.flink.core.fs.Path; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.IterativeStream; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.ProcessFunction; -import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; -import org.apache.flink.util.Collector; -import org.apache.flink.util.OutputTag; - -import java.time.Duration; -import java.util.Random; - -/** - * Example illustrating iterations in Flink streaming. - * - *

The program sums up random numbers and counts additions it performs to reach a specific - * threshold in an iterative streaming fashion. - * - *

This example shows how to use: - * - *

    - *
  • streaming iterations, - *
  • buffer timeout to enhance latency, - *
  • directed outputs. - *
- */ -public class IterateExample { - - private static final int BOUND = 100; - - private static final OutputTag> - ITERATE_TAG = - new OutputTag>( - "iterate") {}; - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - // Checking input parameters - final ParameterTool params = ParameterTool.fromArgs(args); - - // set up input for the stream of integer pairs - - // obtain execution environment and set setBufferTimeout to 1 to enable - // continuous flushing of the output buffers (lowest latency) - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment().setBufferTimeout(1); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - // create input stream of integer pairs - DataStream> inputStream; - if (params.has("input")) { - FileSource fileSource = - FileSource.forRecordStreamFormat( - new TextLineInputFormat(), new Path(params.get("input"))) - .build(); - inputStream = - env.fromSource(fileSource, WatermarkStrategy.noWatermarks(), "Tuples Source") - .map(new FibonacciInputMap()); - } else { - System.out.println("Executing Iterate example with default input data set."); - System.out.println("Use --input to specify file input."); - - GeneratorFunction> dataGenerator = - new RandomFibonacciGenerator(); - DataGeneratorSource> generatorSource = - new DataGeneratorSource<>( - dataGenerator, - BOUND, - RateLimiterStrategy.perSecond(20), - Types.TUPLE(Types.INT, Types.INT)); - - inputStream = - env.fromSource( - generatorSource, - WatermarkStrategy.noWatermarks(), - "Generated tuples Source"); - } - - // create an iterative data stream from the input with 5 second timeout - IterativeStream> it = - inputStream.map(new InputMap()).iterate(5000L); - - // apply the step function to get the next Fibonacci number - // increment the counter and split the output - SingleOutputStreamOperator> step = - it.process(new Step()); - - // close the iteration by selecting the tuples that were directed to the - // 'iterate' channel in the output selector - it.closeWith(step.getSideOutput(ITERATE_TAG)); - - // to produce the final get the input pairs that have the greatest iteration counter - // on a 1 second sliding window - DataStream, Integer>> numbers = step.map(new OutputMap()); - - // emit results - if (params.has("output")) { - numbers.sinkTo( - FileSink., Integer>>forRowFormat( - new Path(params.get("output")), new SimpleStringEncoder<>()) - .withRollingPolicy( - DefaultRollingPolicy.builder() - .withMaxPartSize(MemorySize.ofMebiBytes(1)) - .withRolloverInterval(Duration.ofSeconds(10)) - .build()) - .build()); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - numbers.print(); - } - - // execute the program - env.execute("Streaming Iteration Example"); - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** Generate BOUND number of random integer pairs from the range from 1 to BOUND/2. */ - private static class RandomFibonacciGenerator - implements GeneratorFunction> { - private static final long serialVersionUID = 1L; - - private final Random rnd = new Random(); - - @Override - public Tuple2 map(Long ignoredIndex) throws Exception { - int first = rnd.nextInt(BOUND / 2 - 1) + 1; - int second = rnd.nextInt(BOUND / 2 - 1) + 1; - return new Tuple2<>(first, second); - } - } - - /** Generate random integer pairs from the range from 0 to BOUND/2. */ - private static class FibonacciInputMap - implements MapFunction> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 map(String value) throws Exception { - String record = value.substring(1, value.length() - 1); - String[] splitted = record.split(","); - return new Tuple2<>(Integer.parseInt(splitted[0]), Integer.parseInt(splitted[1])); - } - } - - /** - * Map the inputs so that the next Fibonacci numbers can be calculated while preserving the - * original input tuple. A counter is attached to the tuple and incremented in every iteration - * step. - */ - public static class InputMap - implements MapFunction< - Tuple2, Tuple5> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple5 map( - Tuple2 value) throws Exception { - return new Tuple5<>(value.f0, value.f1, value.f0, value.f1, 0); - } - } - - /** Iteration step function that calculates the next Fibonacci number. */ - public static class Step - extends ProcessFunction< - Tuple5, - Tuple5> { - private static final long serialVersionUID = 1L; - - @Override - public void processElement( - Tuple5 value, - Context ctx, - Collector> out) - throws Exception { - Tuple5 element = - new Tuple5<>(value.f0, value.f1, value.f3, value.f2 + value.f3, ++value.f4); - - if (value.f2 < BOUND && value.f3 < BOUND) { - ctx.output(ITERATE_TAG, element); - } else { - out.collect(element); - } - } - } - - /** Giving back the input pair and the counter. */ - public static class OutputMap - implements MapFunction< - Tuple5, - Tuple2, Integer>> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2, Integer> map( - Tuple5 value) throws Exception { - return new Tuple2<>(new Tuple2<>(value.f0, value.f1), value.f4); - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java deleted file mode 100644 index 031fa69..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.examples.iteration.util; - -/** Data for IterateExampleITCase. */ -public class IterateExampleData { - public static final String INPUT_PAIRS = - "(1,40)\n" - + "(29,38)\n" - + "(11,15)\n" - + "(17,39)\n" - + "(24,41)\n" - + "(7,33)\n" - + "(20,2)\n" - + "(11,5)\n" - + "(3,16)\n" - + "(23,36)\n" - + "(15,23)\n" - + "(28,13)\n" - + "(1,1)\n" - + "(10,6)\n" - + "(21,5)\n" - + "(14,36)\n" - + "(17,15)\n" - + "(7,9)"; - - public static final String RESULTS = - "((1,40),3)\n" - + "((24,41),2)\n" - + "((3,16),5)\n" - + "((1,1),10)\n" - + "((17,15),4)\n" - + "((29,38),2)\n" - + "((7,33),3)\n" - + "((23,36),3)\n" - + "((10,6),6)\n" - + "((7,9),5)\n" - + "((11,15),4)\n" - + "((20,2),5)\n" - + "((15,23),4)\n" - + "((21,5),5)\n" - + "((17,39),3)\n" - + "((11,5),6)\n" - + "((28,13),4)\n" - + "((14,36),3)"; - - private IterateExampleData() {} -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java deleted file mode 100644 index abd4439..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * 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.examples.join; - -import org.apache.flink.api.common.eventtime.AscendingTimestampsWatermarks; -import org.apache.flink.api.common.eventtime.TimestampAssigner; -import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; -import org.apache.flink.api.common.eventtime.WatermarkGenerator; -import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.connector.file.sink.FileSink; -import org.apache.flink.core.fs.Path; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; -import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; - -import java.time.Duration; - -/** - * Example illustrating a windowed stream join between two data streams. - * - *

The example works on two input streams with pairs (name, grade) and (name, salary) - * respectively. It joins the streams based on "name" within a configurable window. - * - *

The example uses a built-in sample data generator that generates the streams of pairs at a - * configurable rate. - */ -public class WindowJoin { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - // parse the parameters - final ParameterTool params = ParameterTool.fromArgs(args); - final long windowSize = params.getLong("windowSize", 2000); - final long rate = params.getLong("rate", 3L); - final boolean fileOutput = params.has("output"); - - System.out.println("Using windowSize=" + windowSize + ", data rate=" + rate); - System.out.println( - "To customize example, use: WindowJoin [--windowSize ] [--rate ]"); - - // obtain execution environment, run this example in "ingestion time" - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - // create the data sources for both grades and salaries - DataStream> grades = - env.fromSource( - WindowJoinSampleData.getGradeGeneratorSource(rate), - IngestionTimeWatermarkStrategy.create(), - "Grades Data Generator") - .setParallelism(1); - - DataStream> salaries = - env.fromSource( - WindowJoinSampleData.getSalaryGeneratorSource(rate), - IngestionTimeWatermarkStrategy.create(), - "Grades Data Generator") - .setParallelism(1); - - // run the actual window join program - // for testability, this functionality is in a separate method. - DataStream> joinedStream = - runWindowJoin(grades, salaries, windowSize); - - if (fileOutput) { - joinedStream - .sinkTo( - FileSink.>forRowFormat( - new Path(params.get("output")), - new SimpleStringEncoder<>()) - .withRollingPolicy( - DefaultRollingPolicy.builder() - .withMaxPartSize(MemorySize.ofMebiBytes(1)) - .withRolloverInterval(Duration.ofSeconds(10)) - .build()) - .build()) - .name("output"); - } else { - // print the results with a single thread, rather than in parallel - joinedStream.print().setParallelism(1); - } - - // execute program - env.execute("Windowed Join Example"); - } - - public static DataStream> runWindowJoin( - DataStream> grades, - DataStream> salaries, - long windowSize) { - - return grades.join(salaries) - .where(new NameKeySelector()) - .equalTo(new NameKeySelector()) - .window(TumblingEventTimeWindows.of(Duration.ofMillis(windowSize))) - .apply( - new JoinFunction< - Tuple2, - Tuple2, - Tuple3>() { - - @Override - public Tuple3 join( - Tuple2 first, Tuple2 second) { - return new Tuple3( - first.f0, first.f1, second.f1); - } - }); - } - - private static class NameKeySelector implements KeySelector, String> { - @Override - public String getKey(Tuple2 value) { - return value.f0; - } - } - - /** - * This {@link WatermarkStrategy} assigns the current system time as the event-time timestamp. - * In a real use case you should use proper timestamps and an appropriate {@link - * WatermarkStrategy}. - */ - private static class IngestionTimeWatermarkStrategy implements WatermarkStrategy { - - private IngestionTimeWatermarkStrategy() {} - - public static IngestionTimeWatermarkStrategy create() { - return new IngestionTimeWatermarkStrategy<>(); - } - - @Override - public WatermarkGenerator createWatermarkGenerator( - WatermarkGeneratorSupplier.Context context) { - return new AscendingTimestampsWatermarks<>(); - } - - @Override - public TimestampAssigner createTimestampAssigner( - TimestampAssignerSupplier.Context context) { - return (event, timestamp) -> System.currentTimeMillis(); - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoinSampleData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoinSampleData.java deleted file mode 100644 index 55f59ed..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoinSampleData.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.examples.join; - -import org.apache.flink.api.common.typeinfo.TypeHint; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.datagen.source.GeneratorFunction; - -import java.util.Random; - -/** Sample data for the {@link WindowJoin} example. */ -@SuppressWarnings("serial") -public class WindowJoinSampleData { - - static final String[] NAMES = {"tom", "jerry", "alice", "bob", "john", "grace"}; - static final int GRADE_COUNT = 5; - static final int SALARY_MAX = 10000; - - /** Continuously generates (name, grade). */ - public static DataGeneratorSource> getGradeGeneratorSource( - double elementsPerSecond) { - return getTupleGeneratorSource(GRADE_COUNT, elementsPerSecond); - } - - /** Continuously generates (name, salary). */ - public static DataGeneratorSource> getSalaryGeneratorSource( - double elementsPerSecond) { - return getTupleGeneratorSource(SALARY_MAX, elementsPerSecond); - } - - private static DataGeneratorSource> getTupleGeneratorSource( - int maxValue, double elementsPerSecond) { - final Random rnd = new Random(); - final GeneratorFunction> generatorFunction = - index -> new Tuple2<>(NAMES[rnd.nextInt(NAMES.length)], rnd.nextInt(maxValue) + 1); - - return new DataGeneratorSource<>( - generatorFunction, - Long.MAX_VALUE, - RateLimiterStrategy.perSecond(elementsPerSecond), - TypeInformation.of(new TypeHint>() {})); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java deleted file mode 100644 index 6e33f5d..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java +++ /dev/null @@ -1,206 +0,0 @@ -/* - * 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.examples.sideoutput; - -import org.apache.flink.api.common.eventtime.AscendingTimestampsWatermarks; -import org.apache.flink.api.common.eventtime.TimestampAssigner; -import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; -import org.apache.flink.api.common.eventtime.WatermarkGenerator; -import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.connector.file.sink.FileSink; -import org.apache.flink.connector.file.src.FileSource; -import org.apache.flink.connector.file.src.reader.TextLineInputFormat; -import org.apache.flink.core.fs.Path; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.ProcessFunction; -import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; -import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; -import org.apache.flink.streaming.examples.wordcount.util.WordCountData; -import org.apache.flink.util.Collector; -import org.apache.flink.util.OutputTag; - -import java.time.Duration; - -/** - * An example that illustrates the use of side output. - * - *

This is a modified version of {@link - * org.apache.flink.streaming.examples.windowing.WindowWordCount} that has a filter in the tokenizer - * and only emits some words for counting while emitting the other words to a side output. - */ -public class SideOutputExample { - - /** - * We need to create an {@link OutputTag} so that we can reference it when emitting data to a - * side output and also to retrieve the side output stream from an operation. - */ - private static final OutputTag rejectedWordsTag = new OutputTag("rejected") {}; - - public static void main(String[] args) throws Exception { - - // Checking input parameters - final ParameterTool params = ParameterTool.fromArgs(args); - - // set up the execution environment - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - // get input data - DataStream textWithTimestampAndWatermark; - if (params.has("input")) { - // read the text file from given input path - FileSource fileSource = - FileSource.forRecordStreamFormat( - new TextLineInputFormat(), new Path(params.get("input"))) - .build(); - textWithTimestampAndWatermark = - env.fromSource( - fileSource, IngestionTimeWatermarkStrategy.create(), "Words Source"); - } else { - System.out.println("Executing WordCount example with default input data set."); - System.out.println("Use --input to specify file input."); - // get default test text data - DataStreamSource text = env.fromData(WordCountData.WORDS); - // We assign the WatermarkStrategy after creating the source because - // StreamExecutionEnvironment#fromElemenets() methods currently does not accept - // WatermarkStrategies. In a real-world job you should integrate the WatermarkStrategy - // in the source as shown above for the FileSource. - textWithTimestampAndWatermark = - text.assignTimestampsAndWatermarks(IngestionTimeWatermarkStrategy.create()); - } - - SingleOutputStreamOperator> tokenized = - textWithTimestampAndWatermark.process(new Tokenizer()); - - DataStream rejectedWords = - tokenized - .getSideOutput(rejectedWordsTag) - .map(value -> "rejected: " + value, Types.STRING); - - DataStream> counts = - tokenized - .keyBy(value -> value.f0) - .window(TumblingEventTimeWindows.of(Duration.ofSeconds(5))) - // group by the tuple field "0" and sum up tuple field "1" - .sum(1); - - // emit result - if (params.has("output")) { - counts.sinkTo( - FileSink.>forRowFormat( - new Path(params.get("output")), - new SimpleStringEncoder<>()) - .withRollingPolicy( - DefaultRollingPolicy.builder() - .withMaxPartSize(MemorySize.ofMebiBytes(1)) - .withRolloverInterval(Duration.ofSeconds(10)) - .build()) - .build()) - .name("output"); - - rejectedWords - .sinkTo( - FileSink.forRowFormat( - new Path(params.get("rejected-words-output")), - new SimpleStringEncoder<>()) - .withRollingPolicy( - DefaultRollingPolicy.builder() - .withMaxPartSize(MemorySize.ofMebiBytes(1)) - .withRolloverInterval(Duration.ofSeconds(10)) - .build()) - .build()) - .name("rejected-words-output"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - counts.print(); - rejectedWords.print(); - } - - // execute program - env.execute("Streaming WordCount SideOutput"); - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** - * Implements the string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into multiple pairs in the - * form of "(word,1)" ({@code Tuple2}). - * - *

This rejects words that are longer than 5 characters long. - */ - public static final class Tokenizer extends ProcessFunction> { - private static final long serialVersionUID = 1L; - - @Override - public void processElement( - String value, Context ctx, Collector> out) - throws Exception { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 5) { - ctx.output(rejectedWordsTag, token); - } else if (token.length() > 0) { - out.collect(new Tuple2<>(token, 1)); - } - } - } - } - - /** - * This {@link WatermarkStrategy} assigns the current system time as the event-time timestamp. - * In a real use case you should use proper timestamps and an appropriate {@link - * WatermarkStrategy}. - */ - private static class IngestionTimeWatermarkStrategy implements WatermarkStrategy { - - private IngestionTimeWatermarkStrategy() {} - - public static IngestionTimeWatermarkStrategy create() { - return new IngestionTimeWatermarkStrategy<>(); - } - - @Override - public WatermarkGenerator createWatermarkGenerator( - WatermarkGeneratorSupplier.Context context) { - return new AscendingTimestampsWatermarks<>(); - } - - @Override - public TimestampAssigner createTimestampAssigner( - TimestampAssignerSupplier.Context context) { - return (event, timestamp) -> System.currentTimeMillis(); - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java deleted file mode 100644 index 584183a..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.examples.socket; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows; - -import java.time.Duration; - -/** - * Implements a streaming windowed version of the "WordCount" program. - * - *

This program connects to a server socket and reads strings from the socket. The easiest way to - * try this out is to open a text server (at port 12345) using the netcat tool via - * - *

- * nc -l 12345 on Linux or nc -l -p 12345 on Windows
- * 
- * - *

The state transition diagram is as follows: - * - *

- *           +--[a]--> W --[b]--> Y --[e]---+
- *           |                    ^         |
- *   Initial-+                    |         |
- *           |                    |         +--> (Z)-----[g]---> Terminal
- *           +--[c]--> X --[b]----+         |
- *                     |                    |
- *                     +--------[d]---------+
- * 
- */ -public enum State { - - /** The terminal state in the state machine. */ - Terminal, - - /** - * Special state returned by the State.transition(...) function when attempting an illegal state - * transition. - */ - InvalidTransition, - - /** State 'Z'. */ - Z(new Transition(EventType.g, Terminal, 1.0f)), - - /** State 'Y'. */ - Y(new Transition(EventType.e, Z, 1.0f)), - - /** State 'X'. */ - X(new Transition(EventType.b, Y, 0.2f), new Transition(EventType.d, Z, 0.8f)), - - /** State 'W'. */ - W(new Transition(EventType.b, Y, 1.0f)), - - /** The initial state from which all state sequences start. */ - Initial(new Transition(EventType.a, W, 0.6f), new Transition(EventType.c, X, 0.4f)); - - // ------------------------------------------------------------------------ - - private final Transition[] transitions; - - State(Transition... transitions) { - this.transitions = transitions; - } - - /** Checks if this state is a terminal state. A terminal state has no outgoing transitions. */ - public boolean isTerminal() { - return transitions.length == 0; - } - - // ------------------------------------------------------------------------ - - /** - * Gets the state after transitioning from this state based on the given event. If the - * transition is valid, this returns the new state, and if this transition is illegal, it - * returns [[InvalidTransition]]. - * - * @param evt The event that defined the transition. - * @return The new state, or [[InvalidTransition]]. - */ - public State transition(EventType evt) { - for (Transition t : transitions) { - if (t.eventType() == evt) { - return t.targetState(); - } - } - - // no transition found - return InvalidTransition; - } - - /** - * Picks a random transition, based on the probabilities of the outgoing transitions of this - * state. - * - * @param rnd The random number generator to use. - * @return A pair of (transition event , new state). - */ - public EventTypeAndState randomTransition(Random rnd) { - if (isTerminal()) { - throw new RuntimeException("Cannot transition from state " + name()); - } else { - final float p = rnd.nextFloat(); - float mass = 0.0f; - Transition transition = null; - - for (Transition t : transitions) { - mass += t.prob(); - if (p <= mass) { - transition = t; - break; - } - } - - assert transition != null; - return new EventTypeAndState(transition.eventType(), transition.targetState()); - } - } - - /** - * Returns an event type that, if applied as a transition on this state, will result in an - * illegal state transition. - * - * @param rnd The random number generator to use. - * @return And event type for an illegal state transition. - */ - public EventType randomInvalidTransition(Random rnd) { - while (true) { - EventType candidate = EventType.values()[rnd.nextInt(EventType.values().length)]; - if (transition(candidate) == InvalidTransition) { - return candidate; - } - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/Transition.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/Transition.java deleted file mode 100644 index 4f4b55d..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/Transition.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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.examples.statemachine.dfa; - -import org.apache.flink.streaming.examples.statemachine.event.EventType; - -import java.io.Serializable; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * A possible transition on a given event into a target state. The transition belongs to its - * originating state and has an associated probability that is used to generate random transition - * events. - */ -public class Transition implements Serializable { - - // this class is serializable to be able to interact cleanly with enums. - private static final long serialVersionUID = 1L; - - /** The event that triggers the transition. */ - private final EventType eventType; - - /** The target state after the transition. */ - private final State targetState; - - /** The probability of the transition. */ - private final float prob; - - /** - * Creates a new transition. - * - * @param eventType The event that triggers the transition. - * @param targetState The target state after the transition. - * @param prob The probability of the transition. - */ - public Transition(EventType eventType, State targetState, float prob) { - this.eventType = checkNotNull(eventType); - this.targetState = checkNotNull(targetState); - this.prob = prob; - } - - // ------------------------------------------------------------------------ - - public EventType eventType() { - return eventType; - } - - public State targetState() { - return targetState; - } - - public float prob() { - return prob; - } - - // ------------------------------------------------------------------------ - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } else if (obj == null || getClass() != obj.getClass()) { - return false; - } else { - final Transition that = (Transition) obj; - return this.eventType == that.eventType - && this.targetState == that.targetState - && Float.compare(this.prob, that.prob) == 0; - } - } - - @Override - public int hashCode() { - int code = 31 * eventType.hashCode() + targetState.hashCode(); - return 31 * code + (prob != +0.0f ? Float.floatToIntBits(prob) : 0); - } - - @Override - public String toString() { - return "--[" + eventType.name() + "]--> " + targetState.name() + " (" + prob + ')'; - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Alert.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Alert.java deleted file mode 100644 index 0dc204a..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Alert.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.examples.statemachine.event; - -import org.apache.flink.streaming.examples.statemachine.dfa.State; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** Data type for alerts. */ -public class Alert { - - private final int address; - - private final State state; - - private final EventType transition; - - /** - * Creates a new alert. - * - * @param address The originating address (think 32 bit IPv4 address). - * @param state The state that the event state machine found. - * @param transition The transition that was considered invalid. - */ - public Alert(int address, State state, EventType transition) { - this.address = address; - this.state = checkNotNull(state); - this.transition = checkNotNull(transition); - } - - // ------------------------------------------------------------------------ - - public int address() { - return address; - } - - public State state() { - return state; - } - - public EventType transition() { - return transition; - } - - // ------------------------------------------------------------------------ - - @Override - public int hashCode() { - int code = 31 * address + state.hashCode(); - return 31 * code + transition.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } else if (obj == null || getClass() != obj.getClass()) { - return false; - } else { - final Alert that = (Alert) obj; - return this.address == that.address - && this.transition == that.transition - && this.state == that.state; - } - } - - @Override - public String toString() { - return "ALERT " - + Event.formatAddress(address) - + " : " - + state.name() - + " -> " - + transition.name(); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Event.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Event.java deleted file mode 100644 index ec15bd1..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/Event.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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.examples.statemachine.event; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** Data type for events, consisting of the originating IP address and an event type. */ -public class Event { - - private final EventType type; - - private final int sourceAddress; - - /** - * Creates a new event. - * - * @param type The event type. - * @param sourceAddress The originating address (think 32 bit IPv4 address). - */ - public Event(EventType type, int sourceAddress) { - this.type = checkNotNull(type); - this.sourceAddress = sourceAddress; - } - - /** Gets the event's type. */ - public EventType type() { - return type; - } - - /** Gets the event's source address. */ - public int sourceAddress() { - return sourceAddress; - } - - // ------------------------------------------------------------------------ - // Miscellaneous - // ------------------------------------------------------------------------ - - @Override - public int hashCode() { - return 31 * type.hashCode() + sourceAddress; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } else if (obj == null || getClass() != obj.getClass()) { - return false; - } else { - final Event that = (Event) obj; - return this.type == that.type && this.sourceAddress == that.sourceAddress; - } - } - - @Override - public String toString() { - return "Event " + formatAddress(sourceAddress) + " : " + type.name(); - } - - // ------------------------------------------------------------------------ - // Utils - // ------------------------------------------------------------------------ - - /** - * Util method to create a string representation of a 32 bit integer representing an IPv4 - * address. - * - * @param address The address, MSB first. - * @return The IP address string. - */ - public static String formatAddress(int address) { - int b1 = (address >>> 24) & 0xff; - int b2 = (address >>> 16) & 0xff; - int b3 = (address >>> 8) & 0xff; - int b4 = address & 0xff; - - return "" + b1 + '.' + b2 + '.' + b3 + '.' + b4; - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/EventType.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/EventType.java deleted file mode 100644 index 9b81ff0..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/event/EventType.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.examples.statemachine.event; - -/** The type of the event processed by the state machine. */ -public enum EventType { - a, - b, - c, - d, - e, - f, - g; -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGenerator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGenerator.java deleted file mode 100644 index 6f78e73..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGenerator.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * 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.examples.statemachine.generator; - -import org.apache.flink.streaming.examples.statemachine.dfa.EventTypeAndState; -import org.apache.flink.streaming.examples.statemachine.dfa.State; -import org.apache.flink.streaming.examples.statemachine.event.Event; -import org.apache.flink.streaming.examples.statemachine.event.EventType; - -import javax.annotation.Nullable; - -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.Map.Entry; -import java.util.Random; - -import static org.apache.flink.util.Preconditions.checkArgument; - -/** - * A generator for events. The generator internally maintains a series of state machines (addresses - * and current associated state) and returns transition events from those state machines. Each time - * the next event is generators, this generator picks a random state machine and creates a random - * transition on that state machine. - * - *

The generator randomly adds new state machines, and removes state machines as soon as they - * reach the terminal state. This implementation maintains up to 1000 state machines concurrently. - */ -public class EventsGenerator { - - /** The random number generator. */ - private final Random rnd; - - /** The currently active state machines. */ - private final LinkedHashMap states; - - /** Probability with this generator generates an illegal state transition. */ - private final double errorProb; - - public EventsGenerator() { - this(0.0); - } - - public EventsGenerator(double errorProb) { - checkArgument(errorProb >= 0.0 && errorProb <= 1.0, "Invalid error probability"); - this.errorProb = errorProb; - - this.rnd = new Random(); - this.states = new LinkedHashMap<>(); - } - - // ------------------------------------------------------------------------ - - /** - * Creates a new random event. This method randomly pick either one of its currently running - * state machines, or start a new state machine for a random IP address. - * - *

With {@link #errorProb} probability, the generated event will be from an illegal state - * transition of one of the currently running state machines. - * - * @param minIp The lower bound for the range from which a new IP address may be picked. - * @param maxIp The upper bound for the range from which a new IP address may be picked. - * @return A next random event. - */ - public Event next(int minIp, int maxIp) { - final double p = rnd.nextDouble(); - - if (p * 1000 >= states.size()) { - // create a new state machine - final int nextIP = rnd.nextInt(maxIp - minIp) + minIp; - - if (!states.containsKey(nextIP)) { - EventTypeAndState eventAndState = State.Initial.randomTransition(rnd); - states.put(nextIP, eventAndState.state); - return new Event(eventAndState.eventType, nextIP); - } else { - // collision on IP address, try again - return next(minIp, maxIp); - } - } else { - // pick an existing state machine - - // skip over some elements in the linked map, then take the next - // update it, and insert it at the end - - int numToSkip = Math.min(20, rnd.nextInt(states.size())); - Iterator> iter = states.entrySet().iterator(); - - for (int i = numToSkip; i > 0; --i) { - iter.next(); - } - - Entry entry = iter.next(); - State currentState = entry.getValue(); - int address = entry.getKey(); - - iter.remove(); - - if (p < errorProb) { - EventType event = currentState.randomInvalidTransition(rnd); - return new Event(event, address); - } else { - EventTypeAndState eventAndState = currentState.randomTransition(rnd); - if (!eventAndState.state.isTerminal()) { - // reinsert - states.put(address, eventAndState.state); - } - - return new Event(eventAndState.eventType, address); - } - } - } - - /** - * Creates an event for an illegal state transition of one of the internal state machines. If - * the generator has not yet started any state machines (for example, because no call to {@link - * #next(int, int)} was made, yet), this will return null. - * - * @return An event for a illegal state transition, or null, if not possible. - */ - @Nullable - public Event nextInvalid() { - final Iterator> iter = states.entrySet().iterator(); - if (iter.hasNext()) { - final Entry entry = iter.next(); - - State currentState = entry.getValue(); - int address = entry.getKey(); - iter.remove(); - - EventType event = currentState.randomInvalidTransition(rnd); - return new Event(event, address); - } else { - return null; - } - } - - public int numActiveEntries() { - return states.size(); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorFunction.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorFunction.java deleted file mode 100644 index 70d6ec3..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorFunction.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.examples.statemachine.generator; - -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.datagen.source.GeneratorFunction; -import org.apache.flink.streaming.examples.statemachine.event.Event; - -import static org.apache.flink.util.Preconditions.checkArgument; - -/** A generator function that produces the events on the fly. Useful for self-contained demos. */ -@SuppressWarnings("serial") -public class EventsGeneratorFunction implements GeneratorFunction { - - private final double errorProbability; - - transient EventsGenerator generator; - private int min; - private int max; - - public EventsGeneratorFunction(double errorProbability) { - checkArgument( - errorProbability >= 0.0 && errorProbability <= 1.0, - "error probability must be in [0.0, 1.0]"); - - this.errorProbability = errorProbability; - } - - @Override - public void open(SourceReaderContext readerContext) throws Exception { - final int range = Integer.MAX_VALUE / readerContext.currentParallelism(); - min = range * readerContext.getIndexOfSubtask(); - max = min + range; - generator = new EventsGenerator(errorProbability); - } - - @Override - public Event map(Long value) throws Exception { - return generator.next(min, max); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/StandaloneThreadedGenerator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/StandaloneThreadedGenerator.java deleted file mode 100644 index 6ab8f38..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/StandaloneThreadedGenerator.java +++ /dev/null @@ -1,252 +0,0 @@ -/* - * 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.examples.statemachine.generator; - -import org.apache.flink.streaming.examples.statemachine.event.Event; -import org.apache.flink.util.Collector; - -import java.io.IOException; - -/** - * Base for standalone generators that use the state machine to create event sequences and push them - * for example into Kafka. - */ -public class StandaloneThreadedGenerator { - - public static void runGenerator(Collector[] collectors) throws IOException { - - final GeneratorThread[] threads = new GeneratorThread[collectors.length]; - final int range = Integer.MAX_VALUE / collectors.length; - - // create the generator threads - for (int i = 0; i < threads.length; i++) { - int min = range * i; - int max = min + range; - GeneratorThread thread = new GeneratorThread(collectors[i], min, max); - threads[i] = thread; - thread.setName("Generator " + i); - } - - long delay = 2L; - int nextErroneous = 0; - boolean running = true; - - for (GeneratorThread t : threads) { - t.setDelay(delay); - t.start(); - } - - final ThroughputLogger throughputLogger = new ThroughputLogger(threads); - throughputLogger.start(); - - System.out.println("Commands:"); - System.out.println(" -> q : Quit"); - System.out.println(" -> + : increase latency"); - System.out.println(" -> - : decrease latency"); - System.out.println(" -> e : inject invalid state transition"); - - // input loop - - while (running) { - final int next = System.in.read(); - - switch (next) { - case 'q': - System.out.println("Quitting..."); - running = false; - break; - - case 'e': - System.out.println("Injecting erroneous transition ..."); - threads[nextErroneous].sendInvalidStateTransition(); - nextErroneous = (nextErroneous + 1) % threads.length; - break; - - case '+': - delay = Math.max(delay * 2, 1); - System.out.println("Delay is " + delay); - for (GeneratorThread t : threads) { - t.setDelay(delay); - } - break; - - case '-': - delay /= 2; - System.out.println("Delay is " + delay); - for (GeneratorThread t : threads) { - t.setDelay(delay); - } - break; - - default: - // do nothing - } - } - - // shutdown - throughputLogger.shutdown(); - - for (GeneratorThread t : threads) { - t.shutdown(); - - try { - t.join(); - } catch (InterruptedException e) { - // restore interrupted status - Thread.currentThread().interrupt(); - } - } - } - - // ------------------------------------------------------------------------ - - /** - * A thread running a {@link EventsGenerator} and pushing generated events to the given - * collector (such as Kafka / Socket / ...). - */ - private static class GeneratorThread extends Thread { - - private final Collector out; - - private final int minAddress; - private final int maxAddress; - - private long delay; - - private long count; - - private volatile boolean running; - - private volatile boolean injectInvalidNext; - - /** - * Creates a new generator thread. - * - * @param out The collector to push the generated records to. - * @param minAddress The lower bound for the range from which a new IP address may be - * picked. - * @param maxAddress The upper bound for the range from which a new IP address may be - * picked. - */ - GeneratorThread(Collector out, int minAddress, int maxAddress) { - this.out = out; - this.minAddress = minAddress; - this.maxAddress = maxAddress; - this.running = true; - } - - @Override - public void run() { - final EventsGenerator generator = new EventsGenerator(); - - while (running) { - if (injectInvalidNext) { - injectInvalidNext = false; - Event next = generator.nextInvalid(); - if (next != null) { - out.collect(next); - } - } else { - out.collect(generator.next(minAddress, maxAddress)); - } - - count += 1; - - // sleep the delay to throttle - if (delay > 0) { - try { - Thread.sleep(delay); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - } - } - - public long currentCount() { - return count; - } - - public void shutdown() { - running = false; - interrupt(); - } - - public void setDelay(long delay) { - this.delay = delay; - } - - public void sendInvalidStateTransition() { - injectInvalidNext = true; - } - } - - // ------------------------------------------------------------------------ - - /** Thread that periodically print the number of elements generated per second. */ - private static class ThroughputLogger extends Thread { - - private final GeneratorThread[] generators; - - private volatile boolean running; - - /** - * Instantiates the throughput logger. - * - * @param generators The generator threads whose aggregate throughput should be logged. - */ - ThroughputLogger(GeneratorThread[] generators) { - this.generators = generators; - this.running = true; - } - - @Override - public void run() { - long lastCount = 0L; - long lastTimeStamp = System.currentTimeMillis(); - - while (running) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - break; - } - - long ts = System.currentTimeMillis(); - long currCount = 0L; - for (GeneratorThread generator : generators) { - currCount += generator.currentCount(); - } - - double factor = (ts - lastTimeStamp) / 1000.0; - double perSec = (currCount - lastCount) / factor; - - lastTimeStamp = ts; - lastCount = currCount; - - System.out.println(perSec + " / sec"); - } - } - - public void shutdown() { - running = false; - interrupt(); - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/EventDeSerializationSchema.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/EventDeSerializationSchema.java deleted file mode 100644 index 42bd675..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/EventDeSerializationSchema.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.examples.statemachine.kafka; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.examples.statemachine.event.Event; -import org.apache.flink.streaming.examples.statemachine.event.EventType; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; - -/** A serializer and deserializer for the {@link Event} type. */ -public class EventDeSerializationSchema - implements DeserializationSchema, SerializationSchema { - - private static final long serialVersionUID = 1L; - - @Override - public byte[] serialize(Event evt) { - ByteBuffer byteBuffer = ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN); - byteBuffer.putInt(0, evt.sourceAddress()); - byteBuffer.putInt(4, evt.type().ordinal()); - return byteBuffer.array(); - } - - @Override - public Event deserialize(byte[] message) throws IOException { - ByteBuffer buffer = ByteBuffer.wrap(message).order(ByteOrder.LITTLE_ENDIAN); - int address = buffer.getInt(0); - int typeOrdinal = buffer.getInt(4); - return new Event(EventType.values()[typeOrdinal], address); - } - - @Override - public boolean isEndOfStream(Event nextElement) { - return false; - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(Event.class); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/KafkaStandaloneGenerator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/KafkaStandaloneGenerator.java deleted file mode 100644 index 781cb09..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/kafka/KafkaStandaloneGenerator.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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.examples.statemachine.kafka; - -import org.apache.flink.streaming.examples.statemachine.event.Event; -import org.apache.flink.streaming.examples.statemachine.generator.StandaloneThreadedGenerator; -import org.apache.flink.util.Collector; - -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.ByteArraySerializer; - -import java.util.Properties; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** A generator that pushes the data into Kafka. */ -public class KafkaStandaloneGenerator extends StandaloneThreadedGenerator { - - public static final String BROKER_ADDRESS = "localhost:9092"; - - public static final String TOPIC = "flink-demo-topic-1"; - - public static final int NUM_PARTITIONS = 1; - - /** Entry point to the kafka data producer. */ - public static void main(String[] args) throws Exception { - - final KafkaCollector[] collectors = new KafkaCollector[NUM_PARTITIONS]; - - // create the generator threads - for (int i = 0; i < collectors.length; i++) { - collectors[i] = new KafkaCollector(BROKER_ADDRESS, TOPIC, i); - } - - StandaloneThreadedGenerator.runGenerator(collectors); - } - - // ------------------------------------------------------------------------ - - private static class KafkaCollector implements Collector, AutoCloseable { - - private final KafkaProducer producer; - - private final EventDeSerializationSchema serializer; - - private final String topic; - - private final int partition; - - KafkaCollector(String brokerAddress, String topic, int partition) { - this.topic = checkNotNull(topic); - this.partition = partition; - this.serializer = new EventDeSerializationSchema(); - - // create Kafka producer - Properties properties = new Properties(); - properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerAddress); - properties.put( - ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - ByteArraySerializer.class.getCanonicalName()); - properties.put( - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - ByteArraySerializer.class.getCanonicalName()); - this.producer = new KafkaProducer<>(properties); - } - - @Override - public void collect(Event evt) { - byte[] serialized = serializer.serialize(evt); - producer.send(new ProducerRecord<>(topic, partition, null, serialized)); - } - - @Override - public void close() { - producer.close(); - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/utils/ThrottledIterator.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/utils/ThrottledIterator.java deleted file mode 100644 index 830f0d9..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/utils/ThrottledIterator.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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.examples.utils; - -import java.io.Serializable; -import java.util.Iterator; - -import static java.util.Objects.requireNonNull; - -/** - * A variant of the collection source (emits a sequence of elements as a stream) that supports - * throttling the emission rate. - * - * @param - */ -public class ThrottledIterator implements Iterator, Serializable { - - private static final long serialVersionUID = 1L; - - @SuppressWarnings("NonSerializableFieldInSerializableClass") - private final Iterator source; - - private final long sleepBatchSize; - private final long sleepBatchTime; - - private long lastBatchCheckTime; - private long num; - - public ThrottledIterator(Iterator source, long elementsPerSecond) { - this.source = requireNonNull(source); - - if (!(source instanceof Serializable)) { - throw new IllegalArgumentException("source must be java.io.Serializable"); - } - - if (elementsPerSecond >= 100) { - // how many elements would we emit per 50ms - this.sleepBatchSize = elementsPerSecond / 20; - this.sleepBatchTime = 50; - } else if (elementsPerSecond >= 1) { - // how long does element take - this.sleepBatchSize = 1; - this.sleepBatchTime = 1000 / elementsPerSecond; - } else { - throw new IllegalArgumentException( - "'elements per second' must be positive and not zero"); - } - } - - @Override - public boolean hasNext() { - return source.hasNext(); - } - - @Override - public T next() { - // delay if necessary - if (lastBatchCheckTime > 0) { - if (++num >= sleepBatchSize) { - num = 0; - - final long now = System.currentTimeMillis(); - final long elapsed = now - lastBatchCheckTime; - if (elapsed < sleepBatchTime) { - try { - Thread.sleep(sleepBatchTime - elapsed); - } catch (InterruptedException e) { - // restore interrupt flag and proceed - Thread.currentThread().interrupt(); - } - } - lastBatchCheckTime = now; - } - } else { - lastBatchCheckTime = System.currentTimeMillis(); - } - - return source.next(); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java deleted file mode 100644 index 9db4e35..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * 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.examples.windowing; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.datagen.source.GeneratorFunction; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; -import org.apache.flink.streaming.api.functions.windowing.WindowFunction; -import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows; -import org.apache.flink.streaming.api.windowing.windows.Window; -import org.apache.flink.util.Collector; - -import java.time.Duration; - -/** An example of grouped stream windowing into sliding time windows. */ -public class GroupedProcessingTimeWindowExample { - - public static void main(String[] args) throws Exception { - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - final long numElementsPerParallel = 20000000; - final long numKeys = 10000; - - GeneratorFunction> generatorFunction = - new DataGeneratorFunction(numElementsPerParallel, numKeys); - - DataGeneratorSource> generatorSource = - new DataGeneratorSource<>( - generatorFunction, - numElementsPerParallel * env.getParallelism(), - Types.TUPLE(Types.LONG, Types.LONG)); - - DataStream> stream = - env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); - - stream.keyBy(value -> value.f0) - .window( - SlidingProcessingTimeWindows.of( - Duration.ofMillis(2500), Duration.ofMillis(500))) - .reduce(new SummingReducer()) - - // alternative: use a apply function which does not pre-aggregate - // .keyBy(new FirstFieldKeyExtractor, Long>()) - // .window(SlidingProcessingTimeWindows.of(Time.milliseconds(2500), - // Time.milliseconds(500))) - // .apply(new SummingWindowFunction()) - - .sinkTo(new DiscardingSink<>()); - - env.execute(); - } - - private static class FirstFieldKeyExtractor - implements KeySelector { - - @Override - @SuppressWarnings("unchecked") - public Key getKey(Type value) { - return (Key) value.getField(0); - } - } - - private static class SummingWindowFunction - implements WindowFunction, Tuple2, Long, Window> { - - @Override - public void apply( - Long key, - Window window, - Iterable> values, - Collector> out) { - long sum = 0L; - for (Tuple2 value : values) { - sum += value.f1; - } - - out.collect(new Tuple2<>(key, sum)); - } - } - - private static class SummingReducer implements ReduceFunction> { - - @Override - public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { - return new Tuple2<>(value1.f0, value1.f1 + value2.f1); - } - } - - /** - * This class represents a data generator function that generates a stream of tuples. Each tuple - * contains a key and a value. The function measures and prints the time it takes to generate - * numElements. The key space is limited to numKeys. The value is always 1. - */ - private static class DataGeneratorFunction - implements GeneratorFunction> { - - private final long numElements; - private final long numKeys; - private long startTime; - - public DataGeneratorFunction(long numElements, long numKeys) { - this.numElements = numElements; - this.numKeys = numKeys; - } - - @Override - public Tuple2 map(Long value) throws Exception { - if ((value % numElements) == 0) { - startTime = System.currentTimeMillis(); - } - if ((value % numElements + 1) == numElements) { - final long endTime = System.currentTimeMillis(); - System.out.println( - Thread.currentThread() - + ": Took " - + (endTime - startTime) - + " msecs for " - + numElements - + " values"); - } - return new Tuple2<>(value % numKeys, 1L); - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java deleted file mode 100644 index 5708a7a..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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.examples.windowing; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; -import org.apache.flink.api.common.typeinfo.TypeHint; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.datagen.source.GeneratorFunction; -import org.apache.flink.connector.file.sink.FileSink; -import org.apache.flink.core.fs.Path; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; -import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.List; - -/** - * An example of session windowing that keys events by ID and groups and counts them in session with - * gaps of 3 milliseconds. - */ -public class SessionWindowing { - - public static void main(String[] args) throws Exception { - - final ParameterTool params = ParameterTool.fromArgs(args); - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - env.getConfig().setGlobalJobParameters(params); - env.setParallelism(1); - - final boolean fileOutput = params.has("output"); - - final List> input = new ArrayList<>(); - - input.add(new Tuple3<>("a", 1L, 1)); - input.add(new Tuple3<>("b", 1L, 1)); - input.add(new Tuple3<>("b", 3L, 1)); - input.add(new Tuple3<>("b", 5L, 1)); - input.add(new Tuple3<>("c", 6L, 1)); - // We expect to detect the session "a" earlier than this point (the old - // functionality can only detect here when the next starts) - input.add(new Tuple3<>("a", 10L, 1)); - // We expect to detect session "b" and "c" at this point as well - input.add(new Tuple3<>("c", 11L, 1)); - - GeneratorFunction> dataGenerator = - index -> input.get(index.intValue()); - DataGeneratorSource> generatorSource = - new DataGeneratorSource<>( - dataGenerator, - input.size(), - TypeInformation.of(new TypeHint>() {})); - - DataStream> source = - env.fromSource( - generatorSource, - WatermarkStrategy.>forMonotonousTimestamps() - .withTimestampAssigner((event, timestamp) -> event.f1), - "Generated data source"); - - // We create sessions for each id with max timeout of 3 time units - DataStream> aggregated = - source.keyBy(value -> value.f0) - .window(EventTimeSessionWindows.withGap(Duration.ofMillis(3L))) - .sum(2); - - if (fileOutput) { - aggregated - .sinkTo( - FileSink.>forRowFormat( - new Path(params.get("output")), - new SimpleStringEncoder<>()) - .withRollingPolicy( - DefaultRollingPolicy.builder() - .withMaxPartSize(MemorySize.ofMebiBytes(1)) - .withRolloverInterval(Duration.ofSeconds(10)) - .build()) - .build()) - .name("output"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - aggregated.print(); - } - - env.execute(); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java deleted file mode 100644 index 135e48f..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java +++ /dev/null @@ -1,194 +0,0 @@ -/* - * 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.examples.windowing; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; -import org.apache.flink.api.common.typeinfo.TypeHint; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.util.ratelimit.GuavaRateLimiter; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.file.sink.FileSink; -import org.apache.flink.connector.file.src.FileSource; -import org.apache.flink.connector.file.src.reader.TextLineInputFormat; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; -import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction; -import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; -import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; -import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger; -import org.apache.flink.streaming.examples.windowing.util.CarGeneratorFunction; -import org.apache.flink.streaming.examples.wordcount.util.CLI; - -import java.time.Duration; - -/** - * An example of grouped stream windowing where different eviction and trigger policies can be used. - * A source fetches events from cars containing their id, their current speed (kmh), overall elapsed - * distance (m) and a timestamp. The streaming example triggers the top speed of each car every x - * meters elapsed for the last y seconds. - */ -public class TopSpeedWindowing { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - final CLI params = CLI.fromArgs(args); - - // Create the execution environment. This is the main entrypoint - // to building a Flink application. - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // Apache Flink’s unified approach to stream and batch processing means that a DataStream - // application executed over bounded input will produce the same final results regardless - // of the configured execution mode. It is important to note what final means here: a job - // executing in STREAMING mode might produce incremental updates (think upserts in - // a database) while a BATCH job would only produce one final result at the end. The final - // result will be the same if interpreted correctly, but getting there can be different. - // - // The “classic” execution behavior of the DataStream API is called STREAMING execution - // mode. Applications should use streaming execution for unbounded jobs that require - // continuous incremental processing and are expected to stay online indefinitely. - // - // By enabling BATCH execution, we allow Flink to apply additional optimizations that we - // can only do when we know that our input is bounded. For example, different - // join/aggregation strategies can be used, in addition to a different shuffle - // implementation that allows more efficient task scheduling and failure recovery behavior. - // - // By setting the runtime mode to AUTOMATIC, Flink will choose BATCH if all sources - // are bounded and otherwise STREAMING. - env.setRuntimeMode(params.getExecutionMode()); - - // This optional step makes the input parameters - // available in the Flink UI. - env.getConfig().setGlobalJobParameters(params); - - SingleOutputStreamOperator> carData; - if (params.getInputs().isPresent()) { - // Create a new file source that will read files from a given set of directories. - // Each file will be processed as plain text and split based on newlines. - FileSource.FileSourceBuilder builder = - FileSource.forRecordStreamFormat( - new TextLineInputFormat(), params.getInputs().get()); - - // If a discovery interval is provided, the source will - // continuously watch the given directories for new files. - params.getDiscoveryInterval().ifPresent(builder::monitorContinuously); - - carData = - env.fromSource(builder.build(), WatermarkStrategy.noWatermarks(), "file-input") - .map(new ParseCarData()) - .name("parse-input"); - } else { - CarGeneratorFunction carGenerator = new CarGeneratorFunction(2); - DataGeneratorSource> carGeneratorSource = - new DataGeneratorSource<>( - carGenerator, - Long.MAX_VALUE, - parallelismIgnored -> new GuavaRateLimiter(10), - TypeInformation.of( - new TypeHint>() {})); - carData = - env.fromSource( - carGeneratorSource, - WatermarkStrategy.noWatermarks(), - "Car data generator source"); - carData.setParallelism(1); - } - - int evictionSec = 10; - double triggerMeters = 50; - DataStream> topSpeeds = - carData.assignTimestampsAndWatermarks( - WatermarkStrategy - .> - forMonotonousTimestamps() - .withTimestampAssigner((car, ts) -> car.f3)) - .keyBy(value -> value.f0) - .window(GlobalWindows.create()) - .evictor(TimeEvictor.of(Duration.ofSeconds(evictionSec))) - .trigger( - DeltaTrigger.of( - triggerMeters, - new DeltaFunction< - Tuple4>() { - private static final long serialVersionUID = 1L; - - @Override - public double getDelta( - Tuple4 - oldDataPoint, - Tuple4 - newDataPoint) { - return newDataPoint.f2 - oldDataPoint.f2; - } - }, - carData.getType() - .createSerializer( - env.getConfig().getSerializerConfig()))) - .maxBy(1); - - if (params.getOutput().isPresent()) { - // Given an output directory, Flink will write the results to a file - // using a simple string encoding. In a production environment, this might - // be something more structured like CSV, Avro, JSON, or Parquet. - topSpeeds - .sinkTo( - FileSink.>forRowFormat( - params.getOutput().get(), new SimpleStringEncoder<>()) - .withRollingPolicy( - DefaultRollingPolicy.builder() - .withMaxPartSize(MemorySize.ofMebiBytes(1)) - .withRolloverInterval(Duration.ofSeconds(10)) - .build()) - .build()) - .name("file-sink"); - } else { - topSpeeds.print(); - } - - env.execute("CarTopSpeedWindowingExample"); - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - private static class ParseCarData - extends RichMapFunction> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple4 map(String record) { - String rawData = record.substring(1, record.length() - 1); - String[] data = rawData.split(","); - return new Tuple4<>( - Integer.valueOf(data[0]), - Integer.valueOf(data[1]), - Double.valueOf(data[2]), - Long.valueOf(data[3])); - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java deleted file mode 100644 index 6075edf..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * 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.examples.windowing; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.connector.file.sink.FileSink; -import org.apache.flink.connector.file.src.FileSource; -import org.apache.flink.connector.file.src.reader.TextLineInputFormat; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; -import org.apache.flink.streaming.examples.wordcount.WordCount; -import org.apache.flink.streaming.examples.wordcount.util.CLI; -import org.apache.flink.streaming.examples.wordcount.util.WordCountData; - -import java.time.Duration; - -/** - * Implements a windowed version of the streaming "WordCount" program. - * - *

The input is a plain text file with lines separated by newline characters. - * - *

Usage: - * WordCount --input <path> --output <path> --window <n> --slide <n> - *
- * If no parameters are provided, the program is run with default data from {@link WordCountData}. - * - *

This example shows how to: - * - *

    - *
  • write a simple Flink Streaming program, - *
  • use tuple data types, - *
  • use basic windowing abstractions. - *
- */ -public class WindowWordCount { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - final CLI params = CLI.fromArgs(args); - - // Create the execution environment. This is the main entrypoint - // to building a Flink application. - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // Apache Flink’s unified approach to stream and batch processing means that a DataStream - // application executed over bounded input will produce the same final results regardless - // of the configured execution mode. It is important to note what final means here: a job - // executing in STREAMING mode might produce incremental updates (think upserts in - // a database) while a BATCH job would only produce one final result at the end. The final - // result will be the same if interpreted correctly, but getting there can be different. - // - // The “classic” execution behavior of the DataStream API is called STREAMING execution - // mode. Applications should use streaming execution for unbounded jobs that require - // continuous incremental processing and are expected to stay online indefinitely. - // - // By enabling BATCH execution, we allow Flink to apply additional optimizations that we - // can only do when we know that our input is bounded. For example, different - // join/aggregation strategies can be used, in addition to a different shuffle - // implementation that allows more efficient task scheduling and failure recovery behavior. - // - // By setting the runtime mode to AUTOMATIC, Flink will choose BATCH if all sources - // are bounded and otherwise STREAMING. - env.setRuntimeMode(params.getExecutionMode()); - - // This optional step makes the input parameters - // available in the Flink UI. - env.getConfig().setGlobalJobParameters(params); - - DataStream text; - if (params.getInputs().isPresent()) { - // Create a new file source that will read files from a given set of directories. - // Each file will be processed as plain text and split based on newlines. - FileSource.FileSourceBuilder builder = - FileSource.forRecordStreamFormat( - new TextLineInputFormat(), params.getInputs().get()); - - // If a discovery interval is provided, the source will - // continuously watch the given directories for new files. - params.getDiscoveryInterval().ifPresent(builder::monitorContinuously); - - text = env.fromSource(builder.build(), WatermarkStrategy.noWatermarks(), "file-input"); - } else { - text = env.fromData(WordCountData.WORDS).name("in-memory-input"); - } - - int windowSize = params.getInt("window").orElse(250); - int slideSize = params.getInt("slide").orElse(150); - - DataStream> counts = - // The text lines read from the source are split into words - // using a user-defined function. The tokenizer, implemented below, - // will output each words as a (2-tuple) containing (word, 1) - text.flatMap(new WordCount.Tokenizer()) - .name("tokenizer") - // keyBy groups tuples based on the "0" field, the word. - // Using a keyBy allows performing aggregations and other - // stateful transformations over data on a per-key basis. - // This is similar to a GROUP BY clause in a SQL query. - .keyBy(value -> value.f0) - // create windows of windowSize records slided every slideSize records - .countWindow(windowSize, slideSize) - // For each key, we perform a simple sum of the "1" field, the count. - // If the input data set is bounded, sum will output a final count for - // each word. If it is unbounded, it will continuously output updates - // each time it sees a new instance of each word in the stream. - .sum(1) - .name("counter"); - - if (params.getOutput().isPresent()) { - // Given an output directory, Flink will write the results to a file - // using a simple string encoding. In a production environment, this might - // be something more structured like CSV, Avro, JSON, or Parquet. - counts.sinkTo( - FileSink.>forRowFormat( - params.getOutput().get(), new SimpleStringEncoder<>()) - .withRollingPolicy( - DefaultRollingPolicy.builder() - .withMaxPartSize(MemorySize.ofMebiBytes(1)) - .withRolloverInterval(Duration.ofSeconds(10)) - .build()) - .build()) - .name("file-sink"); - } else { - counts.print().name("print-sink"); - } - - // Apache Flink applications are composed lazily. Calling execute - // submits the Job and begins processing. - env.execute("WindowWordCount"); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.java deleted file mode 100644 index eb7cecd..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/CarGeneratorFunction.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.examples.windowing.util; - -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.connector.datagen.source.GeneratorFunction; - -import java.util.Arrays; -import java.util.Random; - -/** - * A generator function for simulating car data. - * - *

This generator function generates a stream of car data in a form of a four-element tuple. The - * data includes the car's ID, its speed in kilometers per hour, the distance it has traveled in - * meters, and the timestamp of the data generation. The speed and distance of each car are randomly - * updated in each invocation of the {@link #map(Long)} method. - */ -public class CarGeneratorFunction - implements GeneratorFunction> { - - private static final long serialVersionUID = 1L; - // in kilometers per hour - private final int[] speeds; - // in meters - private final double[] distances; - // in milliseconds - private final long[] lastUpdate; - private int nextCar; - - private static final int MILLIS_IN_HOUR = 1000 * 60 * 60; - private static final double HOURS_IN_MILLI = 1d / MILLIS_IN_HOUR; - private static final int METERS_IN_KILOMETER = 1000; - - private final Random rand = new Random(); - - // Previous version (CarSource) was overestimating the speed. This factor is used to preserve - // the original behaviour of the example. - private static final int COMPAT_FACTOR = 10; - - public CarGeneratorFunction(int numOfCars) { - speeds = new int[numOfCars]; - distances = new double[numOfCars]; - lastUpdate = new long[numOfCars]; - Arrays.fill(speeds, 50); - Arrays.fill(distances, 0d); - Arrays.fill(lastUpdate, 0); - } - - @Override - public Tuple4 map(Long ignoredIndex) throws Exception { - if (rand.nextBoolean()) { - speeds[nextCar] = Math.min(100, speeds[nextCar] + 5); - } else { - speeds[nextCar] = Math.max(0, speeds[nextCar] - 5); - } - long now = System.currentTimeMillis(); - long timeDiffMillis = lastUpdate[nextCar] == 0 ? 0 : now - lastUpdate[nextCar]; - lastUpdate[nextCar] = now; - distances[nextCar] += - speeds[nextCar] - * (timeDiffMillis * HOURS_IN_MILLI) - * METERS_IN_KILOMETER - * COMPAT_FACTOR; - nextCar = (++nextCar) % speeds.length; - return new Tuple4<>(nextCar, speeds[nextCar], distances[nextCar], now); - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java deleted file mode 100644 index 955c6aa..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.examples.windowing.util; - -/** Data for SessionWindowingITCase. */ -public class SessionWindowingData { - - public static final String EXPECTED = - "(a,1,1)\n" + "(c,6,1)\n" + "(c,11,1)\n" + "(b,1,3)\n" + "(a,10,1)"; - - private SessionWindowingData() {} -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java deleted file mode 100644 index ab02b3b..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java +++ /dev/null @@ -1,372 +0,0 @@ -/* - * 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.examples.windowing.util; - -/** Data for TopSpeedWindowingExampleITCase. */ -public class TopSpeedWindowingExampleData { - - public static final String CAR_DATA = - "(0,55,15.277777777777777,1424951918630)\n" - + "(1,45,12.5,1424951918632)\n" - + "(0,50,29.166666666666664,1424951919632)\n" - + "(1,50,26.38888888888889,1424951919632)\n" - + "(0,55,44.44444444444444,1424951920633)\n" - + "(1,45,38.888888888888886,1424951920633)\n" - + "(0,50,58.33333333333333,1424951921634)\n" - + "(1,40,50.0,1424951921634)\n" - + "(0,55,73.6111111111111,1424951922634)\n" - + "(1,35,59.72222222222222,1424951922634)\n" - + "(0,60,90.27777777777777,1424951923634)\n" - + "(1,40,70.83333333333333,1424951923634)\n" - + "(0,65,108.33333333333333,1424951924635)\n" - + "(1,35,80.55555555555554,1424951924635)\n" - + "(0,60,125.0,1424951925635)\n" - + "(1,40,91.66666666666666,1424951925635)\n" - + "(0,55,140.27777777777777,1424951926635)\n" - + "(1,45,104.16666666666666,1424951926636)\n" - + "(0,60,156.94444444444443,1424951927636)\n" - + "(1,50,118.05555555555554,1424951927636)\n" - + "(0,55,172.2222222222222,1424951928636)\n" - + "(1,45,130.55555555555554,1424951928636)\n" - + "(0,50,186.1111111111111,1424951929636)\n" - + "(1,50,144.44444444444443,1424951929637)\n" - + "(0,55,201.38888888888886,1424951930637)\n" - + "(1,55,159.7222222222222,1424951930637)\n" - + "(0,60,218.05555555555551,1424951931637)\n" - + "(1,60,176.38888888888886,1424951931637)\n" - + "(0,55,233.3333333333333,1424951932637)\n" - + "(1,65,194.4444444444444,1424951932638)\n" - + "(0,50,247.22222222222217,1424951933638)\n" - + "(1,70,213.88888888888886,1424951933638)\n" - + "(0,45,259.7222222222222,1424951934638)\n" - + "(1,65,231.9444444444444,1424951934638)\n" - + "(0,50,273.6111111111111,1424951935638)\n" - + "(1,70,251.38888888888886,1424951935639)\n" - + "(0,55,288.88888888888886,1424951936639)\n" - + "(1,75,272.2222222222222,1424951936639)\n" - + "(0,50,302.77777777777777,1424951937639)\n" - + "(1,70,291.66666666666663,1424951937639)\n" - + "(0,45,315.27777777777777,1424951938640)\n" - + "(1,65,309.7222222222222,1424951938640)\n" - + "(0,50,329.1666666666667,1424951939640)\n" - + "(1,70,329.16666666666663,1424951939640)\n" - + "(0,55,344.44444444444446,1424951940640)\n" - + "(1,65,347.2222222222222,1424951940640)\n" - + "(0,50,358.33333333333337,1424951941641)\n" - + "(1,70,366.66666666666663,1424951941641)\n" - + "(0,55,373.61111111111114,1424951942641)\n" - + "(1,65,384.7222222222222,1424951942641)\n" - + "(0,50,387.50000000000006,1424951943641)\n" - + "(1,70,404.16666666666663,1424951943641)\n" - + "(0,45,400.00000000000006,1424951944642)\n" - + "(1,65,422.2222222222222,1424951944642)\n" - + "(0,50,413.88888888888897,1424951945642)\n" - + "(1,60,438.88888888888886,1424951945642)\n" - + "(0,45,426.38888888888897,1424951946642)\n" - + "(1,65,456.9444444444444,1424951946642)\n" - + "(0,40,437.50000000000006,1424951947643)\n" - + "(1,70,476.38888888888886,1424951947643)\n" - + "(0,45,450.00000000000006,1424951948643)\n" - + "(1,75,497.2222222222222,1424951948643)\n" - + "(0,40,461.11111111111114,1424951949643)\n" - + "(1,80,519.4444444444443,1424951949644)\n" - + "(0,45,473.61111111111114,1424951950644)\n" - + "(1,75,540.2777777777777,1424951950644)\n" - + "(0,50,487.50000000000006,1424951951644)\n" - + "(1,80,562.4999999999999,1424951951644)\n" - + "(0,45,500.00000000000006,1424951952644)\n" - + "(1,85,586.111111111111,1424951952645)\n" - + "(0,40,511.11111111111114,1424951953645)\n" - + "(1,80,608.3333333333331,1424951953645)\n" - + "(0,35,520.8333333333334,1424951954645)\n" - + "(1,75,629.1666666666665,1424951954645)\n" - + "(0,40,531.9444444444445,1424951955645)\n" - + "(1,70,648.611111111111,1424951955646)\n" - + "(0,45,544.4444444444445,1424951956646)\n" - + "(1,75,669.4444444444443,1424951956646)\n" - + "(0,50,558.3333333333334,1424951957646)\n" - + "(1,80,691.6666666666665,1424951957646)\n" - + "(0,55,573.6111111111112,1424951958646)\n" - + "(1,85,715.2777777777776,1424951958647)\n" - + "(0,60,590.2777777777778,1424951959647)\n" - + "(1,80,737.4999999999998,1424951959647)\n" - + "(0,65,608.3333333333334,1424951960647)\n" - + "(1,85,761.1111111111109,1424951960647)\n" - + "(0,70,627.7777777777778,1424951961647)\n" - + "(1,80,783.333333333333,1424951961648)\n" - + "(0,75,648.6111111111112,1424951962648)\n" - + "(1,85,806.9444444444441,1424951962648)\n" - + "(0,80,670.8333333333334,1424951963648)\n" - + "(1,90,831.9444444444441,1424951963648)\n" - + "(0,75,691.6666666666667,1424951964649)\n" - + "(1,95,858.333333333333,1424951964649)\n" - + "(0,70,711.1111111111112,1424951965649)\n" - + "(1,90,883.333333333333,1424951965649)\n" - + "(0,75,731.9444444444446,1424951966649)\n" - + "(1,95,909.722222222222,1424951966649)\n" - + "(0,70,751.388888888889,1424951967649)\n" - + "(1,100,937.4999999999998,1424951967650)\n" - + "(0,75,772.2222222222224,1424951968650)\n" - + "(1,100,965.2777777777776,1424951968650)\n" - + "(0,80,794.4444444444446,1424951969650)\n" - + "(1,100,993.0555555555554,1424951969650)\n" - + "(0,75,815.2777777777779,1424951970651)\n" - + "(1,100,1020.8333333333333,1424951970651)\n" - + "(0,80,837.5000000000001,1424951971651)\n" - + "(1,100,1048.611111111111,1424951971651)\n" - + "(0,85,861.1111111111112,1424951972651)\n" - + "(1,100,1076.388888888889,1424951972651)\n" - + "(0,80,883.3333333333334,1424951973652)\n" - + "(1,95,1102.7777777777778,1424951973652)\n" - + "(0,75,904.1666666666667,1424951974652)\n" - + "(1,100,1130.5555555555557,1424951974652)\n" - + "(0,70,923.6111111111112,1424951975652)\n" - + "(1,100,1158.3333333333335,1424951975652)\n" - + "(0,75,944.4444444444446,1424951976653)\n" - + "(1,100,1186.1111111111113,1424951976653)\n" - + "(0,80,966.6666666666667,1424951977653)\n" - + "(1,95,1212.5000000000002,1424951977653)\n" - + "(0,75,987.5000000000001,1424951978653)\n" - + "(1,100,1240.277777777778,1424951978653)\n" - + "(0,80,1009.7222222222223,1424951979654)\n" - + "(1,100,1268.0555555555559,1424951979654)\n" - + "(0,85,1033.3333333333335,1424951980654)\n" - + "(1,100,1295.8333333333337,1424951980654)\n" - + "(0,90,1058.3333333333335,1424951981654)\n" - + "(1,100,1323.6111111111115,1424951981654)\n" - + "(0,85,1081.9444444444446,1424951982655)\n" - + "(1,100,1351.3888888888894,1424951982655)\n" - + "(0,90,1106.9444444444446,1424951983655)\n" - + "(1,100,1379.1666666666672,1424951983655)\n" - + "(0,95,1133.3333333333335,1424951984655)\n" - + "(1,100,1406.944444444445,1424951984656)\n" - + "(0,90,1158.3333333333335,1424951985656)\n" - + "(1,95,1433.333333333334,1424951985656)\n" - + "(0,95,1184.7222222222224,1424951986656)\n" - + "(1,90,1458.333333333334,1424951986656)\n" - + "(0,90,1209.7222222222224,1424951987656)\n" - + "(1,95,1484.7222222222229,1424951987657)\n" - + "(0,85,1233.3333333333335,1424951988657)\n" - + "(1,90,1509.7222222222229,1424951988657)\n" - + "(0,80,1255.5555555555557,1424951989657)\n" - + "(1,95,1536.1111111111118,1424951989657)\n" - + "(0,85,1279.1666666666667,1424951990657)\n" - + "(1,100,1563.8888888888896,1424951990658)\n" - + "(0,90,1304.1666666666667,1424951991658)\n" - + "(1,95,1590.2777777777785,1424951991658)\n" - + "(0,95,1330.5555555555557,1424951992658)\n" - + "(1,90,1615.2777777777785,1424951992658)\n" - + "(0,100,1358.3333333333335,1424951993659)\n" - + "(1,95,1641.6666666666674,1424951993659)\n" - + "(0,100,1386.1111111111113,1424951994659)\n" - + "(1,100,1669.4444444444453,1424951994659)\n" - + "(0,95,1412.5000000000002,1424951995659)\n" - + "(1,95,1695.8333333333342,1424951995660)\n" - + "(0,100,1440.277777777778,1424951996660)\n" - + "(1,90,1720.8333333333342,1424951996660)\n" - + "(0,100,1468.0555555555559,1424951997660)\n" - + "(1,85,1744.4444444444453,1424951997660)\n" - + "(0,95,1494.4444444444448,1424951998660)\n" - + "(1,80,1766.6666666666674,1424951998661)\n" - + "(0,100,1522.2222222222226,1424951999661)\n" - + "(1,75,1787.5000000000007,1424951999661)\n" - + "(0,95,1548.6111111111115,1424952000661)\n" - + "(1,80,1809.7222222222229,1424952000661)\n" - + "(0,90,1573.6111111111115,1424952001662)\n" - + "(1,75,1830.555555555556,1424952001662)\n" - + "(0,95,1600.0000000000005,1424952002662)\n" - + "(1,80,1852.7777777777783,1424952002662)\n" - + "(0,100,1627.7777777777783,1424952003662)\n" - + "(1,85,1876.3888888888894,1424952003662)\n" - + "(0,100,1655.555555555556,1424952004663)\n" - + "(1,80,1898.6111111111115,1424952004663)\n" - + "(0,95,1681.944444444445,1424952005663)\n" - + "(1,85,1922.2222222222226,1424952005663)\n" - + "(0,100,1709.7222222222229,1424952006663)\n" - + "(1,90,1947.2222222222226,1424952006664)\n" - + "(0,100,1737.5000000000007,1424952007664)\n" - + "(1,95,1973.6111111111115,1424952007664)\n" - + "(0,95,1763.8888888888896,1424952008664)\n" - + "(1,90,1998.6111111111115,1424952008664)\n" - + "(0,100,1791.6666666666674,1424952009664)\n" - + "(1,85,2022.2222222222226,1424952009665)\n" - + "(0,95,1818.0555555555563,1424952010665)\n" - + "(1,80,2044.4444444444448,1424952010665)\n" - + "(0,90,1843.0555555555563,1424952011665)\n" - + "(1,75,2065.2777777777783,1424952011665)\n" - + "(0,95,1869.4444444444453,1424952012666)\n" - + "(1,80,2087.5000000000005,1424952012666)\n" - + "(0,100,1897.222222222223,1424952013666)\n" - + "(1,85,2111.1111111111118,1424952013666)\n" - + "(0,95,1923.611111111112,1424952014666)\n" - + "(1,90,2136.1111111111118,1424952014666)\n" - + "(0,100,1951.3888888888898,1424952015667)\n" - + "(1,85,2159.722222222223,1424952015667)\n" - + "(0,95,1977.7777777777787,1424952016667)\n" - + "(1,90,2184.722222222223,1424952016667)\n" - + "(0,100,2005.5555555555566,1424952017667)\n" - + "(1,95,2211.1111111111118,1424952017668)"; - - public static final String TOP_SPEEDS = - "(0,55,15.277777777777777,1424951918630)\n" - + "(1,50,26.38888888888889,1424951919632)\n" - + "(0,65,108.33333333333333,1424951924635)\n" - + "(1,50,26.38888888888889,1424951919632)\n" - + "(0,65,108.33333333333333,1424951924635)\n" - + "(1,65,194.4444444444444,1424951932638)\n" - + "(0,65,108.33333333333333,1424951924635)\n" - + "(1,70,213.88888888888886,1424951933638)\n" - + "(0,60,218.05555555555551,1424951931637)\n" - + "(1,75,272.2222222222222,1424951936639)\n" - + "(0,55,233.3333333333333,1424951932637)\n" - + "(1,75,272.2222222222222,1424951936639)\n" - + "(1,75,272.2222222222222,1424951936639)\n" - + "(0,55,288.88888888888886,1424951936639)\n" - + "(1,70,329.16666666666663,1424951939640)\n" - + "(0,55,373.61111111111114,1424951942641)\n" - + "(1,80,519.4444444444443,1424951949644)\n" - + "(1,85,586.111111111111,1424951952645)\n" - + "(0,50,487.50000000000006,1424951951644)\n" - + "(1,85,586.111111111111,1424951952645)\n" - + "(0,60,590.2777777777778,1424951959647)\n" - + "(1,85,586.111111111111,1424951952645)\n" - + "(0,75,648.6111111111112,1424951962648)\n" - + "(1,85,715.2777777777776,1424951958647)\n" - + "(1,95,858.333333333333,1424951964649)\n" - + "(0,80,670.8333333333334,1424951963648)\n" - + "(1,95,858.333333333333,1424951964649)\n" - + "(0,80,670.8333333333334,1424951963648)\n" - + "(1,100,937.4999999999998,1424951967650)\n" - + "(1,100,937.4999999999998,1424951967650)\n" - + "(0,80,670.8333333333334,1424951963648)\n" - + "(1,100,937.4999999999998,1424951967650)\n" - + "(0,85,861.1111111111112,1424951972651)\n" - + "(1,100,937.4999999999998,1424951967650)\n" - + "(1,100,937.4999999999998,1424951967650)\n" - + "(0,85,861.1111111111112,1424951972651)\n" - + "(1,100,993.0555555555554,1424951969650)\n" - + "(0,85,861.1111111111112,1424951972651)\n" - + "(1,100,1048.611111111111,1424951971651)\n" - + "(1,100,1130.5555555555557,1424951974652)\n" - + "(0,90,1058.3333333333335,1424951981654)\n" - + "(1,100,1158.3333333333335,1424951975652)\n" - + "(0,95,1133.3333333333335,1424951984655)\n" - + "(1,100,1240.277777777778,1424951978653)\n" - + "(0,95,1133.3333333333335,1424951984655)\n" - + "(1,100,1268.0555555555559,1424951979654)\n" - + "(0,95,1133.3333333333335,1424951984655)\n" - + "(1,100,1323.6111111111115,1424951981654)\n" - + "(0,95,1133.3333333333335,1424951984655)\n" - + "(1,100,1379.1666666666672,1424951983655)\n" - + "(0,100,1358.3333333333335,1424951993659)\n" - + "(1,100,1563.8888888888896,1424951990658)\n" - + "(0,100,1358.3333333333335,1424951993659)\n" - + "(1,100,1563.8888888888896,1424951990658)\n" - + "(0,100,1358.3333333333335,1424951993659)\n" - + "(1,100,1563.8888888888896,1424951990658)\n" - + "(0,100,1358.3333333333335,1424951993659)\n" - + "(0,100,1358.3333333333335,1424951993659)\n" - + "(1,100,1669.4444444444453,1424951994659)\n" - + "(0,100,1440.277777777778,1424951996660)\n" - + "(1,90,1720.8333333333342,1424951996660)\n" - + "(0,100,1468.0555555555559,1424951997660)\n" - + "(1,95,1973.6111111111115,1424952007664)\n" - + "(0,100,1522.2222222222226,1424951999661)\n" - + "(0,100,1627.7777777777783,1424952003662)\n" - + "(1,95,1973.6111111111115,1424952007664)\n" - + "(0,100,1627.7777777777783,1424952003662)\n" - + "(1,95,1973.6111111111115,1424952007664)\n" - + "(0,100,1709.7222222222229,1424952006663)\n" - + "(0,100,1737.5000000000007,1424952007664)\n" - + "(1,95,1973.6111111111115,1424952007664)\n"; - - public static final String TOP_CASE_CLASS_SPEEDS = - "CarEvent(0,55,15.277777777777777,1424951918630)\n" - + "CarEvent(1,50,26.38888888888889,1424951919632)\n" - + "CarEvent(0,65,108.33333333333333,1424951924635)\n" - + "CarEvent(1,50,26.38888888888889,1424951919632)\n" - + "CarEvent(0,65,108.33333333333333,1424951924635)\n" - + "CarEvent(1,65,194.4444444444444,1424951932638)\n" - + "CarEvent(0,65,108.33333333333333,1424951924635)\n" - + "CarEvent(1,70,213.88888888888886,1424951933638)\n" - + "CarEvent(0,60,218.05555555555551,1424951931637)\n" - + "CarEvent(1,75,272.2222222222222,1424951936639)\n" - + "CarEvent(0,55,233.3333333333333,1424951932637)\n" - + "CarEvent(1,75,272.2222222222222,1424951936639)\n" - + "CarEvent(1,75,272.2222222222222,1424951936639)\n" - + "CarEvent(0,55,288.88888888888886,1424951936639)\n" - + "CarEvent(1,70,329.16666666666663,1424951939640)\n" - + "CarEvent(0,55,373.61111111111114,1424951942641)\n" - + "CarEvent(1,80,519.4444444444443,1424951949644)\n" - + "CarEvent(1,85,586.111111111111,1424951952645)\n" - + "CarEvent(0,50,487.50000000000006,1424951951644)\n" - + "CarEvent(1,85,586.111111111111,1424951952645)\n" - + "CarEvent(0,60,590.2777777777778,1424951959647)\n" - + "CarEvent(1,85,586.111111111111,1424951952645)\n" - + "CarEvent(0,75,648.6111111111112,1424951962648)\n" - + "CarEvent(1,85,715.2777777777776,1424951958647)\n" - + "CarEvent(1,95,858.333333333333,1424951964649)\n" - + "CarEvent(0,80,670.8333333333334,1424951963648)\n" - + "CarEvent(1,95,858.333333333333,1424951964649)\n" - + "CarEvent(0,80,670.8333333333334,1424951963648)\n" - + "CarEvent(1,100,937.4999999999998,1424951967650)\n" - + "CarEvent(1,100,937.4999999999998,1424951967650)\n" - + "CarEvent(0,80,670.8333333333334,1424951963648)\n" - + "CarEvent(1,100,937.4999999999998,1424951967650)\n" - + "CarEvent(0,85,861.1111111111112,1424951972651)\n" - + "CarEvent(1,100,937.4999999999998,1424951967650)\n" - + "CarEvent(1,100,937.4999999999998,1424951967650)\n" - + "CarEvent(0,85,861.1111111111112,1424951972651)\n" - + "CarEvent(1,100,993.0555555555554,1424951969650)\n" - + "CarEvent(0,85,861.1111111111112,1424951972651)\n" - + "CarEvent(1,100,1048.611111111111,1424951971651)\n" - + "CarEvent(1,100,1130.5555555555557,1424951974652)\n" - + "CarEvent(0,90,1058.3333333333335,1424951981654)\n" - + "CarEvent(1,100,1158.3333333333335,1424951975652)\n" - + "CarEvent(0,95,1133.3333333333335,1424951984655)\n" - + "CarEvent(1,100,1240.277777777778,1424951978653)\n" - + "CarEvent(0,95,1133.3333333333335,1424951984655)\n" - + "CarEvent(1,100,1268.0555555555559,1424951979654)\n" - + "CarEvent(0,95,1133.3333333333335,1424951984655)\n" - + "CarEvent(1,100,1323.6111111111115,1424951981654)\n" - + "CarEvent(0,95,1133.3333333333335,1424951984655)\n" - + "CarEvent(1,100,1379.1666666666672,1424951983655)\n" - + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" - + "CarEvent(1,100,1563.8888888888896,1424951990658)\n" - + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" - + "CarEvent(1,100,1563.8888888888896,1424951990658)\n" - + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" - + "CarEvent(1,100,1563.8888888888896,1424951990658)\n" - + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" - + "CarEvent(0,100,1358.3333333333335,1424951993659)\n" - + "CarEvent(1,100,1669.4444444444453,1424951994659)\n" - + "CarEvent(0,100,1440.277777777778,1424951996660)\n" - + "CarEvent(1,90,1720.8333333333342,1424951996660)\n" - + "CarEvent(0,100,1468.0555555555559,1424951997660)\n" - + "CarEvent(1,95,1973.6111111111115,1424952007664)\n" - + "CarEvent(0,100,1522.2222222222226,1424951999661)\n" - + "CarEvent(0,100,1627.7777777777783,1424952003662)\n" - + "CarEvent(1,95,1973.6111111111115,1424952007664)\n" - + "CarEvent(0,100,1627.7777777777783,1424952003662)\n" - + "CarEvent(1,95,1973.6111111111115,1424952007664)\n" - + "CarEvent(0,100,1709.7222222222229,1424952006663)\n" - + "CarEvent(0,100,1737.5000000000007,1424952007664)\n" - + "CarEvent(1,95,1973.6111111111115,1424952007664)\n"; - - private TopSpeedWindowingExampleData() {} -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java deleted file mode 100644 index 1c545b6..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * 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.examples.wordcount; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.connector.file.sink.FileSink; -import org.apache.flink.connector.file.src.FileSource; -import org.apache.flink.connector.file.src.reader.TextLineInputFormat; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; -import org.apache.flink.streaming.examples.wordcount.util.CLI; -import org.apache.flink.streaming.examples.wordcount.util.WordCountData; -import org.apache.flink.util.Collector; - -import java.time.Duration; - -/** - * Implements the "WordCount" program that computes a simple word occurrence histogram over text - * files. This Job can be executed in both streaming and batch execution modes. - * - *

The input is a [list of] plain text file[s] with lines separated by a newline character. - * - *

Usage: - * - *

    - *
  • --input <path>A list of input files and / or directories to read. If no - * input is provided, the program is run with default data from {@link WordCountData}. - *
  • --discovery-interval <duration>Turns the file reader into a continuous - * source that will monitor the provided input directories every interval and read any new - * files. - *
  • --output <path>The output directory where the Job will write the - * results. If no output path is provided, the Job will print the results to stdout - * . - *
  • --execution-mode <mode>The execution mode (BATCH, STREAMING, or - * AUTOMATIC) of this pipeline. - *
- * - *

This example shows how to: - * - *

    - *
  • Write a simple Flink DataStream program - *
  • Use tuple data types - *
  • Write and use a user-defined function - *
- */ -public class WordCount { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - final CLI params = CLI.fromArgs(args); - - // Create the execution environment. This is the main entrypoint - // to building a Flink application. - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // Apache Flink’s unified approach to stream and batch processing means that a DataStream - // application executed over bounded input will produce the same final results regardless - // of the configured execution mode. It is important to note what final means here: a job - // executing in STREAMING mode might produce incremental updates (think upserts in - // a database) while in BATCH mode, it would only produce one final result at the end. The - // final result will be the same if interpreted correctly, but getting there can be - // different. - // - // The “classic” execution behavior of the DataStream API is called STREAMING execution - // mode. Applications should use streaming execution for unbounded jobs that require - // continuous incremental processing and are expected to stay online indefinitely. - // - // By enabling BATCH execution, we allow Flink to apply additional optimizations that we - // can only do when we know that our input is bounded. For example, different - // join/aggregation strategies can be used, in addition to a different shuffle - // implementation that allows more efficient task scheduling and failure recovery behavior. - // - // By setting the runtime mode to AUTOMATIC, Flink will choose BATCH if all sources - // are bounded and otherwise STREAMING. - env.setRuntimeMode(params.getExecutionMode()); - - // This optional step makes the input parameters - // available in the Flink UI. - env.getConfig().setGlobalJobParameters(params); - - DataStream text; - if (params.getInputs().isPresent()) { - // Create a new file source that will read files from a given set of directories. - // Each file will be processed as plain text and split based on newlines. - FileSource.FileSourceBuilder builder = - FileSource.forRecordStreamFormat( - new TextLineInputFormat(), params.getInputs().get()); - - // If a discovery interval is provided, the source will - // continuously watch the given directories for new files. - params.getDiscoveryInterval().ifPresent(builder::monitorContinuously); - - text = env.fromSource(builder.build(), WatermarkStrategy.noWatermarks(), "file-input"); - } else { - text = env.fromData(WordCountData.WORDS).name("in-memory-input"); - } - - DataStream> counts = - // The text lines read from the source are split into words - // using a user-defined function. The tokenizer, implemented below, - // will output each word as a (2-tuple) containing (word, 1) - text.flatMap(new Tokenizer()) - .name("tokenizer") - // keyBy groups tuples based on the "0" field, the word. - // Using a keyBy allows performing aggregations and other - // stateful transformations over data on a per-key basis. - // This is similar to a GROUP BY clause in a SQL query. - .keyBy(value -> value.f0) - // For each key, we perform a simple sum of the "1" field, the count. - // If the input data stream is bounded, sum will output a final count for - // each word. If it is unbounded, it will continuously output updates - // each time it sees a new instance of each word in the stream. - .sum(1) - .name("counter"); - - if (params.getOutput().isPresent()) { - // Given an output directory, Flink will write the results to a file - // using a simple string encoding. In a production environment, this might - // be something more structured like CSV, Avro, JSON, or Parquet. - counts.sinkTo( - FileSink.>forRowFormat( - params.getOutput().get(), new SimpleStringEncoder<>()) - .withRollingPolicy( - DefaultRollingPolicy.builder() - .withMaxPartSize(MemorySize.ofMebiBytes(1)) - .withRolloverInterval(Duration.ofSeconds(10)) - .build()) - .build()) - .name("file-sink"); - } else { - counts.print().name("print-sink"); - } - - // Apache Flink applications are composed lazily. Calling execute - // submits the Job and begins processing. - env.execute("WordCount"); - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** - * Implements the string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into multiple pairs in the - * form of "(word,1)" ({@code Tuple2}). - */ - public static final class Tokenizer - implements FlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Tuple2<>(token, 1)); - } - } - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java deleted file mode 100644 index ddf1111..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * 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.examples.wordcount.util; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.java.utils.MultipleParameterTool; -import org.apache.flink.configuration.ExecutionOptions; -import org.apache.flink.core.fs.Path; -import org.apache.flink.util.TimeUtils; - -import java.time.Duration; -import java.util.Arrays; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.OptionalInt; - -/** - * A simple CLI parser for the {@link org.apache.flink.streaming.examples.wordcount.WordCount} - * example application. - */ -public class CLI extends ExecutionConfig.GlobalJobParameters { - - public static final String INPUT_KEY = "input"; - public static final String OUTPUT_KEY = "output"; - public static final String DISCOVERY_INTERVAL = "discovery-interval"; - public static final String EXECUTION_MODE = "execution-mode"; - - public static CLI fromArgs(String[] args) throws Exception { - MultipleParameterTool params = MultipleParameterTool.fromArgs(args); - Path[] inputs = null; - if (params.has(INPUT_KEY)) { - inputs = - params.getMultiParameterRequired(INPUT_KEY).stream() - .map(Path::new) - .toArray(Path[]::new); - } else { - System.out.println("Executing example with default input data."); - System.out.println("Use --input to specify file input."); - } - - Path output = null; - if (params.has(OUTPUT_KEY)) { - output = new Path(params.get(OUTPUT_KEY)); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - } - - Duration watchInterval = null; - if (params.has(DISCOVERY_INTERVAL)) { - watchInterval = TimeUtils.parseDuration(params.get(DISCOVERY_INTERVAL)); - } - - RuntimeExecutionMode executionMode = ExecutionOptions.RUNTIME_MODE.defaultValue(); - if (params.has(EXECUTION_MODE)) { - executionMode = RuntimeExecutionMode.valueOf(params.get(EXECUTION_MODE).toUpperCase()); - } - - return new CLI(inputs, output, watchInterval, executionMode, params); - } - - private final Path[] inputs; - private final Path output; - private final Duration discoveryInterval; - private final RuntimeExecutionMode executionMode; - private final MultipleParameterTool params; - - private CLI( - Path[] inputs, - Path output, - Duration discoveryInterval, - RuntimeExecutionMode executionMode, - MultipleParameterTool params) { - this.inputs = inputs; - this.output = output; - this.discoveryInterval = discoveryInterval; - this.executionMode = executionMode; - this.params = params; - } - - public Optional getInputs() { - return Optional.ofNullable(inputs); - } - - public Optional getDiscoveryInterval() { - return Optional.ofNullable(discoveryInterval); - } - - public Optional getOutput() { - return Optional.ofNullable(output); - } - - public RuntimeExecutionMode getExecutionMode() { - return executionMode; - } - - public OptionalInt getInt(String key) { - if (params.has(key)) { - return OptionalInt.of(params.getInt(key)); - } - - return OptionalInt.empty(); - } - - @Override - public Map toMap() { - return params.toMap(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - CLI cli = (CLI) o; - return Arrays.equals(inputs, cli.inputs) - && Objects.equals(output, cli.output) - && Objects.equals(discoveryInterval, cli.discoveryInterval); - } - - @Override - public int hashCode() { - int result = Objects.hash(output, discoveryInterval); - result = 31 * result + Arrays.hashCode(inputs); - return result; - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/WordCountData.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/WordCountData.java deleted file mode 100644 index 6a08872..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/WordCountData.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.examples.wordcount.util; - -/** - * Provides the default data sets used for the WordCount example program. The default data sets are - * used, if no parameters are given to the program. - */ -public class WordCountData { - - public static final String[] WORDS = - new String[] { - "To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", - "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles,", - "And by opposing end them?--To die,--to sleep,--", - "No more; and by a sleep to say we end", - "The heartache, and the thousand natural shocks", - "That flesh is heir to,--'tis a consummation", - "Devoutly to be wish'd. To die,--to sleep;--", - "To sleep! perchance to dream:--ay, there's the rub;", - "For in that sleep of death what dreams may come,", - "When we have shuffled off this mortal coil,", - "Must give us pause: there's the respect", - "That makes calamity of so long life;", - "For who would bear the whips and scorns of time,", - "The oppressor's wrong, the proud man's contumely,", - "The pangs of despis'd love, the law's delay,", - "The insolence of office, and the spurns", - "That patient merit of the unworthy takes,", - "When he himself might his quietus make", - "With a bare bodkin? who would these fardels bear,", - "To grunt and sweat under a weary life,", - "But that the dread of something after death,--", - "The undiscover'd country, from whose bourn", - "No traveller returns,--puzzles the will,", - "And makes us rather bear those ills we have", - "Than fly to others that we know not of?", - "Thus conscience does make cowards of us all;", - "And thus the native hue of resolution", - "Is sicklied o'er with the pale cast of thought;", - "And enterprises of great pith and moment,", - "With this regard, their currents turn awry,", - "And lose the name of action.--Soft you now!", - "The fair Ophelia!--Nymph, in thy orisons", - "Be all my sins remember'd." - }; -} diff --git a/java/Streaming/src/main/resources/log4j2.properties b/java/Streaming/src/main/resources/log4j2.properties deleted file mode 100644 index 9206863..0000000 --- a/java/Streaming/src/main/resources/log4j2.properties +++ /dev/null @@ -1,25 +0,0 @@ -################################################################################ -# 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. -################################################################################ - -rootLogger.level = INFO -rootLogger.appenderRef.console.ref = ConsoleAppender - -appender.console.name = ConsoleAppender -appender.console.type = CONSOLE -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/java/Streaming/src/main/resources/logback.xml b/java/Streaming/src/main/resources/logback.xml deleted file mode 100644 index 95f2d04..0000000 --- a/java/Streaming/src/main/resources/logback.xml +++ /dev/null @@ -1,29 +0,0 @@ - - - - - - %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n - - - - - - - \ No newline at end of file diff --git a/java/Streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java b/java/Streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java deleted file mode 100644 index a4b4685..0000000 --- a/java/Streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * 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.test; - -import org.apache.flink.api.common.eventtime.AscendingTimestampsWatermarks; -import org.apache.flink.api.common.eventtime.TimestampAssigner; -import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; -import org.apache.flink.api.common.eventtime.WatermarkGenerator; -import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.examples.iteration.util.IterateExampleData; -import org.apache.flink.streaming.test.examples.join.WindowJoinData; -import org.apache.flink.test.testdata.WordCountData; -import org.apache.flink.test.util.AbstractTestBaseJUnit4; - -import org.apache.commons.io.FileUtils; -import org.junit.Test; - -import java.io.File; - -import static org.apache.flink.test.util.TestBaseUtils.checkLinesAgainstRegexp; -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; - -/** Integration test for streaming programs in Java examples. */ -public class StreamingExamplesITCase extends AbstractTestBaseJUnit4 { - - @Test - public void testIterateExample() throws Exception { - final String inputPath = - createTempFile("fibonacciInput.txt", IterateExampleData.INPUT_PAIRS); - final String resultPath = getTempDirPath("result"); - - // the example is inherently non-deterministic. The iteration timeout of 5000 ms - // is frequently not enough to make the test run stable on CI infrastructure - // with very small containers, so we cannot do a validation here - org.apache.flink.streaming.examples.iteration.IterateExample.main( - new String[] { - "--input", inputPath, - "--output", resultPath - }); - } - - @Test - public void testWindowJoin() throws Exception { - - final String resultPath = File.createTempFile("result-path", "dir").toURI().toString(); - - final class Parser implements MapFunction> { - - @Override - public Tuple2 map(String value) throws Exception { - String[] fields = value.split(","); - return new Tuple2<>(fields[1], Integer.parseInt(fields[2])); - } - } - - try { - final StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStream> grades = - env.fromData(WindowJoinData.GRADES_INPUT.split("\n")) - .assignTimestampsAndWatermarks(IngestionTimeWatermarkStrategy.create()) - .map(new Parser()); - - DataStream> salaries = - env.fromData(WindowJoinData.SALARIES_INPUT.split("\n")) - .assignTimestampsAndWatermarks(IngestionTimeWatermarkStrategy.create()) - .map(new Parser()); - - org.apache.flink.streaming.examples.join.WindowJoin.runWindowJoin(grades, salaries, 100) - .writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE); - - env.execute(); - - // since the two sides of the join might have different speed - // the exact output can not be checked just whether it is well-formed - // checks that the result lines look like e.g. (bob, 2, 2015) - checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d),(\\d)+\\)"); - } finally { - try { - FileUtils.deleteDirectory(new File(resultPath)); - } catch (Throwable ignored) { - } - } - } - - @Test - public void testSessionWindowing() throws Exception { - final String resultPath = getTempDirPath("result"); - org.apache.flink.streaming.examples.windowing.SessionWindowing.main( - new String[] {"--output", resultPath}); - } - - @Test - public void testWindowWordCount() throws Exception { - final String windowSize = "25"; - final String slideSize = "15"; - final String textPath = createTempFile("text.txt", WordCountData.TEXT); - final String resultPath = getTempDirPath("result"); - - org.apache.flink.streaming.examples.windowing.WindowWordCount.main( - new String[] { - "--input", textPath, - "--output", resultPath, - "--window", windowSize, - "--slide", slideSize - }); - - // since the parallel tokenizers might have different speed - // the exact output can not be checked just whether it is well-formed - // checks that the result lines look like e.g. (faust, 2) - checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d)+\\)"); - } - - @Test - public void testWordCount() throws Exception { - final String textPath = createTempFile("text.txt", WordCountData.TEXT); - final String resultPath = getTempDirPath("result"); - - org.apache.flink.streaming.examples.wordcount.WordCount.main( - new String[] { - "--input", textPath, - "--output", resultPath, - "--execution-mode", "automatic" - }); - - compareResultsByLinesInMemory(WordCountData.COUNTS_AS_TUPLES, resultPath); - } - - /** - * This {@link WatermarkStrategy} assigns the current system time as the event-time timestamp. - * In a real use case you should use proper timestamps and an appropriate {@link - * WatermarkStrategy}. - */ - private static class IngestionTimeWatermarkStrategy implements WatermarkStrategy { - - private IngestionTimeWatermarkStrategy() {} - - public static IngestionTimeWatermarkStrategy create() { - return new IngestionTimeWatermarkStrategy<>(); - } - - @Override - public WatermarkGenerator createWatermarkGenerator( - WatermarkGeneratorSupplier.Context context) { - return new AscendingTimestampsWatermarks<>(); - } - - @Override - public TimestampAssigner createTimestampAssigner( - TimestampAssignerSupplier.Context context) { - return (event, timestamp) -> System.currentTimeMillis(); - } - } -} diff --git a/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/join/WindowJoinData.java b/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/join/WindowJoinData.java deleted file mode 100644 index d15c512..0000000 --- a/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/join/WindowJoinData.java +++ /dev/null @@ -1,229 +0,0 @@ -/* - * 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.test.examples.join; - -/** Class with sample data for window join examples. */ -public class WindowJoinData { - - public static final String GRADES_INPUT = - "0,john,5\n" - + "0,tom,3\n" - + "0,alice,1\n" - + "0,grace,5\n" - + "1,john,4\n" - + "1,bob,1\n" - + "1,alice,2\n" - + "1,alice,3\n" - + "1,bob,5\n" - + "1,alice,3\n" - + "1,tom,5\n" - + "2,john,2\n" - + "2,john,1\n" - + "2,grace,2\n" - + "2,jerry,2\n" - + "2,tom,4\n" - + "2,bob,4\n" - + "2,bob,2\n" - + "3, tom,2\n" - + "3,alice,5\n" - + "3,grace,5\n" - + "3,grace,1\n" - + "3,alice,1\n" - + "3,grace,3\n" - + "3,tom,1\n" - + "4,jerry,5\n" - + "4,john,3\n" - + "4,john,4\n" - + "4,john,1\n" - + "4,jerry,3\n" - + "4,grace,3\n" - + "4,bob,3\n" - + "5,john,3\n" - + "5,jerry,4\n" - + "5,tom,5\n" - + "5,tom,4\n" - + "5,john,2\n" - + "5,jerry,1\n" - + "5,bob,1\n" - + "6,john,5\n" - + "6,grace,4\n" - + "6,tom,5\n" - + "6,john,4\n" - + "6,tom,1\n" - + "6,grace,1\n" - + "6,john,2\n" - + "7,jerry,3\n" - + "7,jerry,5\n" - + "7,tom,2\n" - + "7,tom,2\n" - + "7,alice,4\n" - + "7,tom,4\n" - + "7,jerry,4\n" - + "8,john,3\n" - + "8,grace,4\n" - + "8,tom,3\n" - + "8,jerry,4\n" - + "8,john,5\n" - + "8,john,4\n" - + "8,jerry,1\n" - + "9,john,5\n" - + "9,alice,2\n" - + "9,tom,1\n" - + "9,alice,5\n" - + "9,grace,4\n" - + "9,bob,4\n" - + "9,jerry,1\n" - + "10,john,5\n" - + "10,tom,4\n" - + "10,tom,5\n" - + "10,jerry,5\n" - + "10,tom,1\n" - + "10,grace,3\n" - + "10,bob,5\n" - + "11,john,1\n" - + "11,alice,1\n" - + "11,grace,3\n" - + "11,grace,1\n" - + "11,jerry,1\n" - + "11,jerry,4\n" - + "12,bob,4\n" - + "12,alice,3\n" - + "12,tom,5\n" - + "12,alice,4\n" - + "12,alice,4\n" - + "12,grace,4\n" - + "12,john,5\n" - + "13,john,5\n" - + "13,grace,4\n" - + "13,tom,4\n" - + "13,john,4\n" - + "13,john,5\n" - + "13,alice,5\n" - + "13,jerry,5\n" - + "14,john,3\n" - + "14,tom,5\n" - + "14,jerry,4\n" - + "14,grace,4\n" - + "14,john,3\n" - + "14,bob,2"; - - public static final String SALARIES_INPUT = - "0,john,6469\n" - + "0,jerry,6760\n" - + "0,jerry,8069\n" - + "1,tom,3662\n" - + "1,grace,8427\n" - + "1,john,9425\n" - + "1,bob,9018\n" - + "1,john,352\n" - + "1,tom,3770\n" - + "2,grace,7622\n" - + "2,jerry,7441\n" - + "2,alice,1468\n" - + "2,bob,5472\n" - + "2,grace,898\n" - + "3,tom,3849\n" - + "3,grace,1865\n" - + "3,alice,5582\n" - + "3,john,9511\n" - + "3,alice,1541\n" - + "4,john,2477\n" - + "4,grace,3561\n" - + "4,john,1670\n" - + "4,grace,7290\n" - + "4,grace,6565\n" - + "5,tom,6179\n" - + "5,tom,1601\n" - + "5,john,2940\n" - + "5,bob,4685\n" - + "5,bob,710\n" - + "5,bob,5936\n" - + "6,jerry,1412\n" - + "6,grace,6515\n" - + "6,grace,3321\n" - + "6,tom,8088\n" - + "6,john,2876\n" - + "7,bob,9896\n" - + "7,grace,7368\n" - + "7,grace,9749\n" - + "7,bob,2048\n" - + "7,alice,4782\n" - + "8,alice,3375\n" - + "8,tom,5841\n" - + "8,bob,958\n" - + "8,bob,5258\n" - + "8,tom,3935\n" - + "8,jerry,4394\n" - + "9,alice,102\n" - + "9,alice,4931\n" - + "9,alice,5240\n" - + "9,jerry,7951\n" - + "9,john,5675\n" - + "10,bob,609\n" - + "10,alice,5997\n" - + "10,jerry,9651\n" - + "10,alice,1328\n" - + "10,bob,1022\n" - + "11,grace,2578\n" - + "11,jerry,9704\n" - + "11,tom,4476\n" - + "11,grace,3784\n" - + "11,alice,6144\n" - + "12,bob,6213\n" - + "12,alice,7525\n" - + "12,jerry,2908\n" - + "12,grace,8464\n" - + "12,jerry,9920\n" - + "13,bob,3720\n" - + "13,bob,7612\n" - + "13,alice,7211\n" - + "13,jerry,6484\n" - + "13,alice,1711\n" - + "14,jerry,5994\n" - + "14,grace,928\n" - + "14,jerry,2492\n" - + "14,grace,9080\n" - + "14,tom,4330\n" - + "15,bob,8302\n" - + "15,john,4981\n" - + "15,tom,1781\n" - + "15,grace,1379\n" - + "15,jerry,3700\n" - + "16,jerry,3584\n" - + "16,jerry,2038\n" - + "16,jerry,3902\n" - + "16,tom,1336\n" - + "16,jerry,7500\n" - + "17,tom,3648\n" - + "17,alice,2533\n" - + "17,tom,8685\n" - + "17,bob,3968\n" - + "17,tom,3241\n" - + "17,bob,7461\n" - + "18,jerry,2138\n" - + "18,alice,7503\n" - + "18,alice,6424\n" - + "18,tom,140\n" - + "18,john,9802\n" - + "19,grace,2977\n" - + "19,grace,889\n" - + "19,john,1338"; - - /** Utility class, should not be instantiated. */ - private WindowJoinData() {} -} diff --git a/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/windowing/TopSpeedWindowingExampleITCase.java b/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/windowing/TopSpeedWindowingExampleITCase.java deleted file mode 100644 index 72dd508..0000000 --- a/java/Streaming/src/test/java/org/apache/flink/streaming/test/examples/windowing/TopSpeedWindowingExampleITCase.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.test.examples.windowing; - -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.examples.windowing.TopSpeedWindowing; -import org.apache.flink.streaming.examples.windowing.util.TopSpeedWindowingExampleData; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.util.FileUtils; -import org.apache.flink.util.TestLogger; - -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; - -/** Tests for {@link TopSpeedWindowing}. */ -public class TopSpeedWindowingExampleITCase extends TestLogger { - - @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); - - @ClassRule - public static MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(1) - .build()); - - @Test - public void testTopSpeedWindowingExampleITCase() throws Exception { - File inputFile = temporaryFolder.newFile(); - FileUtils.writeFileUtf8(inputFile, TopSpeedWindowingExampleData.CAR_DATA); - - final String resultPath = temporaryFolder.newFolder().toURI().toString(); - - TopSpeedWindowing.main( - new String[] { - "--input", - inputFile.getAbsolutePath(), - "--output", - resultPath, - "--execution-mode", - "AUTOMATIC" - }); - - compareResultsByLinesInMemory(TopSpeedWindowingExampleData.TOP_SPEEDS, resultPath); - } -} diff --git a/java/Streaming/src/test/java/org/apache/flink/streaming/test/socket/SocketWindowWordCountITCase.java b/java/Streaming/src/test/java/org/apache/flink/streaming/test/socket/SocketWindowWordCountITCase.java deleted file mode 100644 index f8dbca2..0000000 --- a/java/Streaming/src/test/java/org/apache/flink/streaming/test/socket/SocketWindowWordCountITCase.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.test.socket; - -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.streaming.examples.socket.SocketWindowWordCount; -import org.apache.flink.test.testdata.WordCountData; -import org.apache.flink.test.util.AbstractTestBaseJUnit4; -import org.apache.flink.util.NetUtils; - -import org.junit.Test; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.io.PrintStream; -import java.io.PrintWriter; -import java.net.InetAddress; -import java.net.ServerSocket; -import java.net.Socket; - -import static org.junit.Assert.fail; - -/** Tests for {@link SocketWindowWordCount}. */ -public class SocketWindowWordCountITCase extends AbstractTestBaseJUnit4 { - - @Test - public void testJavaProgram() throws Exception { - InetAddress localhost = InetAddress.getByName("localhost"); - - // suppress sysout messages from this example - final PrintStream originalSysout = System.out; - final PrintStream originalSyserr = System.err; - - final ByteArrayOutputStream errorMessages = new ByteArrayOutputStream(); - - System.setOut(new PrintStream(new NullStream())); - System.setErr(new PrintStream(errorMessages)); - - try { - try (ServerSocket server = new ServerSocket(0, 10, localhost)) { - - final ServerThread serverThread = new ServerThread(server); - serverThread.setDaemon(true); - serverThread.start(); - - final int serverPort = server.getLocalPort(); - - SocketWindowWordCount.main(new String[] {"--port", String.valueOf(serverPort)}); - - if (errorMessages.size() != 0) { - fail( - "Found error message: " - + new String( - errorMessages.toByteArray(), - ConfigConstants.DEFAULT_CHARSET)); - } - - serverThread.join(); - serverThread.checkError(); - } - } finally { - System.setOut(originalSysout); - System.setErr(originalSyserr); - } - } - - // ------------------------------------------------------------------------ - - private static class ServerThread extends Thread { - - private final ServerSocket serverSocket; - - private volatile Throwable error; - - public ServerThread(ServerSocket serverSocket) { - super("Socket Server Thread"); - - this.serverSocket = serverSocket; - } - - @Override - public void run() { - try { - try (Socket socket = NetUtils.acceptWithoutTimeout(serverSocket); - PrintWriter writer = new PrintWriter(socket.getOutputStream(), true)) { - - writer.println(WordCountData.TEXT); - } - } catch (Throwable t) { - this.error = t; - } - } - - public void checkError() throws IOException { - if (error != null) { - throw new IOException("Error in server thread: " + error.getMessage(), error); - } - } - } - - private static final class NullStream extends OutputStream { - - @Override - public void write(int b) {} - } -} diff --git a/java/Streaming/src/test/resources/log4j2-test.properties b/java/Streaming/src/test/resources/log4j2-test.properties deleted file mode 100644 index 835c2ec..0000000 --- a/java/Streaming/src/test/resources/log4j2-test.properties +++ /dev/null @@ -1,28 +0,0 @@ -################################################################################ -# 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. -################################################################################ - -# Set root logger level to OFF to not flood build logs -# set manually to INFO for debugging purposes -rootLogger.level = OFF -rootLogger.appenderRef.test.ref = TestLogger - -appender.testlogger.name = TestLogger -appender.testlogger.type = CONSOLE -appender.testlogger.target = SYSTEM_ERR -appender.testlogger.layout.type = PatternLayout -appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n From 22747c8d1edc4abaaf4ade96847e7b057ea5534b Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Tue, 3 Dec 2024 19:15:34 +0530 Subject: [PATCH 20/23] Add printer application --- .github/workflows/maven_packages.yml | 4 +- java/Printer/README.md | 10 + java/Printer/pom.xml | 184 ++++++++++++++++++ .../amazonaws/services/msf/StockPrice.java | 52 +++++ .../msf/StockPriceGeneratorFunction.java | 20 ++ .../amazonaws/services/msf/StreamingJob.java | 63 ++++++ .../src/main/resources/log4j2.properties | 12 ++ 7 files changed, 343 insertions(+), 2 deletions(-) create mode 100644 java/Printer/README.md create mode 100644 java/Printer/pom.xml create mode 100644 java/Printer/src/main/java/com/amazonaws/services/msf/StockPrice.java create mode 100644 java/Printer/src/main/java/com/amazonaws/services/msf/StockPriceGeneratorFunction.java create mode 100644 java/Printer/src/main/java/com/amazonaws/services/msf/StreamingJob.java create mode 100644 java/Printer/src/main/resources/log4j2.properties diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index 1c3a5ca..56327c4 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -21,7 +21,7 @@ jobs: strategy: matrix: include: - - path: "java/Streaming" + - path: "java/Printer" - path: "java/S3Sink" runs-on: ubuntu-latest permissions: @@ -41,7 +41,7 @@ jobs: - name: Build with Maven working-directory: ${{ matrix.path }} - run: mvn --batch-mode package -Denforcer.skip=true -Dcheckstyle.skip=true -Dspotless.check.skip=true + run: mvn --batch-mode package - name: Publish to GitHub Packages Apache Maven working-directory: ${{ matrix.path }} diff --git a/java/Printer/README.md b/java/Printer/README.md new file mode 100644 index 0000000..389e930 --- /dev/null +++ b/java/Printer/README.md @@ -0,0 +1,10 @@ +# Printer + +* Flink version: 1.20 +* Flink API: DataStream API +* Language Java (11) + +The Flink application uses a synthetic source to generate records, +parses the records, and it just prints the results. + +It is based on the S3Sink application in this repo. diff --git a/java/Printer/pom.xml b/java/Printer/pom.xml new file mode 100644 index 0000000..08b3348 --- /dev/null +++ b/java/Printer/pom.xml @@ -0,0 +1,184 @@ + + + 4.0.0 + + com.amazonaws + msaf-printer + 1.0-SNAPSHOT + + + + github + GitHub Packages + https://maven.pkg.github.com/localstack-samples/amazon-managed-service-for-apache-flink-examples + + + + + UTF-8 + ${project.basedir}/target + ${project.name} + 11 + ${target.java.version} + ${target.java.version} + 1.20.0 + 5.0.0-1.20 + 1.2.0 + 2.23.1 + + + + + + com.amazonaws + aws-java-sdk-bom + + 1.12.677 + pom + import + + + + + + + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + org.apache.flink + flink-clients + ${flink.version} + provided + + + org.apache.flink + flink-runtime-web + ${flink.version} + provided + + + + + com.amazonaws + aws-kinesisanalytics-runtime + ${kda.runtime.version} + provided + + + + + org.apache.flink + flink-connector-datagen + ${flink.version} + + + org.apache.flink + flink-connector-files + ${flink.version} + provided + + + org.apache.flink + flink-s3-fs-hadoop + ${flink.version} + + + org.apache.flink + flink-connector-base + ${flink.version} + provided + + + + + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + + + org.apache.logging.log4j + log4j-api + ${log4j.version} + + + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + + + ${buildDirectory} + ${jar.finalName} + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.1 + + ${target.java.version} + ${target.java.version} + true + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.1 + + + + package + + shade + + + + + org.apache.flink:force-shading + com.google.code.findbugs:jsr305 + org.slf4j:* + org.apache.logging.log4j:* + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + com.amazonaws.services.msf.StreamingJob + + + + + + + + + + + diff --git a/java/Printer/src/main/java/com/amazonaws/services/msf/StockPrice.java b/java/Printer/src/main/java/com/amazonaws/services/msf/StockPrice.java new file mode 100644 index 0000000..da60ea7 --- /dev/null +++ b/java/Printer/src/main/java/com/amazonaws/services/msf/StockPrice.java @@ -0,0 +1,52 @@ +package com.amazonaws.services.msf; + +import java.sql.Timestamp; + +public class StockPrice { + private Timestamp eventTime; + private String ticker; + private Double price; + + + public StockPrice() { + } + + public StockPrice(Timestamp eventTime, String ticker, Double price) { + this.eventTime = eventTime; + this.ticker = ticker; + this.price = price; + } + + public Timestamp getEventTime() { + return eventTime; + } + + public void setEventTime(Timestamp eventTime) { + this.eventTime = eventTime; + } + + public String getTicker() { + return ticker; + } + + public void setTicker(String ticker) { + this.ticker = ticker; + } + + public Double getPrice() { + return price; + } + + public void setPrice(Double price) { + this.price = price; + } + + @Override + public String toString() { + return "StockPrice{" + + "eventTime=" + eventTime + + ", ticker='" + ticker + '\'' + + ", price=" + price + + '}'; + } +} \ No newline at end of file diff --git a/java/Printer/src/main/java/com/amazonaws/services/msf/StockPriceGeneratorFunction.java b/java/Printer/src/main/java/com/amazonaws/services/msf/StockPriceGeneratorFunction.java new file mode 100644 index 0000000..7001e04 --- /dev/null +++ b/java/Printer/src/main/java/com/amazonaws/services/msf/StockPriceGeneratorFunction.java @@ -0,0 +1,20 @@ +package com.amazonaws.services.msf; + +import org.apache.commons.lang3.RandomUtils; +import org.apache.flink.connector.datagen.source.GeneratorFunction; + +import java.sql.Timestamp; +import java.time.Instant; + +public class StockPriceGeneratorFunction implements GeneratorFunction { + private static final String[] TICKERS = {"AAPL", "AMZN", "MSFT", "INTC", "TBV"}; + + @Override + public StockPrice map(Long aLong) { + return new StockPrice( + new Timestamp(Instant.now().toEpochMilli()), + TICKERS[RandomUtils.nextInt(0, TICKERS.length)], + RandomUtils.nextDouble(10,100) + ); + } +} \ No newline at end of file diff --git a/java/Printer/src/main/java/com/amazonaws/services/msf/StreamingJob.java b/java/Printer/src/main/java/com/amazonaws/services/msf/StreamingJob.java new file mode 100644 index 0000000..d29cba8 --- /dev/null +++ b/java/Printer/src/main/java/com/amazonaws/services/msf/StreamingJob.java @@ -0,0 +1,63 @@ +package com.amazonaws.services.msf; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +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.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + +public class StreamingJob { + + private static final Logger LOGGER = LogManager.getLogger(StreamingJob.class); + + // Create ObjectMapper instance to serialise POJOs into JSONs + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static boolean isLocal(StreamExecutionEnvironment env) { + return env instanceof LocalStreamEnvironment; + } + + private static DataGeneratorSource getStockPriceDataGeneratorSource() { + long recordPerSecond = 100; + return new DataGeneratorSource<>( + new StockPriceGeneratorFunction(), + Long.MAX_VALUE, + RateLimiterStrategy.perSecond(recordPerSecond), + TypeInformation.of(StockPrice.class)); + } + + public static void main(String[] args) throws Exception { + // Set up the streaming execution environment + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + + // Local dev specific settings + if (isLocal(env)) { + // Checkpointing and parallelism are set by Amazon Managed Service for Apache Flink when running on AWS + env.enableCheckpointing(60000); + env.setParallelism(2); + } + + // Source + DataGeneratorSource source = getStockPriceDataGeneratorSource(); + + // DataStream from Source + DataStream kinesis = env.fromSource( + source, WatermarkStrategy.noWatermarks(), "data-generator").setParallelism(1); + + // Print + kinesis.print(); + + env.execute("Flink Demo Printer Job"); + } +} diff --git a/java/Printer/src/main/resources/log4j2.properties b/java/Printer/src/main/resources/log4j2.properties new file mode 100644 index 0000000..1d8f61f --- /dev/null +++ b/java/Printer/src/main/resources/log4j2.properties @@ -0,0 +1,12 @@ +appender.console.name = ConsoleAppender +appender.console.type = CONSOLE +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss} %-5p %c{1} - %m%n + +#logger.verbose.name = com.amazonaws.services.msf +#logger.verbose.level = debug +#logger.verbose.additivity = false +#logger.verbose.appenderRef.console.ref = ConsoleAppender + +rootLogger.level = INFO +rootLogger.appenderRef.console.ref = ConsoleAppender From 75ce2992f43e682d5f4ba2da808dae13ab930123 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Tue, 3 Dec 2024 19:28:49 +0530 Subject: [PATCH 21/23] Revert "Omit branch filter" This reverts commit a368d5bf9476b7b519c3814955551c6b91397589. --- .github/workflows/maven_packages.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml index 56327c4..c420a18 100644 --- a/.github/workflows/maven_packages.yml +++ b/.github/workflows/maven_packages.yml @@ -5,14 +5,14 @@ on: paths: - java/** - .github/workflows/maven_packages.yml -# branches: -# - master + branches: + - master pull_request: paths: - java/** -# branches: -# - master + branches: + - master workflow_dispatch: From 0a54e6a073a72516051ac51d9259e039efd065b3 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Tue, 3 Dec 2024 19:49:24 +0530 Subject: [PATCH 22/23] Update readme --- java/Printer/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/java/Printer/README.md b/java/Printer/README.md index 389e930..16e3636 100644 --- a/java/Printer/README.md +++ b/java/Printer/README.md @@ -4,7 +4,7 @@ * Flink API: DataStream API * Language Java (11) -The Flink application uses a synthetic source to generate records, -parses the records, and it just prints the results. +The Flink application uses a synthetic source to generate ticker records, +parses them, and just prints the results. It is based on the S3Sink application in this repo. From 13dbede96aad2f692c6b09b2e2555a48ccb97f41 Mon Sep 17 00:00:00 2001 From: Viren Nadkarni Date: Mon, 9 Dec 2024 15:08:20 +0530 Subject: [PATCH 23/23] Remove workflow --- .github/workflows/maven_packages.yml | 50 ---------------------------- 1 file changed, 50 deletions(-) delete mode 100644 .github/workflows/maven_packages.yml diff --git a/.github/workflows/maven_packages.yml b/.github/workflows/maven_packages.yml deleted file mode 100644 index c420a18..0000000 --- a/.github/workflows/maven_packages.yml +++ /dev/null @@ -1,50 +0,0 @@ -name: Maven Packages - -on: - push: - paths: - - java/** - - .github/workflows/maven_packages.yml - branches: - - master - - pull_request: - paths: - - java/** - branches: - - master - - workflow_dispatch: - -jobs: - build-publish: - strategy: - matrix: - include: - - path: "java/Printer" - - path: "java/S3Sink" - runs-on: ubuntu-latest - permissions: - contents: read - packages: write - steps: - - name: Checkout - uses: actions/checkout@v4 - - - name: Set up JDK - uses: actions/setup-java@v4 - with: - java-version: '21' - distribution: 'temurin' - server-id: github # Value of the distributionManagement/repository/id field of the pom.xml - settings-path: ${{ github.workspace }} - - - name: Build with Maven - working-directory: ${{ matrix.path }} - run: mvn --batch-mode package - - - name: Publish to GitHub Packages Apache Maven - working-directory: ${{ matrix.path }} - run: mvn --batch-mode deploy -s $GITHUB_WORKSPACE/settings.xml - env: - GITHUB_TOKEN: ${{ github.token }}

and run this example with the hostname and the port as arguments. - */ -public class SocketWindowWordCount { - - public static void main(String[] args) throws Exception { - - // the host and the port to connect to - final String hostname; - final int port; - try { - final ParameterTool params = ParameterTool.fromArgs(args); - hostname = params.has("hostname") ? params.get("hostname") : "localhost"; - port = params.getInt("port"); - } catch (Exception e) { - System.err.println( - "No port specified. Please run 'SocketWindowWordCount " - + "--hostname --port ', where hostname (localhost by default) " - + "and port is the address of the text server"); - System.err.println( - "To start a simple text server, run 'netcat -l ' and " - + "type the input text into the command line"); - return; - } - - // get the execution environment - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // get input data by connecting to the socket - DataStream text = env.socketTextStream(hostname, port, "\n"); - - // parse the data, group it, window it, and aggregate the counts - DataStream windowCounts = - text.flatMap( - (FlatMapFunction) - (value, out) -> { - for (String word : value.split("\\s")) { - out.collect(new WordWithCount(word, 1L)); - } - }, - Types.POJO(WordWithCount.class)) - .keyBy(value -> value.word) - .window(TumblingProcessingTimeWindows.of(Duration.ofSeconds(5))) - .reduce((a, b) -> new WordWithCount(a.word, a.count + b.count)) - .returns(WordWithCount.class); - - // print the results with a single thread, rather than in parallel - windowCounts.print().setParallelism(1); - - env.execute("Socket Window WordCount"); - } - - // ------------------------------------------------------------------------ - - /** Data type for words with count. */ - public static class WordWithCount { - - public String word; - public long count; - - @SuppressWarnings("unused") - public WordWithCount() {} - - public WordWithCount(String word, long count) { - this.word = word; - this.count = count; - } - - @Override - public String toString() { - return word + " : " + count; - } - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java deleted file mode 100644 index 776fcfd..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.examples.statemachine; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.datagen.source.GeneratorFunction; -import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; -import org.apache.flink.connector.kafka.sink.KafkaSink; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.examples.statemachine.event.Event; -import org.apache.flink.streaming.examples.statemachine.generator.EventsGeneratorFunction; -import org.apache.flink.streaming.examples.statemachine.kafka.EventDeSerializationSchema; - -/** - * Job to generate input events that are written to Kafka, for the {@link StateMachineExample} job. - */ -public class KafkaEventsGeneratorJob { - - public static void main(String[] args) throws Exception { - - final ParameterTool params = ParameterTool.fromArgs(args); - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - final double errorRate = params.getDouble("error-rate", 0.0); - final int sleep = params.getInt("sleep", 1); - final double recordsPerSecond = - params.getDouble("rps", rpsFromSleep(sleep, env.getParallelism())); - System.out.printf( - "Generating events to Kafka with standalone source with error rate %f and %.1f records per second\n", - errorRate, recordsPerSecond); - System.out.println(); - - String kafkaTopic = params.get("kafka-topic"); - String brokers = params.get("brokers", "localhost:9092"); - - GeneratorFunction generatorFunction = new EventsGeneratorFunction(errorRate); - DataGeneratorSource eventGeneratorSource = - new DataGeneratorSource<>( - generatorFunction, - Long.MAX_VALUE, - RateLimiterStrategy.perSecond(recordsPerSecond), - TypeInformation.of(Event.class)); - - env.fromSource( - eventGeneratorSource, - WatermarkStrategy.noWatermarks(), - "Events Generator Source") - .sinkTo( - KafkaSink.builder() - .setBootstrapServers(brokers) - .setRecordSerializer( - KafkaRecordSerializationSchema.builder() - .setValueSerializationSchema( - new EventDeSerializationSchema()) - .setTopic(kafkaTopic) - .build()) - .build()); - - // trigger program execution - env.execute("State machine example Kafka events generator job"); - } - - // Used for backwards compatibility to convert legacy 'sleep' parameter to records per second. - private static double rpsFromSleep(int sleep, int parallelism) { - return (1000d / sleep) * parallelism; - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/README.md b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/README.md deleted file mode 100644 index b32ffe6..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/README.md +++ /dev/null @@ -1,52 +0,0 @@ -Example: Running a state machine for pattern detection -====================================================== - -This example illustrates a minimal roll-your-own event pattern detection scenario, -using a simple state machine that is evaluated over the stream. - -While this example is much simpler and more manual than what the CEP library supports, -it illustrates the use of event processing and state management for a medium -complex scenario. - -**Scenario Description** - -Events in streams are expected to occur in certain patterns. Any deviation from -these patterns indicates an anomaly that the streaming system should recognize and that -should trigger an alert. - -You can, for example, think of events as being generated by network devices and services, -such as firewalls login-, and registration with an authentication service, etc. A deviation -from the expected pattern might indicate an intrusion detection. - -The event patterns are tracked per interacting party (here simplified per source IP address) -and are validated by a state machine. The state machine's states define what possible -events may occur next, and what new states these events will result in. - -The following diagram depicts the state machine used in this example. - -``` - +----> W ----> Y -----+ - | ^ | - INITIAL-+ | | - | | +--> (Z) --------> TERM - +----> X ------+ | - | | - +-----------------+ -``` - -**Example Program** - -The main class of this example program is `org.apache.flink.streaming.examples.statemachine.StateMachineExample`. -The core logic is in the `flatMap` function, which runs the state machines per IP address. - -The streaming data flow is as shown below, where the source stream may come from either -an embedded data generator, or from a from a Kafka topic: - -``` - [ stream partition 1] --> source --> partition -+---> flatMap(state machine) --> sink - \/ - /\ - [ stream partition 2] --> source --> partition -+---> flatMap(state machine) --> sink -``` - - diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java deleted file mode 100644 index 940b836..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java +++ /dev/null @@ -1,242 +0,0 @@ -/* - * 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.examples.statemachine; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.common.serialization.SimpleStringEncoder; -import org.apache.flink.api.common.state.ValueState; -import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.CheckpointingOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.StateBackendOptions; -import org.apache.flink.connector.datagen.source.DataGeneratorSource; -import org.apache.flink.connector.datagen.source.GeneratorFunction; -import org.apache.flink.connector.file.sink.FileSink; -import org.apache.flink.connector.kafka.source.KafkaSource; -import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; -import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; -import org.apache.flink.core.fs.Path; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; -import org.apache.flink.streaming.examples.statemachine.dfa.State; -import org.apache.flink.streaming.examples.statemachine.event.Alert; -import org.apache.flink.streaming.examples.statemachine.event.Event; -import org.apache.flink.streaming.examples.statemachine.generator.EventsGeneratorFunction; -import org.apache.flink.streaming.examples.statemachine.kafka.EventDeSerializationSchema; -import org.apache.flink.util.Collector; - -import java.time.Duration; - -/** - * Main class of the state machine example. This class implements the streaming application that - * receives the stream of events and evaluates a state machine (per originating address) to validate - * that the events follow the state machine's rules. - */ -public class StateMachineExample { - - /** - * Main entry point for the program. - * - * @param args The command line arguments. - */ - public static void main(String[] args) throws Exception { - - // ---- print some usage help ---- - - System.out.println( - "Usage with built-in data generator: StateMachineExample [--error-rate ] [--sleep | --rps ]"); - System.out.println( - "Usage with Kafka: StateMachineExample --kafka-topic [--brokers ]"); - System.out.println("Options for both the above setups: "); - System.out.println("\t[--backend ]"); - System.out.println("\t[--checkpoint-dir ]"); - System.out.println("\t[--incremental-checkpoints ]"); - System.out.println("\t[--output OR null for stdout]"); - System.out.println(); - - // ---- determine whether to use the built-in source, or read from Kafka ---- - - final DataStream events; - final ParameterTool params = ParameterTool.fromArgs(args); - - // create the environment to create streams and configure execution - Configuration configuration = new Configuration(); - - final String stateBackend = params.get("backend", "memory"); - if ("hashmap".equals(stateBackend)) { - final String checkpointDir = params.get("checkpoint-dir"); - configuration.set(StateBackendOptions.STATE_BACKEND, "hashmap"); - configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); - configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir); - } else if ("rocks".equals(stateBackend)) { - final String checkpointDir = params.get("checkpoint-dir"); - boolean incrementalCheckpoints = params.getBoolean("incremental-checkpoints", false); - configuration.set( - StateBackendOptions.STATE_BACKEND, - "org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackendFactory"); - configuration.set(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, incrementalCheckpoints); - configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); - configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir); - } - final StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment(configuration); - env.enableCheckpointing(2000L); - - if (params.has("kafka-topic")) { - // set up the Kafka reader - String kafkaTopic = params.get("kafka-topic"); - String brokers = params.get("brokers", "localhost:9092"); - - System.out.printf("Reading from kafka topic %s @ %s\n", kafkaTopic, brokers); - System.out.println(); - - KafkaSource source = - KafkaSource.builder() - .setBootstrapServers(brokers) - .setGroupId("stateMachineExample") - .setTopics(kafkaTopic) - .setDeserializer( - KafkaRecordDeserializationSchema.valueOnly( - new EventDeSerializationSchema())) - .setStartingOffsets(OffsetsInitializer.latest()) - .build(); - events = - env.fromSource( - source, WatermarkStrategy.noWatermarks(), "StateMachineExampleSource"); - } else { - final double errorRate = params.getDouble("error-rate", 0.0); - final int sleep = params.getInt("sleep", 1); - final double recordsPerSecond = - params.getDouble("rps", rpsFromSleep(sleep, env.getParallelism())); - System.out.printf( - "Using standalone source with error rate %f and %.1f records per second\n", - errorRate, recordsPerSecond); - System.out.println(); - - GeneratorFunction generatorFunction = - new EventsGeneratorFunction(errorRate); - DataGeneratorSource eventGeneratorSource = - new DataGeneratorSource<>( - generatorFunction, - Long.MAX_VALUE, - RateLimiterStrategy.perSecond(recordsPerSecond), - TypeInformation.of(Event.class)); - - events = - env.fromSource( - eventGeneratorSource, - WatermarkStrategy.noWatermarks(), - "Events Generator Source"); - } - - // ---- main program ---- - - final String outputFile = params.get("output"); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - DataStream alerts = - events - // partition on the address to make sure equal addresses - // end up in the same state machine flatMap function - .keyBy(Event::sourceAddress) - - // the function that evaluates the state machine over the sequence of events - .flatMap(new StateMachineMapper()); - - // output the alerts to std-out - if (outputFile == null) { - alerts.print(); - } else { - alerts.sinkTo( - FileSink.forRowFormat( - new Path(outputFile), new SimpleStringEncoder<>()) - .withRollingPolicy( - DefaultRollingPolicy.builder() - .withMaxPartSize(MemorySize.ofMebiBytes(1)) - .withRolloverInterval(Duration.ofSeconds(10)) - .build()) - .build()) - .setParallelism(1) - .name("output"); - } - - // trigger program execution - env.execute("State machine job"); - } - - // ------------------------------------------------------------------------ - - /** - * The function that maintains the per-IP-address state machines and verifies that the events - * are consistent with the current state of the state machine. If the event is not consistent - * with the current state, the function produces an alert. - */ - @SuppressWarnings("serial") - static class StateMachineMapper extends RichFlatMapFunction { - - /** The state for the current key. */ - private ValueState currentState; - - @Override - public void open(OpenContext openContext) { - // get access to the state object - currentState = - getRuntimeContext().getState(new ValueStateDescriptor<>("state", State.class)); - } - - @Override - public void flatMap(Event evt, Collector out) throws Exception { - // get the current state for the key (source address) - // if no state exists, yet, the state must be the state machine's initial state - State state = currentState.value(); - if (state == null) { - state = State.Initial; - } - - // ask the state machine what state we should go to based on the given event - State nextState = state.transition(evt.type()); - - if (nextState == State.InvalidTransition) { - // the current event resulted in an invalid transition - // raise an alert! - out.collect(new Alert(evt.sourceAddress(), state, evt.type())); - } else if (nextState.isTerminal()) { - // we reached a terminal state, clean up the current state - currentState.clear(); - } else { - // remember the new state - currentState.update(nextState); - } - } - } - - // Used for backwards compatibility to convert legacy 'sleep' parameter to records per second. - private static double rpsFromSleep(int sleep, int parallelism) { - return (1000d / sleep) * parallelism; - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/EventTypeAndState.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/EventTypeAndState.java deleted file mode 100644 index 2ad77b4..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/EventTypeAndState.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.examples.statemachine.dfa; - -import org.apache.flink.streaming.examples.statemachine.event.EventType; - -/** Simple combination of EventType and State. */ -public class EventTypeAndState { - - public final EventType eventType; - - public final State state; - - public EventTypeAndState(EventType eventType, State state) { - this.eventType = eventType; - this.state = state; - } -} diff --git a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/State.java b/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/State.java deleted file mode 100644 index eb79fb8..0000000 --- a/java/Streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/dfa/State.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * 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.examples.statemachine.dfa; - -import org.apache.flink.streaming.examples.statemachine.event.EventType; - -import java.util.Random; - -/** - * The State captures the main functionality of the state machine. It represents a specific state in - * the state machine, and holds all transitions possible from a specific state. - * - *