-
Notifications
You must be signed in to change notification settings - Fork 6
Add an example for ProcessTableFunctions #31
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
Open
Martijn Visser (MartijnVisser)
wants to merge
2
commits into
confluentinc:master
Choose a base branch
from
MartijnVisser:ptf-examples
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+118
−0
Open
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
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
111 changes: 111 additions & 0 deletions
111
src/main/java/io/confluent/flink/examples/table/Example_11_ProcessTableFunction.java
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,111 @@ | ||
| package io.confluent.flink.examples.table; | ||
|
|
||
| import io.confluent.flink.plugin.ConfluentSettings; | ||
|
|
||
| import org.apache.flink.table.annotation.ArgumentHint; | ||
| import org.apache.flink.table.annotation.StateHint; | ||
| import org.apache.flink.table.api.EnvironmentSettings; | ||
| import org.apache.flink.table.api.TableEnvironment; | ||
| import org.apache.flink.table.functions.ProcessTableFunction; | ||
| import org.apache.flink.types.Row; | ||
|
|
||
| import java.time.Duration; | ||
| import java.time.Instant; | ||
|
|
||
| import static org.apache.flink.table.annotation.ArgumentTrait.REQUIRE_ON_TIME; | ||
| import static org.apache.flink.table.annotation.ArgumentTrait.SET_SEMANTIC_TABLE; | ||
| import static org.apache.flink.table.api.Expressions.$; | ||
| import static org.apache.flink.table.api.Expressions.descriptor; | ||
| import static org.apache.flink.table.api.Expressions.lit; | ||
|
|
||
| /** | ||
| * A table program example illustrating how to use a {@link ProcessTableFunction} (PTF) in the Flink | ||
| * Table API. | ||
| * | ||
| * <p>This example detects inactive users on the {@code examples.marketplace.clicks} table. For each | ||
| * user, it counts clicks and registers an event-time timer. When no new click arrives within the | ||
| * configured timeout, the timer fires and emits an alert with the user's accumulated click count. | ||
| * | ||
| * <p>Unlike a windowed aggregation, the PTF emits exactly once per inactivity period, combining | ||
| * running state with the absence of events. | ||
| */ | ||
| public class Example_11_ProcessTableFunction { | ||
|
|
||
| // Fill this with an environment you have write access to | ||
| static final String TARGET_CATALOG = ""; | ||
|
|
||
| // Fill this with a Kafka cluster you have write access to | ||
| static final String TARGET_DATABASE = ""; | ||
|
|
||
| // All logic is defined in a main() method. It can run both in an IDE or CI/CD system. | ||
| public static void main(String[] args) { | ||
| // Setup connection properties to Confluent Cloud | ||
| EnvironmentSettings settings = ConfluentSettings.fromResource("/cloud.properties"); | ||
|
|
||
| // Initialize the session context to get started | ||
| TableEnvironment env = TableEnvironment.create(settings); | ||
|
|
||
| // Set default catalog and database | ||
| env.useCatalog(TARGET_CATALOG); | ||
| env.useDatabase(TARGET_DATABASE); | ||
|
|
||
| // Invoke the PTF inline against the clicks table, partitioned by user_id. | ||
| // The function's lifecycle is bound to this query. | ||
| System.out.println("Executing inline ProcessTableFunction..."); | ||
| env.from("`examples`.`marketplace`.`clicks`") | ||
| .partitionBy($("user_id")) | ||
| .process( | ||
| ClickInactivityMonitor.class, | ||
| lit(30).asArgument("timeoutSeconds"), | ||
| descriptor("$rowtime").asArgument("on_time")) | ||
| .execute() | ||
| .print(); | ||
| } | ||
|
|
||
| /** | ||
| * A ProcessTableFunction that detects user inactivity based on click events. | ||
| * | ||
| * <p>For each user (partitioned by {@code user_id}), it counts incoming clicks and registers a | ||
| * named event-time timer. Each new click replaces the previous timer, resetting the inactivity | ||
| * clock. When the timer fires (no new clicks within the timeout), an alert is emitted and the | ||
| * partition's state is cleared so a returning user starts a fresh inactivity window. | ||
| */ | ||
| public static class ClickInactivityMonitor | ||
| extends ProcessTableFunction<ClickInactivityMonitor.InactivityAlert> { | ||
|
|
||
| /** Output POJO. The framework adds the user_id partition key and rowtime automatically. */ | ||
| public static class InactivityAlert { | ||
| public int clickCount; | ||
| } | ||
|
|
||
| /** Per-user state. */ | ||
| public static class ClickState { | ||
| public int clickCount = 0; | ||
| } | ||
|
|
||
| public void eval( | ||
| Context ctx, | ||
| @StateHint ClickState state, | ||
| @ArgumentHint({SET_SEMANTIC_TABLE, REQUIRE_ON_TIME}) Row input, | ||
| Integer timeoutSeconds) { | ||
|
|
||
| state.clickCount++; | ||
|
|
||
| // Each new click pushes the timeout forward; the timer fires only after true | ||
| // inactivity. | ||
| TimeContext<Instant> timeCtx = ctx.timeContext(Instant.class); | ||
| timeCtx.registerOnTime( | ||
| "inactivity", timeCtx.time().plus(Duration.ofSeconds(timeoutSeconds))); | ||
| } | ||
|
|
||
| public void onTimer(OnTimerContext ctx, ClickState state) { | ||
| InactivityAlert alert = new InactivityAlert(); | ||
| alert.clickCount = state.clickCount; | ||
| collect(alert); | ||
|
|
||
| // Reset the partition: drop the click counter (and any leftover timers) so a returning | ||
| // user starts a new inactivity window instead of retaining state indefinitely. | ||
| ctx.clearAll(); | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Timo Walther (@twalthr) I would appreciate your thoughts if we should use |
||
| } | ||
| } | ||
| } | ||
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't see any signs of state cleanup. Our examples should be designed in a state efficient way.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Totally missed it, thnx. Fixed