-
Notifications
You must be signed in to change notification settings - Fork 8
Refactor flink job and add support to run chained groupBys #1247
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
18 commits
Select commit
Hold shift + click to select a range
0d0a6ef
Refactor existing Flink job code to make way to kick off join src runner
piyush-zlai 3e11855
Update SparkExprEval to take query objects instead of GBs
piyush-zlai 654ead7
Update FlinkJob to pass correct query
piyush-zlai 710fc56
Basic first cut Flink Join src runner job
piyush-zlai f52ba21
Remove one ts
piyush-zlai ae3c326
Minor cleanups
piyush-zlai 3b521af
Fix parallelism
piyush-zlai 0a25c9d
Add planner support for chaining
piyush-zlai e70e1bb
style: Apply scalafix and scalafmt changes
piyush-zlai b2056b6
Streamline redundant code
piyush-zlai 779bd44
Fix join request name
piyush-zlai 12932fc
Add metrics
piyush-zlai d59681e
Refactor common code to TableDeps
piyush-zlai f6a8f1e
style: Apply scalafix and scalafmt changes
piyush-zlai 5cd11a5
Make data model mandatory
piyush-zlai f1b3cad
Fix futures and error handling of join enrichment fn
piyush-zlai 47e727f
Dont poison pill on join source query sql errors
piyush-zlai 167c36b
Address comments
piyush-zlai File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
219 changes: 219 additions & 0 deletions
219
flink/src/main/scala/ai/chronon/flink/FlinkGroupByStreamingJob.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,219 @@ | ||
| package ai.chronon.flink | ||
|
|
||
| import ai.chronon.aggregator.windowing.ResolutionUtils | ||
| import ai.chronon.api.Extensions.{GroupByOps, SourceOps} | ||
| import ai.chronon.api.DataType | ||
| import ai.chronon.flink.FlinkJob.watermarkStrategy | ||
| import ai.chronon.flink.deser.ProjectedEvent | ||
| import ai.chronon.flink.source.FlinkSource | ||
| import ai.chronon.flink.types.{AvroCodecOutput, TimestampedTile, WriteResponse} | ||
| import ai.chronon.flink.window.{ | ||
| AlwaysFireOnElementTrigger, | ||
| BufferedProcessingTimeTrigger, | ||
| FlinkRowAggProcessFunction, | ||
| FlinkRowAggregationFunction, | ||
| KeySelectorBuilder | ||
| } | ||
| import ai.chronon.online.{GroupByServingInfoParsed, TopicInfo} | ||
| import org.apache.flink.streaming.api.datastream.{DataStream, SingleOutputStreamOperator} | ||
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment | ||
| import org.apache.flink.streaming.api.functions.async.RichAsyncFunction | ||
| import org.apache.flink.streaming.api.windowing.assigners.{TumblingEventTimeWindows, WindowAssigner} | ||
| import org.apache.flink.streaming.api.windowing.time.Time | ||
| import org.apache.flink.streaming.api.windowing.triggers.Trigger | ||
| import org.apache.flink.streaming.api.windowing.windows.TimeWindow | ||
| import org.apache.flink.util.OutputTag | ||
|
|
||
| import scala.collection.Seq | ||
|
|
||
| /** Flink job that processes a single streaming GroupBy and writes out the results (in the form of pre-aggregated tiles) to the KV store. | ||
| * | ||
| * @param eventSrc - Provider of a Flink Datastream[ ProjectedEvent ] for the given topic and groupBy. The event | ||
| * consists of a field Map as well as metadata columns such as processing start time (to track | ||
| * metrics). The Map contains projected columns from the source data based on projections and filters | ||
| * in the GroupBy. | ||
| * @param sinkFn - Async Flink writer function to help us write to the KV store | ||
| * @param groupByServingInfoParsed - The GroupBy we are working with | ||
| * @param parallelism - Parallelism to use for the Flink job | ||
| * @param enableDebug - If enabled will log additional debug info per processed event | ||
| */ | ||
| class FlinkGroupByStreamingJob(eventSrc: FlinkSource[ProjectedEvent], | ||
| inputSchema: Seq[(String, DataType)], | ||
| sinkFn: RichAsyncFunction[AvroCodecOutput, WriteResponse], | ||
| val groupByServingInfoParsed: GroupByServingInfoParsed, | ||
| parallelism: Int, | ||
| props: Map[String, String], | ||
| topicInfo: TopicInfo, | ||
| enableDebug: Boolean = false) | ||
| extends BaseFlinkJob { | ||
|
|
||
| val groupByName: String = groupByServingInfoParsed.groupBy.getMetaData.getName | ||
| logger.info(f"Creating Flink GroupBy streaming job. groupByName=${groupByName}") | ||
|
|
||
| if (groupByServingInfoParsed.groupBy.streamingSource.isEmpty) { | ||
| throw new IllegalArgumentException( | ||
| s"Invalid groupBy: $groupByName. No streaming source" | ||
| ) | ||
| } | ||
|
|
||
| private val kvStoreCapacity = FlinkUtils | ||
| .getProperty("kv_concurrency", props, topicInfo) | ||
| .map(_.toInt) | ||
| .getOrElse(AsyncKVStoreWriter.kvStoreConcurrency) | ||
|
|
||
| // The source of our Flink application is a topic | ||
| val topic: String = groupByServingInfoParsed.groupBy.streamingSource.get.topic | ||
|
|
||
| /** The "untiled" version of the Flink app. | ||
| * | ||
| * At a high level, the operators are structured as follows: | ||
| * source -> Spark expression eval -> Avro conversion -> KV store writer | ||
| * source - Reads objects of type T (specific case class, Thrift / Proto) from a topic | ||
| * Spark expression eval - Evaluates the Spark SQL expression in the GroupBy and projects and filters the input data | ||
| * Avro conversion - Converts the Spark expr eval output to a form that can be written out to the KV store | ||
| * (PutRequest object) | ||
| * KV store writer - Writes the PutRequest objects to the KV store using the AsyncDataStream API | ||
| * | ||
| * In this untiled version, there are no shuffles and thus this ends up being a single node in the Flink DAG | ||
| * (with the above 4 operators and parallelism as injected by the user). | ||
| */ | ||
| def runGroupByJob(env: StreamExecutionEnvironment): DataStream[WriteResponse] = { | ||
|
|
||
| logger.info( | ||
| f"Running Flink job for groupByName=${groupByName}, Topic=${topic}. " + | ||
| "Tiling is disabled.") | ||
|
|
||
| // we expect parallelism on the source stream to be set by the source provider | ||
| val sourceSparkProjectedStream: DataStream[ProjectedEvent] = | ||
| eventSrc | ||
| .getDataStream(topic, groupByName)(env, parallelism) | ||
| .uid(s"source-$groupByName") | ||
| .name(s"Source for $groupByName") | ||
|
|
||
| val sparkExprEvalDSWithWatermarks: DataStream[ProjectedEvent] = sourceSparkProjectedStream | ||
| .assignTimestampsAndWatermarks(watermarkStrategy) | ||
| .uid(s"spark-expr-eval-timestamps-$groupByName") | ||
| .name(s"Spark expression eval with timestamps for $groupByName") | ||
| .setParallelism(sourceSparkProjectedStream.getParallelism) | ||
|
|
||
| val putRecordDS: DataStream[AvroCodecOutput] = sparkExprEvalDSWithWatermarks | ||
| .flatMap(AvroCodecFn(groupByServingInfoParsed)) | ||
| .uid(s"avro-conversion-$groupByName") | ||
| .name(s"Avro conversion for $groupByName") | ||
| .setParallelism(sourceSparkProjectedStream.getParallelism) | ||
|
|
||
| AsyncKVStoreWriter.withUnorderedWaits( | ||
| putRecordDS, | ||
| sinkFn, | ||
| groupByName, | ||
| capacity = kvStoreCapacity | ||
| ) | ||
| } | ||
|
|
||
| /** The "tiled" version of the Flink app. | ||
| * | ||
| * The operators are structured as follows: | ||
| * 1. source - Reads objects of type T (specific case class, Thrift / Proto) from a topic | ||
| * 2. Spark expression eval - Evaluates the Spark SQL expression in the GroupBy and projects and filters the input | ||
| * data | ||
| * 3. Window/tiling - This window aggregates incoming events, keeps track of the IRs, and sends them forward so | ||
| * they are written out to the KV store | ||
| * 4. Avro conversion - Finishes converting the output of the window (the IRs) to a form that can be written out | ||
| * to the KV store (PutRequest object) | ||
| * 5. KV store writer - Writes the PutRequest objects to the KV store using the AsyncDataStream API | ||
| * | ||
| * The window causes a split in the Flink DAG, so there are two nodes, (1+2) and (3+4+5). | ||
| */ | ||
| override def runTiledGroupByJob(env: StreamExecutionEnvironment): DataStream[WriteResponse] = { | ||
| logger.info( | ||
| f"Running Flink job for groupByName=${groupByName}, Topic=${topic}. " + | ||
| "Tiling is enabled.") | ||
|
|
||
| val tilingWindowSizeInMillis: Long = | ||
| ResolutionUtils.getSmallestTailHopMillis(groupByServingInfoParsed.groupBy) | ||
|
|
||
| // we expect parallelism on the source stream to be set by the source provider | ||
| val sourceSparkProjectedStream: DataStream[ProjectedEvent] = | ||
| eventSrc | ||
| .getDataStream(topic, groupByName)(env, parallelism) | ||
| .uid(s"source-$groupByName") | ||
| .name(s"Source for $groupByName") | ||
|
|
||
| val sparkExprEvalDSAndWatermarks: DataStream[ProjectedEvent] = sourceSparkProjectedStream | ||
| .assignTimestampsAndWatermarks(watermarkStrategy) | ||
| .uid(s"spark-expr-eval-timestamps-$groupByName") | ||
| .name(s"Spark expression eval with timestamps for $groupByName") | ||
| .setParallelism(sourceSparkProjectedStream.getParallelism) | ||
|
|
||
| val window = TumblingEventTimeWindows | ||
| .of(Time.milliseconds(tilingWindowSizeInMillis)) | ||
| .asInstanceOf[WindowAssigner[ProjectedEvent, TimeWindow]] | ||
|
|
||
| // We default to the AlwaysFireOnElementTrigger which will cause the window to "FIRE" on every element. | ||
| // An alternative is the BufferedProcessingTimeTrigger (trigger=buffered in topic info | ||
| // or properties) which will buffer writes and only "FIRE" every X milliseconds per GroupBy & key. | ||
| val trigger = getTrigger() | ||
|
|
||
| // We use Flink "Side Outputs" to track any late events that aren't computed. | ||
| val tilingLateEventsTag = new OutputTag[ProjectedEvent]("tiling-late-events") {} | ||
|
|
||
| // The tiling operator works the following way: | ||
| // 1. Input: Spark expression eval (previous operator) | ||
| // 2. Key by the entity key(s) defined in the groupby | ||
| // 3. Window by a tumbling window | ||
| // 4. Use our custom trigger that will "FIRE" on every element | ||
| // 5. the AggregationFunction merges each incoming element with the current IRs which are kept in state | ||
| // - Each time a "FIRE" is triggered (i.e. on every event), getResult() is called and the current IRs are emitted | ||
| // 6. A process window function does additional processing each time the AggregationFunction emits results | ||
| // - The only purpose of this window function is to mark tiles as closed so we can do client-side caching in SFS | ||
| // 7. Output: TimestampedTile, containing the current IRs (Avro encoded) and the timestamp of the current element | ||
|
|
||
| val tilingDS: SingleOutputStreamOperator[TimestampedTile] = | ||
| sparkExprEvalDSAndWatermarks | ||
| .keyBy(KeySelectorBuilder.build(groupByServingInfoParsed.groupBy)) | ||
| .window(window) | ||
| .trigger(trigger) | ||
| .sideOutputLateData(tilingLateEventsTag) | ||
| .aggregate( | ||
| // See Flink's "ProcessWindowFunction with Incremental Aggregation" | ||
| new FlinkRowAggregationFunction(groupByServingInfoParsed.groupBy, inputSchema, enableDebug), | ||
| new FlinkRowAggProcessFunction(groupByServingInfoParsed.groupBy, inputSchema, enableDebug) | ||
| ) | ||
| .uid(s"tiling-01-$groupByName") | ||
| .name(s"Tiling for $groupByName") | ||
| .setParallelism(sourceSparkProjectedStream.getParallelism) | ||
|
|
||
| // Track late events | ||
| tilingDS | ||
| .getSideOutput(tilingLateEventsTag) | ||
| .flatMap(new LateEventCounter(groupByName)) | ||
| .uid(s"tiling-side-output-01-$groupByName") | ||
| .name(s"Tiling Side Output Late Data for $groupByName") | ||
| .setParallelism(sourceSparkProjectedStream.getParallelism) | ||
|
|
||
| val putRecordDS: DataStream[AvroCodecOutput] = tilingDS | ||
| .flatMap(TiledAvroCodecFn(groupByServingInfoParsed, tilingWindowSizeInMillis, enableDebug)) | ||
| .uid(s"avro-conversion-01-$groupByName") | ||
| .name(s"Avro conversion for $groupByName") | ||
| .setParallelism(sourceSparkProjectedStream.getParallelism) | ||
|
|
||
| AsyncKVStoreWriter.withUnorderedWaits( | ||
| putRecordDS, | ||
| sinkFn, | ||
| groupByName, | ||
| capacity = kvStoreCapacity | ||
| ) | ||
| } | ||
|
|
||
| private def getTrigger(): Trigger[ProjectedEvent, TimeWindow] = { | ||
| FlinkUtils | ||
| .getProperty("trigger", props, topicInfo) | ||
| .map { | ||
| case "always_fire" => new AlwaysFireOnElementTrigger() | ||
| case "buffered" => new BufferedProcessingTimeTrigger(100L) | ||
| case t => | ||
| throw new IllegalArgumentException(s"Unsupported trigger type: $t. Supported: 'always_fire', 'buffered'") | ||
| } | ||
| .getOrElse(new AlwaysFireOnElementTrigger()) | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.