-
Notifications
You must be signed in to change notification settings - Fork 13.4k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-36881][table] Introduce GroupTableAggFunction in GroupTableAgg…
…regate with Async State API
- Loading branch information
Showing
9 changed files
with
547 additions
and
46 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
196 changes: 196 additions & 0 deletions
196
.../apache/flink/streaming/api/operators/asyncprocessing/AsyncStateKeyedProcessOperator.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,196 @@ | ||
/* | ||
* 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.streaming.api.operators.asyncprocessing; | ||
|
||
import org.apache.flink.annotation.Internal; | ||
import org.apache.flink.runtime.asyncprocessing.operators.AbstractAsyncStateUdfStreamOperator; | ||
import org.apache.flink.runtime.state.VoidNamespace; | ||
import org.apache.flink.runtime.state.VoidNamespaceSerializer; | ||
import org.apache.flink.streaming.api.SimpleTimerService; | ||
import org.apache.flink.streaming.api.TimeDomain; | ||
import org.apache.flink.streaming.api.TimerService; | ||
import org.apache.flink.streaming.api.functions.KeyedProcessFunction; | ||
import org.apache.flink.streaming.api.operators.InternalTimer; | ||
import org.apache.flink.streaming.api.operators.InternalTimerService; | ||
import org.apache.flink.streaming.api.operators.KeyedProcessOperator; | ||
import org.apache.flink.streaming.api.operators.OneInputStreamOperator; | ||
import org.apache.flink.streaming.api.operators.StreamOperator; | ||
import org.apache.flink.streaming.api.operators.TimestampedCollector; | ||
import org.apache.flink.streaming.api.operators.Triggerable; | ||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; | ||
import org.apache.flink.util.OutputTag; | ||
|
||
import static org.apache.flink.util.Preconditions.checkNotNull; | ||
import static org.apache.flink.util.Preconditions.checkState; | ||
|
||
/** | ||
* A {@link StreamOperator} for executing {@link KeyedProcessFunction KeyedProcessFunctions}. | ||
* | ||
* <p>This class is nearly identical with {@link KeyedProcessOperator}, but extending from {@link | ||
* AbstractAsyncStateUdfStreamOperator} to integrate with asynchronous state access. Another | ||
* difference is this class is internal. | ||
*/ | ||
@Internal | ||
public class AsyncStateKeyedProcessOperator<K, IN, OUT> | ||
extends AbstractAsyncStateUdfStreamOperator<OUT, KeyedProcessFunction<K, IN, OUT>> | ||
implements OneInputStreamOperator<IN, OUT>, Triggerable<K, VoidNamespace> { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
private transient TimestampedCollector<OUT> collector; | ||
|
||
private transient ContextImpl context; | ||
|
||
private transient OnTimerContextImpl onTimerContext; | ||
|
||
public AsyncStateKeyedProcessOperator(KeyedProcessFunction<K, IN, OUT> function) { | ||
super(function); | ||
} | ||
|
||
@Override | ||
public void open() throws Exception { | ||
super.open(); | ||
collector = new TimestampedCollector<>(output); | ||
|
||
InternalTimerService<VoidNamespace> internalTimerService = | ||
getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this); | ||
|
||
TimerService timerService = new SimpleTimerService(internalTimerService); | ||
|
||
context = new ContextImpl(userFunction, timerService); | ||
onTimerContext = new OnTimerContextImpl(userFunction, timerService); | ||
} | ||
|
||
@Override | ||
public void onEventTime(InternalTimer<K, VoidNamespace> timer) throws Exception { | ||
collector.setAbsoluteTimestamp(timer.getTimestamp()); | ||
invokeUserFunction(TimeDomain.EVENT_TIME, timer); | ||
} | ||
|
||
@Override | ||
public void onProcessingTime(InternalTimer<K, VoidNamespace> timer) throws Exception { | ||
collector.eraseTimestamp(); | ||
invokeUserFunction(TimeDomain.PROCESSING_TIME, timer); | ||
} | ||
|
||
@Override | ||
public void processElement(StreamRecord<IN> element) throws Exception { | ||
collector.setTimestamp(element); | ||
context.element = element; | ||
userFunction.processElement(element.getValue(), context, collector); | ||
context.element = null; | ||
} | ||
|
||
private void invokeUserFunction(TimeDomain timeDomain, InternalTimer<K, VoidNamespace> timer) | ||
throws Exception { | ||
onTimerContext.timeDomain = timeDomain; | ||
onTimerContext.timer = timer; | ||
userFunction.onTimer(timer.getTimestamp(), onTimerContext, collector); | ||
onTimerContext.timeDomain = null; | ||
onTimerContext.timer = null; | ||
} | ||
|
||
private class ContextImpl extends KeyedProcessFunction<K, IN, OUT>.Context { | ||
|
||
private final TimerService timerService; | ||
|
||
private StreamRecord<IN> element; | ||
|
||
ContextImpl(KeyedProcessFunction<K, IN, OUT> function, TimerService timerService) { | ||
function.super(); | ||
this.timerService = checkNotNull(timerService); | ||
} | ||
|
||
@Override | ||
public Long timestamp() { | ||
checkState(element != null); | ||
|
||
if (element.hasTimestamp()) { | ||
return element.getTimestamp(); | ||
} else { | ||
return null; | ||
} | ||
} | ||
|
||
@Override | ||
public TimerService timerService() { | ||
return timerService; | ||
} | ||
|
||
@Override | ||
public <X> void output(OutputTag<X> outputTag, X value) { | ||
if (outputTag == null) { | ||
throw new IllegalArgumentException("OutputTag must not be null."); | ||
} | ||
|
||
output.collect(outputTag, new StreamRecord<>(value, element.getTimestamp())); | ||
} | ||
|
||
@Override | ||
@SuppressWarnings("unchecked") | ||
public K getCurrentKey() { | ||
return (K) AsyncStateKeyedProcessOperator.this.getCurrentKey(); | ||
} | ||
} | ||
|
||
private class OnTimerContextImpl extends KeyedProcessFunction<K, IN, OUT>.OnTimerContext { | ||
|
||
private final TimerService timerService; | ||
|
||
private TimeDomain timeDomain; | ||
|
||
private InternalTimer<K, VoidNamespace> timer; | ||
|
||
OnTimerContextImpl(KeyedProcessFunction<K, IN, OUT> function, TimerService timerService) { | ||
function.super(); | ||
this.timerService = checkNotNull(timerService); | ||
} | ||
|
||
@Override | ||
public Long timestamp() { | ||
checkState(timer != null); | ||
return timer.getTimestamp(); | ||
} | ||
|
||
@Override | ||
public TimerService timerService() { | ||
return timerService; | ||
} | ||
|
||
@Override | ||
public <X> void output(OutputTag<X> outputTag, X value) { | ||
if (outputTag == null) { | ||
throw new IllegalArgumentException("OutputTag must not be null."); | ||
} | ||
|
||
output.collect(outputTag, new StreamRecord<>(value, timer.getTimestamp())); | ||
} | ||
|
||
@Override | ||
public TimeDomain timeDomain() { | ||
checkState(timeDomain != null); | ||
return timeDomain; | ||
} | ||
|
||
@Override | ||
public K getCurrentKey() { | ||
return timer.getKey(); | ||
} | ||
} | ||
} |
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
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
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
Oops, something went wrong.