Skip to content

Commit

Permalink
[FLINK-36926][table] Introduce window join operator with async state api
Browse files Browse the repository at this point in the history
This closes #25815
  • Loading branch information
xuyangzhong authored Jan 14, 2025
1 parent 29edd60 commit 8ada0ba
Show file tree
Hide file tree
Showing 13 changed files with 1,062 additions and 494 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.runtime.state.v2.adaptor;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.api.common.state.InternalCheckpointListener;
import org.apache.flink.api.common.state.v2.State;
Expand Down Expand Up @@ -197,4 +198,9 @@ public boolean requiresLegacySynchronousTimerSnapshots(SnapshotType checkpointTy
public boolean isSafeToReuseKVState() {
return keyedStateBackend.isSafeToReuseKVState();
}

@VisibleForTesting
public CheckpointableKeyedStateBackend<K> getKeyedStateBackend() {
return keyedStateBackend;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,10 @@ public int numKeyedStateEntries() {
if (keyedStateBackend instanceof HeapKeyedStateBackend) {
return ((HeapKeyedStateBackend) keyedStateBackend).numKeyValueStateEntries();
} else {
throw new UnsupportedOperationException();
throw new UnsupportedOperationException(
String.format(
"Unsupported keyed state backend: %s",
keyedStateBackend.getClass().getCanonicalName()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,13 @@

package org.apache.flink.streaming.util.asyncprocessing;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.runtime.asyncprocessing.operators.AbstractAsyncStateStreamOperator;
import org.apache.flink.runtime.state.AsyncKeyedStateBackend;
import org.apache.flink.runtime.state.KeyedStateBackend;
import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
import org.apache.flink.runtime.state.v2.adaptor.AsyncKeyedStateBackendAdaptor;
import org.apache.flink.streaming.api.operators.BoundedMultiInput;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
Expand All @@ -31,7 +34,7 @@
import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.function.FunctionWithException;
import org.apache.flink.util.function.RunnableWithException;
Expand All @@ -53,7 +56,7 @@
* async processing, please use methods of test harness instead of operator.
*/
public class AsyncKeyedTwoInputStreamOperatorTestHarness<K, IN1, IN2, OUT>
extends TwoInputStreamOperatorTestHarness<IN1, IN2, OUT> {
extends KeyedTwoInputStreamOperatorTestHarness<K, IN1, IN2, OUT> {

private final TwoInputStreamOperator<IN1, IN2, OUT> twoInputOperator;

Expand Down Expand Up @@ -128,15 +131,14 @@ public AsyncKeyedTwoInputStreamOperatorTestHarness(
int numSubtasks,
int subtaskIndex)
throws Exception {
super(operator, maxParallelism, numSubtasks, subtaskIndex);

ClosureCleaner.clean(keySelector1, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, false);
ClosureCleaner.clean(keySelector2, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, false);
config.setStatePartitioner(0, keySelector1);
config.setStatePartitioner(1, keySelector2);
config.setStateKeySerializer(
keyType.createSerializer(executionConfig.getSerializerConfig()));
config.serializeAllConfigs();
super(
operator,
keySelector1,
keySelector2,
keyType,
maxParallelism,
numSubtasks,
subtaskIndex);

Preconditions.checkState(
operator instanceof AsyncStateProcessingOperator,
Expand Down Expand Up @@ -239,6 +241,34 @@ public void close() throws Exception {
executor.shutdown();
}

@Override
public int numKeyedStateEntries() {
AbstractAsyncStateStreamOperator<OUT> asyncOp =
(AbstractAsyncStateStreamOperator<OUT>) operator;
AsyncKeyedStateBackend<Object> asyncKeyedStateBackend = asyncOp.getAsyncKeyedStateBackend();
KeyedStateBackend<?> keyedStateBackend;
if (asyncKeyedStateBackend instanceof AsyncKeyedStateBackendAdaptor) {
keyedStateBackend =
((AsyncKeyedStateBackendAdaptor<?>) asyncKeyedStateBackend)
.getKeyedStateBackend();

} else {
throw new UnsupportedOperationException(
String.format(
"Unsupported async keyed state backend: %s",
asyncKeyedStateBackend.getClass().getCanonicalName()));
}

if (keyedStateBackend instanceof HeapKeyedStateBackend) {
return ((HeapKeyedStateBackend) keyedStateBackend).numKeyValueStateEntries();
} else {
throw new UnsupportedOperationException(
String.format(
"Unsupported keyed state backend: %s",
keyedStateBackend.getClass().getCanonicalName()));
}
}

private void executeAndGet(RunnableWithException runnable) throws Exception {
try {
execute(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,10 @@
import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.api.config.ExecutionConfigOptions;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.logical.WindowAttachedWindowingStrategy;
Expand All @@ -42,7 +44,6 @@
import org.apache.flink.table.planner.utils.TableConfigUtils;
import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
import org.apache.flink.table.runtime.operators.join.window.WindowJoinOperator;
import org.apache.flink.table.runtime.operators.join.window.WindowJoinOperatorBuilder;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.runtime.util.TimeWindowUtil;
Expand Down Expand Up @@ -172,7 +173,7 @@ protected Transformation<RowData> translateToPlanInternal(
TimeWindowUtil.getShiftTimeZone(
leftWindowing.getTimeAttributeType(),
TableConfigUtils.getLocalTimeZone(config));
WindowJoinOperator operator =
WindowJoinOperatorBuilder operatorBuilder =
WindowJoinOperatorBuilder.builder()
.leftSerializer(leftTypeInfo.toRowSerializer())
.rightSerializer(rightTypeInfo.toRowSerializer())
Expand All @@ -181,8 +182,12 @@ protected Transformation<RowData> translateToPlanInternal(
.rightWindowEndIndex(rightWindowEndIndex)
.filterNullKeys(joinSpec.getFilterNulls())
.joinType(joinSpec.getJoinType())
.withShiftTimezone(shiftTimeZone)
.build();
.withShiftTimezone(shiftTimeZone);
if (config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_STATE_ENABLED)) {
operatorBuilder.enableAsyncState();
}

TwoInputStreamOperator<RowData, RowData, RowData> operator = operatorBuilder.build();

final RowType returnType = (RowType) getOutputType();
final TwoInputTransformation<RowData, RowData, RowData> transform =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.runtime.stream.sql
import org.apache.flink.configuration.{Configuration, RestartStrategyOptions}
import org.apache.flink.core.execution.CheckpointingMode
import org.apache.flink.table.api.bridge.scala._
import org.apache.flink.table.api.config.ExecutionConfigOptions
import org.apache.flink.table.planner.factories.TestValuesTableFactory
import org.apache.flink.table.planner.runtime.utils.{FailingCollectionSource, StreamingWithStateTestBase, TestData, TestingAppendSink}
import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode}
Expand All @@ -35,7 +36,7 @@ import java.util
import scala.collection.JavaConversions._

@ExtendWith(Array(classOf[ParameterizedTestExtension]))
class WindowJoinITCase(mode: StateBackendMode, useTimestampLtz: Boolean)
class WindowJoinITCase(mode: StateBackendMode, useTimestampLtz: Boolean, enableAsyncState: Boolean)
extends StreamingWithStateTestBase(mode) {

val SHANGHAI_ZONE = ZoneId.of("Asia/Shanghai")
Expand All @@ -55,6 +56,10 @@ class WindowJoinITCase(mode: StateBackendMode, useTimestampLtz: Boolean)
env.configure(configuration, Thread.currentThread.getContextClassLoader)
FailingCollectionSource.reset()

tEnv.getConfig.set(
ExecutionConfigOptions.TABLE_EXEC_ASYNC_STATE_ENABLED,
Boolean.box(enableAsyncState))

val dataId1 = TestValuesTableFactory.registerData(TestData.windowDataWithTimestamp)
val dataIdWithLtz = TestValuesTableFactory.registerData(TestData.windowDataWithLtzInShanghai)
tEnv.executeSql(
Expand Down Expand Up @@ -1159,13 +1164,15 @@ class WindowJoinITCase(mode: StateBackendMode, useTimestampLtz: Boolean)

object WindowJoinITCase {

@Parameters(name = "StateBackend={0}, UseTimestampLtz = {1}")
@Parameters(name = "StateBackend={0}, UseTimestampLtz = {1}, EnableAsyncState = {2}")
def parameters(): util.Collection[Array[java.lang.Object]] = {
Seq[Array[AnyRef]](
Array(HEAP_BACKEND, java.lang.Boolean.TRUE),
Array(HEAP_BACKEND, java.lang.Boolean.FALSE),
Array(ROCKSDB_BACKEND, java.lang.Boolean.TRUE),
Array(ROCKSDB_BACKEND, java.lang.Boolean.FALSE)
Array(HEAP_BACKEND, java.lang.Boolean.TRUE, Boolean.box(false)),
Array(HEAP_BACKEND, java.lang.Boolean.FALSE, Boolean.box(false)),
Array(HEAP_BACKEND, java.lang.Boolean.TRUE, Boolean.box(true)),
Array(HEAP_BACKEND, java.lang.Boolean.FALSE, Boolean.box(true)),
Array(ROCKSDB_BACKEND, java.lang.Boolean.TRUE, Boolean.box(false)),
Array(ROCKSDB_BACKEND, java.lang.Boolean.FALSE, Boolean.box(false))
)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,122 @@
/*
* 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.table.runtime.operators;

import org.apache.flink.runtime.asyncprocessing.operators.AbstractAsyncStateStreamOperator;
import org.apache.flink.streaming.api.TimerService;
import org.apache.flink.streaming.api.operators.InternalTimerService;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;

import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;

/**
* Table operator to invoke close always. This is a base class for both batch and stream operators
* without key.
*
* <p>This class is nearly identical with {@link TableStreamOperator}, but extending from {@link
* AbstractAsyncStateStreamOperator} to integrate with asynchronous state access.
*/
public abstract class AsyncStateTableStreamOperator<OUT>
extends AbstractAsyncStateStreamOperator<OUT> {

/** We listen to this ourselves because we don't have an {@link InternalTimerService}. */
protected long currentWatermark = Long.MIN_VALUE;

protected transient ContextImpl ctx;

@Override
public void open() throws Exception {
super.open();
this.ctx = new ContextImpl(getProcessingTimeService());
}

@Override
public boolean useSplittableTimers() {
return true;
}

@Override
public Watermark preProcessWatermark(Watermark mark) throws Exception {
currentWatermark = mark.getTimestamp();
return super.preProcessWatermark(mark);
}

/** Information available in an invocation of processElement. */
protected class ContextImpl implements TimerService {

protected final ProcessingTimeService timerService;

public StreamRecord<?> element;

ContextImpl(ProcessingTimeService timerService) {
this.timerService = checkNotNull(timerService);
}

public Long timestamp() {
checkState(element != null);

if (element.hasTimestamp()) {
return element.getTimestamp();
} else {
return null;
}
}

@Override
public long currentProcessingTime() {
return timerService.getCurrentProcessingTime();
}

@Override
public long currentWatermark() {
return currentWatermark;
}

@Override
public void registerProcessingTimeTimer(long time) {
throw new UnsupportedOperationException(
"Setting timers is only supported on a keyed streams.");
}

@Override
public void registerEventTimeTimer(long time) {
throw new UnsupportedOperationException(
"Setting timers is only supported on a keyed streams.");
}

@Override
public void deleteProcessingTimeTimer(long time) {
throw new UnsupportedOperationException(
"Delete timers is only supported on a keyed streams.");
}

@Override
public void deleteEventTimeTimer(long time) {
throw new UnsupportedOperationException(
"Delete timers is only supported on a keyed streams.");
}

public TimerService timerService() {
return this;
}
}
}
Loading

0 comments on commit 8ada0ba

Please sign in to comment.