ctx) {
31 | for (int i = 0; (i < testStream.length) && running; i++) {
32 | if (testStream[i] instanceof Long) {
33 | Long ts = (Long) testStream[i];
34 | ctx.emitWatermark(new Watermark(ts));
35 | } else {
36 | //noinspection unchecked
37 | T element = (T) testStream[i];
38 | ctx.collectWithTimestamp(element, getTimestamp(element));
39 | }
40 | }
41 | // test sources are finite, so they have a Long.MAX_VALUE watermark when they finishes
42 | }
43 |
44 | abstract long getTimestamp(T element);
45 |
46 | @Override
47 | public void cancel() {
48 | running = false;
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/common/src/main/java/org/apache/flink/training/exercises/common/sources/TaxiFareGenerator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.common.sources;
20 |
21 | import org.apache.flink.streaming.api.functions.source.SourceFunction;
22 | import org.apache.flink.streaming.api.watermark.Watermark;
23 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
24 |
25 | /**
26 | * This SourceFunction generates a data stream of TaxiFare records that include event time
27 | * timestamps.
28 | *
29 | * The stream is generated in order, and it includes Watermarks.
30 | *
31 | */
32 | public class TaxiFareGenerator implements SourceFunction {
33 |
34 | private volatile boolean running = true;
35 |
36 | @Override
37 | public void run(SourceContext ctx) throws Exception {
38 |
39 | long id = 1;
40 |
41 | while (running) {
42 | TaxiFare fare = new TaxiFare(id);
43 | id += 1;
44 |
45 | ctx.collectWithTimestamp(fare, fare.getEventTime());
46 | ctx.emitWatermark(new Watermark(fare.getEventTime()));
47 |
48 | // match our event production rate to that of the TaxiRideGenerator
49 | Thread.sleep(TaxiRideGenerator.SLEEP_MILLIS_PER_EVENT);
50 | }
51 | }
52 |
53 | @Override
54 | public void cancel() {
55 | running = false;
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/hourly-tips/src/main/scala/org/apache/flink/training/exercises/hourlytips/scala/HourlyTipsExercise.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.hourlytips.scala
20 |
21 | import org.apache.flink.streaming.api.TimeCharacteristic
22 | import org.apache.flink.streaming.api.scala._
23 | import org.apache.flink.training.exercises.common.sources.TaxiFareGenerator
24 | import org.apache.flink.training.exercises.common.utils.ExerciseBase._
25 | import org.apache.flink.training.exercises.common.utils.{ExerciseBase, MissingSolutionException}
26 |
27 | /**
28 | * The "Hourly Tips" exercise of the Flink training in the docs.
29 | *
30 | * The task of the exercise is to first calculate the total tips collected by each driver, hour by hour, and
31 | * then from that stream, find the highest tip total in each hour.
32 | *
33 | */
34 | object HourlyTipsExercise {
35 |
36 | def main(args: Array[String]) {
37 |
38 | // set up streaming execution environment
39 | val env = StreamExecutionEnvironment.getExecutionEnvironment
40 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
41 | env.setParallelism(ExerciseBase.parallelism)
42 |
43 | // start the data generator
44 | val fares = env.addSource(fareSourceOrTest(new TaxiFareGenerator()))
45 |
46 | throw new MissingSolutionException()
47 |
48 | // print result on stdout
49 | // printOrTest(hourlyMax)
50 |
51 | // execute the transformation pipeline
52 | env.execute("Hourly Tips (scala)")
53 | }
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/long-ride-alerts/DISCUSSION.md:
--------------------------------------------------------------------------------
1 |
19 |
20 | # Lab Discussion: `ProcessFunction` and Timers (Long Ride Alerts)
21 |
22 | (Discussion of [Lab: `ProcessFunction` and Timers (Long Ride Alerts)](./))
23 |
24 | It would be interesting to test that the solution does not leak state.
25 |
26 | A good way to write unit tests for a `KeyedProcessFunction` to check for state retention, etc., is to
27 | use the test harnesses described in the
28 | [documentation on testing](https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/testing.html#unit-testing-stateful-or-timely-udfs--custom-operators).
29 |
30 | In fact, the reference solutions will leak state in the case where a START event is missing. They also
31 | leak in the case where the alert is generated, but then the END event does eventually arrive (after `onTimer()`
32 | has cleared the matching START event).
33 |
34 | This could be addressed either by using [state TTL](https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/state/state.html#state-time-to-live-ttl),
35 | or by using another timer that eventually
36 | clears any remaining state. There is a tradeoff here, however: once that state has been removed,
37 | then if the matching events are not actually missing, but are instead very, very late, they will cause erroneous alerts.
38 |
39 | This tradeoff between keeping state indefinitely versus occasionally getting things wrong when events are
40 | exceptionally late is a challenge that is inherent to stateful stream processing.
41 |
42 | -----
43 |
44 | [**Back to Labs Overview**](../LABS-OVERVIEW.md)
45 |
--------------------------------------------------------------------------------
/ride-cleansing/src/solution/scala/org/apache/flink/training/solutions/ridecleansing/scala/RideCleansingSolution.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.solutions.ridecleansing.scala
20 |
21 | import org.apache.flink.streaming.api.scala._
22 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator
23 | import org.apache.flink.training.exercises.common.utils.ExerciseBase._
24 | import org.apache.flink.training.exercises.common.utils.{ExerciseBase, GeoUtils}
25 |
26 | /**
27 | * Scala reference implementation for the "Ride Cleansing" exercise of the Flink training in the docs.
28 | *
29 | * The task of the exercise is to filter a data stream of taxi ride records to keep only rides that
30 | * start and end within New York City. The resulting stream should be printed to the
31 | * standard out.
32 | *
33 | */
34 | object RideCleansingSolution {
35 |
36 | def main(args: Array[String]) {
37 |
38 | // set up the execution environment
39 | val env = StreamExecutionEnvironment.getExecutionEnvironment
40 | env.setParallelism(ExerciseBase.parallelism)
41 |
42 | // get the taxi ride data stream
43 | val rides = env.addSource(rideSourceOrTest(new TaxiRideGenerator()))
44 |
45 | val filteredRides = rides
46 | .filter(r => GeoUtils.isInNYC(r.startLon, r.startLat) && GeoUtils.isInNYC(r.endLon, r.endLat))
47 |
48 | // print the filtered stream
49 | printOrTest(filteredRides)
50 |
51 | // run the cleansing pipeline
52 | env.execute("Taxi Ride Cleansing")
53 | }
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/ride-cleansing/src/main/scala/org/apache/flink/training/exercises/ridecleansing/scala/RideCleansingExercise.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.ridecleansing.scala
20 |
21 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator
22 | import org.apache.flink.training.exercises.common.utils.ExerciseBase._
23 | import org.apache.flink.training.exercises.common.utils.{ExerciseBase, GeoUtils, MissingSolutionException}
24 | import org.apache.flink.streaming.api.scala._
25 |
26 | /**
27 | * The "Ride Cleansing" exercise of the Flink training in the docs.
28 | *
29 | * The task of the exercise is to filter a data stream of taxi ride records to keep only rides that
30 | * start and end within New York City. The resulting stream should be printed to the
31 | * standard out.
32 | *
33 | */
34 | object RideCleansingExercise extends ExerciseBase {
35 |
36 | def main(args: Array[String]) {
37 |
38 | // set up the execution environment
39 | val env = StreamExecutionEnvironment.getExecutionEnvironment
40 | env.setParallelism(parallelism)
41 |
42 | // get the taxi ride data stream
43 | val rides = env.addSource(rideSourceOrTest(new TaxiRideGenerator()))
44 |
45 | val filteredRides = rides
46 | // filter out rides that do not start and end in NYC
47 | .filter(ride => throw new MissingSolutionException)
48 |
49 | // print the filtered stream
50 | printOrTest(filteredRides)
51 |
52 | // run the cleansing pipeline
53 | env.execute("Taxi Ride Cleansing")
54 | }
55 |
56 | }
57 |
--------------------------------------------------------------------------------
/hourly-tips/src/test/scala/org/apache/flink/training/exercises/hourlytips/scala/HourlyTipsTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.hourlytips.scala
20 |
21 | import java.util
22 |
23 | import org.apache.flink.api.java.tuple
24 | import org.apache.flink.training.exercises.hourlytips
25 | import org.apache.flink.training.exercises.testing.TaxiRideTestBase
26 | import org.apache.flink.training.solutions.hourlytips.scala.HourlyTipsSolution
27 |
28 | class HourlyTipsTest extends hourlytips.HourlyTipsTest {
29 | private val scalaExercise: TaxiRideTestBase.Testable = () => HourlyTipsExercise.main(Array.empty[String])
30 |
31 | @throws[Exception]
32 | override protected def results(source: TaxiRideTestBase.TestFareSource): util.List[tuple.Tuple3[java.lang.Long, java.lang.Long, java.lang.Float]] = {
33 | val scalaSolution: TaxiRideTestBase.Testable = () => HourlyTipsSolution.main(Array.empty[String])
34 | val tuples: util.List[_] = runApp(source, new TaxiRideTestBase.TestSink[tuple.Tuple3[java.lang.Long, java.lang.Long, java.lang.Float]], scalaExercise, scalaSolution)
35 | javaTuples(tuples.asInstanceOf[util.List[(Long, Long, Float)]])
36 | }
37 |
38 | private def javaTuples(a: util.List[(Long, Long, Float)]): util.ArrayList[tuple.Tuple3[java.lang.Long, java.lang.Long, java.lang.Float]] = {
39 | val javaCopy: util.ArrayList[tuple.Tuple3[java.lang.Long, java.lang.Long, java.lang.Float]] = new util.ArrayList[tuple.Tuple3[java.lang.Long, java.lang.Long, java.lang.Float]](a.size)
40 | a.iterator.forEachRemaining((t: (Long, Long, Float)) => javaCopy.add(tuple.Tuple3.of(t._1, t._2, t._3)))
41 | javaCopy
42 | }
43 |
44 | }
45 |
--------------------------------------------------------------------------------
/rides-and-fares/src/test/scala/org/apache/flink/training/exercises/ridesandfares/scala/RidesAndFaresTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.ridesandfares.scala
20 |
21 | import java.util
22 |
23 | import org.apache.flink.api.java.tuple
24 | import org.apache.flink.training.exercises.common.datatypes.{TaxiFare, TaxiRide}
25 | import org.apache.flink.training.exercises.ridesandfares
26 | import org.apache.flink.training.exercises.testing.TaxiRideTestBase
27 | import org.apache.flink.training.solutions.ridesandfares.scala.RidesAndFaresSolution
28 |
29 | class RidesAndFaresTest extends ridesandfares.RidesAndFaresTest{
30 | private val scalaExercise: TaxiRideTestBase.Testable = () => RidesAndFaresExercise.main(Array.empty[String])
31 |
32 | @throws[Exception]
33 | override protected def results(rides: TaxiRideTestBase.TestRideSource, fares: TaxiRideTestBase.TestFareSource): util.List[tuple.Tuple2[TaxiRide, TaxiFare]] = {
34 | val scalaSolution: TaxiRideTestBase.Testable = () => RidesAndFaresSolution.main(Array.empty[String])
35 | val tuples: util.List[_] = runApp(rides, fares, new TaxiRideTestBase.TestSink[tuple.Tuple2[TaxiRide, TaxiFare]], scalaExercise, scalaSolution)
36 | javaTuples(tuples.asInstanceOf[util.List[(TaxiRide, TaxiFare)]])
37 | }
38 |
39 | private def javaTuples(a: util.List[(TaxiRide, TaxiFare)]): util.ArrayList[tuple.Tuple2[TaxiRide, TaxiFare]] = {
40 | val javaCopy: util.ArrayList[tuple.Tuple2[TaxiRide, TaxiFare]] = new util.ArrayList[tuple.Tuple2[TaxiRide, TaxiFare]](a.size)
41 | a.iterator.forEachRemaining((t: (TaxiRide, TaxiFare)) => javaCopy.add(tuple.Tuple2.of(t._1, t._2)))
42 | javaCopy
43 | }
44 |
45 | }
46 |
--------------------------------------------------------------------------------
/hourly-tips/src/main/java/org/apache/flink/training/exercises/hourlytips/HourlyTipsExercise.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.hourlytips;
20 |
21 | import org.apache.flink.streaming.api.TimeCharacteristic;
22 | import org.apache.flink.streaming.api.datastream.DataStream;
23 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
24 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
25 | import org.apache.flink.training.exercises.common.sources.TaxiFareGenerator;
26 | import org.apache.flink.training.exercises.common.utils.ExerciseBase;
27 | import org.apache.flink.training.exercises.common.utils.MissingSolutionException;
28 |
29 | /**
30 | * The "Hourly Tips" exercise of the Flink training in the docs.
31 | *
32 | * The task of the exercise is to first calculate the total tips collected by each driver, hour by hour, and
33 | * then from that stream, find the highest tip total in each hour.
34 | *
35 | */
36 | public class HourlyTipsExercise extends ExerciseBase {
37 |
38 | /**
39 | * Main method.
40 | *
41 | * @throws Exception which occurs during job execution.
42 | */
43 | public static void main(String[] args) throws Exception {
44 |
45 | // set up streaming execution environment
46 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
47 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
48 | env.setParallelism(ExerciseBase.parallelism);
49 |
50 | // start the data generator
51 | DataStream fares = env.addSource(fareSourceOrTest(new TaxiFareGenerator()));
52 |
53 | throw new MissingSolutionException();
54 |
55 | // printOrTest(hourlyMax);
56 |
57 | // execute the transformation pipeline
58 | // env.execute("Hourly Tips (java)");
59 | }
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/ride-cleansing/src/test/java/org/apache/flink/training/exercises/ridecleansing/RideCleansingTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.ridecleansing;
20 |
21 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
22 | import org.apache.flink.training.exercises.testing.TaxiRideTestBase;
23 | import org.apache.flink.training.solutions.ridecleansing.RideCleansingSolution;
24 |
25 | import org.junit.Test;
26 |
27 | import java.time.Instant;
28 | import java.util.Collections;
29 | import java.util.List;
30 |
31 | import static org.junit.Assert.assertEquals;
32 |
33 | public class RideCleansingTest extends TaxiRideTestBase {
34 |
35 | static final Testable JAVA_EXERCISE = () -> RideCleansingExercise.main(new String[]{});
36 |
37 | @Test
38 | public void testInNYC() throws Exception {
39 | TaxiRide atPennStation = testRide(-73.9947F, 40.750626F, -73.9947F, 40.750626F);
40 |
41 | TestRideSource source = new TestRideSource(atPennStation);
42 |
43 | assertEquals(Collections.singletonList(atPennStation), results(source));
44 | }
45 |
46 | @Test
47 | public void testNotInNYC() throws Exception {
48 | TaxiRide toThePole = testRide(-73.9947F, 40.750626F, 0, 90);
49 | TaxiRide fromThePole = testRide(0, 90, -73.9947F, 40.750626F);
50 | TaxiRide atNorthPole = testRide(0, 90, 0, 90);
51 |
52 | TestRideSource source = new TestRideSource(toThePole, fromThePole, atNorthPole);
53 |
54 | assertEquals(Collections.emptyList(), results(source));
55 | }
56 |
57 | private TaxiRide testRide(float startLon, float startLat, float endLon, float endLat) {
58 | return new TaxiRide(1L, true, Instant.EPOCH, Instant.EPOCH,
59 | startLon, startLat, endLon, endLat, (short) 1, 0, 0);
60 | }
61 |
62 | protected List> results(TestRideSource source) throws Exception {
63 | Testable javaSolution = () -> RideCleansingSolution.main(new String[]{});
64 | return runApp(source, new TestSink<>(), JAVA_EXERCISE, javaSolution);
65 | }
66 |
67 | }
68 |
--------------------------------------------------------------------------------
/ride-cleansing/src/main/java/org/apache/flink/training/exercises/ridecleansing/RideCleansingExercise.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.ridecleansing;
20 |
21 | import org.apache.flink.api.common.functions.FilterFunction;
22 | import org.apache.flink.streaming.api.datastream.DataStream;
23 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
24 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
25 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator;
26 | import org.apache.flink.training.exercises.common.utils.ExerciseBase;
27 | import org.apache.flink.training.exercises.common.utils.MissingSolutionException;
28 |
29 | /**
30 | * The "Ride Cleansing" exercise from the Flink training in the docs.
31 | *
32 | * The task of the exercise is to filter a data stream of taxi ride records to keep only rides that
33 | * start and end within New York City. The resulting stream should be printed.
34 | *
35 | */
36 | public class RideCleansingExercise extends ExerciseBase {
37 |
38 | /**
39 | * Main method.
40 | *
41 | * @throws Exception which occurs during job execution.
42 | */
43 | public static void main(String[] args) throws Exception {
44 |
45 | // set up streaming execution environment
46 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
47 | env.setParallelism(ExerciseBase.parallelism);
48 |
49 | // start the data generator
50 | DataStream rides = env.addSource(rideSourceOrTest(new TaxiRideGenerator()));
51 |
52 | DataStream filteredRides = rides
53 | // filter out rides that do not start or stop in NYC
54 | .filter(new NYCFilter());
55 |
56 | // print the filtered stream
57 | printOrTest(filteredRides);
58 |
59 | // run the cleansing pipeline
60 | env.execute("Taxi Ride Cleansing");
61 | }
62 |
63 | private static class NYCFilter implements FilterFunction {
64 |
65 | @Override
66 | public boolean filter(TaxiRide taxiRide) throws Exception {
67 | throw new MissingSolutionException();
68 | }
69 | }
70 |
71 | }
72 |
--------------------------------------------------------------------------------
/rides-and-fares/src/main/scala/org/apache/flink/training/exercises/ridesandfares/scala/RidesAndFaresExercise.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.ridesandfares.scala
20 |
21 | import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction
22 | import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, _}
23 | import org.apache.flink.training.exercises.common.datatypes.{TaxiFare, TaxiRide}
24 | import org.apache.flink.training.exercises.common.sources.{TaxiFareGenerator, TaxiRideGenerator}
25 | import org.apache.flink.training.exercises.common.utils.ExerciseBase._
26 | import org.apache.flink.training.exercises.common.utils.{ExerciseBase, MissingSolutionException}
27 | import org.apache.flink.util.Collector
28 |
29 | /**
30 | * The "Stateful Enrichment" exercise of the Flink training in the docs.
31 | *
32 | * The goal for this exercise is to enrich TaxiRides with fare information.
33 | *
34 | */
35 | object RidesAndFaresExercise {
36 |
37 | def main(args: Array[String]) {
38 |
39 | // set up streaming execution environment
40 | val env = StreamExecutionEnvironment.getExecutionEnvironment
41 | env.setParallelism(ExerciseBase.parallelism)
42 |
43 | val rides = env
44 | .addSource(rideSourceOrTest(new TaxiRideGenerator()))
45 | .filter { ride => ride.isStart }
46 | .keyBy { ride => ride.rideId }
47 |
48 | val fares = env
49 | .addSource(fareSourceOrTest(new TaxiFareGenerator()))
50 | .keyBy { fare => fare.rideId }
51 |
52 | val processed = rides
53 | .connect(fares)
54 | .flatMap(new EnrichmentFunction)
55 |
56 | printOrTest(processed)
57 |
58 | env.execute("Join Rides with Fares (scala RichCoFlatMap)")
59 | }
60 |
61 | class EnrichmentFunction extends RichCoFlatMapFunction[TaxiRide, TaxiFare, (TaxiRide, TaxiFare)] {
62 |
63 | override def flatMap1(ride: TaxiRide, out: Collector[(TaxiRide, TaxiFare)]): Unit = {
64 | throw new MissingSolutionException()
65 | }
66 |
67 | override def flatMap2(fare: TaxiFare, out: Collector[(TaxiRide, TaxiFare)]): Unit = {
68 | }
69 |
70 | }
71 |
72 | }
73 |
--------------------------------------------------------------------------------
/ride-cleansing/src/solution/java/org/apache/flink/training/solutions/ridecleansing/RideCleansingSolution.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.solutions.ridecleansing;
20 |
21 | import org.apache.flink.api.common.functions.FilterFunction;
22 | import org.apache.flink.streaming.api.datastream.DataStream;
23 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
24 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
25 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator;
26 | import org.apache.flink.training.exercises.common.utils.ExerciseBase;
27 | import org.apache.flink.training.exercises.common.utils.GeoUtils;
28 |
29 | /**
30 | * Solution to the "Ride Cleansing" exercise of the Flink training in the docs.
31 | *
32 | * The task of the exercise is to filter a data stream of taxi ride records to keep only rides that
33 | * start and end within New York City. The resulting stream should be printed.
34 | *
35 | */
36 | public class RideCleansingSolution extends ExerciseBase {
37 |
38 | /**
39 | * Main method.
40 | *
41 | * @throws Exception which occurs during job execution.
42 | */
43 | public static void main(String[] args) throws Exception {
44 |
45 | // set up streaming execution environment
46 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
47 | env.setParallelism(ExerciseBase.parallelism);
48 |
49 | // start the data generator
50 | DataStream rides = env.addSource(rideSourceOrTest(new TaxiRideGenerator()));
51 |
52 | DataStream filteredRides = rides
53 | // keep only those rides and both start and end in NYC
54 | .filter(new NYCFilter());
55 |
56 | // print the filtered stream
57 | printOrTest(filteredRides);
58 |
59 | // run the cleansing pipeline
60 | env.execute("Taxi Ride Cleansing");
61 | }
62 |
63 | public static class NYCFilter implements FilterFunction {
64 | @Override
65 | public boolean filter(TaxiRide taxiRide) {
66 | return GeoUtils.isInNYC(taxiRide.startLon, taxiRide.startLat) &&
67 | GeoUtils.isInNYC(taxiRide.endLon, taxiRide.endLat);
68 | }
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/common/src/main/java/org/apache/flink/training/examples/ridecount/RideCountExample.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.examples.ridecount;
20 |
21 | import org.apache.flink.api.common.functions.MapFunction;
22 | import org.apache.flink.api.java.tuple.Tuple2;
23 | import org.apache.flink.streaming.api.datastream.DataStream;
24 | import org.apache.flink.streaming.api.datastream.KeyedStream;
25 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
26 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
27 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator;
28 |
29 | /**
30 | * Example that counts the rides for each driver.
31 | *
32 | * Note that this is implicitly keeping state for each driver.
33 | * This sort of simple, non-windowed aggregation on an unbounded set of keys will use an unbounded amount of state.
34 | * When this is an issue, look at the SQL/Table API, or ProcessFunction, or state TTL, all of which provide
35 | * mechanisms for expiring state for stale keys.
36 | */
37 | public class RideCountExample {
38 |
39 | /**
40 | * Main method.
41 | *
42 | * @throws Exception which occurs during job execution.
43 | */
44 | public static void main(String[] args) throws Exception {
45 |
46 | // set up streaming execution environment
47 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
48 |
49 | // start the data generator
50 | DataStream rides = env.addSource(new TaxiRideGenerator());
51 |
52 | // map each ride to a tuple of (driverId, 1)
53 | DataStream> tuples = rides.map(new MapFunction>() {
54 | @Override
55 | public Tuple2 map(TaxiRide ride) {
56 | return Tuple2.of(ride.driverId, 1L);
57 | }
58 | });
59 |
60 | // partition the stream by the driverId
61 | KeyedStream, Long> keyedByDriverId = tuples.keyBy(t -> t.f0);
62 |
63 | // count the rides for each driver
64 | DataStream> rideCounts = keyedByDriverId.sum(1);
65 |
66 | // we could, in fact, print out any or all of these streams
67 | rideCounts.print();
68 |
69 | // run the cleansing pipeline
70 | env.execute("Ride Count");
71 | }
72 | }
73 |
--------------------------------------------------------------------------------
/common/src/main/java/org/apache/flink/training/exercises/common/utils/ExerciseBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.common.utils;
20 |
21 | import org.apache.flink.streaming.api.functions.sink.SinkFunction;
22 | import org.apache.flink.streaming.api.functions.source.SourceFunction;
23 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
24 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
25 |
26 | /**
27 | * Base for all exercises with a few helper methods.
28 | */
29 | public class ExerciseBase {
30 | public static SourceFunction rides = null;
31 | public static SourceFunction fares = null;
32 | public static SourceFunction strings = null;
33 | public static SinkFunction out = null;
34 | public static int parallelism = 4;
35 |
36 | /**
37 | * Retrieves a test source during unit tests and the given one during normal execution.
38 | */
39 | public static SourceFunction rideSourceOrTest(SourceFunction source) {
40 | if (rides == null) {
41 | return source;
42 | }
43 | return rides;
44 | }
45 |
46 | /**
47 | * Retrieves a test source during unit tests and the given one during normal execution.
48 | */
49 | public static SourceFunction fareSourceOrTest(SourceFunction source) {
50 | if (fares == null) {
51 | return source;
52 | }
53 | return fares;
54 | }
55 |
56 | /**
57 | * Retrieves a test source during unit tests and the given one during normal execution.
58 | */
59 | public static SourceFunction stringSourceOrTest(SourceFunction source) {
60 | if (strings == null) {
61 | return source;
62 | }
63 | return strings;
64 | }
65 |
66 | /**
67 | * Prints the given data stream during normal execution and collects outputs during tests.
68 | */
69 | public static void printOrTest(org.apache.flink.streaming.api.datastream.DataStream> ds) {
70 | if (out == null) {
71 | ds.print();
72 | } else {
73 | ds.addSink(out);
74 | }
75 | }
76 |
77 | /**
78 | * Prints the given data stream during normal execution and collects outputs during tests.
79 | */
80 | public static void printOrTest(org.apache.flink.streaming.api.scala.DataStream> ds) {
81 | if (out == null) {
82 | ds.print();
83 | } else {
84 | ds.addSink(out);
85 | }
86 | }
87 | }
88 |
--------------------------------------------------------------------------------
/long-ride-alerts/src/main/scala/org/apache/flink/training/exercises/longrides/scala/LongRidesExercise.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.longrides.scala
20 |
21 | import org.apache.flink.configuration.Configuration
22 | import org.apache.flink.streaming.api.TimeCharacteristic
23 | import org.apache.flink.streaming.api.functions.KeyedProcessFunction
24 | import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, _}
25 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide
26 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator
27 | import org.apache.flink.training.exercises.common.utils.ExerciseBase._
28 | import org.apache.flink.training.exercises.common.utils.{ExerciseBase, MissingSolutionException}
29 | import org.apache.flink.util.Collector
30 |
31 | /**
32 | * The "Long Ride Alerts" exercise of the Flink training in the docs.
33 | *
34 | * The goal for this exercise is to emit START events for taxi rides that have not been matched
35 | * by an END event during the first 2 hours of the ride.
36 | *
37 | */
38 | object LongRidesExercise {
39 |
40 | def main(args: Array[String]) {
41 |
42 | // set up the execution environment
43 | val env = StreamExecutionEnvironment.getExecutionEnvironment
44 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
45 | env.setParallelism(ExerciseBase.parallelism)
46 |
47 | val rides = env.addSource(rideSourceOrTest(new TaxiRideGenerator()))
48 |
49 | val longRides = rides
50 | .keyBy(_.rideId)
51 | .process(new ImplementMeFunction())
52 |
53 | printOrTest(longRides)
54 |
55 | env.execute("Long Taxi Rides")
56 | }
57 |
58 | class ImplementMeFunction extends KeyedProcessFunction[Long, TaxiRide, TaxiRide] {
59 |
60 | override def open(parameters: Configuration): Unit = {
61 | throw new MissingSolutionException()
62 | }
63 |
64 | override def processElement(ride: TaxiRide,
65 | context: KeyedProcessFunction[Long, TaxiRide, TaxiRide]#Context,
66 | out: Collector[TaxiRide]): Unit = {
67 | }
68 |
69 | override def onTimer(timestamp: Long,
70 | ctx: KeyedProcessFunction[Long, TaxiRide, TaxiRide]#OnTimerContext,
71 | out: Collector[TaxiRide]): Unit = {
72 | }
73 |
74 | }
75 |
76 | }
77 |
--------------------------------------------------------------------------------
/long-ride-alerts/src/main/java/org/apache/flink/training/exercises/longrides/LongRidesExercise.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.longrides;
20 |
21 | import org.apache.flink.configuration.Configuration;
22 | import org.apache.flink.streaming.api.TimeCharacteristic;
23 | import org.apache.flink.streaming.api.TimerService;
24 | import org.apache.flink.streaming.api.datastream.DataStream;
25 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
26 | import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
27 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
28 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator;
29 | import org.apache.flink.training.exercises.common.utils.ExerciseBase;
30 | import org.apache.flink.training.exercises.common.utils.MissingSolutionException;
31 | import org.apache.flink.util.Collector;
32 |
33 | /**
34 | * The "Long Ride Alerts" exercise of the Flink training in the docs.
35 | *
36 | * The goal for this exercise is to emit START events for taxi rides that have not been matched
37 | * by an END event during the first 2 hours of the ride.
38 | *
39 | */
40 | public class LongRidesExercise extends ExerciseBase {
41 |
42 | /**
43 | * Main method.
44 | *
45 | * @throws Exception which occurs during job execution.
46 | */
47 | public static void main(String[] args) throws Exception {
48 |
49 | // set up streaming execution environment
50 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
51 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
52 | env.setParallelism(ExerciseBase.parallelism);
53 |
54 | // start the data generator
55 | DataStream rides = env.addSource(rideSourceOrTest(new TaxiRideGenerator()));
56 |
57 | DataStream longRides = rides
58 | .keyBy((TaxiRide ride) -> ride.rideId)
59 | .process(new MatchFunction());
60 |
61 | printOrTest(longRides);
62 |
63 | env.execute("Long Taxi Rides");
64 | }
65 |
66 | public static class MatchFunction extends KeyedProcessFunction {
67 |
68 | @Override
69 | public void open(Configuration config) throws Exception {
70 | throw new MissingSolutionException();
71 | }
72 |
73 | @Override
74 | public void processElement(TaxiRide ride, Context context, Collector out) throws Exception {
75 | TimerService timerService = context.timerService();
76 | }
77 |
78 | @Override
79 | public void onTimer(long timestamp, OnTimerContext context, Collector out) throws Exception {
80 | }
81 | }
82 | }
83 |
--------------------------------------------------------------------------------
/rides-and-fares/src/test/java/org/apache/flink/training/exercises/ridesandfares/RidesAndFaresTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.ridesandfares;
20 |
21 | import org.apache.flink.api.java.tuple.Tuple2;
22 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
23 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
24 | import org.apache.flink.training.exercises.testing.TaxiRideTestBase;
25 | import org.apache.flink.training.solutions.ridesandfares.RidesAndFaresSolution;
26 |
27 | import org.junit.Test;
28 |
29 | import java.time.Instant;
30 | import java.util.Arrays;
31 | import java.util.List;
32 |
33 | import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
34 | import static org.junit.Assert.assertThat;
35 |
36 | public class RidesAndFaresTest extends TaxiRideTestBase> {
37 |
38 | static final Testable JAVA_EXERCISE = () -> RidesAndFaresExercise.main(new String[]{});
39 |
40 | final TaxiRide ride1 = testRide(1);
41 | final TaxiRide ride2 = testRide(2);
42 | final TaxiFare fare1 = testFare(1);
43 | final TaxiFare fare2 = testFare(2);
44 |
45 | @Test
46 | public void testInOrder() throws Exception {
47 | TestRideSource rides = new TestRideSource(ride1, ride2);
48 | TestFareSource fares = new TestFareSource(fare1, fare2);
49 |
50 | List> expected = Arrays.asList(
51 | Tuple2.of(ride1, fare1),
52 | Tuple2.of(ride2, fare2));
53 |
54 | assertThat("Join results don't match", results(rides, fares), containsInAnyOrder(expected.toArray()));
55 | }
56 |
57 | @Test
58 | public void testOutOfOrder() throws Exception {
59 | TestRideSource rides = new TestRideSource(ride1, ride2);
60 | TestFareSource fares = new TestFareSource(fare2, fare1);
61 |
62 | List> expected = Arrays.asList(
63 | Tuple2.of(ride1, fare1),
64 | Tuple2.of(ride2, fare2));
65 |
66 | assertThat("Join results don't match", results(rides, fares), containsInAnyOrder(expected.toArray()));
67 | }
68 |
69 | private TaxiRide testRide(long rideId) {
70 | return new TaxiRide(rideId, true, Instant.EPOCH, Instant.EPOCH,
71 | 0F, 0F, 0F, 0F, (short) 1, 0, rideId);
72 | }
73 |
74 | private TaxiFare testFare(long rideId) {
75 | return new TaxiFare(rideId, 0, rideId, Instant.EPOCH, "", 0F, 0F, 0F);
76 | }
77 |
78 | protected List> results(TestRideSource rides, TestFareSource fares) throws Exception {
79 | Testable javaSolution = () -> RidesAndFaresSolution.main(new String[]{});
80 | return runApp(rides, fares, new TestSink<>(), JAVA_EXERCISE, javaSolution);
81 | }
82 |
83 | }
84 |
--------------------------------------------------------------------------------
/common/src/main/java/org/apache/flink/training/exercises/common/sources/TaxiRideGenerator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.common.sources;
20 |
21 | import org.apache.flink.streaming.api.functions.source.SourceFunction;
22 | import org.apache.flink.streaming.api.watermark.Watermark;
23 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
24 |
25 | import java.util.ArrayList;
26 | import java.util.List;
27 | import java.util.PriorityQueue;
28 | import java.util.Random;
29 |
30 | /**
31 | * This SourceFunction generates a data stream of TaxiRide records that include event time
32 | * timestamps.
33 | *
34 | * The stream is produced out-of-order, and includes Watermarks (with no late events).
35 | *
36 | */
37 | public class TaxiRideGenerator implements SourceFunction {
38 |
39 | public static final int SLEEP_MILLIS_PER_EVENT = 10;
40 | private static final int BATCH_SIZE = 5;
41 | private volatile boolean running = true;
42 |
43 | @Override
44 | public void run(SourceContext ctx) throws Exception {
45 |
46 | PriorityQueue endEventQ = new PriorityQueue<>(100);
47 | long id = 0;
48 | long maxStartTime = 0;
49 |
50 | while (running) {
51 |
52 | // generate a batch of START events
53 | List startEvents = new ArrayList(BATCH_SIZE);
54 | for (int i = 1; i <= BATCH_SIZE; i++) {
55 | TaxiRide ride = new TaxiRide(id + i, true);
56 | startEvents.add(ride);
57 | // the start times may be in order, but let's not assume that
58 | maxStartTime = Math.max(maxStartTime, ride.startTime.toEpochMilli());
59 | }
60 |
61 | // enqueue the corresponding END events
62 | for (int i = 1; i <= BATCH_SIZE; i++) {
63 | endEventQ.add(new TaxiRide(id + i, false));
64 | }
65 |
66 | // release the END events coming before the end of this new batch
67 | // (this allows a few END events to precede their matching START event)
68 | while (endEventQ.peek().getEventTime() <= maxStartTime) {
69 | TaxiRide ride = endEventQ.poll();
70 | ctx.collectWithTimestamp(ride, ride.getEventTime());
71 | }
72 |
73 | // then emit the new START events (out-of-order)
74 | java.util.Collections.shuffle(startEvents, new Random(id));
75 | startEvents.iterator().forEachRemaining(r -> ctx.collectWithTimestamp(r, r.getEventTime()));
76 |
77 | // produce a Watermark
78 | ctx.emitWatermark(new Watermark(maxStartTime));
79 |
80 | // prepare for the next batch
81 | id += BATCH_SIZE;
82 |
83 | // don't go too fast
84 | Thread.sleep(BATCH_SIZE * SLEEP_MILLIS_PER_EVENT);
85 | }
86 | }
87 |
88 | @Override
89 | public void cancel() {
90 | running = false;
91 | }
92 | }
93 |
--------------------------------------------------------------------------------
/gradlew.bat:
--------------------------------------------------------------------------------
1 | @rem
2 | @rem Copyright 2015 the original author or authors.
3 | @rem
4 | @rem Licensed under the Apache License, Version 2.0 (the "License");
5 | @rem you may not use this file except in compliance with the License.
6 | @rem You may obtain a copy of the License at
7 | @rem
8 | @rem https://www.apache.org/licenses/LICENSE-2.0
9 | @rem
10 | @rem Unless required by applicable law or agreed to in writing, software
11 | @rem distributed under the License is distributed on an "AS IS" BASIS,
12 | @rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | @rem See the License for the specific language governing permissions and
14 | @rem limitations under the License.
15 | @rem
16 |
17 | @if "%DEBUG%" == "" @echo off
18 | @rem ##########################################################################
19 | @rem
20 | @rem Gradle startup script for Windows
21 | @rem
22 | @rem ##########################################################################
23 |
24 | @rem Set local scope for the variables with windows NT shell
25 | if "%OS%"=="Windows_NT" setlocal
26 |
27 | set DIRNAME=%~dp0
28 | if "%DIRNAME%" == "" set DIRNAME=.
29 | set APP_BASE_NAME=%~n0
30 | set APP_HOME=%DIRNAME%
31 |
32 | @rem Resolve any "." and ".." in APP_HOME to make it shorter.
33 | for %%i in ("%APP_HOME%") do set APP_HOME=%%~fi
34 |
35 | @rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
36 | set DEFAULT_JVM_OPTS="-Xmx64m" "-Xms64m"
37 |
38 | @rem Find java.exe
39 | if defined JAVA_HOME goto findJavaFromJavaHome
40 |
41 | set JAVA_EXE=java.exe
42 | %JAVA_EXE% -version >NUL 2>&1
43 | if "%ERRORLEVEL%" == "0" goto init
44 |
45 | echo.
46 | echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH.
47 | echo.
48 | echo Please set the JAVA_HOME variable in your environment to match the
49 | echo location of your Java installation.
50 |
51 | goto fail
52 |
53 | :findJavaFromJavaHome
54 | set JAVA_HOME=%JAVA_HOME:"=%
55 | set JAVA_EXE=%JAVA_HOME%/bin/java.exe
56 |
57 | if exist "%JAVA_EXE%" goto init
58 |
59 | echo.
60 | echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME%
61 | echo.
62 | echo Please set the JAVA_HOME variable in your environment to match the
63 | echo location of your Java installation.
64 |
65 | goto fail
66 |
67 | :init
68 | @rem Get command-line arguments, handling Windows variants
69 |
70 | if not "%OS%" == "Windows_NT" goto win9xME_args
71 |
72 | :win9xME_args
73 | @rem Slurp the command line arguments.
74 | set CMD_LINE_ARGS=
75 | set _SKIP=2
76 |
77 | :win9xME_args_slurp
78 | if "x%~1" == "x" goto execute
79 |
80 | set CMD_LINE_ARGS=%*
81 |
82 | :execute
83 | @rem Setup the command line
84 |
85 | set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar
86 |
87 | @rem Execute Gradle
88 | "%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS%
89 |
90 | :end
91 | @rem End local scope for the variables with windows NT shell
92 | if "%ERRORLEVEL%"=="0" goto mainEnd
93 |
94 | :fail
95 | rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of
96 | rem the _cmd.exe /c_ return code!
97 | if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1
98 | exit /b 1
99 |
100 | :mainEnd
101 | if "%OS%"=="Windows_NT" endlocal
102 |
103 | :omega
104 |
--------------------------------------------------------------------------------
/common/src/main/java/org/apache/flink/training/exercises/common/datatypes/TaxiFare.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.common.datatypes;
20 |
21 | import org.apache.flink.training.exercises.common.utils.DataGenerator;
22 |
23 | import java.io.Serializable;
24 | import java.time.Instant;
25 |
26 | /**
27 | * A TaxiFare has payment information about a taxi ride.
28 | *
29 | * It has these fields in common with the TaxiRides
30 | * - the rideId
31 | * - the taxiId
32 | * - the driverId
33 | * - the startTime
34 | *
35 | *
It also includes
36 | * - the paymentType
37 | * - the tip
38 | * - the tolls
39 | * - the totalFare
40 | */
41 | public class TaxiFare implements Serializable {
42 |
43 | /**
44 | * Creates a TaxiFare with now as the start time.
45 | */
46 | public TaxiFare() {
47 | this.startTime = Instant.now();
48 | }
49 |
50 | /**
51 | * Invents a TaxiFare.
52 | */
53 | public TaxiFare(long rideId) {
54 | DataGenerator g = new DataGenerator(rideId);
55 |
56 | this.rideId = rideId;
57 | this.taxiId = g.taxiId();
58 | this.driverId = g.driverId();
59 | this.startTime = g.startTime();
60 | this.paymentType = g.paymentType();
61 | this.tip = g.tip();
62 | this.tolls = g.tolls();
63 | this.totalFare = g.totalFare();
64 | }
65 |
66 | /**
67 | * Creates a TaxiFare with the given parameters.
68 | */
69 | public TaxiFare(long rideId, long taxiId, long driverId, Instant startTime, String paymentType, float tip, float tolls, float totalFare) {
70 | this.rideId = rideId;
71 | this.taxiId = taxiId;
72 | this.driverId = driverId;
73 | this.startTime = startTime;
74 | this.paymentType = paymentType;
75 | this.tip = tip;
76 | this.tolls = tolls;
77 | this.totalFare = totalFare;
78 | }
79 |
80 | public long rideId;
81 | public long taxiId;
82 | public long driverId;
83 | public Instant startTime;
84 | public String paymentType;
85 | public float tip;
86 | public float tolls;
87 | public float totalFare;
88 |
89 | @Override
90 | public String toString() {
91 |
92 | return rideId + "," +
93 | taxiId + "," +
94 | driverId + "," +
95 | startTime.toString() + "," +
96 | paymentType + "," +
97 | tip + "," +
98 | tolls + "," +
99 | totalFare;
100 | }
101 |
102 | @Override
103 | public boolean equals(Object other) {
104 | return other instanceof TaxiFare &&
105 | this.rideId == ((TaxiFare) other).rideId;
106 | }
107 |
108 | @Override
109 | public int hashCode() {
110 | return (int) this.rideId;
111 | }
112 |
113 | /**
114 | * Gets the fare's start time.
115 | */
116 | public long getEventTime() {
117 | return startTime.toEpochMilli();
118 | }
119 |
120 | }
121 |
--------------------------------------------------------------------------------
/hourly-tips/src/solution/scala/org/apache/flink/training/solutions/hourlytips/scala/HourlyTipsSolution.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.solutions.hourlytips.scala
20 |
21 | import org.apache.flink.streaming.api.TimeCharacteristic
22 | import org.apache.flink.streaming.api.scala._
23 | import org.apache.flink.streaming.api.scala.function.ProcessWindowFunction
24 | import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows
25 | import org.apache.flink.streaming.api.windowing.time.Time
26 | import org.apache.flink.streaming.api.windowing.windows.TimeWindow
27 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare
28 | import org.apache.flink.training.exercises.common.sources.TaxiFareGenerator
29 | import org.apache.flink.training.exercises.common.utils.ExerciseBase
30 | import org.apache.flink.training.exercises.common.utils.ExerciseBase._
31 | import org.apache.flink.util.Collector
32 |
33 | /**
34 | * Scala reference implementation for the "Hourly Tips" exercise of the Flink training in the docs.
35 | *
36 | * The task of the exercise is to first calculate the total tips collected by each driver, hour by hour, and
37 | * then from that stream, find the highest tip total in each hour.
38 | *
39 | */
40 | object HourlyTipsSolution {
41 |
42 | def main(args: Array[String]) {
43 |
44 | // set up streaming execution environment
45 | val env = StreamExecutionEnvironment.getExecutionEnvironment
46 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
47 | env.setParallelism(ExerciseBase.parallelism)
48 |
49 | // start the data generator
50 | val fares = env.addSource(fareSourceOrTest(new TaxiFareGenerator()))
51 |
52 | // total tips per hour by driver
53 | val hourlyTips = fares
54 | .map((f: TaxiFare) => (f.driverId, f.tip))
55 | .keyBy(_._1)
56 | .window(TumblingEventTimeWindows.of(Time.hours(1)))
57 | .reduce(
58 | (f1: (Long, Float), f2: (Long, Float)) => { (f1._1, f1._2 + f2._2) },
59 | new WrapWithWindowInfo())
60 |
61 | // max tip total in each hour
62 | val hourlyMax = hourlyTips
63 | .windowAll(TumblingEventTimeWindows.of(Time.hours(1)))
64 | .maxBy(2)
65 |
66 | // print result on stdout
67 | printOrTest(hourlyMax)
68 |
69 | // execute the transformation pipeline
70 | env.execute("Hourly Tips (scala)")
71 | }
72 |
73 | class WrapWithWindowInfo() extends ProcessWindowFunction[(Long, Float), (Long, Long, Float), Long, TimeWindow] {
74 | override def process(key: Long, context: Context, elements: Iterable[(Long, Float)], out: Collector[(Long, Long, Float)]): Unit = {
75 | val sumOfTips = elements.iterator.next()._2
76 | out.collect((context.window.getEnd, key, sumOfTips))
77 | }
78 | }
79 |
80 | }
81 |
--------------------------------------------------------------------------------
/rides-and-fares/src/main/java/org/apache/flink/training/exercises/ridesandfares/RidesAndFaresExercise.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.ridesandfares;
20 |
21 | import org.apache.flink.api.java.tuple.Tuple2;
22 | import org.apache.flink.configuration.Configuration;
23 | import org.apache.flink.streaming.api.datastream.DataStream;
24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
25 | import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
26 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
27 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
28 | import org.apache.flink.training.exercises.common.sources.TaxiFareGenerator;
29 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator;
30 | import org.apache.flink.training.exercises.common.utils.ExerciseBase;
31 | import org.apache.flink.training.exercises.common.utils.MissingSolutionException;
32 | import org.apache.flink.util.Collector;
33 |
34 | /**
35 | * The "Stateful Enrichment" exercise of the Flink training in the docs.
36 | *
37 | *
The goal for this exercise is to enrich TaxiRides with fare information.
38 | *
39 | */
40 | public class RidesAndFaresExercise extends ExerciseBase {
41 |
42 | /**
43 | * Main method.
44 | *
45 | * @throws Exception which occurs during job execution.
46 | */
47 | public static void main(String[] args) throws Exception {
48 |
49 | // set up streaming execution environment
50 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
51 | env.setParallelism(ExerciseBase.parallelism);
52 |
53 | DataStream rides = env
54 | .addSource(rideSourceOrTest(new TaxiRideGenerator()))
55 | .filter((TaxiRide ride) -> ride.isStart)
56 | .keyBy((TaxiRide ride) -> ride.rideId);
57 |
58 | DataStream fares = env
59 | .addSource(fareSourceOrTest(new TaxiFareGenerator()))
60 | .keyBy((TaxiFare fare) -> fare.rideId);
61 |
62 | DataStream> enrichedRides = rides
63 | .connect(fares)
64 | .flatMap(new EnrichmentFunction());
65 |
66 | printOrTest(enrichedRides);
67 |
68 | env.execute("Join Rides with Fares (java RichCoFlatMap)");
69 | }
70 |
71 | public static class EnrichmentFunction extends RichCoFlatMapFunction> {
72 |
73 | @Override
74 | public void open(Configuration config) throws Exception {
75 | throw new MissingSolutionException();
76 | }
77 |
78 | @Override
79 | public void flatMap1(TaxiRide ride, Collector> out) throws Exception {
80 | }
81 |
82 | @Override
83 | public void flatMap2(TaxiFare fare, Collector> out) throws Exception {
84 | }
85 | }
86 | }
87 |
--------------------------------------------------------------------------------
/rides-and-fares/src/solution/scala/org/apache/flink/training/solutions/ridesandfares/scala/RidesAndFaresSolution.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.solutions.ridesandfares.scala
20 |
21 | import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
22 | import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction
23 | import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, _}
24 | import org.apache.flink.training.exercises.common.datatypes.{TaxiFare, TaxiRide}
25 | import org.apache.flink.training.exercises.common.sources.{TaxiFareGenerator, TaxiRideGenerator}
26 | import org.apache.flink.training.exercises.common.utils.ExerciseBase
27 | import org.apache.flink.training.exercises.common.utils.ExerciseBase._
28 | import org.apache.flink.util.Collector
29 |
30 | /**
31 | * Scala reference implementation for the "Stateful Enrichment" exercise of the Flink training in the docs.
32 | *
33 | * The goal for this exercise is to enrich TaxiRides with fare information.
34 | *
35 | */
36 | object RidesAndFaresSolution {
37 |
38 | def main(args: Array[String]) {
39 |
40 | // set up streaming execution environment
41 | val env = StreamExecutionEnvironment.getExecutionEnvironment
42 | env.setParallelism(ExerciseBase.parallelism)
43 |
44 | val rides = env
45 | .addSource(rideSourceOrTest(new TaxiRideGenerator()))
46 | .filter { ride => ride.isStart }
47 | .keyBy { ride => ride.rideId }
48 |
49 | val fares = env
50 | .addSource(fareSourceOrTest(new TaxiFareGenerator()))
51 | .keyBy { fare => fare.rideId }
52 |
53 | val processed = rides
54 | .connect(fares)
55 | .flatMap(new EnrichmentFunction)
56 |
57 | printOrTest(processed)
58 |
59 | env.execute("Join Rides with Fares (scala RichCoFlatMap)")
60 | }
61 |
62 | class EnrichmentFunction extends RichCoFlatMapFunction[TaxiRide, TaxiFare, (TaxiRide, TaxiFare)] {
63 | // keyed, managed state
64 | lazy val rideState: ValueState[TaxiRide] = getRuntimeContext.getState(
65 | new ValueStateDescriptor[TaxiRide]("saved ride", classOf[TaxiRide]))
66 | lazy val fareState: ValueState[TaxiFare] = getRuntimeContext.getState(
67 | new ValueStateDescriptor[TaxiFare]("saved fare", classOf[TaxiFare]))
68 |
69 | override def flatMap1(ride: TaxiRide, out: Collector[(TaxiRide, TaxiFare)]): Unit = {
70 | val fare = fareState.value
71 | if (fare != null) {
72 | fareState.clear()
73 | out.collect((ride, fare))
74 | }
75 | else {
76 | rideState.update(ride)
77 | }
78 | }
79 |
80 | override def flatMap2(fare: TaxiFare, out: Collector[(TaxiRide, TaxiFare)]): Unit = {
81 | val ride = rideState.value
82 | if (ride != null) {
83 | rideState.clear()
84 | out.collect((ride, fare))
85 | }
86 | else {
87 | fareState.update(fare)
88 | }
89 | }
90 | }
91 |
92 | }
93 |
--------------------------------------------------------------------------------
/hourly-tips/src/test/java/org/apache/flink/training/exercises/hourlytips/HourlyTipsTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.hourlytips;
20 |
21 | import org.apache.flink.api.java.tuple.Tuple3;
22 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
23 | import org.apache.flink.training.exercises.testing.TaxiRideTestBase;
24 | import org.apache.flink.training.solutions.hourlytips.HourlyTipsSolution;
25 |
26 | import org.junit.Test;
27 |
28 | import java.time.Instant;
29 | import java.util.Arrays;
30 | import java.util.Collections;
31 | import java.util.List;
32 |
33 | import static org.junit.Assert.assertEquals;
34 |
35 | public class HourlyTipsTest extends TaxiRideTestBase> {
36 |
37 | static final Testable JAVA_EXERCISE = () -> HourlyTipsExercise.main(new String[]{});
38 |
39 | @Test
40 | public void testOneDriverOneTip() throws Exception {
41 | TaxiFare one = testFare(1, t(0), 1.0F);
42 |
43 | TestFareSource source = new TestFareSource(
44 | one
45 | );
46 |
47 | Tuple3 max = Tuple3.of(t(60).toEpochMilli(), 1L, 1.0F);
48 |
49 | List> expected = Collections.singletonList(max);
50 |
51 | assertEquals(expected, results(source));
52 | }
53 |
54 | @Test
55 | public void testTipsAreSummedByHour() throws Exception {
56 | TaxiFare oneIn1 = testFare(1, t(0), 1.0F);
57 | TaxiFare fiveIn1 = testFare(1, t(15), 5.0F);
58 | TaxiFare tenIn2 = testFare(1, t(90), 10.0F);
59 |
60 | TestFareSource source = new TestFareSource(
61 | oneIn1,
62 | fiveIn1,
63 | tenIn2
64 | );
65 |
66 | Tuple3 hour1 = Tuple3.of(t(60).toEpochMilli(), 1L, 6.0F);
67 | Tuple3 hour2 = Tuple3.of(t(120).toEpochMilli(), 1L, 10.0F);
68 |
69 | List> expected = Arrays.asList(hour1, hour2);
70 |
71 | assertEquals(expected, results(source));
72 | }
73 |
74 | @Test
75 | public void testMaxAcrossDrivers() throws Exception {
76 | TaxiFare oneFor1In1 = testFare(1, t(0), 1.0F);
77 | TaxiFare fiveFor1In1 = testFare(1, t(15), 5.0F);
78 | TaxiFare tenFor1In2 = testFare(1, t(90), 10.0F);
79 | TaxiFare twentyFor2In2 = testFare(2, t(90), 20.0F);
80 |
81 | TestFareSource source = new TestFareSource(
82 | oneFor1In1,
83 | fiveFor1In1,
84 | tenFor1In2,
85 | twentyFor2In2
86 | );
87 |
88 | Tuple3 hour1 = Tuple3.of(t(60).toEpochMilli(), 1L, 6.0F);
89 | Tuple3 hour2 = Tuple3.of(t(120).toEpochMilli(), 2L, 20.0F);
90 |
91 | List> expected = Arrays.asList(hour1, hour2);
92 |
93 | assertEquals(expected, results(source));
94 | }
95 |
96 | private Instant t(int minutes) {
97 | return Instant.parse("2020-01-01T12:00:00.00Z").plusSeconds(60 * minutes);
98 | }
99 |
100 | private TaxiFare testFare(long driverId, Instant startTime, float tip) {
101 | return new TaxiFare(0, 0, driverId, startTime, "", tip, 0F, 0F);
102 | }
103 |
104 | protected List> results(TestFareSource source) throws Exception {
105 | Testable javaSolution = () -> HourlyTipsSolution.main(new String[]{});
106 | return runApp(source, new TestSink<>(), JAVA_EXERCISE, javaSolution);
107 | }
108 |
109 | }
110 |
--------------------------------------------------------------------------------
/hourly-tips/src/solution/java/org/apache/flink/training/solutions/hourlytips/HourlyTipsSolution.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.solutions.hourlytips;
20 |
21 | import org.apache.flink.api.java.tuple.Tuple3;
22 | import org.apache.flink.streaming.api.TimeCharacteristic;
23 | import org.apache.flink.streaming.api.datastream.DataStream;
24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
25 | import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
26 | import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
27 | import org.apache.flink.streaming.api.windowing.time.Time;
28 | import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
29 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
30 | import org.apache.flink.training.exercises.common.sources.TaxiFareGenerator;
31 | import org.apache.flink.training.exercises.common.utils.ExerciseBase;
32 | import org.apache.flink.util.Collector;
33 |
34 | /**
35 | * Java reference implementation for the "Hourly Tips" exercise of the Flink training in the docs.
36 | *
37 | * The task of the exercise is to first calculate the total tips collected by each driver, hour by hour, and
38 | * then from that stream, find the highest tip total in each hour.
39 | *
40 | */
41 | public class HourlyTipsSolution extends ExerciseBase {
42 |
43 | /**
44 | * Main method.
45 | *
46 | * @throws Exception which occurs during job execution.
47 | */
48 | public static void main(String[] args) throws Exception {
49 |
50 | // set up streaming execution environment
51 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
52 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
53 | env.setParallelism(ExerciseBase.parallelism);
54 |
55 | // start the data generator
56 | DataStream fares = env.addSource(fareSourceOrTest(new TaxiFareGenerator()));
57 |
58 | // compute tips per hour for each driver
59 | DataStream> hourlyTips = fares
60 | .keyBy((TaxiFare fare) -> fare.driverId)
61 | .window(TumblingEventTimeWindows.of(Time.hours(1)))
62 | .process(new AddTips());
63 |
64 | DataStream> hourlyMax = hourlyTips
65 | .windowAll(TumblingEventTimeWindows.of(Time.hours(1)))
66 | .maxBy(2);
67 |
68 | // You should explore how this alternative behaves. In what ways is the same as,
69 | // and different from, the solution above (using a windowAll)?
70 |
71 | // DataStream> hourlyMax = hourlyTips
72 | // .keyBy(t -> t.f0)
73 | // .maxBy(2);
74 |
75 | printOrTest(hourlyMax);
76 |
77 | // execute the transformation pipeline
78 | env.execute("Hourly Tips (java)");
79 | }
80 |
81 | /*
82 | * Wraps the pre-aggregated result into a tuple along with the window's timestamp and key.
83 | */
84 | public static class AddTips extends ProcessWindowFunction<
85 | TaxiFare, Tuple3, Long, TimeWindow> {
86 |
87 | @Override
88 | public void process(Long key, Context context, Iterable fares, Collector> out) {
89 | float sumOfTips = 0F;
90 | for (TaxiFare f : fares) {
91 | sumOfTips += f.tip;
92 | }
93 | out.collect(Tuple3.of(context.window().getEnd(), key, sumOfTips));
94 | }
95 | }
96 | }
97 |
--------------------------------------------------------------------------------
/long-ride-alerts/src/solution/scala/org/apache/flink/training/solutions/longrides/scala/LongRidesSolution.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.solutions.longrides.scala
20 |
21 | import scala.concurrent.duration._
22 | import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
23 | import org.apache.flink.streaming.api.TimeCharacteristic
24 | import org.apache.flink.streaming.api.functions.KeyedProcessFunction
25 | import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, _}
26 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide
27 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator
28 | import org.apache.flink.training.exercises.common.utils.ExerciseBase
29 | import org.apache.flink.training.exercises.common.utils.ExerciseBase._
30 | import org.apache.flink.util.Collector
31 |
32 | /**
33 | * Scala reference implementation for the "Long Ride Alerts" exercise of the Flink training in the docs.
34 | *
35 | * The goal for this exercise is to emit START events for taxi rides that have not been matched
36 | * by an END event during the first 2 hours of the ride.
37 | *
38 | */
39 | object LongRidesSolution {
40 |
41 | def main(args: Array[String]) {
42 |
43 | // set up the execution environment
44 | val env = StreamExecutionEnvironment.getExecutionEnvironment
45 | // operate in Event-time
46 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
47 | env.setParallelism(ExerciseBase.parallelism)
48 |
49 | val rides = env.addSource(rideSourceOrTest(new TaxiRideGenerator()))
50 |
51 | val longRides = rides
52 | .keyBy(_.rideId)
53 | .process(new MatchFunction())
54 |
55 | printOrTest(longRides)
56 |
57 | env.execute("Long Taxi Rides")
58 | }
59 |
60 | class MatchFunction extends KeyedProcessFunction[Long, TaxiRide, TaxiRide] {
61 | lazy val rideState: ValueState[TaxiRide] = getRuntimeContext.getState(
62 | new ValueStateDescriptor[TaxiRide]("ride event", classOf[TaxiRide]))
63 |
64 | override def processElement(ride: TaxiRide,
65 | context: KeyedProcessFunction[Long, TaxiRide, TaxiRide]#Context,
66 | out: Collector[TaxiRide]): Unit = {
67 |
68 | val previousRideEvent = rideState.value()
69 |
70 | if (previousRideEvent == null) {
71 | rideState.update(ride)
72 | if (ride.isStart) {
73 | context.timerService().registerEventTimeTimer(getTimerTime(ride))
74 | }
75 | } else {
76 | if (!ride.isStart) {
77 | // it's an END event, so event saved was the START event and has a timer
78 | // the timer hasn't fired yet, and we can safely kill the timer
79 | context.timerService().deleteEventTimeTimer(getTimerTime(previousRideEvent))
80 | }
81 | // both events have now been seen, we can clear the state
82 | rideState.clear()
83 | }
84 | }
85 |
86 | override def onTimer(timestamp: Long,
87 | ctx: KeyedProcessFunction[Long, TaxiRide, TaxiRide]#OnTimerContext,
88 | out: Collector[TaxiRide]): Unit = {
89 |
90 | // if we get here, we know that the ride started two hours ago, and the END hasn't been processed
91 | out.collect(rideState.value())
92 | rideState.clear()
93 | }
94 |
95 | private def getTimerTime(ride: TaxiRide) = {
96 | ride.startTime.toEpochMilli + 2.hours.toMillis
97 | }
98 | }
99 |
100 | }
101 |
--------------------------------------------------------------------------------
/hourly-tips/README.md:
--------------------------------------------------------------------------------
1 |
19 |
20 | # Lab: Windowed Analytics (Hourly Tips)
21 |
22 | The task of the "Hourly Tips" exercise is to identify, for each hour, the driver earning the most tips. It's easiest to approach this in two steps: first use hour-long windows that compute the total tips for each driver during the hour, and then from that stream of window results, find the driver with the maximum tip total for each hour.
23 |
24 | Please note that the program should operate in event time.
25 |
26 | ### Input Data
27 |
28 | The input data of this exercise is a stream of `TaxiFare` events generated by the [Taxi Fare Stream Generator](../README.md#using-the-taxi-data-streams).
29 |
30 | The `TaxiFareGenerator` annotates the generated `DataStream` with timestamps and watermarks. Hence, there is no need to provide a custom timestamp and watermark assigner in order to correctly use event time.
31 |
32 | ### Expected Output
33 |
34 | The result of this exercise is a data stream of `Tuple3` records, one for each hour. Each hourly record should contain the timestamp at the end of the hour, the driverId of the driver earning the most in tips during that hour, and the actual total of their tips.
35 |
36 | The resulting stream should be printed to standard out.
37 |
38 | ## Getting Started
39 |
40 | > :information_source: Rather than following these links to the sources, you might prefer to open these classes in your IDE.
41 |
42 | ### Exercise Classes
43 |
44 | - Java: [`org.apache.flink.training.exercises.hourlytips.HourlyTipsExercise`](src/main/java/org/apache/flink/training/exercises/hourlytips/HourlyTipsExercise.java)
45 | - Scala: [`org.apache.flink.training.exercises.hourlytips.scala.HourlyTipsExercise`](src/main/scala/org/apache/flink/training/exercises/hourlytips/scala/HourlyTipsExercise.scala)
46 |
47 | ### Tests
48 |
49 | - Java: [`org.apache.flink.training.exercises.hourlytips.HourlyTipsTest`](src/test/java/org/apache/flink/training/exercises/hourlytips/HourlyTipsTest.java)
50 | - Scala: [`org.apache.flink.training.exercises.hourlytips.scala.HourlyTipsTest`](src/test/scala/org/apache/flink/training/exercises/hourlytips/scala/HourlyTipsTest.scala)
51 |
52 | ## Implementation Hints
53 |
54 |
55 | Program Structure
56 |
57 | Note that it is possible to cascade one set of time windows after another, so long as the timeframes are compatible (the second set of windows needs to have a duration that is a multiple of the first set). So you can have a initial set of hour-long windows that is keyed by the `driverId` and use this to create a stream of `(endOfHourTimestamp, driverId, totalTips)`, and then follow this with another hour-long window (this window is not keyed) that finds the record from the first window with the maximum `totalTips`.
58 |
59 |
60 | ## Documentation
61 |
62 | - [Windows](https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/operators/windows.html)
63 | - [See the section on aggregations on windows](https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/operators/#datastream-transformations)
64 |
65 | ## Reference Solutions
66 |
67 | Reference solutions are available at GitHub:
68 |
69 | - Java: [`org.apache.flink.training.solutions.hourlytips.HourlyTipsSolution`](src/solution/java/org/apache/flink/training/solutions/hourlytips/HourlyTipsSolution.java)
70 | - Scala: [`org.apache.flink.training.solutions.hourlytips.scala.HourlyTipsSolution`](src/solution/scala/org/apache/flink/training/solutions/hourlytips/scala/HourlyTipsSolution.scala)
71 |
72 | -----
73 |
74 | [**Lab Discussion: Windowed Analytics (Hourly Tips)**](DISCUSSION.md)
75 |
76 | [**Back to Labs Overview**](../LABS-OVERVIEW.md)
77 |
--------------------------------------------------------------------------------
/long-ride-alerts/src/solution/java/org/apache/flink/training/solutions/longrides/LongRidesSolution.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.solutions.longrides;
20 |
21 | import org.apache.flink.api.common.state.ValueState;
22 | import org.apache.flink.api.common.state.ValueStateDescriptor;
23 | import org.apache.flink.configuration.Configuration;
24 | import org.apache.flink.streaming.api.TimeCharacteristic;
25 | import org.apache.flink.streaming.api.datastream.DataStream;
26 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
27 | import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
28 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
29 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator;
30 | import org.apache.flink.training.exercises.common.utils.ExerciseBase;
31 | import org.apache.flink.util.Collector;
32 |
33 | /**
34 | * Solution to the "Long Ride Alerts" exercise of the Flink training in the docs.
35 | *
36 | * The goal for this exercise is to emit START events for taxi rides that have not been matched
37 | * by an END event during the first 2 hours of the ride.
38 | *
39 | */
40 | public class LongRidesSolution extends ExerciseBase {
41 |
42 | /**
43 | * Main method.
44 | *
45 | * @throws Exception which occurs during job execution.
46 | */
47 | public static void main(String[] args) throws Exception {
48 |
49 | // set up streaming execution environment
50 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
51 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
52 | env.setParallelism(ExerciseBase.parallelism);
53 |
54 | // start the data generator
55 | DataStream rides = env.addSource(rideSourceOrTest(new TaxiRideGenerator()));
56 |
57 | DataStream longRides = rides
58 | .keyBy((TaxiRide ride) -> ride.rideId)
59 | .process(new MatchFunction());
60 |
61 | printOrTest(longRides);
62 |
63 | env.execute("Long Taxi Rides");
64 | }
65 |
66 | private static class MatchFunction extends KeyedProcessFunction {
67 |
68 | private ValueState rideState;
69 |
70 | @Override
71 | public void open(Configuration config) {
72 | ValueStateDescriptor stateDescriptor =
73 | new ValueStateDescriptor<>("ride event", TaxiRide.class);
74 | rideState = getRuntimeContext().getState(stateDescriptor);
75 | }
76 |
77 | @Override
78 | public void processElement(TaxiRide ride, Context context, Collector out) throws Exception {
79 | TaxiRide previousRideEvent = rideState.value();
80 |
81 | if (previousRideEvent == null) {
82 | rideState.update(ride);
83 | if (ride.isStart) {
84 | context.timerService().registerEventTimeTimer(getTimerTime(ride));
85 | }
86 | } else {
87 | if (!ride.isStart) {
88 | // it's an END event, so event saved was the START event and has a timer
89 | // the timer hasn't fired yet, and we can safely kill the timer
90 | context.timerService().deleteEventTimeTimer(getTimerTime(previousRideEvent));
91 | }
92 | // both events have now been seen, we can clear the state
93 | rideState.clear();
94 | }
95 | }
96 |
97 | @Override
98 | public void onTimer(long timestamp, OnTimerContext context, Collector out) throws Exception {
99 |
100 | // if we get here, we know that the ride started two hours ago, and the END hasn't been processed
101 | out.collect(rideState.value());
102 | rideState.clear();
103 | }
104 |
105 | private long getTimerTime(TaxiRide ride) {
106 | return ride.startTime.plusSeconds(120 * 60).toEpochMilli();
107 | }
108 | }
109 |
110 | }
111 |
--------------------------------------------------------------------------------
/rides-and-fares/README.md:
--------------------------------------------------------------------------------
1 |
19 |
20 | # Lab: Stateful Enrichment (Rides and Fares)
21 |
22 | The goal of this exercise is to join together the `TaxiRide` and `TaxiFare` records for each ride.
23 |
24 | For each distinct `rideId`, there are exactly three events:
25 |
26 | 1. a `TaxiRide` START event
27 | 1. a `TaxiRide` END event
28 | 1. a `TaxiFare` event (whose timestamp happens to match the start time)
29 |
30 | The result should be a `DataStream>`, with one record for each distinct `rideId`. Each tuple should pair the `TaxiRide` START event for some `rideId` with its matching `TaxiFare`.
31 |
32 | ### Input Data
33 |
34 | For this exercise you will work with two data streams, one with `TaxiRide` events generated by a `TaxiRideSource` and the other with `TaxiFare` events generated by a `TaxiFareSource`. See [Using the Taxi Data Streams](../README.md#using-the-taxi-data-streams) for information on how to download the data and how to work with these stream generators.
35 |
36 | ### Expected Output
37 |
38 | The result of this exercise is a data stream of `Tuple2` records, one for each distinct `rideId`. The exercise is setup to ignore the END events, and you should join the event for the START of each ride with its corresponding fare event.
39 |
40 | The resulting stream is printed to standard out.
41 |
42 | ## Getting Started
43 |
44 | > :information_source: Rather than following these links to the sources, you might prefer to open these classes in your IDE.
45 |
46 | ### Exercise Classes
47 |
48 | - Java: [`org.apache.flink.training.exercises.ridesandfares.RidesAndFaresExercise`](src/main/java/org/apache/flink/training/exercises/ridesandfares/RidesAndFaresExercise.java)
49 | - Scala: [`org.apache.flink.training.exercises.ridesandfares.scala.RidesAndFaresExercise`](src/main/scala/org/apache/flink/training/exercises/ridesandfares/scala/RidesAndFaresExercise.scala)
50 |
51 | ### Tests
52 |
53 | - Java: [`org.apache.flink.training.exercises.ridesandfares.RidesAndFaresTest`](src/test/java/org/apache/flink/training/exercises/ridesandfares/RidesAndFaresTest.java)
54 | - Scala: [`org.apache.flink.training.exercises.ridesandfares.scala.RidesAndFaresTest`](src/test/scala/org/apache/flink/training/exercises/ridesandfares/scala/RidesAndFaresTest.scala)
55 |
56 | ## Implementation Hints
57 |
58 |
59 | Program Structure
60 |
61 | You can use a `RichCoFlatMap` to implement this join operation. Note that you have no control over the order of arrival of the ride and fare records for each rideId, so you'll need to be prepared to store either piece of information until the matching info arrives, at which point you can emit a `Tuple2` joining the two records together.
62 |
63 |
64 |
65 | Working with State
66 |
67 | You should be using Flink's managed, keyed state to buffer the data that is being held until the matching event arrives. And be sure to clear the state once it is no longer needed.
68 |
69 |
70 | ## Discussion
71 |
72 | For the purposes of this exercise it's okay to assume that the START and fare events are perfectly paired. But in a real-world application you should worry about the fact that whenever an event is missing, the other event for the same `rideId` will be held in state forever. In a [later lab](../long-ride-alerts) we'll look at the `ProcessFunction` and Timers which may also help the situation here.
73 |
74 | ## Documentation
75 |
76 | - [Working with State](https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/state/index.html)
77 |
78 | ## Reference Solutions
79 |
80 | Reference solutions are available in this project:
81 |
82 | - Java: [`org.apache.flink.training.solutions.ridesandfares.RidesAndFaresSolution`](src/solution/java/org/apache/flink/training/solutions/ridesandfares/RidesAndFaresSolution.java)
83 | - Scala: [`org.apache.flink.training.solutions.ridesandfares.scala.RidesAndFaresSolution`](src/solution/scala/org/apache/flink/training/solutions/ridesandfares/scala/RidesAndFaresSolution.scala)
84 |
85 | -----
86 |
87 | [**Back to Labs Overview**](../LABS-OVERVIEW.md)
88 |
--------------------------------------------------------------------------------
/ride-cleansing/README.md:
--------------------------------------------------------------------------------
1 |
19 |
20 | # Lab: Filtering a Stream (Ride Cleansing)
21 |
22 | If you haven't already done so, you'll need to first [setup your Flink development environment](../README.md). See [How to do the Labs](../README.md#how-to-do-the-labs) for an overall introduction to these exercises.
23 |
24 | The task of the "Taxi Ride Cleansing" exercise is to cleanse a stream of TaxiRide events by removing events that start or end outside of New York City.
25 |
26 | The `GeoUtils` utility class provides a static method `isInNYC(float lon, float lat)` to check if a location is within the NYC area.
27 |
28 | ### Input Data
29 |
30 | This series of exercises is based a stream of `TaxiRide` events, as described in [Using the Taxi Data Streams](../README.md#using-the-taxi-data-streams).
31 |
32 | ### Expected Output
33 |
34 | The result of the exercise should be a `DataStream` that only contains events of taxi rides which both start and end in the New York City area as defined by `GeoUtils.isInNYC()`.
35 |
36 | The resulting stream should be printed to standard out.
37 |
38 | ## Getting Started
39 |
40 | > :information_source: Rather than following the links to the sources in this section, you'll do better to find these classes in the flink-training project in your IDE.
41 | > Both IntelliJ and Eclipse have ways to make it easy to search for and navigate to classes and files. For IntelliJ, see [the help on searching](https://www.jetbrains.com/help/idea/searching-everywhere.html), or simply press the Shift key twice and then continue typing something like `RideCleansing` and then select from the choices that popup.
42 |
43 | ### Exercise Classes
44 |
45 | This exercise uses these classes:
46 |
47 | - Java: [`org.apache.flink.training.exercises.ridecleansing.RideCleansingExercise`](src/main/java/org/apache/flink/training/exercises/ridecleansing/RideCleansingExercise.java)
48 | - Scala: [`org.apache.flink.training.exercises.ridecleansing.scala.RideCleansingExercise`](src/main/scala/org/apache/flink/training/exercises/ridecleansing/scala/RideCleansingExercise.scala)
49 |
50 | ### Tests
51 |
52 | You will find the test for this exercise in
53 |
54 | - Java: [`org.apache.flink.training.exercises.ridecleansing.RideCleansingTest`](src/test/java/org/apache/flink/training/exercises/ridecleansing/RideCleansingTest.java)
55 | - Scala: [`org.apache.flink.training.exercises.ridecleansing.scala.RideCleansingTest`](src/test/scala/org/apache/flink/training/exercises/ridecleansing/scala/RideCleansingTest.scala)
56 |
57 | Like most of these exercises, at some point the `RideCleansingExercise` class throws an exception
58 |
59 | ```java
60 | throw new MissingSolutionException();
61 | ```
62 |
63 | Once you remove this line, the test will fail until you provide a working solution. You might want to first try something clearly broken, such as
64 |
65 | ```java
66 | return false;
67 | ```
68 |
69 | in order to verify that the test does indeed fail when you make a mistake, and then work on implementing a proper solution.
70 |
71 | ## Implementation Hints
72 |
73 |
74 | Filtering Events
75 |
76 | Flink's DataStream API features a `DataStream.filter(FilterFunction)` transformation to filter events from a data stream. The `GeoUtils.isInNYC()` function can be called within a `FilterFunction` to check if a location is in the New York City area. Your filter function should check both the starting and ending locations of each ride.
77 |
78 |
79 | ## Documentation
80 |
81 | - [DataStream API](https://ci.apache.org/projects/flink/flink-docs-stable/dev/datastream_api.html)
82 | - [Flink JavaDocs](https://ci.apache.org/projects/flink/flink-docs-stable/api/java/)
83 |
84 | ## Reference Solutions
85 |
86 | Reference solutions are available in this project:
87 |
88 | - Java: [`org.apache.flink.training.solutions.ridecleansing.RideCleansingSolution`](src/solution/java/org/apache/flink/training/solutions/ridecleansing/RideCleansingSolution.java)
89 | - Scala: [`org.apache.flink.training.solutions.ridecleansing.scala.RideCleansingSolution`](src/solution/scala/org/apache/flink/training/solutions/ridecleansing/scala/RideCleansingSolution.scala)
90 |
91 | -----
92 |
93 | [**Back to Labs Overview**](../LABS-OVERVIEW.md)
94 |
--------------------------------------------------------------------------------
/long-ride-alerts/src/test/java/org/apache/flink/training/exercises/longrides/LongRidesTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.longrides;
20 |
21 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
22 | import org.apache.flink.training.exercises.testing.TaxiRideTestBase;
23 | import org.apache.flink.training.solutions.longrides.LongRidesSolution;
24 |
25 | import org.junit.Test;
26 |
27 | import java.time.Instant;
28 | import java.util.Collections;
29 | import java.util.List;
30 |
31 | import static org.junit.Assert.assertEquals;
32 |
33 | public class LongRidesTest extends TaxiRideTestBase {
34 |
35 | static final Testable JAVA_EXERCISE = () -> LongRidesExercise.main(new String[]{});
36 |
37 | private static final Instant BEGINNING = Instant.parse("2020-01-01T12:00:00.00Z");
38 |
39 | @Test
40 | public void shortRide() throws Exception {
41 | Instant oneMinLater = BEGINNING.plusSeconds(60);
42 | TaxiRide rideStarted = startRide(1, BEGINNING);
43 | TaxiRide endedOneMinLater = endRide(rideStarted, oneMinLater);
44 | Long markOneMinLater = oneMinLater.toEpochMilli();
45 |
46 | TestRideSource source = new TestRideSource(rideStarted, endedOneMinLater, markOneMinLater);
47 | assert(results(source).isEmpty());
48 | }
49 |
50 | @Test
51 | public void outOfOrder() throws Exception {
52 | Instant oneMinLater = BEGINNING.plusSeconds(60);
53 | TaxiRide rideStarted = startRide(1, BEGINNING);
54 | TaxiRide endedOneMinLater = endRide(rideStarted, oneMinLater);
55 | Long markOneMinLater = oneMinLater.toEpochMilli();
56 |
57 | TestRideSource source = new TestRideSource(endedOneMinLater, rideStarted, markOneMinLater);
58 | assert(results(source).isEmpty());
59 | }
60 |
61 | @Test
62 | public void noStartShort() throws Exception {
63 | Instant oneMinLater = BEGINNING.plusSeconds(60);
64 | TaxiRide rideStarted = startRide(1, BEGINNING);
65 | TaxiRide endedOneMinLater = endRide(rideStarted, oneMinLater);
66 | Long markOneMinLater = oneMinLater.toEpochMilli();
67 |
68 | TestRideSource source = new TestRideSource(endedOneMinLater, markOneMinLater);
69 | assert(results(source).isEmpty());
70 | }
71 |
72 | @Test
73 | public void noEnd() throws Exception {
74 | TaxiRide rideStarted = startRide(1, BEGINNING);
75 | Long markThreeHoursLater = BEGINNING.plusSeconds(180 * 60).toEpochMilli();
76 |
77 | TestRideSource source = new TestRideSource(rideStarted, markThreeHoursLater);
78 | assertEquals(Collections.singletonList(rideStarted), results(source));
79 | }
80 |
81 | @Test
82 | public void longRide() throws Exception {
83 | TaxiRide rideStarted = startRide(1, BEGINNING);
84 | Long mark2HoursLater = BEGINNING.plusSeconds(120 * 60).toEpochMilli();
85 | TaxiRide rideEnded3HoursLater = endRide(rideStarted, BEGINNING.plusSeconds(180 * 60));
86 |
87 | TestRideSource source = new TestRideSource(rideStarted, mark2HoursLater, rideEnded3HoursLater);
88 | assertEquals(Collections.singletonList(rideStarted), results(source));
89 | }
90 |
91 | @Test
92 | public void startIsDelayedMoreThanTwoHours() throws Exception {
93 | TaxiRide rideStarted = startRide(1, BEGINNING);
94 | TaxiRide rideEndedAfter1Hour = endRide(rideStarted, BEGINNING.plusSeconds(60 * 60));
95 | Long mark2HoursAfterEnd = BEGINNING.plusSeconds(180 * 60).toEpochMilli();
96 |
97 | TestRideSource source = new TestRideSource(rideEndedAfter1Hour, mark2HoursAfterEnd, rideStarted);
98 | assert(results(source).isEmpty());
99 | }
100 |
101 | private TaxiRide testRide(long rideId, Boolean isStart, Instant startTime, Instant endTime) {
102 | return new TaxiRide(rideId, isStart, startTime, endTime, -73.9947F, 40.750626F, -73.9947F, 40.750626F, (short) 1, 0, 0);
103 | }
104 |
105 | private TaxiRide startRide(long rideId, Instant startTime) {
106 | return testRide(rideId, true, startTime, Instant.EPOCH);
107 | }
108 |
109 | private TaxiRide endRide(TaxiRide started, Instant endTime) {
110 | return testRide(started.rideId, false, started.startTime, endTime);
111 | }
112 |
113 | protected List results(TestRideSource source) throws Exception {
114 | Testable javaSolution = () -> LongRidesSolution.main(new String[]{});
115 | return runApp(source, new TestSink<>(), JAVA_EXERCISE, javaSolution);
116 | }
117 |
118 | }
119 |
--------------------------------------------------------------------------------
/rides-and-fares/src/solution/java/org/apache/flink/training/solutions/ridesandfares/RidesAndFaresSolution.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.solutions.ridesandfares;
20 |
21 | import org.apache.flink.api.common.state.ValueState;
22 | import org.apache.flink.api.common.state.ValueStateDescriptor;
23 | import org.apache.flink.api.java.tuple.Tuple2;
24 | import org.apache.flink.configuration.Configuration;
25 | import org.apache.flink.streaming.api.datastream.DataStream;
26 | import org.apache.flink.streaming.api.environment.CheckpointConfig;
27 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
28 | import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
29 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
30 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
31 | import org.apache.flink.training.exercises.common.sources.TaxiFareGenerator;
32 | import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator;
33 | import org.apache.flink.training.exercises.common.utils.ExerciseBase;
34 | import org.apache.flink.util.Collector;
35 |
36 | /**
37 | * Java reference implementation for the "Stateful Enrichment" exercise of the Flink training in the docs.
38 | *
39 | * The goal for this exercise is to enrich TaxiRides with fare information.
40 | *
41 | */
42 | public class RidesAndFaresSolution extends ExerciseBase {
43 |
44 | /**
45 | * Main method.
46 | *
47 | * @throws Exception which occurs during job execution.
48 | */
49 | public static void main(String[] args) throws Exception {
50 |
51 | // Set up streaming execution environment, including Web UI and REST endpoint.
52 | // Checkpointing isn't needed for the RidesAndFares exercise; this setup is for
53 | // using the State Processor API.
54 |
55 | Configuration conf = new Configuration();
56 | conf.setString("state.backend", "filesystem");
57 | conf.setString("state.savepoints.dir", "file:///tmp/savepoints");
58 | conf.setString("state.checkpoints.dir", "file:///tmp/checkpoints");
59 | StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironmentWithWebUI(conf);
60 | env.setParallelism(ExerciseBase.parallelism);
61 |
62 | env.enableCheckpointing(10000L);
63 | CheckpointConfig config = env.getCheckpointConfig();
64 | config.enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
65 |
66 | DataStream rides = env
67 | .addSource(rideSourceOrTest(new TaxiRideGenerator()))
68 | .filter((TaxiRide ride) -> ride.isStart)
69 | .keyBy((TaxiRide ride) -> ride.rideId);
70 |
71 | DataStream fares = env
72 | .addSource(fareSourceOrTest(new TaxiFareGenerator()))
73 | .keyBy((TaxiFare fare) -> fare.rideId);
74 |
75 | // Set a UID on the stateful flatmap operator so we can read its state using the State Processor API.
76 | DataStream> enrichedRides = rides
77 | .connect(fares)
78 | .flatMap(new EnrichmentFunction())
79 | .uid("enrichment");
80 |
81 | printOrTest(enrichedRides);
82 |
83 | env.execute("Join Rides with Fares (java RichCoFlatMap)");
84 | }
85 |
86 | public static class EnrichmentFunction extends RichCoFlatMapFunction> {
87 | // keyed, managed state
88 | private ValueState rideState;
89 | private ValueState fareState;
90 |
91 | @Override
92 | public void open(Configuration config) {
93 | rideState = getRuntimeContext().getState(new ValueStateDescriptor<>("saved ride", TaxiRide.class));
94 | fareState = getRuntimeContext().getState(new ValueStateDescriptor<>("saved fare", TaxiFare.class));
95 | }
96 |
97 | @Override
98 | public void flatMap1(TaxiRide ride, Collector> out) throws Exception {
99 | TaxiFare fare = fareState.value();
100 | if (fare != null) {
101 | fareState.clear();
102 | out.collect(Tuple2.of(ride, fare));
103 | } else {
104 | rideState.update(ride);
105 | }
106 | }
107 |
108 | @Override
109 | public void flatMap2(TaxiFare fare, Collector> out) throws Exception {
110 | TaxiRide ride = rideState.value();
111 | if (ride != null) {
112 | rideState.clear();
113 | out.collect(Tuple2.of(ride, fare));
114 | } else {
115 | fareState.update(fare);
116 | }
117 | }
118 | }
119 | }
120 |
--------------------------------------------------------------------------------
/long-ride-alerts/README.md:
--------------------------------------------------------------------------------
1 |
19 |
20 | # Lab: `ProcessFunction` and Timers (Long Ride Alerts)
21 |
22 | The goal of the "Long Ride Alerts" exercise is to provide a real-time warning whenever a taxi ride
23 | started two hours ago, and is still ongoing.
24 |
25 | This should be done using the event time timestamps and watermarks that are provided in the data stream.
26 |
27 | The stream is out-of-order, and it is possible that the END event for a ride will be processed before
28 | its START event. But in such cases, we never care to create an alert, since we do know that the ride
29 | has ended.
30 |
31 | ### Input Data
32 |
33 | The input data of this exercise is a `DataStream` of taxi ride events.
34 |
35 | ### Expected Output
36 |
37 | The goal of this exercise is _not_ to find all rides that lasted for more than two hours, but rather
38 | to create an alert _in real time_ at the moment it becomes known that a ride has been going on for
39 | more than two hours.
40 |
41 | The result of the exercise should be a `DataStream` that only contains START events of
42 | taxi rides that started two hours earlier, and whose END event hasn't yet arrived.
43 |
44 | The resulting stream should be printed to standard out.
45 |
46 | ## Getting Started
47 |
48 | > :information_source: Rather than following these links to the sources, you might prefer to open these classes in your IDE.
49 |
50 | ### Exercise Classes
51 |
52 | - Java: [`org.apache.flink.training.exercises.longrides.LongRidesExercise`](src/main/java/org/apache/flink/training/exercises/longrides/LongRidesExercise.java)
53 | - Scala: [`org.apache.flink.training.exercises.longrides.scala.LongRidesExercise`](src/main/scala/org/apache/flink/training/exercises/longrides/scala/LongRidesExercise.scala)
54 |
55 | ### Tests
56 |
57 | - Java: [`org.apache.flink.training.exercises.longrides.LongRidesTest`](src/test/java/org/apache/flink/training/exercises/longrides/LongRidesTest.java)
58 | - Scala: [`org.apache.flink.training.exercises.longrides.scala.LongRidesTest`](src/test/scala/org/apache/flink/training/exercises/longrides/scala/LongRidesTest.scala)
59 |
60 | ## Implementation Hints
61 |
62 |
63 | Overall approach
64 |
65 | This exercise revolves around using a `ProcessFunction` to manage some keyed state and event time timers,
66 | and doing so in a way that works even when the END event for a given `rideId` arrives before the START (which can happen).
67 | The challenge is figuring out what state to keep, and when to set and clear that state.
68 | You will want to use event time timers that fire two hours after an incoming START event, and in the `onTimer()` method,
69 | collect START events to the output only if a matching END event hasn't yet arrived.
70 |
71 |
72 |
73 | State and timers
74 |
75 | There are many possible solutions for this exercise, but in general it is enough to keep one
76 | `TaxiRide` in state (one `TaxiRide` for each key, or `rideId`). The approach used in the reference solution is to
77 | store whichever event arrives first (the START or the END), and if it's a START event,
78 | create a timer for two hours later. If and when the other event (for the same `rideId`) arrives,
79 | carefully clean things up.
80 |
81 | It is possible to arrange this so that if `onTimer()` is called, you are guaranteed that
82 | an alert (i.e., the ride kept in state) should be emitted. Writing the code this way conveniently
83 | puts all of the complex business logic together in one place (in the `processElement()` method).
84 |
85 |
86 | ## Documentation
87 |
88 | - [ProcessFunction](https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/operators/process_function.html)
89 | - [Working with State](https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/state/index.html)
90 |
91 | ## Reference Solutions
92 |
93 | Reference solutions are available at GitHub:
94 |
95 | - Java API: [`org.apache.flink.training.solutions.longrides.LongRidesSolution`](src/solution/java/org/apache/flink/training/solutions/longrides/LongRidesSolution.java)
96 | - Scala API: [`org.apache.flink.training.solutions.longrides.scala.LongRidesSolution`](src/solution/scala/org/apache/flink/training/solutions/longrides/scala/LongRidesSolution.scala)
97 |
98 | -----
99 |
100 | [**Lab Discussion: `ProcessFunction` and Timers (Long Ride Alerts)**](DISCUSSION.md)
101 |
102 | [**Back to Labs Overview**](../LABS-OVERVIEW.md)
103 |
--------------------------------------------------------------------------------
/common/src/test/java/org/apache/flink/training/exercises/testing/TaxiRideTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.testing;
20 |
21 | import org.apache.flink.api.common.typeinfo.TypeInformation;
22 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
23 | import org.apache.flink.streaming.api.functions.sink.SinkFunction;
24 | import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
25 | import org.apache.flink.training.exercises.common.datatypes.TaxiRide;
26 | import org.apache.flink.training.exercises.common.utils.ExerciseBase;
27 | import org.apache.flink.training.exercises.common.utils.MissingSolutionException;
28 |
29 | import java.util.ArrayList;
30 | import java.util.List;
31 |
32 | public abstract class TaxiRideTestBase {
33 | public static class TestRideSource extends TestSource implements ResultTypeQueryable {
34 | public TestRideSource(Object ... eventsOrWatermarks) {
35 | this.testStream = eventsOrWatermarks;
36 | }
37 |
38 | @Override
39 | long getTimestamp(TaxiRide ride) {
40 | return ride.getEventTime();
41 | }
42 |
43 | @Override
44 | public TypeInformation getProducedType() {
45 | return TypeInformation.of(TaxiRide.class);
46 | }
47 | }
48 |
49 | public static class TestFareSource extends TestSource implements ResultTypeQueryable {
50 | public TestFareSource(Object ... eventsOrWatermarks) {
51 | this.testStream = eventsOrWatermarks;
52 | }
53 |
54 | @Override
55 | long getTimestamp(TaxiFare fare) {
56 | return fare.getEventTime();
57 | }
58 |
59 | @Override
60 | public TypeInformation getProducedType() {
61 | return TypeInformation.of(TaxiFare.class);
62 | }
63 | }
64 |
65 | public static class TestStringSource extends TestSource implements ResultTypeQueryable {
66 | public TestStringSource(Object ... eventsOrWatermarks) {
67 | this.testStream = eventsOrWatermarks;
68 | }
69 |
70 | @Override
71 | long getTimestamp(String s) {
72 | return 0L;
73 | }
74 |
75 | @Override
76 | public TypeInformation getProducedType() {
77 | return TypeInformation.of(String.class);
78 | }
79 | }
80 |
81 | public static class TestSink implements SinkFunction {
82 |
83 | // must be static
84 | public static final List VALUES = new ArrayList<>();
85 |
86 | @Override
87 | public void invoke(OUT value, Context context) {
88 | VALUES.add(value);
89 | }
90 | }
91 |
92 | public interface Testable {
93 | void main() throws Exception;
94 | }
95 |
96 | protected List runApp(TestRideSource source, TestSink sink, Testable exercise, Testable solution) throws Exception {
97 | ExerciseBase.rides = source;
98 |
99 | return execute(sink, exercise, solution);
100 | }
101 |
102 | protected List runApp(TestFareSource source, TestSink sink, Testable exercise, Testable solution) throws Exception {
103 | ExerciseBase.fares = source;
104 |
105 | return execute(sink, exercise, solution);
106 | }
107 |
108 | protected List runApp(TestRideSource rides, TestFareSource fares, TestSink sink, Testable exercise, Testable solution) throws Exception {
109 | ExerciseBase.rides = rides;
110 | ExerciseBase.fares = fares;
111 |
112 | return execute(sink, exercise, solution);
113 | }
114 |
115 | protected List runApp(TestRideSource rides, TestSink sink, Testable solution) throws Exception {
116 | ExerciseBase.rides = rides;
117 |
118 | return execute(sink, solution);
119 | }
120 |
121 | protected List runApp(TestRideSource rides, TestStringSource strings, TestSink sink, Testable exercise, Testable solution) throws Exception {
122 | ExerciseBase.rides = rides;
123 | ExerciseBase.strings = strings;
124 |
125 | return execute(sink, exercise, solution);
126 | }
127 |
128 | private List execute(TestSink sink, Testable exercise, Testable solution) throws Exception {
129 | sink.VALUES.clear();
130 |
131 | ExerciseBase.out = sink;
132 | ExerciseBase.parallelism = 1;
133 |
134 | try {
135 | exercise.main();
136 | } catch (Exception e) {
137 | if (ultimateCauseIsMissingSolution(e)) {
138 | sink.VALUES.clear();
139 | solution.main();
140 | } else {
141 | throw e;
142 | }
143 | }
144 |
145 | return sink.VALUES;
146 | }
147 |
148 | private List execute(TestSink sink, Testable solution) throws Exception {
149 | sink.VALUES.clear();
150 |
151 | ExerciseBase.out = sink;
152 | ExerciseBase.parallelism = 1;
153 |
154 | solution.main();
155 |
156 | return sink.VALUES;
157 | }
158 |
159 | private boolean ultimateCauseIsMissingSolution(Throwable e) {
160 | if (e instanceof MissingSolutionException) {
161 | return true;
162 | } else if (e.getCause() != null) {
163 | return ultimateCauseIsMissingSolution(e.getCause());
164 | } else {
165 | return false;
166 | }
167 | }
168 | }
169 |
--------------------------------------------------------------------------------
/common/src/main/java/org/apache/flink/training/exercises/common/datatypes/TaxiRide.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.common.datatypes;
20 |
21 | import org.apache.flink.training.exercises.common.utils.DataGenerator;
22 | import org.apache.flink.training.exercises.common.utils.GeoUtils;
23 |
24 | import javax.annotation.Nullable;
25 |
26 | import java.io.Serializable;
27 | import java.time.Instant;
28 |
29 | /**
30 | * A TaxiRide is a taxi ride event. There are two types of events, a taxi ride start event and a
31 | * taxi ride end event. The isStart flag specifies the type of the event.
32 | *
33 | * A TaxiRide consists of
34 | * - the rideId of the event which is identical for start and end record
35 | * - the type of the event (start or end)
36 | * - the time of the event
37 | * - the longitude of the start location
38 | * - the latitude of the start location
39 | * - the longitude of the end location
40 | * - the latitude of the end location
41 | * - the passengerCnt of the ride
42 | * - the taxiId
43 | * - the driverId
44 | */
45 | public class TaxiRide implements Comparable, Serializable {
46 |
47 | /**
48 | * Creates a new TaxiRide with now as start and end time.
49 | */
50 | public TaxiRide() {
51 | this.startTime = Instant.now();
52 | this.endTime = Instant.now();
53 | }
54 |
55 | /**
56 | * Invents a TaxiRide.
57 | */
58 | public TaxiRide(long rideId, boolean isStart) {
59 | DataGenerator g = new DataGenerator(rideId);
60 |
61 | this.rideId = rideId;
62 | this.isStart = isStart;
63 | this.startTime = g.startTime();
64 | this.endTime = isStart ? Instant.ofEpochMilli(0) : g.endTime();
65 | this.startLon = g.startLon();
66 | this.startLat = g.startLat();
67 | this.endLon = g.endLon();
68 | this.endLat = g.endLat();
69 | this.passengerCnt = g.passengerCnt();
70 | this.taxiId = g.taxiId();
71 | this.driverId = g.driverId();
72 | }
73 |
74 | /**
75 | * Creates a TaxiRide with the given parameters.
76 | */
77 | public TaxiRide(long rideId, boolean isStart, Instant startTime, Instant endTime,
78 | float startLon, float startLat, float endLon, float endLat,
79 | short passengerCnt, long taxiId, long driverId) {
80 | this.rideId = rideId;
81 | this.isStart = isStart;
82 | this.startTime = startTime;
83 | this.endTime = endTime;
84 | this.startLon = startLon;
85 | this.startLat = startLat;
86 | this.endLon = endLon;
87 | this.endLat = endLat;
88 | this.passengerCnt = passengerCnt;
89 | this.taxiId = taxiId;
90 | this.driverId = driverId;
91 | }
92 |
93 | public long rideId;
94 | public boolean isStart;
95 | public Instant startTime;
96 | public Instant endTime;
97 | public float startLon;
98 | public float startLat;
99 | public float endLon;
100 | public float endLat;
101 | public short passengerCnt;
102 | public long taxiId;
103 | public long driverId;
104 |
105 | @Override
106 | public String toString() {
107 |
108 | return rideId + "," +
109 | (isStart ? "START" : "END") + "," +
110 | startTime.toString() + "," +
111 | endTime.toString() + "," +
112 | startLon + "," +
113 | startLat + "," +
114 | endLon + "," +
115 | endLat + "," +
116 | passengerCnt + "," +
117 | taxiId + "," +
118 | driverId;
119 | }
120 |
121 | /**
122 | * Compares this TaxiRide with the given one.
123 | *
124 | *
125 | * - sort by timestamp,
126 | * - putting START events before END events if they have the same timestamp
127 | *
128 | */
129 | public int compareTo(@Nullable TaxiRide other) {
130 | if (other == null) {
131 | return 1;
132 | }
133 | int compareTimes = Long.compare(this.getEventTime(), other.getEventTime());
134 | if (compareTimes == 0) {
135 | if (this.isStart == other.isStart) {
136 | return 0;
137 | }
138 | else {
139 | if (this.isStart) {
140 | return -1;
141 | }
142 | else {
143 | return 1;
144 | }
145 | }
146 | }
147 | else {
148 | return compareTimes;
149 | }
150 | }
151 |
152 | @Override
153 | public boolean equals(Object other) {
154 | return other instanceof TaxiRide &&
155 | this.rideId == ((TaxiRide) other).rideId;
156 | }
157 |
158 | @Override
159 | public int hashCode() {
160 | return (int) this.rideId;
161 | }
162 |
163 | /**
164 | * Gets the ride's time stamp (start or end time depending on {@link #isStart}).
165 | */
166 | public long getEventTime() {
167 | if (isStart) {
168 | return startTime.toEpochMilli();
169 | }
170 | else {
171 | return endTime.toEpochMilli();
172 | }
173 | }
174 |
175 | /**
176 | * Gets the distance from the ride location to the given one.
177 | */
178 | public double getEuclideanDistance(double longitude, double latitude) {
179 | if (this.isStart) {
180 | return GeoUtils.getEuclideanDistance((float) longitude, (float) latitude, this.startLon, this.startLat);
181 | } else {
182 | return GeoUtils.getEuclideanDistance((float) longitude, (float) latitude, this.endLon, this.endLat);
183 | }
184 | }
185 | }
186 |
--------------------------------------------------------------------------------
/hourly-tips/DISCUSSION.md:
--------------------------------------------------------------------------------
1 |
19 |
20 | # Lab Discussion: Windowed Analytics (Hourly Tips)
21 |
22 | (Discussion of [Lab: Windowed Analytics (Hourly Tips)](./))
23 |
24 | The Java and Scala reference solutions illustrate two different approaches, though they have a lot of similarities. Both first compute the sum of the tips for every hour for each driver. In [`HourlyTipsSolution.java`](src/solution/java/org/apache/flink/training/solutions/hourlytips/HourlyTipsSolution.java) that looks like this,
25 |
26 | ```java
27 | DataStream> hourlyTips = fares
28 | .keyBy((TaxiFare fare) -> fare.driverId)
29 | .window(TumblingEventTimeWindows.of(Time.hours(1)))
30 | .process(new AddTips());
31 | ```
32 |
33 | where a `ProcessWindowFunction` does all the heavy lifting:
34 |
35 | ```java
36 | public static class AddTips extends ProcessWindowFunction<
37 | TaxiFare, Tuple3, Long, TimeWindow> {
38 | @Override
39 | public void process(Long key, Context context, Iterable fares, Collector> out) throws Exception {
40 | Float sumOfTips = 0F;
41 | for (TaxiFare f : fares) {
42 | sumOfTips += f.tip;
43 | }
44 | out.collect(Tuple3.of(context.window().getEnd(), key, sumOfTips));
45 | }
46 | }
47 | ```
48 |
49 | This is straightforward, but has the drawback that it is buffering all of the `TaxiFare` objects in the windows until the windows are triggered, which is less efficient than computing the sum of the tips incrementally, using a `reduce` or `agggregate` function.
50 |
51 | The [Scala solution](src/solution/scala/org/apache/flink/training/solutions/hourlytips/scala/HourlyTipsSolution.scala) uses a `reduce` function
52 |
53 | ```scala
54 | val hourlyTips = fares
55 | .map((f: TaxiFare) => (f.driverId, f.tip))
56 | .keyBy(_._1)
57 | .window(TumblingEventTimeWindows.of(Time.hours(1)))
58 | .reduce(
59 | (f1: (Long, Float), f2: (Long, Float)) => { (f1._1, f1._2 + f2._2) },
60 | new WrapWithWindowInfo())
61 | ```
62 |
63 | along with this `ProcessWindowFunction`
64 |
65 | ```scala
66 | class WrapWithWindowInfo() extends ProcessWindowFunction[(Long, Float), (Long, Long, Float), Long, TimeWindow] {
67 | override def process(key: Long, context: Context, elements: Iterable[(Long, Float)], out: Collector[(Long, Long, Float)]): Unit = {
68 | val sumOfTips = elements.iterator.next()._2
69 | out.collect((context.window.getEnd(), key, sumOfTips))
70 | }
71 | }
72 | ```
73 |
74 | to compute `hourlyTips`.
75 |
76 | Having computed `hourlyTips`, it is a good idea to take a look at what this stream looks like. `hourlyTips.print()` yields something like this,
77 |
78 | ```
79 | 2> (1577883600000,2013000185,33.0)
80 | 4> (1577883600000,2013000108,14.0)
81 | 3> (1577883600000,2013000087,14.0)
82 | 1> (1577883600000,2013000036,23.0)
83 | 4> (1577883600000,2013000072,13.0)
84 | 2> (1577883600000,2013000041,28.0)
85 | 3> (1577883600000,2013000123,33.0)
86 | 4> (1577883600000,2013000188,18.0)
87 | 1> (1577883600000,2013000098,23.0)
88 | 2> (1577883600000,2013000047,13.0)
89 | ...
90 | ```
91 |
92 | or in other words, lots of tuples for each hour that show for each driver, the sum of their tips for that hour.
93 |
94 | Now, how to find the maximum within each hour? The reference solutions both do this, more or less:
95 |
96 | ```java
97 | DataStream> hourlyMax = hourlyTips
98 | .windowAll(TumblingEventTimeWindows.of(Time.hours(1)))
99 | .maxBy(2);
100 | ```
101 |
102 | which works just fine, producing this stream of results:
103 |
104 | ```
105 | 3> (1577883600000,2013000089,76.0)
106 | 4> (1577887200000,2013000197,71.0)
107 | 1> (1577890800000,2013000118,83.0)
108 | 2> (1577894400000,2013000119,81.0)
109 | 3> (1577898000000,2013000195,73.0)
110 | 4> (1577901600000,2013000072,123.0)
111 | ```
112 |
113 | But, what if we were to do this, instead?
114 |
115 | ```java
116 | DataStream> hourlyMax = hourlyTips
117 | .keyBy(t -> t.f0)
118 | .maxBy(2);
119 | ```
120 |
121 | This says to group the stream of `hourlyTips` by timestamp, and within each timestamp, find the maximum of the sum of the tips.
122 | That sounds like it is exactly what we want. And while this alternative does find the same results,
123 | there are a couple of reasons why it is not a very good solution.
124 |
125 | First, instead of producing a single result at the end of each window, with this approach we get a stream that is
126 | continuously reporting the maximum achieved so far, for each key (i.e., each hour), which is an awkward way to consume
127 | the result if all you wanted was a single value for each hour.
128 |
129 | ```
130 | 1> (1577883600000,2013000108,14.0)
131 | 1> (1577883600000,2013000108,14.0)
132 | 1> (1577883600000,2013000188,18.0)
133 | 1> (1577883600000,2013000188,18.0)
134 | 1> (1577883600000,2013000188,18.0)
135 | 1> (1577883600000,2013000034,36.0)
136 | 1> (1577883600000,2013000183,70.0)
137 | 1> (1577883600000,2013000183,70.0)
138 | ...
139 | 1> (1577883600000,2013000152,73.0)
140 | 1> (1577883600000,2013000152,73.0)
141 | ...
142 | 1> (1577883600000,2013000089,76.0)
143 | ...
144 | ```
145 |
146 | Second, Flink will be keeping in state the maximum seen so far for each key (each hour), forever.
147 | Flink has no idea that these keys are event-time timestamps, and that the watermarks could be used as
148 | an indicator of when this state can be cleared -- to get those semantics, we need to use windows.
149 |
150 | -----
151 |
152 | [**Back to Labs Overview**](../LABS-OVERVIEW.md)
153 |
--------------------------------------------------------------------------------
/gradlew:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env sh
2 |
3 | #
4 | # Copyright 2015 the original author or authors.
5 | #
6 | # Licensed under the Apache License, Version 2.0 (the "License");
7 | # you may not use this file except in compliance with the License.
8 | # You may obtain a copy of the License at
9 | #
10 | # https://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | #
18 |
19 | ##############################################################################
20 | ##
21 | ## Gradle start up script for UN*X
22 | ##
23 | ##############################################################################
24 |
25 | # Attempt to set APP_HOME
26 | # Resolve links: $0 may be a link
27 | PRG="$0"
28 | # Need this for relative symlinks.
29 | while [ -h "$PRG" ] ; do
30 | ls=`ls -ld "$PRG"`
31 | link=`expr "$ls" : '.*-> \(.*\)$'`
32 | if expr "$link" : '/.*' > /dev/null; then
33 | PRG="$link"
34 | else
35 | PRG=`dirname "$PRG"`"/$link"
36 | fi
37 | done
38 | SAVED="`pwd`"
39 | cd "`dirname \"$PRG\"`/" >/dev/null
40 | APP_HOME="`pwd -P`"
41 | cd "$SAVED" >/dev/null
42 |
43 | APP_NAME="Gradle"
44 | APP_BASE_NAME=`basename "$0"`
45 |
46 | # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
47 | DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"'
48 |
49 | # Use the maximum available, or set MAX_FD != -1 to use that value.
50 | MAX_FD="maximum"
51 |
52 | warn () {
53 | echo "$*"
54 | }
55 |
56 | die () {
57 | echo
58 | echo "$*"
59 | echo
60 | exit 1
61 | }
62 |
63 | # OS specific support (must be 'true' or 'false').
64 | cygwin=false
65 | msys=false
66 | darwin=false
67 | nonstop=false
68 | case "`uname`" in
69 | CYGWIN* )
70 | cygwin=true
71 | ;;
72 | Darwin* )
73 | darwin=true
74 | ;;
75 | MINGW* )
76 | msys=true
77 | ;;
78 | NONSTOP* )
79 | nonstop=true
80 | ;;
81 | esac
82 |
83 | CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
84 |
85 | # Determine the Java command to use to start the JVM.
86 | if [ -n "$JAVA_HOME" ] ; then
87 | if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
88 | # IBM's JDK on AIX uses strange locations for the executables
89 | JAVACMD="$JAVA_HOME/jre/sh/java"
90 | else
91 | JAVACMD="$JAVA_HOME/bin/java"
92 | fi
93 | if [ ! -x "$JAVACMD" ] ; then
94 | die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME
95 |
96 | Please set the JAVA_HOME variable in your environment to match the
97 | location of your Java installation."
98 | fi
99 | else
100 | JAVACMD="java"
101 | which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH.
102 |
103 | Please set the JAVA_HOME variable in your environment to match the
104 | location of your Java installation."
105 | fi
106 |
107 | # Increase the maximum file descriptors if we can.
108 | if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then
109 | MAX_FD_LIMIT=`ulimit -H -n`
110 | if [ $? -eq 0 ] ; then
111 | if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then
112 | MAX_FD="$MAX_FD_LIMIT"
113 | fi
114 | ulimit -n $MAX_FD
115 | if [ $? -ne 0 ] ; then
116 | warn "Could not set maximum file descriptor limit: $MAX_FD"
117 | fi
118 | else
119 | warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT"
120 | fi
121 | fi
122 |
123 | # For Darwin, add options to specify how the application appears in the dock
124 | if $darwin; then
125 | GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\""
126 | fi
127 |
128 | # For Cygwin or MSYS, switch paths to Windows format before running java
129 | if [ "$cygwin" = "true" -o "$msys" = "true" ] ; then
130 | APP_HOME=`cygpath --path --mixed "$APP_HOME"`
131 | CLASSPATH=`cygpath --path --mixed "$CLASSPATH"`
132 | JAVACMD=`cygpath --unix "$JAVACMD"`
133 |
134 | # We build the pattern for arguments to be converted via cygpath
135 | ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null`
136 | SEP=""
137 | for dir in $ROOTDIRSRAW ; do
138 | ROOTDIRS="$ROOTDIRS$SEP$dir"
139 | SEP="|"
140 | done
141 | OURCYGPATTERN="(^($ROOTDIRS))"
142 | # Add a user-defined pattern to the cygpath arguments
143 | if [ "$GRADLE_CYGPATTERN" != "" ] ; then
144 | OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)"
145 | fi
146 | # Now convert the arguments - kludge to limit ourselves to /bin/sh
147 | i=0
148 | for arg in "$@" ; do
149 | CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -`
150 | CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option
151 |
152 | if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition
153 | eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"`
154 | else
155 | eval `echo args$i`="\"$arg\""
156 | fi
157 | i=`expr $i + 1`
158 | done
159 | case $i in
160 | 0) set -- ;;
161 | 1) set -- "$args0" ;;
162 | 2) set -- "$args0" "$args1" ;;
163 | 3) set -- "$args0" "$args1" "$args2" ;;
164 | 4) set -- "$args0" "$args1" "$args2" "$args3" ;;
165 | 5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;;
166 | 6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;;
167 | 7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;;
168 | 8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;;
169 | 9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;;
170 | esac
171 | fi
172 |
173 | # Escape application args
174 | save () {
175 | for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done
176 | echo " "
177 | }
178 | APP_ARGS=`save "$@"`
179 |
180 | # Collect all arguments for the java command, following the shell quoting and substitution rules
181 | eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS"
182 |
183 | exec "$JAVACMD" "$@"
184 |
--------------------------------------------------------------------------------
/common/src/main/java/org/apache/flink/training/exercises/common/utils/DataGenerator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.common.utils;
20 |
21 | import java.time.Instant;
22 | import java.util.Random;
23 |
24 | /**
25 | * Data generator for the fields in TaxiRide and TaxiFare objects.
26 | *
27 | * Results are deterministically determined by the rideId. This guarantees (among other things)
28 | * that the startTime for a TaxiRide START event matches the startTime for the TaxiRide END and
29 | * TaxiFare events for that same rideId.
30 | */
31 | public class DataGenerator {
32 |
33 | private static final int SECONDS_BETWEEN_RIDES = 20;
34 | private static final int NUMBER_OF_DRIVERS = 200;
35 | private static final Instant beginTime = Instant.parse("2020-01-01T12:00:00.00Z");
36 |
37 | private transient long rideId;
38 |
39 | /**
40 | * Creates a DataGenerator for the specified rideId.
41 | */
42 | public DataGenerator(long rideId) {
43 | this.rideId = rideId;
44 | }
45 |
46 | /**
47 | * Deterministically generates and returns the startTime for this ride.
48 | */
49 | public Instant startTime() {
50 | return beginTime.plusSeconds(SECONDS_BETWEEN_RIDES * rideId);
51 | }
52 |
53 | /**
54 | * Deterministically generates and returns the endTime for this ride.
55 | */
56 | public Instant endTime() {
57 | return startTime().plusSeconds(60 * rideDurationMinutes());
58 | }
59 |
60 | /**
61 | * Deterministically generates and returns the driverId for this ride.
62 | * The HourlyTips exercise is more interesting if aren't too many drivers.
63 | */
64 | public long driverId() {
65 | Random rnd = new Random(rideId);
66 | return 2013000000 + rnd.nextInt(NUMBER_OF_DRIVERS);
67 | }
68 |
69 | /**
70 | * Deterministically generates and returns the taxiId for this ride.
71 | */
72 | public long taxiId() {
73 | return driverId();
74 | }
75 |
76 | /**
77 | * Deterministically generates and returns the startLat for this ride.
78 | *
79 | *
The locations are used in the RideCleansing exercise.
80 | * We want some rides to be outside of NYC.
81 | */
82 | public float startLat() {
83 | return aFloat((float) (GeoUtils.LAT_SOUTH - 0.1), (float) (GeoUtils.LAT_NORTH + 0.1F));
84 | }
85 |
86 | /**
87 | * Deterministically generates and returns the startLon for this ride.
88 | */
89 | public float startLon() {
90 | return aFloat((float) (GeoUtils.LON_WEST - 0.1), (float) (GeoUtils.LON_EAST + 0.1F));
91 | }
92 |
93 | /**
94 | * Deterministically generates and returns the endLat for this ride.
95 | */
96 | public float endLat() {
97 | return bFloat((float) (GeoUtils.LAT_SOUTH - 0.1), (float) (GeoUtils.LAT_NORTH + 0.1F));
98 | }
99 |
100 | /**
101 | * Deterministically generates and returns the endLon for this ride.
102 | */
103 | public float endLon() {
104 | return bFloat((float) (GeoUtils.LON_WEST - 0.1), (float) (GeoUtils.LON_EAST + 0.1F));
105 | }
106 |
107 | /**
108 | * Deterministically generates and returns the passengerCnt for this ride.
109 | */
110 | public short passengerCnt() {
111 | return (short) aLong(1L, 4L);
112 | }
113 |
114 | /**
115 | * Deterministically generates and returns the paymentType for this ride.
116 | */
117 | public String paymentType() {
118 | return (rideId % 2 == 0) ? "CARD" : "CASH";
119 | }
120 |
121 | /**
122 | * Deterministically generates and returns the tip for this ride.
123 | *
124 | *
The HourlyTips exercise is more interesting if there's some significant variation in tipping.
125 | */
126 | public float tip() {
127 | return aLong(0L, 60L, 10F, 15F);
128 | }
129 |
130 | /**
131 | * Deterministically generates and returns the tolls for this ride.
132 | */
133 | public float tolls() {
134 | return (rideId % 10 == 0) ? aLong(0L, 5L) : 0L;
135 | }
136 |
137 | /**
138 | * Deterministically generates and returns the totalFare for this ride.
139 | */
140 | public float totalFare() {
141 | return (float) (3.0 + (1.0 * rideDurationMinutes()) + tip() + tolls());
142 | }
143 |
144 | /**
145 | * The LongRides exercise needs to have some rides with a duration > 2 hours, but not too many.
146 | */
147 | private long rideDurationMinutes() {
148 | return aLong(0L, 600, 20, 40);
149 | }
150 |
151 | // -------------------------------------
152 |
153 | private long aLong(long min, long max) {
154 | float mean = (min + max) / 2.0F;
155 | float stddev = (max - min) / 8F;
156 |
157 | return aLong(min, max, mean, stddev);
158 | }
159 |
160 | // the rideId is used as the seed to guarantee deterministic results
161 | private long aLong(long min, long max, float mean, float stddev) {
162 | Random rnd = new Random(rideId);
163 | long value;
164 | do {
165 | value = (long) Math.round((stddev * rnd.nextGaussian()) + mean);
166 | } while ((value < min) || (value > max));
167 | return value;
168 | }
169 |
170 | // -------------------------------------
171 |
172 | private float aFloat(float min, float max) {
173 | float mean = (min + max) / 2.0F;
174 | float stddev = (max - min) / 8F;
175 |
176 | return aFloat(rideId, min, max, mean, stddev);
177 | }
178 |
179 | private float bFloat(float min, float max) {
180 | float mean = (min + max) / 2.0F;
181 | float stddev = (max - min) / 8F;
182 |
183 | return aFloat(rideId + 42, min, max, mean, stddev);
184 | }
185 |
186 | // the rideId is used as the seed to guarantee deterministic results
187 | private float aFloat(long seed, float min, float max, float mean, float stddev) {
188 | Random rnd = new Random(seed);
189 | float value;
190 | do {
191 | value = (float) (stddev * rnd.nextGaussian()) + mean;
192 | } while ((value < min) || (value > max));
193 | return value;
194 | }
195 |
196 | }
197 |
--------------------------------------------------------------------------------
/common/src/main/java/org/apache/flink/training/exercises/common/utils/GeoUtils.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.training.exercises.common.utils;
20 |
21 | import java.util.ArrayList;
22 | import java.util.List;
23 | import java.util.Random;
24 |
25 | /**
26 | * GeoUtils provides utility methods to deal with locations for the data streaming exercises.
27 | */
28 | public class GeoUtils {
29 |
30 | // geo boundaries of the area of NYC
31 | public static final double LON_EAST = -73.7;
32 | public static final double LON_WEST = -74.05;
33 | public static final double LAT_NORTH = 41.0;
34 | public static final double LAT_SOUTH = 40.5;
35 |
36 | // area width and height
37 | public static final double LON_WIDTH = 74.05 - 73.7;
38 | public static final double LAT_HEIGHT = 41.0 - 40.5;
39 |
40 | // delta step to create artificial grid overlay of NYC
41 | public static final double DELTA_LON = 0.0014;
42 | public static final double DELTA_LAT = 0.00125;
43 |
44 | // ( |LON_WEST| - |LON_EAST| ) / DELTA_LAT
45 | public static final int NUMBER_OF_GRID_X = 250;
46 | // ( LAT_NORTH - LAT_SOUTH ) / DELTA_LON
47 | public static final int NUMBER_OF_GRID_Y = 400;
48 |
49 | public static final float DEG_LEN = 110.25f;
50 |
51 | /**
52 | * Checks if a location specified by longitude and latitude values is
53 | * within the geo boundaries of New York City.
54 | *
55 | * @param lon longitude of the location to check
56 | * @param lat latitude of the location to check
57 | *
58 | * @return true if the location is within NYC boundaries, otherwise false.
59 | */
60 | public static boolean isInNYC(float lon, float lat) {
61 |
62 | return !(lon > LON_EAST || lon < LON_WEST) &&
63 | !(lat > LAT_NORTH || lat < LAT_SOUTH);
64 | }
65 |
66 | /**
67 | * Maps a location specified by latitude and longitude values to a cell of a
68 | * grid covering the area of NYC.
69 | * The grid cells are roughly 100 x 100 m and sequentially number from north-west
70 | * to south-east starting by zero.
71 | *
72 | * @param lon longitude of the location to map
73 | * @param lat latitude of the location to map
74 | *
75 | * @return id of mapped grid cell.
76 | */
77 | public static int mapToGridCell(float lon, float lat) {
78 | int xIndex = (int) Math.floor((Math.abs(LON_WEST) - Math.abs(lon)) / DELTA_LON);
79 | int yIndex = (int) Math.floor((LAT_NORTH - lat) / DELTA_LAT);
80 |
81 | return xIndex + (yIndex * NUMBER_OF_GRID_X);
82 | }
83 |
84 | /**
85 | * Maps the direct path between two locations specified by longitude and latitude to a list of
86 | * cells of a grid covering the area of NYC.
87 | * The grid cells are roughly 100 x 100 m and sequentially number from north-west
88 | * to south-east starting by zero.
89 | *
90 | * @param lon1 longitude of the first location
91 | * @param lat1 latitude of the first location
92 | * @param lon2 longitude of the second location
93 | * @param lat2 latitude of the second location
94 | *
95 | * @return A list of cell ids
96 | */
97 | public static List mapToGridCellsOnWay(float lon1, float lat1, float lon2, float lat2) {
98 |
99 | int x1 = (int) Math.floor((Math.abs(LON_WEST) - Math.abs(lon1)) / DELTA_LON);
100 | int y1 = (int) Math.floor((LAT_NORTH - lat1) / DELTA_LAT);
101 |
102 | int x2 = (int) Math.floor((Math.abs(LON_WEST) - Math.abs(lon2)) / DELTA_LON);
103 | int y2 = (int) Math.floor((LAT_NORTH - lat2) / DELTA_LAT);
104 |
105 | int startX, startY, endX, endY;
106 | if (x1 <= x2) {
107 | startX = x1;
108 | startY = y1;
109 | endX = x2;
110 | endY = y2;
111 | }
112 | else {
113 | startX = x2;
114 | startY = y2;
115 | endX = x1;
116 | endY = y1;
117 | }
118 |
119 | double slope = (endY - startY) / ((endX - startX) + 0.00000001);
120 |
121 | int curX = startX;
122 | int curY = startY;
123 |
124 | ArrayList cellIds = new ArrayList<>(64);
125 | cellIds.add(curX + (curY * NUMBER_OF_GRID_X));
126 |
127 | while (curX < endX || curY != endY) {
128 |
129 | if (slope > 0) {
130 | double y = (curX - startX + 0.5) * slope + startY - 0.5;
131 |
132 | if (y > curY - 0.05 && y < curY + 0.05) {
133 | curX++;
134 | curY++;
135 | }
136 | else if (y < curY) {
137 | curX++;
138 | }
139 | else {
140 | curY++;
141 | }
142 | }
143 | else {
144 | double y = (curX - startX + 0.5) * slope + startY + 0.5;
145 |
146 | if (y > curY - 0.05 && y < curY + 0.05) {
147 | curX++;
148 | curY--;
149 | }
150 | if (y > curY) {
151 | curX++;
152 | }
153 | else {
154 | curY--;
155 | }
156 |
157 | }
158 |
159 | cellIds.add(curX + (curY * NUMBER_OF_GRID_X));
160 | }
161 |
162 | return cellIds;
163 | }
164 |
165 | /**
166 | * Returns the longitude of the center of a grid cell.
167 | *
168 | * @param gridCellId The grid cell.
169 | *
170 | * @return The longitude value of the cell's center.
171 | */
172 | public static float getGridCellCenterLon(int gridCellId) {
173 |
174 | int xIndex = gridCellId % NUMBER_OF_GRID_X;
175 |
176 | return (float) (Math.abs(LON_WEST) - (xIndex * DELTA_LON) - (DELTA_LON / 2)) * -1.0f;
177 | }
178 |
179 | /**
180 | * Returns the latitude of the center of a grid cell.
181 | *
182 | * @param gridCellId The grid cell.
183 | *
184 | * @return The latitude value of the cell's center.
185 | */
186 | public static float getGridCellCenterLat(int gridCellId) {
187 |
188 | int xIndex = gridCellId % NUMBER_OF_GRID_X;
189 | int yIndex = (gridCellId - xIndex) / NUMBER_OF_GRID_X;
190 |
191 | return (float) (LAT_NORTH - (yIndex * DELTA_LAT) - (DELTA_LAT / 2));
192 |
193 | }
194 |
195 | /**
196 | * Returns a random longitude within the NYC area.
197 | *
198 | * @param rand A random number generator.
199 | * @return A random longitude value within the NYC area.
200 | */
201 | public static float getRandomNYCLon(Random rand) {
202 | return (float) (LON_EAST - (LON_WIDTH * rand.nextFloat()));
203 | }
204 |
205 | /**
206 | * Returns a random latitude within the NYC area.
207 | *
208 | * @param rand A random number generator.
209 | * @return A random latitude value within the NYC area.
210 | */
211 | public static float getRandomNYCLat(Random rand) {
212 | return (float) (LAT_SOUTH + (LAT_HEIGHT * rand.nextFloat()));
213 | }
214 |
215 | /**
216 | * Returns the Euclidean distance between two locations specified as lon/lat pairs.
217 | *
218 | * @param lon1 Longitude of first location
219 | * @param lat1 Latitude of first location
220 | * @param lon2 Longitude of second location
221 | * @param lat2 Latitude of second location
222 | * @return The Euclidean distance between the specified locations.
223 | */
224 | public static double getEuclideanDistance(float lon1, float lat1, float lon2, float lat2) {
225 | double x = lat1 - lat2;
226 | double y = (lon1 - lon2) * Math.cos(lat2);
227 | return (DEG_LEN * Math.sqrt(x * x + y * y));
228 | }
229 |
230 | /**
231 | * Returns the angle in degrees between the vector from the start to the destination
232 | * and the x-axis on which the start is located.
233 | *
234 | * The angle describes in which direction the destination is located from the start, i.e.,
235 | * 0° -> East, 90° -> South, 180° -> West, 270° -> North
236 | *
237 | * @param startLon longitude of start location
238 | * @param startLat latitude of start location
239 | * @param destLon longitude of destination
240 | * @param destLat latitude of destination
241 | * @return The direction from start to destination location
242 | */
243 | public static int getDirectionAngle(
244 | float startLon, float startLat, float destLon, float destLat) {
245 |
246 | double x = destLat - startLat;
247 | double y = (destLon - startLon) * Math.cos(startLat);
248 |
249 | return (int) Math.toDegrees(Math.atan2(x, y)) + 179;
250 | }
251 |
252 | }
253 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 |
19 |
20 | # Apache Flink Training Exercises
21 |
22 | Exercises that go along with the training content in the documentation.
23 |
24 | ## Table of Contents
25 |
26 | [**Setup your Development Environment**](#setup-your-development-environment)
27 |
28 | 1. [Software requirements](#software-requirements)
29 | 1. [Clone and build the flink-training project](#clone-and-build-the-flink-training-project)
30 | 1. [Import the flink-training project into your IDE](#import-the-flink-training-project-into-your-ide)
31 |
32 | [**Using the Taxi Data Streams**](#using-the-taxi-data-streams)
33 |
34 | 1. [Schema of Taxi Ride Events](#schema-of-taxi-ride-events)
35 | 1. [Generating Taxi Ride Data Streams in a Flink program](#generating-taxi-ride-data-streams-in-a-flink-program)
36 |
37 | [**How to do the Labs**](#how-to-do-the-labs)
38 |
39 | 1. [Learn about the data](#learn-about-the-data)
40 | 1. [Modify `ExerciseBase`](#modify-exercisebase)
41 | 1. [Run and debug Flink programs in your IDE](#run-and-debug-flink-programs-in-your-ide)
42 | 1. [Exercises, Tests, and Solutions](#exercises-tests-and-solutions)
43 |
44 | [**Labs**](LABS-OVERVIEW.md)
45 |
46 | [**License**](#license)
47 |
48 | ## Setup your Development Environment
49 |
50 | The following instructions guide you through the process of setting up a development environment for the purpose of developing, debugging, and executing solutions to the Flink developer training exercises and examples.
51 |
52 | ### Software requirements
53 |
54 | Flink supports Linux, OS X, and Windows as development environments for Flink programs and local execution. The following software is required for a Flink development setup and should be installed on your system:
55 |
56 | - a JDK for Java 8 or Java 11 (a JRE is not sufficient; other versions of Java are not supported)
57 | - Git
58 | - an IDE for Java (and/or Scala) development with Gradle support.
59 | We recommend IntelliJ, but Eclipse or Visual Studio Code can also be used so long as you stick to Java. For Scala you will need to use IntelliJ (and its Scala plugin).
60 |
61 | > **:information_source: Note for Windows users:** The examples of shell commands provided in the training instructions are for UNIX systems. To make things easier, you may find it worthwhile to setup cygwin or WSL. For developing Flink jobs, Windows works reasonably well: you can run a Flink cluster on a single machine, submit jobs, run the webUI, and execute jobs in the IDE.
62 |
63 | ### Clone and build the flink-training project
64 |
65 | This `flink-training` project contains exercises, tests, and reference solutions for the programming exercises. Clone the `flink-training` project from Github and build it.
66 |
67 | > **:information_source: Repository Layout:** This repository has several branches set up pointing to different Apache Flink versions, similarly to the [apache/flink](https://github.com/apache/flink) repository with:
68 | > - a release branch for each minor version of Apache Flink, e.g. `release-1.10`, and
69 | > - a `master` branch that points to the current Flink release (not `flink:master`!)
70 | >
71 | > If you want to work on a version other than the current Flink release, make sure to check out the appropriate branch.
72 |
73 | ```bash
74 | git clone https://github.com/apache/flink-training.git
75 | cd flink-training
76 | ./gradlew test shadowJar
77 | ```
78 |
79 | If you haven’t done this before, at this point you’ll end up downloading all of the dependencies for this Flink training project. This usually takes a few minutes, depending on the speed of your internet connection.
80 |
81 | If all of the tests pass and the build is successful, you are off to a good start.
82 |
83 |
84 | Users in China: click here for instructions about using a local maven mirror.
85 |
86 | If you are in China, we recommend configuring the maven repository to use a mirror. You can do this by uncommenting the appropriate line in our [`build.gradle`](build.gradle) like this:
87 |
88 | ```groovy
89 | repositories {
90 | // for access from China, you may need to uncomment this line
91 | maven { url 'http://maven.aliyun.com/nexus/content/groups/public/' }
92 | mavenCentral()
93 | maven {
94 | url "https://repository.apache.org/content/repositories/snapshots/"
95 | mavenContent {
96 | snapshotsOnly()
97 | }
98 | }
99 | }
100 | ```
101 |
102 |
103 |
104 | ### Import the flink-training project into your IDE
105 |
106 | The project needs to be imported as a gradle project into your IDE.
107 |
108 | Once that’s done you should be able to open [`RideCleansingTest`](ride-cleansing/src/test/java/org/apache/flink/training/exercises/ridecleansing/RideCleansingTest.java) and successfully run this test.
109 |
110 | > **:information_source: Note for Scala users:** You will need to use IntelliJ with the JetBrains Scala plugin, and you will need to add a Scala 2.12 SDK to the Global Libraries section of the Project Structure. IntelliJ will ask you for the latter when you open a Scala file.
111 |
112 | ## Using the Taxi Data Streams
113 |
114 | These exercises use data generators that produce simulated event streams inspired by those shared by the
115 | [New York City Taxi & Limousine Commission](http://www.nyc.gov/html/tlc/html/home/home.shtml)
116 | in their public [data set](https://uofi.app.box.com/NYCtaxidata) about taxi rides in New York City.
117 |
118 | ### Schemas of Taxi Ride and Taxi Fare Events
119 |
120 | Our taxi data set contains information about individual taxi rides in New York City. Each ride is represented by two events: a trip start, and a trip end event. Each event consists of eleven fields:
121 |
122 | ```
123 | rideId : Long // a unique id for each ride
124 | taxiId : Long // a unique id for each taxi
125 | driverId : Long // a unique id for each driver
126 | isStart : Boolean // TRUE for ride start events, FALSE for ride end events
127 | startTime : Instant // the start time of a ride
128 | endTime : Instant // the end time of a ride,
129 | // "1970-01-01 00:00:00" for start events
130 | startLon : Float // the longitude of the ride start location
131 | startLat : Float // the latitude of the ride start location
132 | endLon : Float // the longitude of the ride end location
133 | endLat : Float // the latitude of the ride end location
134 | passengerCnt : Short // number of passengers on the ride
135 | ```
136 |
137 | There is also a related data set containing fare data about those same rides, with these fields:
138 |
139 | ```
140 | rideId : Long // a unique id for each ride
141 | taxiId : Long // a unique id for each taxi
142 | driverId : Long // a unique id for each driver
143 | startTime : Instant // the start time of a ride
144 | paymentType : String // CASH or CARD
145 | tip : Float // tip for this ride
146 | tolls : Float // tolls for this ride
147 | totalFare : Float // total fare collected
148 | ```
149 |
150 | ## How to do the Labs
151 |
152 | In the hands-on sessions you will implement Flink programs using various Flink APIs.
153 |
154 | The following steps guide you through the process of using the provided data streams, implementing your first Flink streaming program, and executing your program in your IDE.
155 |
156 | We assume you have setup your development environment according to our [setup guide above](#setup-your-development-environment).
157 |
158 | ### Learn about the data
159 |
160 | The initial set of exercises are all based on data streams of events about taxi rides and taxi fares. These streams are produced by source functions which reads data from input files. Please read the [instructions above](#using-the-taxi-data-streams) to learn how to use them.
161 |
162 | ### Run and debug Flink programs in your IDE
163 |
164 | Flink programs can be executed and debugged from within an IDE. This significantly eases the development process and provides an experience similar to working on any other Java (or Scala) application.
165 |
166 | Starting a Flink program in your IDE is as easy as running its `main()` method. Under the hood, the execution environment will start a local Flink instance within the same process. Hence it is also possible to put breakpoints in your code and debug it.
167 |
168 | Assuming you have an IDE with this `flink-training` project imported, you can run (or debug) a simple streaming job as follows:
169 |
170 | - Open the `org.apache.flink.training.examples.ridecount.RideCountExample` class in your IDE
171 | - Run (or debug) the `main()` method of the `RideCountExample` class using your IDE.
172 |
173 | ### Exercises, Tests, and Solutions
174 |
175 | Each of these exercises includes an `...Exercise` class with most of the necessary boilerplate code for getting started, as well as a JUnit Test class (`...Test`) with a few tests for your implementation, and a `...Solution` class with a complete solution.
176 |
177 | > **:information_source: Note:** As long as your `...Exercise` class is throwing a `MissingSolutionException`, the provided JUnit test classes will ignore that failure and verify the correctness of the solution implementation instead.
178 |
179 | There are Java and Scala versions of all the exercise, test, and solution classes.
180 |
181 | -----
182 |
183 | Now you are ready to begin with the first exercise in our [**Labs**](LABS-OVERVIEW.md).
184 |
185 | -----
186 |
187 | ## License
188 |
189 | The code in this repository is licensed under the Apache Software License 2.
190 |
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 | Apache License
2 | Version 2.0, January 2004
3 | http://www.apache.org/licenses/
4 |
5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
6 |
7 | 1. Definitions.
8 |
9 | "License" shall mean the terms and conditions for use, reproduction,
10 | and distribution as defined by Sections 1 through 9 of this document.
11 |
12 | "Licensor" shall mean the copyright owner or entity authorized by
13 | the copyright owner that is granting the License.
14 |
15 | "Legal Entity" shall mean the union of the acting entity and all
16 | other entities that control, are controlled by, or are under common
17 | control with that entity. For the purposes of this definition,
18 | "control" means (i) the power, direct or indirect, to cause the
19 | direction or management of such entity, whether by contract or
20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
21 | outstanding shares, or (iii) beneficial ownership of such entity.
22 |
23 | "You" (or "Your") shall mean an individual or Legal Entity
24 | exercising permissions granted by this License.
25 |
26 | "Source" form shall mean the preferred form for making modifications,
27 | including but not limited to software source code, documentation
28 | source, and configuration files.
29 |
30 | "Object" form shall mean any form resulting from mechanical
31 | transformation or translation of a Source form, including but
32 | not limited to compiled object code, generated documentation,
33 | and conversions to other media types.
34 |
35 | "Work" shall mean the work of authorship, whether in Source or
36 | Object form, made available under the License, as indicated by a
37 | copyright notice that is included in or attached to the work
38 | (an example is provided in the Appendix below).
39 |
40 | "Derivative Works" shall mean any work, whether in Source or Object
41 | form, that is based on (or derived from) the Work and for which the
42 | editorial revisions, annotations, elaborations, or other modifications
43 | represent, as a whole, an original work of authorship. For the purposes
44 | of this License, Derivative Works shall not include works that remain
45 | separable from, or merely link (or bind by name) to the interfaces of,
46 | the Work and Derivative Works thereof.
47 |
48 | "Contribution" shall mean any work of authorship, including
49 | the original version of the Work and any modifications or additions
50 | to that Work or Derivative Works thereof, that is intentionally
51 | submitted to Licensor for inclusion in the Work by the copyright owner
52 | or by an individual or Legal Entity authorized to submit on behalf of
53 | the copyright owner. For the purposes of this definition, "submitted"
54 | means any form of electronic, verbal, or written communication sent
55 | to the Licensor or its representatives, including but not limited to
56 | communication on electronic mailing lists, source code control systems,
57 | and issue tracking systems that are managed by, or on behalf of, the
58 | Licensor for the purpose of discussing and improving the Work, but
59 | excluding communication that is conspicuously marked or otherwise
60 | designated in writing by the copyright owner as "Not a Contribution."
61 |
62 | "Contributor" shall mean Licensor and any individual or Legal Entity
63 | on behalf of whom a Contribution has been received by Licensor and
64 | subsequently incorporated within the Work.
65 |
66 | 2. Grant of Copyright License. Subject to the terms and conditions of
67 | this License, each Contributor hereby grants to You a perpetual,
68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
69 | copyright license to reproduce, prepare Derivative Works of,
70 | publicly display, publicly perform, sublicense, and distribute the
71 | Work and such Derivative Works in Source or Object form.
72 |
73 | 3. Grant of Patent License. Subject to the terms and conditions of
74 | this License, each Contributor hereby grants to You a perpetual,
75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
76 | (except as stated in this section) patent license to make, have made,
77 | use, offer to sell, sell, import, and otherwise transfer the Work,
78 | where such license applies only to those patent claims licensable
79 | by such Contributor that are necessarily infringed by their
80 | Contribution(s) alone or by combination of their Contribution(s)
81 | with the Work to which such Contribution(s) was submitted. If You
82 | institute patent litigation against any entity (including a
83 | cross-claim or counterclaim in a lawsuit) alleging that the Work
84 | or a Contribution incorporated within the Work constitutes direct
85 | or contributory patent infringement, then any patent licenses
86 | granted to You under this License for that Work shall terminate
87 | as of the date such litigation is filed.
88 |
89 | 4. Redistribution. You may reproduce and distribute copies of the
90 | Work or Derivative Works thereof in any medium, with or without
91 | modifications, and in Source or Object form, provided that You
92 | meet the following conditions:
93 |
94 | (a) You must give any other recipients of the Work or
95 | Derivative Works a copy of this License; and
96 |
97 | (b) You must cause any modified files to carry prominent notices
98 | stating that You changed the files; and
99 |
100 | (c) You must retain, in the Source form of any Derivative Works
101 | that You distribute, all copyright, patent, trademark, and
102 | attribution notices from the Source form of the Work,
103 | excluding those notices that do not pertain to any part of
104 | the Derivative Works; and
105 |
106 | (d) If the Work includes a "NOTICE" text file as part of its
107 | distribution, then any Derivative Works that You distribute must
108 | include a readable copy of the attribution notices contained
109 | within such NOTICE file, excluding those notices that do not
110 | pertain to any part of the Derivative Works, in at least one
111 | of the following places: within a NOTICE text file distributed
112 | as part of the Derivative Works; within the Source form or
113 | documentation, if provided along with the Derivative Works; or,
114 | within a display generated by the Derivative Works, if and
115 | wherever such third-party notices normally appear. The contents
116 | of the NOTICE file are for informational purposes only and
117 | do not modify the License. You may add Your own attribution
118 | notices within Derivative Works that You distribute, alongside
119 | or as an addendum to the NOTICE text from the Work, provided
120 | that such additional attribution notices cannot be construed
121 | as modifying the License.
122 |
123 | You may add Your own copyright statement to Your modifications and
124 | may provide additional or different license terms and conditions
125 | for use, reproduction, or distribution of Your modifications, or
126 | for any such Derivative Works as a whole, provided Your use,
127 | reproduction, and distribution of the Work otherwise complies with
128 | the conditions stated in this License.
129 |
130 | 5. Submission of Contributions. Unless You explicitly state otherwise,
131 | any Contribution intentionally submitted for inclusion in the Work
132 | by You to the Licensor shall be under the terms and conditions of
133 | this License, without any additional terms or conditions.
134 | Notwithstanding the above, nothing herein shall supersede or modify
135 | the terms of any separate license agreement you may have executed
136 | with Licensor regarding such Contributions.
137 |
138 | 6. Trademarks. This License does not grant permission to use the trade
139 | names, trademarks, service marks, or product names of the Licensor,
140 | except as required for reasonable and customary use in describing the
141 | origin of the Work and reproducing the content of the NOTICE file.
142 |
143 | 7. Disclaimer of Warranty. Unless required by applicable law or
144 | agreed to in writing, Licensor provides the Work (and each
145 | Contributor provides its Contributions) on an "AS IS" BASIS,
146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
147 | implied, including, without limitation, any warranties or conditions
148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
149 | PARTICULAR PURPOSE. You are solely responsible for determining the
150 | appropriateness of using or redistributing the Work and assume any
151 | risks associated with Your exercise of permissions under this License.
152 |
153 | 8. Limitation of Liability. In no event and under no legal theory,
154 | whether in tort (including negligence), contract, or otherwise,
155 | unless required by applicable law (such as deliberate and grossly
156 | negligent acts) or agreed to in writing, shall any Contributor be
157 | liable to You for damages, including any direct, indirect, special,
158 | incidental, or consequential damages of any character arising as a
159 | result of this License or out of the use or inability to use the
160 | Work (including but not limited to damages for loss of goodwill,
161 | work stoppage, computer failure or malfunction, or any and all
162 | other commercial damages or losses), even if such Contributor
163 | has been advised of the possibility of such damages.
164 |
165 | 9. Accepting Warranty or Additional Liability. While redistributing
166 | the Work or Derivative Works thereof, You may choose to offer,
167 | and charge a fee for, acceptance of support, warranty, indemnity,
168 | or other liability obligations and/or rights consistent with this
169 | License. However, in accepting such obligations, You may act only
170 | on Your own behalf and on Your sole responsibility, not on behalf
171 | of any other Contributor, and only if You agree to indemnify,
172 | defend, and hold each Contributor harmless for any liability
173 | incurred by, or claims asserted against, such Contributor by reason
174 | of your accepting any such warranty or additional liability.
175 |
176 | END OF TERMS AND CONDITIONS
177 |
178 | APPENDIX: How to apply the Apache License to your work.
179 |
180 | To apply the Apache License to your work, attach the following
181 | boilerplate notice, with the fields enclosed by brackets "[]"
182 | replaced with your own identifying information. (Don't include
183 | the brackets!) The text should be enclosed in the appropriate
184 | comment syntax for the file format. We also recommend that a
185 | file or class name and description of purpose be included on the
186 | same "printed page" as the copyright notice for easier
187 | identification within third-party archives.
188 |
189 | Copyright [yyyy] [name of copyright owner]
190 |
191 | Licensed under the Apache License, Version 2.0 (the "License");
192 | you may not use this file except in compliance with the License.
193 | You may obtain a copy of the License at
194 |
195 | http://www.apache.org/licenses/LICENSE-2.0
196 |
197 | Unless required by applicable law or agreed to in writing, software
198 | distributed under the License is distributed on an "AS IS" BASIS,
199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
200 | See the License for the specific language governing permissions and
201 | limitations under the License.
202 |
--------------------------------------------------------------------------------
/config/checkstyle/checkstyle.xml:
--------------------------------------------------------------------------------
1 |
2 |
20 |
23 |
24 |
31 |
32 |
33 |
34 |
35 |
36 |
37 |
38 |
39 |
40 |
41 |
42 |
43 |
44 |
45 |
46 |
47 |
48 |
49 |
50 |
51 |
52 |
53 |
54 |
55 |
56 |
57 |
58 |
59 |
60 |
61 |
62 |
63 |
64 |
65 |
66 |
67 |
68 |
69 |
70 |
71 |
72 |
73 |
74 |
79 |
80 |
81 |
82 |
83 |
84 |
85 |
86 |
87 |
88 |
89 |
90 |
91 |
92 |
93 |
94 |
99 |
100 |
101 |
102 |
104 |
105 |
106 |
107 |
108 |
109 |
110 |
111 |
112 |
113 |
114 |
115 |
116 |
117 |
118 |
119 |
120 |
121 |
122 |
123 |
124 |
125 |
126 |
131 |
132 |
133 |
134 |
135 |
136 |
137 |
138 |
139 |
140 |
141 |
142 |
143 |
144 |
145 |
146 |
147 |
148 |
149 |
150 |
151 |
152 |
153 |
154 |
155 |
156 |
157 |
158 |
159 |
160 |
161 |
162 |
163 |
164 |
165 |
166 |
167 |
168 |
169 |
170 |
171 |
172 |
173 |
174 |
175 |
176 |
177 |
178 |
179 |
180 |
181 |
182 |
183 |
184 |
185 |
190 |
191 |
192 |
193 |
194 |
196 |
197 |
198 |
199 |
200 |
201 |
202 |
203 |
204 |
205 |
206 |
208 |
209 |
210 |
211 |
212 |
213 |
214 |
215 |
216 |
217 |
218 |
219 |
224 |
225 |
226 |
227 |
231 |
232 |
233 |
234 |
235 |
238 |
239 |
240 |
241 |
242 |
243 |
244 |
245 |
246 |
248 |
249 |
250 |
255 |
256 |
257 |
258 |
259 |
260 |
261 |
262 |
263 |
264 |
265 |
266 |
267 |
268 |
269 |
270 |
271 |
272 |
273 |
274 |
275 |
276 |
277 |
278 |
279 |
280 |
281 |
282 |
283 |
288 |
289 |
290 |
291 |
292 |
294 |
297 |
298 |
299 |
300 |
301 |
302 |
304 |
305 |
306 |
307 |
308 |
309 |
311 |
312 |
313 |
314 |
315 |
316 |
317 |
319 |
320 |
321 |
322 |
323 |
325 |
326 |
327 |
328 |
329 |
330 |
331 |
332 |
333 |
334 |
335 |
336 |
337 |
338 |
339 |
340 |
341 |
342 |
343 |
344 |
345 |
346 |
347 |
348 |
349 |
350 |
351 |
352 |
353 |
355 |
356 |
357 |
358 |
359 |
361 |
362 |
363 |
364 |
365 |
367 |
368 |
369 |
370 |
373 |
374 |
375 |
376 |
377 |
378 |
379 |
380 |
381 |
382 |
383 |
384 |
385 |
386 |
387 |
388 |
393 |
394 |
395 |
396 |
397 |
398 |
399 |
400 |
401 |
402 |
403 |
404 |
405 |
406 |
407 |
408 |
409 |
410 |
411 |
412 |
413 |
414 |
415 |
416 |
417 |
418 |
419 |
420 |
421 |
422 |
423 |
424 |
425 |
426 |
427 |
428 |
429 |
430 |
431 |
432 |
433 |
434 |
435 |
436 |
437 |
438 |
439 |
440 |
441 |
442 |
443 |
444 |
445 |
446 |
447 |
452 |
454 |
455 |
456 |
457 |
458 |
459 |
460 |
461 |
462 |
463 |
464 |
465 |
466 |
467 |
468 |
469 |
470 |
475 |
476 |
477 |
482 |
483 |
484 |
485 |
486 |
491 |
492 |
493 |
497 |
498 |
499 |
502 |
503 |
504 |
505 |
509 |
516 |
517 |
518 |
519 |
520 |
523 |
524 |
525 |
526 |
527 |
530 |
532 |
533 |
534 |
535 |
536 |
537 |
540 |
541 |
542 |
543 |
544 |
545 |
546 |
547 |
548 |
549 |
550 |
551 |
552 |
553 |
554 |
555 |
556 |
557 |
558 |
559 |
560 |
561 |
562 |
565 |
566 |
567 |
568 |
569 |
570 |
571 |
--------------------------------------------------------------------------------