forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-37112][runtime] Process event time extension relted watermarks…
… in EventTimeProcessFunction for DataStream V2
- Loading branch information
Showing
24 changed files
with
2,218 additions
and
4 deletions.
There are no files selected for viewing
This file contains 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
40 changes: 40 additions & 0 deletions
40
...rg/apache/flink/datastream/api/extension/eventtime/function/EventTimeProcessFunction.java
This file contains 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,40 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you 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 org.apache.flink.datastream.api.extension.eventtime.function; | ||
|
||
import org.apache.flink.annotation.Experimental; | ||
import org.apache.flink.datastream.api.extension.eventtime.timer.EventTimeManager; | ||
import org.apache.flink.datastream.api.function.ProcessFunction; | ||
import org.apache.flink.datastream.api.stream.KeyedPartitionStream; | ||
|
||
/** | ||
* The base interface for event time processing, indicating that the {@link ProcessFunction} will be | ||
* enriched with event time processing functions, such as registering event timers and handle event | ||
* time watermarks. | ||
* | ||
* <p>Note that registering event timers can only be used with {@link KeyedPartitionStream}. | ||
*/ | ||
@Experimental | ||
public interface EventTimeProcessFunction extends ProcessFunction { | ||
/** | ||
* Initialize the {@link EventTimeProcessFunction} with an instance of {@link EventTimeManager}. | ||
* Note that this method should be invoked before the open method. | ||
*/ | ||
void initEventTimeProcessFunction(EventTimeManager eventTimeManager); | ||
} |
47 changes: 47 additions & 0 deletions
47
...k/datastream/api/extension/eventtime/function/OneInputEventTimeStreamProcessFunction.java
This file contains 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,47 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you 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 org.apache.flink.datastream.api.extension.eventtime.function; | ||
|
||
import org.apache.flink.annotation.Experimental; | ||
import org.apache.flink.datastream.api.common.Collector; | ||
import org.apache.flink.datastream.api.context.NonPartitionedContext; | ||
import org.apache.flink.datastream.api.context.PartitionedContext; | ||
import org.apache.flink.datastream.api.function.OneInputStreamProcessFunction; | ||
import org.apache.flink.datastream.api.stream.KeyedPartitionStream; | ||
|
||
/** The {@link OneInputStreamProcessFunction} that extends with event time support. */ | ||
@Experimental | ||
public interface OneInputEventTimeStreamProcessFunction<IN, OUT> | ||
extends EventTimeProcessFunction, OneInputStreamProcessFunction<IN, OUT> { | ||
|
||
/** | ||
* The {@code #onEventTimeWatermark} method signifies that the EventTimeProcessFunction has | ||
* received an EventTimeWatermark. Other types of watermarks will be processed by the {@code | ||
* ProcessFunction#onWatermark} method. | ||
*/ | ||
default void onEventTimeWatermark( | ||
long watermarkTimestamp, Collector<OUT> output, NonPartitionedContext<OUT> ctx) | ||
throws Exception {} | ||
|
||
/** | ||
* Invoked when an event-time timer fires. Note that it is only used in {@link | ||
* KeyedPartitionStream}. | ||
*/ | ||
default void onEventTimer(long timestamp, Collector<OUT> output, PartitionedContext<OUT> ctx) {} | ||
} |
47 changes: 47 additions & 0 deletions
47
...eam/api/extension/eventtime/function/TwoInputBroadcastEventTimeStreamProcessFunction.java
This file contains 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,47 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you 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 org.apache.flink.datastream.api.extension.eventtime.function; | ||
|
||
import org.apache.flink.annotation.Experimental; | ||
import org.apache.flink.datastream.api.common.Collector; | ||
import org.apache.flink.datastream.api.context.NonPartitionedContext; | ||
import org.apache.flink.datastream.api.context.PartitionedContext; | ||
import org.apache.flink.datastream.api.function.TwoInputBroadcastStreamProcessFunction; | ||
import org.apache.flink.datastream.api.stream.KeyedPartitionStream; | ||
|
||
/** The {@link TwoInputBroadcastStreamProcessFunction} that extends with event time support. */ | ||
@Experimental | ||
public interface TwoInputBroadcastEventTimeStreamProcessFunction<IN1, IN2, OUT> | ||
extends EventTimeProcessFunction, TwoInputBroadcastStreamProcessFunction<IN1, IN2, OUT> { | ||
|
||
/** | ||
* The {@code #onEventTimeWatermark} method signifies that the EventTimeProcessFunction has | ||
* received an EventTimeWatermark. Other types of watermarks will be processed by the {@code | ||
* ProcessFunction#onWatermark} method. | ||
*/ | ||
default void onEventTimeWatermark( | ||
long watermarkTimestamp, Collector<OUT> output, NonPartitionedContext<OUT> ctx) | ||
throws Exception {} | ||
|
||
/** | ||
* Invoked when an event-time timer fires. Note that it is only used in {@link | ||
* KeyedPartitionStream}. | ||
*/ | ||
default void onEventTimer(long timestamp, Collector<OUT> output, PartitionedContext<OUT> ctx) {} | ||
} |
47 changes: 47 additions & 0 deletions
47
.../api/extension/eventtime/function/TwoInputNonBroadcastEventTimeStreamProcessFunction.java
This file contains 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,47 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you 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 org.apache.flink.datastream.api.extension.eventtime.function; | ||
|
||
import org.apache.flink.annotation.Experimental; | ||
import org.apache.flink.datastream.api.common.Collector; | ||
import org.apache.flink.datastream.api.context.NonPartitionedContext; | ||
import org.apache.flink.datastream.api.context.PartitionedContext; | ||
import org.apache.flink.datastream.api.function.TwoInputNonBroadcastStreamProcessFunction; | ||
import org.apache.flink.datastream.api.stream.KeyedPartitionStream; | ||
|
||
/** The {@link TwoInputNonBroadcastStreamProcessFunction} that extends with event time support. */ | ||
@Experimental | ||
public interface TwoInputNonBroadcastEventTimeStreamProcessFunction<IN1, IN2, OUT> | ||
extends EventTimeProcessFunction, TwoInputNonBroadcastStreamProcessFunction<IN1, IN2, OUT> { | ||
|
||
/** | ||
* The {@code #onEventTimeWatermark} method signifies that the EventTimeProcessFunction has | ||
* received an EventTimeWatermark. Other types of watermarks will be processed by the {@code | ||
* ProcessFunction#onWatermark} method. | ||
*/ | ||
default void onEventTimeWatermark( | ||
long watermarkTimestamp, Collector<OUT> output, NonPartitionedContext<OUT> ctx) | ||
throws Exception {} | ||
|
||
/** | ||
* Invoked when an event-time timer fires. Note that it is only used in {@link | ||
* KeyedPartitionStream}. | ||
*/ | ||
default void onEventTimer(long timestamp, Collector<OUT> output, PartitionedContext<OUT> ctx) {} | ||
} |
54 changes: 54 additions & 0 deletions
54
.../datastream/api/extension/eventtime/function/TwoOutputEventTimeStreamProcessFunction.java
This file contains 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,54 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you 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 org.apache.flink.datastream.api.extension.eventtime.function; | ||
|
||
import org.apache.flink.annotation.Experimental; | ||
import org.apache.flink.datastream.api.common.Collector; | ||
import org.apache.flink.datastream.api.context.TwoOutputNonPartitionedContext; | ||
import org.apache.flink.datastream.api.context.TwoOutputPartitionedContext; | ||
import org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction; | ||
import org.apache.flink.datastream.api.stream.KeyedPartitionStream; | ||
|
||
/** The {@link TwoOutputStreamProcessFunction} that extends with event time support. */ | ||
@Experimental | ||
public interface TwoOutputEventTimeStreamProcessFunction<IN, OUT1, OUT2> | ||
extends EventTimeProcessFunction, TwoOutputStreamProcessFunction<IN, OUT1, OUT2> { | ||
|
||
/** | ||
* The {@code #onEventTimeWatermark} method signifies that the EventTimeProcessFunction has | ||
* received an EventTimeWatermark. Other types of watermarks will be processed by the {@code | ||
* ProcessFunction#onWatermark} method. | ||
*/ | ||
default void onEventTimeWatermark( | ||
long watermarkTimestamp, | ||
Collector<OUT1> output1, | ||
Collector<OUT2> output2, | ||
TwoOutputNonPartitionedContext<OUT1, OUT2> ctx) | ||
throws Exception {} | ||
|
||
/** | ||
* Invoked when an event-time timer fires. Note that it is only used in {@link | ||
* KeyedPartitionStream}. | ||
*/ | ||
default void onEventTimer( | ||
long timestamp, | ||
Collector<OUT1> output1, | ||
Collector<OUT2> output2, | ||
TwoOutputPartitionedContext<OUT1, OUT2> ctx) {} | ||
} |
Oops, something went wrong.