Github user wenlong88 commented on a diff in the pull request: https://github.com/apache/flink/pull/3438#discussion_r104334699 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/ProcessFunction.java --- @@ -19,30 +19,35 @@ 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} as 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 { --- End diff -- hi, changing form `interface` to `class` is incompatible on the user side. Can't ProcessFunction just extend RichFunction?
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---