| /* |
| * 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.ridesandfares; |
| |
| import org.apache.flink.api.common.JobExecutionResult; |
| import org.apache.flink.api.common.state.ValueState; |
| import org.apache.flink.api.common.state.ValueStateDescriptor; |
| import org.apache.flink.configuration.Configuration; |
| import org.apache.flink.streaming.api.datastream.DataStream; |
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; |
| import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; |
| import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; |
| import org.apache.flink.streaming.api.functions.sink.SinkFunction; |
| import org.apache.flink.streaming.api.functions.source.SourceFunction; |
| import org.apache.flink.training.exercises.common.datatypes.RideAndFare; |
| import org.apache.flink.training.exercises.common.datatypes.TaxiFare; |
| import org.apache.flink.training.exercises.common.datatypes.TaxiRide; |
| import org.apache.flink.training.exercises.common.sources.TaxiFareGenerator; |
| import org.apache.flink.training.exercises.common.sources.TaxiRideGenerator; |
| import org.apache.flink.util.Collector; |
| |
| /** |
| * Java reference implementation for the Stateful Enrichment exercise from the Flink training. |
| * |
| * <p>The goal for this exercise is to enrich TaxiRides with fare information. |
| */ |
| public class RidesAndFaresSolution { |
| |
| private final SourceFunction<TaxiRide> rideSource; |
| private final SourceFunction<TaxiFare> fareSource; |
| private final SinkFunction<RideAndFare> sink; |
| |
| /** Creates a job using the sources and sink provided. */ |
| public RidesAndFaresSolution( |
| SourceFunction<TaxiRide> rideSource, |
| SourceFunction<TaxiFare> fareSource, |
| SinkFunction<RideAndFare> sink) { |
| |
| this.rideSource = rideSource; |
| this.fareSource = fareSource; |
| this.sink = sink; |
| } |
| |
| /** |
| * Creates and executes the pipeline using the StreamExecutionEnvironment provided. |
| * |
| * @throws Exception which occurs during job execution. |
| * @param env The {StreamExecutionEnvironment}. |
| * @return {JobExecutionResult} |
| */ |
| public JobExecutionResult execute(StreamExecutionEnvironment env) throws Exception { |
| |
| // A stream of taxi ride START events, keyed by rideId. |
| DataStream<TaxiRide> rides = |
| env.addSource(rideSource).filter(ride -> ride.isStart).keyBy(ride -> ride.rideId); |
| |
| // A stream of taxi fare events, also keyed by rideId. |
| DataStream<TaxiFare> fares = env.addSource(fareSource).keyBy(fare -> fare.rideId); |
| |
| // Create the pipeline. |
| rides.connect(fares) |
| .flatMap(new EnrichmentFunction()) |
| .uid("enrichment") // uid for this operator's state |
| .name("enrichment") // name for this operator in the web UI |
| .addSink(sink); |
| |
| // Execute the pipeline and return the result. |
| return env.execute("Join Rides with Fares"); |
| } |
| |
| /** Creates and executes the pipeline using the default StreamExecutionEnvironment. */ |
| public JobExecutionResult execute() throws Exception { |
| StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); |
| |
| return execute(env); |
| } |
| |
| /** |
| * Main method. |
| * |
| * @throws Exception which occurs during job execution. |
| */ |
| public static void main(String[] args) throws Exception { |
| |
| RidesAndFaresSolution job = |
| new RidesAndFaresSolution( |
| new TaxiRideGenerator(), |
| new TaxiFareGenerator(), |
| new PrintSinkFunction<>()); |
| |
| // Setting up checkpointing so that the state can be explored with the State Processor API. |
| // Generally it's better to separate configuration settings from the code, |
| // but for this example it's convenient to have it here for running in the IDE. |
| Configuration conf = new Configuration(); |
| conf.setString("state.backend", "filesystem"); |
| conf.setString("state.checkpoints.dir", "file:///tmp/checkpoints"); |
| conf.setString("execution.checkpointing.interval", "10s"); |
| conf.setString( |
| "execution.checkpointing.externalized-checkpoint-retention", |
| "RETAIN_ON_CANCELLATION"); |
| |
| StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(conf); |
| |
| job.execute(env); |
| } |
| |
| public static class EnrichmentFunction |
| extends RichCoFlatMapFunction<TaxiRide, TaxiFare, RideAndFare> { |
| |
| private ValueState<TaxiRide> rideState; |
| private ValueState<TaxiFare> fareState; |
| |
| @Override |
| public void open(Configuration config) { |
| |
| rideState = |
| getRuntimeContext() |
| .getState(new ValueStateDescriptor<>("saved ride", TaxiRide.class)); |
| fareState = |
| getRuntimeContext() |
| .getState(new ValueStateDescriptor<>("saved fare", TaxiFare.class)); |
| } |
| |
| @Override |
| public void flatMap1(TaxiRide ride, Collector<RideAndFare> out) throws Exception { |
| |
| TaxiFare fare = fareState.value(); |
| if (fare != null) { |
| fareState.clear(); |
| out.collect(new RideAndFare(ride, fare)); |
| } else { |
| rideState.update(ride); |
| } |
| } |
| |
| @Override |
| public void flatMap2(TaxiFare fare, Collector<RideAndFare> out) throws Exception { |
| |
| TaxiRide ride = rideState.value(); |
| if (ride != null) { |
| rideState.clear(); |
| out.collect(new RideAndFare(ride, fare)); |
| } else { |
| fareState.update(fare); |
| } |
| } |
| } |
| } |