-
Notifications
You must be signed in to change notification settings - Fork 13.2k
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
[FLINK-4460] Allow ProcessFunction on non-keyed streams #3438
Changes from all commits
82eddca
0228676
e12f320
06740fb
746c1ef
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,30 +19,37 @@ | |
package org.apache.flink.streaming.api.functions; | ||
|
||
import org.apache.flink.annotation.PublicEvolving; | ||
import org.apache.flink.api.common.functions.Function; | ||
import org.apache.flink.api.common.functions.AbstractRichFunction; | ||
import org.apache.flink.streaming.api.TimeDomain; | ||
import org.apache.flink.streaming.api.TimerService; | ||
import org.apache.flink.util.Collector; | ||
|
||
/** | ||
* A function that processes elements of a stream. | ||
* | ||
* <p>The function will be called for every element in the input stream and can produce | ||
* zero or more output. The function can also query the time and set timers. When | ||
* reacting to the firing of set timers the function can emit yet more elements. | ||
* <p>For every element in the input stream {@link #processElement(Object, Context, Collector)} | ||
* is invoked. This can produce zero or more elements as output. Implementations can also | ||
* query the time and set timers through the provided {@link Context}. For firing timers | ||
* {@link #onTimer(long, OnTimerContext, Collector)} will be invoked. This can again produce | ||
* zero or more elements as output and register further timers. | ||
* | ||
* <p>The function will be called for every element in the input stream and can produce | ||
* zero or more output elements. Contrary to the | ||
* {@link org.apache.flink.api.common.functions.FlatMapFunction}, this function can also query | ||
* the time (both event and processing) and set timers, through the provided {@link Context}. | ||
* When reacting to the firing of set timers the function can directly emit a result, and/or | ||
* register a timer that will trigger an action in the future. | ||
* <p><b>NOTE:</b> Access to keyed state and timers (which are also scoped to a key) is only | ||
* available if the {@code ProcessFunction} is applied on a {@code KeyedStream}. | ||
* | ||
* <p><b>NOTE:</b> A {@code ProcessFunction} is always a | ||
* {@link org.apache.flink.api.common.functions.RichFunction}. Therefore, access to the | ||
* {@link org.apache.flink.api.common.functions.RuntimeContext} is always available and setup and | ||
* teardown methods can be implemented. See | ||
* {@link org.apache.flink.api.common.functions.RichFunction#open(org.apache.flink.configuration.Configuration)} | ||
* and {@link org.apache.flink.api.common.functions.RichFunction#close()}. | ||
* | ||
* @param <I> Type of the input elements. | ||
* @param <O> Type of the output elements. | ||
*/ | ||
@PublicEvolving | ||
public interface ProcessFunction<I, O> extends Function { | ||
public abstract class ProcessFunction<I, O> extends AbstractRichFunction { | ||
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. hi, changing form 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. I think the problem is that we need a default implementation for 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. hi @wenlong88 in the ML discussion (https://lists.apache.org/thread.html/f3fe7d68986877994ad6b66173f40e72fc454420720a74ea5a834cc2@%3Cdev.flink.apache.org%3E) we decided to make 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. Missing serialVersionUID |
||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
/** | ||
* Process one element from the input stream. | ||
|
@@ -59,7 +66,7 @@ public interface ProcessFunction<I, O> extends Function { | |
* @throws Exception This method may throw exceptions. Throwing an exception will cause the operation | ||
* to fail and may trigger recovery. | ||
*/ | ||
void processElement(I value, Context ctx, Collector<O> out) throws Exception; | ||
public abstract void processElement(I value, Context ctx, Collector<O> out) throws Exception; | ||
|
||
/** | ||
* Called when a timer set using {@link TimerService} fires. | ||
|
@@ -74,36 +81,36 @@ public interface ProcessFunction<I, O> extends Function { | |
* @throws Exception This method may throw exceptions. Throwing an exception will cause the operation | ||
* to fail and may trigger recovery. | ||
*/ | ||
void onTimer(long timestamp, OnTimerContext ctx, Collector<O> out) throws Exception ; | ||
public void onTimer(long timestamp, OnTimerContext ctx, Collector<O> out) throws Exception {} | ||
|
||
/** | ||
* Information available in an invocation of {@link #processElement(Object, Context, Collector)} | ||
* or {@link #onTimer(long, OnTimerContext, Collector)}. | ||
*/ | ||
interface Context { | ||
public abstract class Context { | ||
|
||
/** | ||
* Timestamp of the element currently being processed or timestamp of a firing timer. | ||
* | ||
* <p>This might be {@code null}, for example if the time characteristic of your program | ||
* is set to {@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime}. | ||
*/ | ||
Long timestamp(); | ||
public abstract Long timestamp(); | ||
|
||
/** | ||
* A {@link TimerService} for querying time and registering timers. | ||
*/ | ||
TimerService timerService(); | ||
public abstract TimerService timerService(); | ||
} | ||
|
||
/** | ||
* Information available in an invocation of {@link #onTimer(long, OnTimerContext, Collector)}. | ||
*/ | ||
interface OnTimerContext extends Context { | ||
public abstract class OnTimerContext extends Context { | ||
/** | ||
* The {@link TimeDomain} of the firing timer. | ||
*/ | ||
TimeDomain timeDomain(); | ||
public abstract TimeDomain timeDomain(); | ||
} | ||
|
||
} |
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.
Is this internal method only exposed as
public
for the Scala API? If yes, I'm wondering if it makes sense to calltransform
manually in the ScalaDataStream
API.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.
Yes, it's exposed for that. The pattern, so far, is for methods to also expose a public method that takes a
TypeInformation
because we get theTypeInformation
from the context bound in the Scala API.Calling
transform()
manually is an option but if we do that we would basically not base the Scala API on the Java API anymore and we would have code that instantiates the Stream Operators in both the Java and Scala API. For example, right now we have the code for instantiating a flat map operator in(Java)DataStream
while(Scala)DataStream.flatMap()
calls that method.What do you think?
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.
Makes sense to keep it like that. The benefits to base the Scala API on top of the Java API instead of duplicating it are very persuasive, too. 😄 So +1 to keep it as is. 👍 I was just wondering whether users would be confused by this.