Permalink
Switch branches/tags
Nothing to show
Find file Copy path
Fetching contributors…
Cannot retrieve contributors at this time
119 lines (98 sloc) 4.43 KB
/*
* Copyright 2017 data Artisans GmbH
*
* Licensed 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 com.dataartisans.flinktraining.solutions.datastream_scala.process
import java.util.concurrent.TimeUnit
import com.dataartisans.flinktraining.exercises.datastream_java.datatypes.TaxiRide
import com.dataartisans.flinktraining.exercises.datastream_java.sources.CheckpointedTaxiRideSource
import com.dataartisans.flinktraining.exercises.datastream_java.utils.ExerciseBase._
import com.dataartisans.flinktraining.exercises.datastream_java.utils.{ExerciseBase, GeoUtils}
import org.apache.flink.api.common.restartstrategy.RestartStrategies
import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
import org.apache.flink.api.common.time.Time
import org.apache.flink.api.java.utils.ParameterTool
import org.apache.flink.runtime.state.filesystem.FsStateBackend
import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.functions.KeyedProcessFunction
import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, _}
import org.apache.flink.util.Collector
/**
* Scala reference implementation for the "Long Ride Alerts" exercise of the Flink training
* (http://training.data-artisans.com).
*
* The goal for this exercise is to emit START events for taxi rides that have not been matched
* by an END event during the first 2 hours of the ride.
*
* This version is setup for checkpointing and fault recovery.
*
* Parameters:
* -input path-to-input-file
*
*/
object CheckpointedLongRidesSolution {
def main(args: Array[String]) {
// parse parameters
val params = ParameterTool.fromArgs(args)
val input = params.get("input", ExerciseBase.pathToRideData)
val speed = 1800 // events of 30 minutes are served every second
// set up the execution environment
val env = StreamExecutionEnvironment.getExecutionEnvironment
// operate in Event-time
env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
env.setParallelism(ExerciseBase.parallelism)
// set up checkpointing
env.setStateBackend(new FsStateBackend("file:///tmp/checkpoints"))
env.enableCheckpointing(1000)
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(60, Time.of(10, TimeUnit.SECONDS)))
val rides = env.addSource(rideSourceOrTest(new CheckpointedTaxiRideSource(input, speed)))
val longRides = rides
// remove all rides which are not within NYC
.filter { r => GeoUtils.isInNYC(r.startLon, r.startLat) && GeoUtils.isInNYC(r.endLon, r.endLat) }
.keyBy(_.rideId)
.process(new MatchFunction())
printOrTest(longRides)
env.execute("Long Taxi Rides (checkpointed)")
}
class MatchFunction extends KeyedProcessFunction[Long, TaxiRide, TaxiRide] {
// keyed, managed state
// holds an END event if the ride has ended, otherwise a START event
lazy val rideState: ValueState[TaxiRide] = getRuntimeContext.getState(
new ValueStateDescriptor[TaxiRide]("saved ride", classOf[TaxiRide]))
override def processElement(ride: TaxiRide,
context: KeyedProcessFunction[Long, TaxiRide, TaxiRide]#Context,
out: Collector[TaxiRide]): Unit = {
val timerService = context.timerService
if (ride.isStart) {
// the matching END might have arrived first (out of order); don't overwrite it
if (rideState.value() == null) {
rideState.update(ride)
}
}
else {
rideState.update(ride)
}
timerService.registerEventTimeTimer(ride.getEventTime + 120 * 60 * 1000)
}
override def onTimer(timestamp: Long,
ctx: KeyedProcessFunction[Long, TaxiRide, TaxiRide]#OnTimerContext,
out: Collector[TaxiRide]): Unit = {
val savedRide = rideState.value
if (savedRide != null && savedRide.isStart) {
out.collect(savedRide)
}
rideState.clear()
}
}
}