| /* |
| * Licensed to the Apache Software Foundation (ASF) under one |
| * or more contributor license agreements. See the NOTICE file |
| * distributed with this work for additional information |
| * regarding copyright ownership. The ASF licenses this file |
| * to you under the Apache License, Version 2.0 (the |
| * "License"); you may not use this file except in compliance |
| * with the License. You may obtain a copy of the License at |
| * |
| * http://www.apache.org/licenses/LICENSE-2.0 |
| * |
| * Unless required by applicable law or agreed to in writing, software |
| * distributed under the License is distributed on an "AS IS" BASIS, |
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| * See the License for the specific language governing permissions and |
| * limitations under the License. |
| */ |
| |
| package org.apache.flink.training.solutions.hourlytips; |
| |
| import org.apache.flink.api.java.tuple.Tuple3; |
| import org.apache.flink.api.java.utils.ParameterTool; |
| import org.apache.flink.streaming.api.TimeCharacteristic; |
| import org.apache.flink.streaming.api.datastream.DataStream; |
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; |
| import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; |
| import org.apache.flink.streaming.api.windowing.time.Time; |
| import org.apache.flink.streaming.api.windowing.windows.TimeWindow; |
| import org.apache.flink.training.exercises.common.datatypes.TaxiFare; |
| import org.apache.flink.training.exercises.common.sources.TaxiFareSource; |
| import org.apache.flink.training.exercises.common.utils.ExerciseBase; |
| import org.apache.flink.util.Collector; |
| |
| /** |
| * Java reference implementation for the "Hourly Tips" exercise of the Flink training in the docs. |
| * |
| * <p>The task of the exercise is to first calculate the total tips collected by each driver, hour by hour, and |
| * then from that stream, find the highest tip total in each hour. |
| * |
| * <p>Parameters: |
| * -input path-to-input-file |
| */ |
| public class HourlyTipsSolution extends ExerciseBase { |
| |
| /** |
| * Main method. |
| * |
| * <p>Parameters: |
| * -input path-to-input-file |
| * |
| * @throws Exception which occurs during job execution. |
| */ |
| public static void main(String[] args) throws Exception { |
| |
| // read parameters |
| ParameterTool params = ParameterTool.fromArgs(args); |
| final String input = params.get("input", ExerciseBase.PATH_TO_FARE_DATA); |
| |
| final int maxEventDelay = 60; // events are out of order by max 60 seconds |
| final int servingSpeedFactor = 600; // events of 10 minutes are served in 1 second |
| |
| // set up streaming execution environment |
| StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); |
| env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); |
| env.setParallelism(ExerciseBase.parallelism); |
| |
| // start the data generator |
| DataStream<TaxiFare> fares = env.addSource(fareSourceOrTest(new TaxiFareSource(input, maxEventDelay, servingSpeedFactor))); |
| |
| // compute tips per hour for each driver |
| DataStream<Tuple3<Long, Long, Float>> hourlyTips = fares |
| .keyBy((TaxiFare fare) -> fare.driverId) |
| .timeWindow(Time.hours(1)) |
| .process(new AddTips()); |
| |
| DataStream<Tuple3<Long, Long, Float>> hourlyMax = hourlyTips |
| .timeWindowAll(Time.hours(1)) |
| .maxBy(2); |
| |
| // You should explore how this alternative behaves. In what ways is the same as, |
| // and different from, the solution above (using a timeWindowAll)? |
| |
| // DataStream<Tuple3<Long, Long, Float>> hourlyMax = hourlyTips |
| // .keyBy(0) |
| // .maxBy(2); |
| |
| printOrTest(hourlyMax); |
| |
| // execute the transformation pipeline |
| env.execute("Hourly Tips (java)"); |
| } |
| |
| /* |
| * Wraps the pre-aggregated result into a tuple along with the window's timestamp and key. |
| */ |
| public static class AddTips extends ProcessWindowFunction< |
| TaxiFare, Tuple3<Long, Long, Float>, Long, TimeWindow> { |
| @Override |
| public void process(Long key, Context context, Iterable<TaxiFare> fares, Collector<Tuple3<Long, Long, Float>> out) { |
| float sumOfTips = 0F; |
| for (TaxiFare f : fares) { |
| sumOfTips += f.tip; |
| } |
| out.collect(Tuple3.of(context.window().getEnd(), key, sumOfTips)); |
| } |
| } |
| } |