diff --git a/docs/layouts/shortcodes/generated/execution_config_configuration.html b/docs/layouts/shortcodes/generated/execution_config_configuration.html index 87b89981bc51a..861f0a5fb630b 100644 --- a/docs/layouts/shortcodes/generated/execution_config_configuration.html +++ b/docs/layouts/shortcodes/generated/execution_config_configuration.html @@ -245,6 +245,24 @@

Enum

Because of the disorder of ChangeLog data caused by Shuffle in distributed system, the data received by Sink may not be the order of global upsert. So add upsert materialize operator before upsert sink. It receives the upstream changelog records and generate an upsert view for the downstream.
By default, the materialize operator will be added when a distributed disorder occurs on unique keys. You can also choose no materialization(NONE) or force materialization(FORCE).

Possible values: + +
table.exec.sink.upsert-materialize.adaptive.threshold.high

Streaming + (none) + Integer + When using strategy=ADAPTIVE, defines the number of entries per key when the implementation is changed from VALUE to MAP. If not specified, Flink uses state-backend specific defaults.
The option takes effect during job (re)starting
+ + +
table.exec.sink.upsert-materialize.adaptive.threshold.low

Streaming + (none) + Integer + When using strategy=ADAPTIVE, defines the number of entries per key when the implementation is changed from MAP to VALUE. If not specified, Flink uses state-backend specific defaults.
The option takes effect during job (re)starting
+ + +
table.exec.sink.upsert-materialize.strategy

Streaming + (none) +

Enum

+ Which strategy of SinkUpsertMaterializer to use. Supported strategies:
LEGACY: Simple implementation based on ValueState<List> (the original implementation).
MAP: OrderedMultiSetState-based implementation based on a combination of several MapState maintaining ordering and fast lookup properties.
VALUE: Similar to LEGACY, but compatible with MAP and therefore allows to switch to ADAPTIVE.
ADAPTIVE: Alternate between MAP and VALUE depending on the number of entries for the given key starting with VALUE and switching to MAP upon reaching threshold.high value (and back to VALUE, when reaching low).
The default is LEGACY
The option takes effect during planning / compile plan generation. Existing jobs won't be affected by this option.


Possible values: +
table.exec.sort.async-merge-enabled

Batch true diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java b/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java index 69c513c388a54..b1833c37919bd 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java @@ -338,4 +338,13 @@ org.apache.flink.api.common.state.v2.AggregatingState getAggregatingSta org.apache.flink.api.common.state.v2.AggregatingStateDescriptor< IN, ACC, OUT> stateProperties); + + /** + * @return fixed lower-case string identifying the type of the underlying state backend, e.g. + * rocksdb, hashmap, or unknown. + */ + @Experimental + default String getBackendTypeIdentifier() { + return "unknown"; + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeSerializer.java index dbcfa7367634a..ea07b604c7d19 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeSerializer.java @@ -228,8 +228,15 @@ private PrecomputedParameters( this.stateful = stateful; } - static PrecomputedParameters precompute( + public static PrecomputedParameters precompute( boolean immutableTargetType, TypeSerializer[] fieldSerializers) { + return precompute(immutableTargetType, false, fieldSerializers); + } + + public static PrecomputedParameters precompute( + boolean immutableTargetType, + boolean forceFieldsImmutable, + TypeSerializer[] fieldSerializers) { Preconditions.checkNotNull(fieldSerializers); int totalLength = 0; boolean fieldsImmutable = true; @@ -239,7 +246,7 @@ static PrecomputedParameters precompute( if (fieldSerializer != fieldSerializer.duplicate()) { stateful = true; } - if (!fieldSerializer.isImmutableType()) { + if (!forceFieldsImmutable && !fieldSerializer.isImmutableType()) { fieldsImmutable = false; } if (fieldSerializer.getLength() < 0) { diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/SnapshotUtils.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/SnapshotUtils.java index a3ecabe715f62..14688e4ee742d 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/SnapshotUtils.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/SnapshotUtils.java @@ -72,7 +72,7 @@ public static > TaggedOperatorSubtaskState s operator.snapshotState(checkpointId, timestamp, options, storage); OperatorSubtaskState state = - new OperatorSnapshotFinalizer(snapshotInProgress).getJobManagerOwnedState(); + OperatorSnapshotFinalizer.create(snapshotInProgress).getJobManagerOwnedState(); operator.notifyCheckpointComplete(checkpointId); return new TaggedOperatorSubtaskState(index, state); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AsyncKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AsyncKeyedStateBackend.java index 21fdf4b6e2f35..f65b0a2b4e885 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AsyncKeyedStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AsyncKeyedStateBackend.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.state; +import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.state.InternalCheckpointListener; import org.apache.flink.api.common.state.v2.State; @@ -140,4 +141,13 @@ default boolean isSafeToReuseKVState() { @Override void dispose(); + + /** + * @return fixed lower-case string identifying the type of the underlying state backend, e.g. + * rocksdb, hashmap, or unknown. + */ + @Experimental + default String getBackendTypeIdentifier() { + return "unknown"; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultKeyedStateStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultKeyedStateStore.java index f43e6b8ad0a48..82cdb849d99ee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultKeyedStateStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultKeyedStateStore.java @@ -219,6 +219,15 @@ org.apache.flink.api.common.state.v2.AggregatingState getAggregatingSta } } + @Override + public String getBackendTypeIdentifier() { + if (keyedStateBackend != null) { + return keyedStateBackend.getBackendTypeIdentifier(); + } else { + return asyncKeyedStateBackend.getBackendTypeIdentifier(); + } + } + protected S getPartitionedState( org.apache.flink.api.common.state.v2.StateDescriptor stateDescriptor) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java index 19ef61b8ab524..632d3a6eb5979 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.state; +import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.state.State; import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -166,6 +167,15 @@ default boolean isSafeToReuseKVState() { return false; } + /** + * @return fixed lower-case string identifying the type of the underlying state backend, e.g. + * rocksdb, hashmap, or unknown. + */ + @Experimental + default String getBackendTypeIdentifier() { + return "unknown"; + } + /** Listener is given a callback when {@link #setCurrentKey} is called (key context changes). */ @FunctionalInterface interface KeySelectionListener { diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizer.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizer.java index c308a6cb8aeec..455c2489a741e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizer.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizer.java @@ -27,12 +27,11 @@ import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.util.concurrent.FutureUtils; -import javax.annotation.Nonnull; - import java.util.concurrent.ExecutionException; import static org.apache.flink.runtime.checkpoint.StateObjectCollection.emptyIfNull; import static org.apache.flink.runtime.checkpoint.StateObjectCollection.singletonOrEmpty; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * This class finalizes {@link OperatorSnapshotFutures}. Each object is created with a {@link @@ -47,8 +46,9 @@ public class OperatorSnapshotFinalizer { /** Secondary replica of the operator subtask state for faster, local recovery on TM. */ private final OperatorSubtaskState taskLocalState; - public OperatorSnapshotFinalizer(@Nonnull OperatorSnapshotFutures snapshotFutures) + public static OperatorSnapshotFinalizer create(OperatorSnapshotFutures snapshotFutures) throws ExecutionException, InterruptedException { + checkNotNull(snapshotFutures); SnapshotResult keyedManaged = FutureUtils.runIfNotDoneAndGet(snapshotFutures.getKeyedStateManagedFuture()); @@ -68,7 +68,7 @@ public OperatorSnapshotFinalizer(@Nonnull OperatorSnapshotFutures snapshotFuture SnapshotResult> resultSubpartition = snapshotFutures.getResultSubpartitionStateFuture().get(); - jobManagerOwnedState = + OperatorSubtaskState jobManagerOwnedState = OperatorSubtaskState.builder() .setManagedOperatorState( singletonOrEmpty(operatorManaged.getJobManagerOwnedSnapshot())) @@ -83,7 +83,7 @@ public OperatorSnapshotFinalizer(@Nonnull OperatorSnapshotFutures snapshotFuture emptyIfNull(resultSubpartition.getJobManagerOwnedSnapshot())) .build(); - taskLocalState = + OperatorSubtaskState taskLocalState = OperatorSubtaskState.builder() .setManagedOperatorState( singletonOrEmpty(operatorManaged.getTaskLocalSnapshot())) @@ -94,6 +94,14 @@ public OperatorSnapshotFinalizer(@Nonnull OperatorSnapshotFutures snapshotFuture .setResultSubpartitionState( emptyIfNull(resultSubpartition.getTaskLocalSnapshot())) .build(); + + return new OperatorSnapshotFinalizer(jobManagerOwnedState, taskLocalState); + } + + public OperatorSnapshotFinalizer( + OperatorSubtaskState jobManagerOwnedState, OperatorSubtaskState taskLocalState) { + this.jobManagerOwnedState = checkNotNull(jobManagerOwnedState); + this.taskLocalState = checkNotNull(taskLocalState); } public OperatorSubtaskState getTaskLocalState() { diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java index 2e859d2ab8d2c..ee674fc1c2ad9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java @@ -189,7 +189,7 @@ private SnapshotsFinalizeResult finalizeNonFinishedSnapshots() throws Exception // finalize the async part of all by executing all snapshot runnables OperatorSnapshotFinalizer finalizedSnapshots = - new OperatorSnapshotFinalizer(snapshotInProgress); + OperatorSnapshotFinalizer.create(snapshotInProgress); jobManagerTaskOperatorSubtaskStates.putSubtaskStateByOperatorID( operatorID, finalizedSnapshots.getJobManagerOwnedState()); diff --git a/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizerTest.java b/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizerTest.java index a64905098af06..17d6351e3655d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizerTest.java @@ -93,7 +93,7 @@ void testRunAndExtract() throws Exception { assertThat(f).isNotDone(); } - OperatorSnapshotFinalizer finalizer = new OperatorSnapshotFinalizer(snapshotFutures); + OperatorSnapshotFinalizer finalizer = OperatorSnapshotFinalizer.create(snapshotFutures); for (Future f : snapshotFutures.getAllFutures()) { assertThat(f).isDone(); diff --git a/flink-runtime/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-runtime/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java index 31a6019a50a6c..b72f7299f4b8f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java +++ b/flink-runtime/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java @@ -761,7 +761,7 @@ public OperatorSnapshotFinalizer snapshotWithLocalState( checkpointStorageAccess.resolveCheckpointStorageLocation( checkpointId, locationReference)); - return new OperatorSnapshotFinalizer(operatorStateResult); + return OperatorSnapshotFinalizer.create(operatorStateResult); } /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java index 1e0fb43ebfec0..b6d1bd7a43866 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java @@ -159,6 +159,66 @@ public class ExecutionConfigOptions { + "or force materialization(FORCE).") .build()); + @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) + public static final ConfigOption + TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_LOW = + key("table.exec.sink.upsert-materialize.adaptive.threshold.low") + .longType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "When using strategy=ADAPTIVE, defines the number of entries per key when the implementation is changed from MAP to VALUE. If not specified, Flink uses state-backend specific defaults.") + .linebreak() + .text("The option takes effect during job (re)starting") + .linebreak() + .build()); + + @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) + public static final ConfigOption + TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_HIGH = + key("table.exec.sink.upsert-materialize.adaptive.threshold.high") + .longType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "When using strategy=ADAPTIVE, defines the number of entries per key when the implementation is changed from VALUE to MAP. If not specified, Flink uses state-backend specific defaults.") + .linebreak() + .text("The option takes effect during job (re)starting") + .linebreak() + .build()); + + @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) + public static final ConfigOption + TABLE_EXEC_SINK_UPSERT_MATERIALIZE_STRATEGY = + key("table.exec.sink.upsert-materialize.strategy") + .enumType(SinkUpsertMaterializeStrategy.class) + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Which strategy of SinkUpsertMaterializer to use. Supported strategies:") + .linebreak() + .text( + "LEGACY: Simple implementation based on ValueState (the original implementation).") + .linebreak() + .text( + "MAP: OrderedMultiSetState-based implementation based on a combination of several MapState maintaining ordering and fast lookup properties.") + .linebreak() + .text( + "VALUE: Similar to LEGACY, but compatible with MAP and therefore allows to switch to ADAPTIVE.") + .linebreak() + .text( + "ADAPTIVE: Alternate between MAP and VALUE depending on the number of entries for the given key starting with VALUE and switching to MAP upon reaching threshold.high value (and back to VALUE, when reaching low).") + .linebreak() + .text("The default is LEGACY") + .linebreak() + .text( + "The option takes effect during planning / compile plan generation. Existing jobs won't be affected by this option.") + .linebreak() + .build()); + @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) public static final ConfigOption TABLE_EXEC_SINK_KEYED_SHUFFLE = key("table.exec.sink.keyed-shuffle") @@ -883,6 +943,43 @@ public enum RetryStrategy { FIXED_DELAY } + /** SinkUpsertMaterializer strategy. */ + @PublicEvolving + public enum SinkUpsertMaterializeStrategy { + /** + * Simple implementation based on ValueState (the original implementation). + * + *
    + *
  • optimal for cases with history under approx. 100 elements + *
  • limited TTL support (per key granularity, i.e. no expiration for old history + * elements) + *
+ */ + LEGACY, + /** + * OrderedMultiSetState-based implementation based on a combination of several MapState + * maintaining ordering and fast lookup properties. + * + *
    + *
  • faster and more memory-efficient on long histories + *
  • slower on short histories + *
  • currently, no TTL support (to be added in the future) + *
  • requires more space + *
+ */ + MAP, + /** + * Similar to LEGACY, but compatible with MAP and therefore allows to switch to ADAPTIVE. + */ + VALUE, + /** + * Alternate between MAP and VALUE depending on the number of entries for the given key + * starting with VALUE and switching to MAP upon reaching threshold.high value (and back to + * VALUE, when reaching low). + */ + ADAPTIVE + } + /** Determine if CAST operates using the legacy behaviour or the new one. */ @Deprecated public enum LegacyCastBehaviour implements DescribedEnum { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/ProjectedRowData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/ProjectedRowData.java index 0916ea6d37d51..dd99f2100e38b 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/ProjectedRowData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/ProjectedRowData.java @@ -45,11 +45,17 @@ public class ProjectedRowData implements RowData { private final int[] indexMapping; + private final boolean isNullAtNonProjected; private RowData row; private ProjectedRowData(int[] indexMapping) { + this(indexMapping, false); + } + + protected ProjectedRowData(int[] indexMapping, boolean isNullAtNonProjected) { this.indexMapping = indexMapping; + this.isNullAtNonProjected = isNullAtNonProjected; } /** @@ -82,7 +88,8 @@ public void setRowKind(RowKind kind) { @Override public boolean isNullAt(int pos) { - return row.isNullAt(indexMapping[pos]); + return (pos >= indexMapping.length && isNullAtNonProjected) + || row.isNullAt(indexMapping[pos]); } @Override @@ -186,6 +193,15 @@ public String toString() { + '}'; } + /** + * Returns a new {@link ProjectedRowData} that, depending on isNullAtNonProjected, returns null + * from {@link #isNullAt} if the index is out of range or throws {@link + * ArrayIndexOutOfBoundsException}. + */ + public ProjectedRowData withNullAtNonProjected(boolean isNullAtNonProjected) { + return new ProjectedRowData(this.indexMapping, isNullAtNonProjected); + } + /** * Like {@link #from(int[])}, but throws {@link IllegalArgumentException} if the provided {@code * projection} array contains nested projections, which are not supported by {@link diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/typeutils/RowTypeUtils.java similarity index 95% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/typeutils/RowTypeUtils.java index 4d9879d7b9980..d70c3ef1b98d9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/RowTypeUtils.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/typeutils/RowTypeUtils.java @@ -16,8 +16,9 @@ * limitations under the License. */ -package org.apache.flink.table.planner.typeutils; +package org.apache.flink.table.typeutils; +import org.apache.flink.annotation.Internal; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; @@ -28,7 +29,8 @@ import java.util.Collections; import java.util.List; -/** Utils for deriving row types of {@link org.apache.calcite.rel.RelNode}s. */ +/** Utils for deriving row types of org.apache.calcite.rel.RelNode. */ +@Internal public class RowTypeUtils { public static String getUniqueName(String oldName, List checklist) { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/typeutils/RowTypeUtilsTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/typeutils/RowTypeUtilsTest.java similarity index 98% rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/typeutils/RowTypeUtilsTest.java rename to flink-table/flink-table-common/src/test/java/org/apache/flink/table/typeutils/RowTypeUtilsTest.java index 143a21503e3d2..b1851c62aa514 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/typeutils/RowTypeUtilsTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/typeutils/RowTypeUtilsTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.typeutils; +package org.apache.flink.table.typeutils; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.types.logical.BigIntType; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/runtimefilter/BatchExecLocalRuntimeFilterBuilder.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/runtimefilter/BatchExecLocalRuntimeFilterBuilder.java index 4f8496cc15de8..fa242100a1cc1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/runtimefilter/BatchExecLocalRuntimeFilterBuilder.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/runtimefilter/BatchExecLocalRuntimeFilterBuilder.java @@ -34,12 +34,12 @@ import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecNode; import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; -import org.apache.flink.table.planner.typeutils.RowTypeUtils; import org.apache.flink.table.runtime.generated.GeneratedProjection; import org.apache.flink.table.runtime.operators.runtimefilter.LocalRuntimeFilterBuilderOperator; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.typeutils.RowTypeUtils; import java.util.List; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java index 8b6c5614ee0fb..5a97b86fe7c41 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java @@ -19,16 +19,21 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.dag.Transformation; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.config.ExecutionConfigOptions.RowtimeInserter; +import org.apache.flink.table.api.config.ExecutionConfigOptions.SinkUpsertMaterializeStrategy; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.EqualiserCodeGenerator; +import org.apache.flink.table.planner.codegen.HashCodeGenerator; import org.apache.flink.table.planner.connectors.CollectDynamicSink; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; @@ -42,16 +47,18 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec; import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; -import org.apache.flink.table.planner.typeutils.RowTypeUtils; +import org.apache.flink.table.runtime.generated.GeneratedHashFunction; import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializer; -import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializerV2; +import org.apache.flink.table.runtime.orderedmultisetstate.StateSettings; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.runtime.typeutils.TypeCheckUtils; import org.apache.flink.table.runtime.util.StateConfigUtil; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.typeutils.RowTypeUtils; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; @@ -63,7 +70,9 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; +import java.util.stream.IntStream; /** * Stream {@link ExecNode} to write data into an external sink defined by a {@link @@ -92,6 +101,7 @@ public class StreamExecSink extends CommonExecSink implements StreamExecNode stateMetadataList, @JsonProperty(FIELD_NAME_INPUT_UPSERT_KEY) int[] inputUpsertKey, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @@ -166,6 +184,7 @@ public StreamExecSink( this.upsertMaterialize = upsertMaterialize; this.inputUpsertKey = inputUpsertKey; this.stateMetadataList = stateMetadataList; + this.upsertMaterializeStrategy = sinkUpsertMaterializeStrategy; } @SuppressWarnings("unchecked") @@ -231,9 +250,11 @@ protected Transformation applyUpsertMaterialize( ClassLoader classLoader, RowType physicalRowType, int[] inputUpsertKey) { + final GeneratedRecordEqualiser rowEqualiser = new EqualiserCodeGenerator(physicalRowType, classLoader) .generateRecordEqualiser("SinkMaterializeEqualiser"); + final GeneratedRecordEqualiser upsertKeyEqualiser = inputUpsertKey == null ? null @@ -243,16 +264,37 @@ protected Transformation applyUpsertMaterialize( classLoader) .generateRecordEqualiser("SinkMaterializeUpsertKeyEqualiser"); - final long stateRetentionTime = - StateMetadata.getStateTtlForOneInputOperator(config, stateMetadataList); + GeneratedHashFunction rowHashFunction = + HashCodeGenerator.generateRowHash( + new CodeGeneratorContext(config, classLoader), + physicalRowType, + "hashCode", + IntStream.range(0, physicalRowType.getFieldCount()).toArray()); - SinkUpsertMaterializer operator = - new SinkUpsertMaterializer( - StateConfigUtil.createTtlConfig(stateRetentionTime), - InternalSerializers.create(physicalRowType), - rowEqualiser, + final GeneratedHashFunction upsertKeyHashFunction = + inputUpsertKey == null + ? null + : HashCodeGenerator.generateRowHash( + new CodeGeneratorContext(config, classLoader), + RowTypeUtils.projectRowType(physicalRowType, inputUpsertKey), + "", + IntStream.range(0, inputUpsertKey.length).toArray()); + + StateTtlConfig ttlConfig = + StateConfigUtil.createTtlConfig( + StateMetadata.getStateTtlForOneInputOperator(config, stateMetadataList)); + + final OneInputStreamOperator operator = + createSumOperator( + config, + physicalRowType, + inputUpsertKey, upsertKeyEqualiser, - inputUpsertKey); + upsertKeyHashFunction, + ttlConfig, + rowEqualiser, + rowHashFunction); + final String[] fieldNames = physicalRowType.getFieldNames().toArray(new String[0]); final List pkFieldNames = Arrays.stream(primaryKeys) @@ -280,4 +322,36 @@ protected Transformation applyUpsertMaterialize( materializeTransform.setStateKeyType(keySelector.getProducedType()); return materializeTransform; } + + private OneInputStreamOperator createSumOperator( + ExecNodeConfig config, + RowType physicalRowType, + int[] inputUpsertKey, + GeneratedRecordEqualiser upsertKeyEqualiser, + GeneratedHashFunction upsertKeyHashFunction, + StateTtlConfig ttlConfig, + GeneratedRecordEqualiser rowEqualiser, + GeneratedHashFunction rowHashFunction) { + + SinkUpsertMaterializeStrategy sinkUpsertMaterializeStrategy = + Optional.ofNullable(upsertMaterializeStrategy) + .orElse(SinkUpsertMaterializeStrategy.LEGACY); + + return sinkUpsertMaterializeStrategy == SinkUpsertMaterializeStrategy.LEGACY + ? SinkUpsertMaterializer.create( + ttlConfig, + physicalRowType, + rowEqualiser, + upsertKeyEqualiser, + inputUpsertKey) + : SinkUpsertMaterializerV2.create( + physicalRowType, + rowEqualiser, + upsertKeyEqualiser, + rowHashFunction, + upsertKeyHashFunction, + inputUpsertKey, + StateSettings.forStrategy( + sinkUpsertMaterializeStrategy, config, ttlConfig)); + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java index c55c872b224d9..e790df7c9e04c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/WatermarkAssignerChangelogNormalizeTransposeRule.java @@ -25,7 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner; import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution; import org.apache.flink.table.planner.plan.trait.FlinkRelDistributionTraitDef; -import org.apache.flink.table.planner.typeutils.RowTypeUtils; +import org.apache.flink.table.typeutils.RowTypeUtils; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala index f51fddc005a82..075f7343d73ca 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala @@ -26,12 +26,12 @@ import org.apache.flink.table.functions.DeclarativeAggregateFunction import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, CodeGenUtils, OperatorCodeGenerator, ProjectionCodeGenerator} import org.apache.flink.table.planner.codegen.CodeGenUtils.ROW_DATA import org.apache.flink.table.planner.plan.utils.AggregateInfoList -import org.apache.flink.table.planner.typeutils.RowTypeUtils import org.apache.flink.table.runtime.generated.GeneratedOperator import org.apache.flink.table.runtime.operators.TableStreamOperator import org.apache.flink.table.runtime.operators.aggregate.BytesHashMapSpillMemorySegmentPool import org.apache.flink.table.runtime.util.collections.binary.BytesMap import org.apache.flink.table.types.logical.RowType +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.calcite.tools.RelBuilder diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/SortAggCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/SortAggCodeGenerator.scala index dc2605bb9305e..f68c302239a5b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/SortAggCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/SortAggCodeGenerator.scala @@ -25,10 +25,10 @@ import org.apache.flink.table.functions.AggregateFunction import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, CodeGenUtils, ProjectionCodeGenerator} import org.apache.flink.table.planner.codegen.OperatorCodeGenerator.generateCollect import org.apache.flink.table.planner.plan.utils.AggregateInfoList -import org.apache.flink.table.planner.typeutils.RowTypeUtils import org.apache.flink.table.runtime.generated.GeneratedOperator import org.apache.flink.table.runtime.operators.TableStreamOperator import org.apache.flink.table.types.logical.RowType +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.calcite.tools.RelBuilder diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala index 41bbe7d2beed3..66ea53c2b1e7b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala @@ -38,7 +38,6 @@ import org.apache.flink.table.planner.expressions.ExpressionBuilder._ import org.apache.flink.table.planner.expressions.converter.ExpressionConverter import org.apache.flink.table.planner.plan.logical.{LogicalWindow, SlidingGroupWindow, TumblingGroupWindow} import org.apache.flink.table.planner.plan.utils.{AggregateInfo, AggregateInfoList, AggregateUtil} -import org.apache.flink.table.planner.typeutils.RowTypeUtils import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTypeFactory import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.flink.table.runtime.operators.window.TimeWindow @@ -46,6 +45,7 @@ import org.apache.flink.table.runtime.operators.window.grouping.{HeapWindowsGrou import org.apache.flink.table.runtime.util.RowIterator import org.apache.flink.table.types.logical._ import org.apache.flink.table.types.logical.LogicalTypeRoot.INTERVAL_DAY_TIME +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.flink.table.utils.DateTimeUtils import org.apache.calcite.rel.core.AggregateCall diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/runtimefilter/RuntimeFilterCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/runtimefilter/RuntimeFilterCodeGenerator.scala index 8d2fd41844699..a036aeb420c48 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/runtimefilter/RuntimeFilterCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/runtimefilter/RuntimeFilterCodeGenerator.scala @@ -20,11 +20,11 @@ package org.apache.flink.table.planner.codegen.runtimefilter import org.apache.flink.runtime.operators.util.BloomFilter import org.apache.flink.table.data.RowData import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, OperatorCodeGenerator, ProjectionCodeGenerator} -import org.apache.flink.table.planner.codegen.CodeGenUtils.{className, newName, DEFAULT_INPUT1_TERM, DEFAULT_INPUT2_TERM, ROW_DATA} +import org.apache.flink.table.planner.codegen.CodeGenUtils.{className, newName, DEFAULT_INPUT1_TERM, DEFAULT_INPUT2_TERM} import org.apache.flink.table.planner.codegen.OperatorCodeGenerator.{generateCollect, INPUT_SELECTION} -import org.apache.flink.table.planner.typeutils.RowTypeUtils import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory import org.apache.flink.table.types.logical.RowType +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.flink.util.Preconditions /** Operator code generator for runtime filter operator. */ diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/fusion/spec/HashAggFusionCodegenSpec.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/fusion/spec/HashAggFusionCodegenSpec.scala index 109ea1e61911d..a33c6b11f6506 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/fusion/spec/HashAggFusionCodegenSpec.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/fusion/spec/HashAggFusionCodegenSpec.scala @@ -26,17 +26,17 @@ import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, CodeGenUtil import org.apache.flink.table.planner.codegen.CodeGenUtils.{getReuseRowFieldExprs, newName, newNames} import org.apache.flink.table.planner.codegen.ProjectionCodeGenerator.genAdaptiveLocalHashAggValueProjectionExpr import org.apache.flink.table.planner.codegen.agg.batch.{AggCodeGenHelper, HashAggCodeGenHelper} -import org.apache.flink.table.planner.codegen.agg.batch.AggCodeGenHelper.{buildAggregateArgsMapping, genAggregateByFlatAggregateBuffer, genFlatAggBufferExprs, genGetValueFromFlatAggregateBuffer, genInitFlatAggregateBuffer} -import org.apache.flink.table.planner.codegen.agg.batch.HashAggCodeGenHelper.{buildAggregateAggBuffMapping, genAggregate, genCreateFallbackSorter, genHashAggValueExpr, genRetryAppendToMap, genReusableEmptyAggBuffer, prepareFallbackSorter} +import org.apache.flink.table.planner.codegen.agg.batch.AggCodeGenHelper._ +import org.apache.flink.table.planner.codegen.agg.batch.HashAggCodeGenHelper._ import org.apache.flink.table.planner.plan.fusion.{OpFusionCodegenSpecBase, OpFusionContext} import org.apache.flink.table.planner.plan.fusion.FusionCodegenUtil.{constructDoConsumeCode, constructDoConsumeFunction, evaluateVariables} import org.apache.flink.table.planner.plan.utils.AggregateInfoList -import org.apache.flink.table.planner.typeutils.RowTypeUtils import org.apache.flink.table.planner.utils.JavaScalaConversionUtil.{toJava, toScala} import org.apache.flink.table.runtime.operators.aggregate.BytesHashMapSpillMemorySegmentPool import org.apache.flink.table.runtime.util.KeyValueIterator import org.apache.flink.table.runtime.util.collections.binary.{BytesHashMap, BytesMap} import org.apache.flink.table.types.logical.{LogicalType, RowType} +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.calcite.tools.RelBuilder diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/fusion/spec/RuntimeFilterFusionCodegenSpec.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/fusion/spec/RuntimeFilterFusionCodegenSpec.scala index a7e075b62177b..0f049a844aab9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/fusion/spec/RuntimeFilterFusionCodegenSpec.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/fusion/spec/RuntimeFilterFusionCodegenSpec.scala @@ -22,8 +22,8 @@ import org.apache.flink.table.data.binary.BinaryRowData import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, GeneratedExpression} import org.apache.flink.table.planner.codegen.CodeGenUtils.{className, newName, newNames} import org.apache.flink.table.planner.plan.fusion.{OpFusionCodegenSpecBase, OpFusionContext} -import org.apache.flink.table.planner.typeutils.RowTypeUtils import org.apache.flink.table.types.logical.RowType +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.flink.util.Preconditions import java.util diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Rank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Rank.scala index a7a61f378aab4..671e340125652 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Rank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Rank.scala @@ -19,8 +19,8 @@ package org.apache.flink.table.planner.plan.nodes.calcite import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.plan.utils._ -import org.apache.flink.table.planner.typeutils.RowTypeUtils import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankRange, RankType, VariableRankRange} +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField, RelDataTypeFieldImpl} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala index 1ad00e2b397fb..02f7423de5f6a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.table.planner.plan.nodes.physical.stream +import org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE_STRATEGY import org.apache.flink.table.catalog.ContextResolvedTable import org.apache.flink.table.connector.sink.DynamicTableSink import org.apache.flink.table.planner.calcite.FlinkTypeFactory @@ -94,13 +95,17 @@ class StreamPhysicalSink( .reuseOrCreate(cluster.getMetadataQuery) .getUpsertKeys(inputRel) + val config = unwrapTableConfig(this) new StreamExecSink( - unwrapTableConfig(this), + config, tableSinkSpec, inputChangelogMode, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), upsertMaterialize, + // persist upsertMaterialize strategy separately in the compiled plan to make it immutable; + // later on, it can't be obtained from the node config because it is merged with the new environment + config.getOptional(TABLE_EXEC_SINK_UPSERT_MATERIALIZE_STRATEGY).orElse(null), UpsertKeyUtil.getSmallestKey(inputUpsertKeys), getRelDetailedDescription) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalOverAggregateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalOverAggregateRule.scala index a428d03f95fbf..3a9874915dfb0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalOverAggregateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalOverAggregateRule.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.physical.batch.{BatchPhysicalOv import org.apache.flink.table.planner.plan.utils.{AggregateUtil, OverAggregateUtil, SortUtil} import org.apache.flink.table.planner.plan.utils.PythonUtil.isPythonAggregate import org.apache.flink.table.planner.utils.ShortcutUtils +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.plan.RelOptRule._ diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/OverAggregateUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/OverAggregateUtil.scala index 389c34a30e60c..d3e1128aa48a4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/OverAggregateUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/OverAggregateUtil.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.planner.JArrayList import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.spec.{OverSpec, PartitionSpec} import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec.GroupSpec -import org.apache.flink.table.planner.typeutils.RowTypeUtils +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.calcite.plan.RelOptCluster import org.apache.calcite.rel.`type`.RelDataType diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala index 5be058832c604..9102f68528dcc 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala @@ -29,13 +29,13 @@ import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery import org.apache.flink.table.planner.plan.nodes.logical._ import org.apache.flink.table.planner.plan.utils.AggregateUtil.inferAggAccumulatorNames import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED, TABLE_EXEC_EMIT_LATE_FIRE_ENABLED} -import org.apache.flink.table.planner.typeutils.RowTypeUtils import org.apache.flink.table.runtime.groupwindow._ import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAssigner import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigner import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.types.logical.TimestampType import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.canBeTimeAttributeType +import org.apache.flink.table.typeutils.RowTypeUtils import org.apache.calcite.plan.volcano.RelSubset import org.apache.calcite.rel.`type`.RelDataType diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializer.java index 8ba4b792e2b05..cd5f66f7a8da9 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializer.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializer.java @@ -31,6 +31,8 @@ import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; import org.apache.flink.table.runtime.generated.RecordEqualiser; import org.apache.flink.table.runtime.operators.TableStreamOperator; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.RowKind; import org.apache.flink.util.Preconditions; @@ -222,4 +224,18 @@ private boolean equalsIgnoreRowKind(RowData newRow, RowData oldRow) { } return equaliser.equals(newRow, oldRow); } + + public static SinkUpsertMaterializer create( + StateTtlConfig ttlConfig, + RowType physicalRowType, + GeneratedRecordEqualiser rowEqualiser, + GeneratedRecordEqualiser upsertKeyEqualiser, + int[] inputUpsertKey) { + return new SinkUpsertMaterializer( + ttlConfig, + InternalSerializers.create(physicalRowType), + rowEqualiser, + upsertKeyEqualiser, + inputUpsertKey); + } } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerV2.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerV2.java new file mode 100644 index 0000000000000..6873ebb3098b6 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerV2.java @@ -0,0 +1,180 @@ +/* + * 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.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.TimestampedCollector; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.utils.ProjectedRowData; +import org.apache.flink.table.runtime.generated.GeneratedHashFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.operators.TableStreamOperator; +import org.apache.flink.table.runtime.orderedmultisetstate.OrderedMultiSetState; +import org.apache.flink.table.runtime.orderedmultisetstate.OrderedMultiSetState.RemovalResultType; +import org.apache.flink.table.runtime.orderedmultisetstate.OrderedMultiSetStateParameters; +import org.apache.flink.table.runtime.orderedmultisetstate.StateSettings; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.typeutils.RowTypeUtils; +import org.apache.flink.types.RowKind; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An operator that maintains incoming records in state corresponding to the upsert keys and + * generates an upsert view for the downstream operator. + * + *
    + *
  • Adds an insertion to state and emits it with updated {@link RowKind}. + *
  • Applies a deletion to state. + *
  • Emits a deletion with updated {@link RowKind} iff affects the last record or the state is + * empty afterward. A deletion to an already updated record is swallowed. + *
+ */ +@Internal +public class SinkUpsertMaterializerV2 extends TableStreamOperator + implements OneInputStreamOperator { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(SinkUpsertMaterializerV2.class); + + private final OrderedMultiSetStateParameters stateParameters; + + // Buffer of emitted insertions on which deletions will be applied first. + // The row kind might be +I or +U and will be ignored when applying the deletion. + private transient TimestampedCollector collector; + + private transient OrderedMultiSetState orderedMultiSetState; + private final boolean hasUpsertKey; + + public SinkUpsertMaterializerV2( + boolean hasUpsertKey, OrderedMultiSetStateParameters stateParameters) { + this.hasUpsertKey = hasUpsertKey; + this.stateParameters = stateParameters; + } + + @Override + public void open() throws Exception { + super.open(); + orderedMultiSetState = + OrderedMultiSetState.create( + stateParameters, + getRuntimeContext(), + getKeyedStateStore().getBackendTypeIdentifier()); + collector = new TimestampedCollector<>(output); + LOG.info("Opened {} with upsert key: {}", this.getClass().getSimpleName(), hasUpsertKey); + } + + @SuppressWarnings("OptionalGetWithoutIsPresent") + @Override + public void processElement(StreamRecord element) throws Exception { + final RowData row = element.getValue(); + final long timestamp = element.getTimestamp(); + + switch (row.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + if (hasUpsertKey) { + collect(row, orderedMultiSetState.add(row, timestamp).wasEmpty()); + } else { + collect(row, orderedMultiSetState.append(row, timestamp).wasEmpty()); + } + break; + + case UPDATE_BEFORE: + case DELETE: + Tuple3, OrderedMultiSetState.SizeChangeInfo> + removalResult = orderedMultiSetState.remove(row); + switch (removalResult.f0) { + case REMOVED_OTHER: + // do nothing; + break; + case NOTHING_REMOVED: + LOG.warn("Not found record to retract"); // not logging the record due for + // security + break; + case ALL_REMOVED: + collect(removalResult.f1.get(), RowKind.DELETE); + break; + case REMOVED_LAST_ADDED: + collect(removalResult.f1.get(), RowKind.UPDATE_AFTER); + break; + default: + throw new IllegalArgumentException( + "Unexpected removal result type: " + removalResult.f0); + } + } + } + + private void collect(RowData row, boolean notExisted) { + collect(row, getRowKind(notExisted)); + } + + private RowKind getRowKind(boolean notExisted) { + return notExisted ? RowKind.INSERT : RowKind.UPDATE_AFTER; + } + + private void collect(RowData row, RowKind withKind) { + RowKind orig = row.getRowKind(); + row.setRowKind(withKind); + collector.collect(row); + row.setRowKind(orig); + } + + public static SinkUpsertMaterializerV2 create( + RowType physicalRowType, + GeneratedRecordEqualiser rowEqualiser, + GeneratedRecordEqualiser upsertKeyEqualiser, + GeneratedHashFunction rowHashFunction, + GeneratedHashFunction upsertKeyHashFunction, + int[] inputUpsertKey, + StateSettings stateSettings) { + + boolean hasUpsertKey = inputUpsertKey != null && inputUpsertKey.length > 0; + + return new SinkUpsertMaterializerV2( + hasUpsertKey, + new OrderedMultiSetStateParameters( + checkNotNull( + hasUpsertKey + ? InternalSerializers.create( + RowTypeUtils.projectRowType( + physicalRowType, inputUpsertKey)) + : InternalSerializers.create(physicalRowType)), + checkNotNull(hasUpsertKey ? upsertKeyEqualiser : rowEqualiser), + checkNotNull(hasUpsertKey ? upsertKeyHashFunction : rowHashFunction), + InternalSerializers.create(physicalRowType), + row -> + hasUpsertKey + ? ProjectedRowData.from(inputUpsertKey) + .withNullAtNonProjected(true) + .replaceRow(row) + : row, + stateSettings)); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/AdaptiveOrderedMultiSetState.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/AdaptiveOrderedMultiSetState.java new file mode 100644 index 0000000000000..35995e3f2ea26 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/AdaptiveOrderedMultiSetState.java @@ -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.table.runtime.orderedmultisetstate; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.function.FunctionWithException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Optional; +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * An {@link OrderedMultiSetState} that switches dynamically between {@link ValueStateMultiSetState} + * and {@link LinkedMultiSetState} based on the number of elements. + */ +class AdaptiveOrderedMultiSetState implements OrderedMultiSetState { + private static final Logger LOG = LoggerFactory.getLogger(AdaptiveOrderedMultiSetState.class); + + private final OrderedMultiSetState smallState; + private final OrderedMultiSetState largeState; + private final long switchToLargeThreshold; + private final long switchToSmallThreshold; + + AdaptiveOrderedMultiSetState( + OrderedMultiSetState smallState, + OrderedMultiSetState largeState, + long switchToLargeThreshold, + long switchToSmallThreshold) { + checkArgument(switchToLargeThreshold > switchToSmallThreshold); + this.smallState = smallState; + this.largeState = largeState; + this.switchToLargeThreshold = switchToLargeThreshold; + this.switchToSmallThreshold = switchToSmallThreshold; + LOG.info( + "Created {} with thresholds: {}=>large, {}=>small", + this.getClass().getSimpleName(), + switchToLargeThreshold, + switchToSmallThreshold); + } + + @Override + public SizeChangeInfo add(RowData element, long timestamp) throws Exception { + return execute(state -> state.add(element, timestamp), Function.identity(), "add"); + } + + @Override + public SizeChangeInfo append(RowData element, long timestamp) throws Exception { + return execute(state -> state.append(element, timestamp), Function.identity(), "append"); + } + + @Override + public Iterator> iterator() throws Exception { + if (smallState.isEmpty()) { + return largeState.iterator(); + } else { + return smallState.iterator(); + } + } + + @Override + public boolean isEmpty() throws IOException { + // large state check is faster + return largeState.isEmpty() || smallState.isEmpty(); + } + + @Override + public Tuple3, SizeChangeInfo> remove(RowData element) + throws Exception { + return execute(state -> state.remove(element), ret -> ret.f2, "remove"); + } + + @Override + public void clear() { + clearCache(); + smallState.clear(); + largeState.clear(); + } + + @Override + public void loadCache() throws IOException { + smallState.loadCache(); + largeState.loadCache(); + } + + @Override + public void clearCache() { + smallState.clearCache(); + largeState.clearCache(); + } + + private T execute( + FunctionWithException, T, Exception> stateOp, + Function getSizeChangeInfo, + String action) + throws Exception { + + final boolean isUsingLarge = isEmptyCaching(smallState) && !isEmptyCaching(largeState); + + // start with small state, i.e. choose smallState when both are empty + OrderedMultiSetState currentState = isUsingLarge ? largeState : smallState; + OrderedMultiSetState otherState = isUsingLarge ? smallState : largeState; + + T result = stateOp.apply(currentState); + SizeChangeInfo sizeInfo = getSizeChangeInfo.apply(result); + + final boolean thresholdReached = + isUsingLarge + ? sizeInfo.sizeAfter <= switchToSmallThreshold + : sizeInfo.sizeAfter >= switchToLargeThreshold; + + if (thresholdReached) { + LOG.debug( + "Switch {} -> {} because '{}' resulted in state size change {} -> {}", + currentState.getClass().getSimpleName(), + otherState.getClass().getSimpleName(), + action, + sizeInfo.sizeBefore, + sizeInfo.sizeAfter); + switchState(currentState, otherState); + } + + clearCache(); + return result; + } + + private boolean isEmptyCaching(OrderedMultiSetState state) throws IOException { + state.loadCache(); + return state.isEmpty(); + } + + private void switchState(OrderedMultiSetState src, OrderedMultiSetState dst) + throws Exception { + Iterator> it = src.iterator(); + while (it.hasNext()) { + Tuple2 next = it.next(); + dst.append(next.f0, next.f1); + } + src.clear(); + } + + public static AdaptiveOrderedMultiSetState create( + StateSettings stateSettings, + String backendTypeIdentifier, + OrderedMultiSetState smallState, + OrderedMultiSetState largeState) { + return new AdaptiveOrderedMultiSetState( + smallState, + largeState, + stateSettings + .getAdaptiveHighThresholdOverride() + .orElse( + isHeap(backendTypeIdentifier) + ? ADAPTIVE_HEAP_HIGH_THRESHOLD + : ADAPTIVE_ROCKSDB_HIGH_THRESHOLD), + stateSettings + .getAdaptiveLowThresholdOverride() + .orElse( + isHeap(backendTypeIdentifier) + ? ADAPTIVE_HEAP_LOW_THRESHOLD + : ADAPTIVE_ROCKSDB_LOW_THRESHOLD)); + } + + private static final long ADAPTIVE_HEAP_HIGH_THRESHOLD = 400; + private static final long ADAPTIVE_HEAP_LOW_THRESHOLD = 300; + private static final long ADAPTIVE_ROCKSDB_HIGH_THRESHOLD = 50; + private static final long ADAPTIVE_ROCKSDB_LOW_THRESHOLD = 40; + + private static boolean isHeap(String stateBackend) { + String trim = stateBackend.trim(); + return trim.equalsIgnoreCase("hashmap") || trim.equalsIgnoreCase("heap"); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/LinkedMultiSetState.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/LinkedMultiSetState.java new file mode 100644 index 0000000000000..ab6f16a5ae560 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/LinkedMultiSetState.java @@ -0,0 +1,336 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.HashFunction; +import org.apache.flink.table.runtime.generated.RecordEqualiser; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.function.Function; + +import static org.apache.flink.table.runtime.orderedmultisetstate.OrderedMultiSetState.RemovalResultType.ALL_REMOVED; +import static org.apache.flink.table.runtime.orderedmultisetstate.OrderedMultiSetState.RemovalResultType.NOTHING_REMOVED; +import static org.apache.flink.table.runtime.orderedmultisetstate.OrderedMultiSetState.RemovalResultType.REMOVED_LAST_ADDED; +import static org.apache.flink.table.runtime.orderedmultisetstate.OrderedMultiSetState.RemovalResultType.REMOVED_OTHER; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This class implements an ordered multi-set state backend using Flink's state primitives. It + * maintains the insertion order of elements and supports operations such as adding, appending, and + * removing elements. The state is backed by Flink's `MapState` and `ValueState` to store and manage + * the relationships between rows and sequence numbers (SQNs). + * + *

Key features of this state implementation: + * + *

    + *
  • Maintains insertion order of elements using a doubly-linked list structure. + *
  • Supports both normal set semantics (replacing existing elements) and multi-set semantics + * (allowing duplicates). + *
  • Efficiently tracks the highest sequence number and links between elements for fast + * traversal and updates. + *
  • Provides methods to add, append, and remove elements with appropriate handling of state + * transitions. + *
+ * + *

Note: This implementation is marked as {@code @Internal} and is intended for internal use + * within Flink. It may be subject to changes in future versions. + * + *

Usage: + * + *

    + *
  • Use the {@link #add(RowData, long)} method to add an element, replacing any existing + * matching element. + *
  • Use the {@link #append(RowData, long)} method to append an element, allowing duplicates. + *
  • Use the {@link #remove(RowData)} method to remove an element, with detailed removal result + * types. + *
+ * + * @see OrderedMultiSetState + * @see org.apache.flink.api.common.state.MapState + * @see org.apache.flink.api.common.state.ValueState + */ +@Internal +public class LinkedMultiSetState implements OrderedMultiSetState { + + // maps rows to SQNs (single SQN per RowData in case of upsert key; last SQN otherwise) + private final MapState rowToSqnState; + // maps SQNs to Nodes, which comprise a doubly-linked list + private final MapState sqnToNodeState; + // highest sequence number; also latest emitted downstream + // todo: replace tuple with class? that would make it easier to add more fields + // in particular, lowestSqn is needed for iterator + private final ValueState> highestSqnStateAndSize; + + private final RecordEqualiser keyEqualiser; + private final HashFunction keyHashFunction; + private final Function keyExtractor; + private final TimeSelector timeSelector; + + private LinkedMultiSetState( + MapState rowToSqnState, + MapState sqnToNodeState, + ValueState> highestSqnStateAndSize, + RecordEqualiser keyEqualiser, + HashFunction keyHashFunction, + Function keyExtractor, + TimeSelector timeSelector) { + this.rowToSqnState = checkNotNull(rowToSqnState); + this.sqnToNodeState = checkNotNull(sqnToNodeState); + this.highestSqnStateAndSize = checkNotNull(highestSqnStateAndSize); + this.keyEqualiser = checkNotNull(keyEqualiser); + this.keyHashFunction = checkNotNull(keyHashFunction); + this.keyExtractor = keyExtractor; + this.timeSelector = timeSelector; + } + + public static OrderedMultiSetState create( + OrderedMultiSetStateParameters p, RuntimeContext ctx) { + + RecordEqualiser keyEqualiser = + p.generatedKeyEqualiser.newInstance(ctx.getUserCodeClassLoader()); + HashFunction keyHashFunction = + p.generatedKeyHashFunction.newInstance(ctx.getUserCodeClassLoader()); + + MapState rowToSqnState = + ctx.getMapState( + new MapStateDescriptor<>( + "rowToSqnState", + new RowDataKeySerializer( + p.keySerializer, + keyEqualiser, + keyHashFunction, + p.generatedKeyEqualiser, + p.generatedKeyHashFunction), + LongSerializer.INSTANCE)); + MapState sqnToNodeState = + ctx.getMapState( + new MapStateDescriptor<>( + "sqnToNodeState", + LongSerializer.INSTANCE, + new NodeSerializer(p.recordSerializer))); + + //noinspection rawtypes,unchecked + ValueState> highestSqnState = + ctx.getState( + new ValueStateDescriptor>( + "highestSqnState", + new TupleSerializer( + Tuple2.class, + new TypeSerializer[] { + LongSerializer.INSTANCE, LongSerializer.INSTANCE + }))); + + return new LinkedMultiSetState( + rowToSqnState, + sqnToNodeState, + highestSqnState, + keyEqualiser, + keyHashFunction, + p.keyExtractor, + TimeSelector.getTimeDomain(p.stateSettings.getTtlTimeDomain())); + } + + /** + * Add row, replacing any matching existing ones. + * + * @return RowKind.UPDATE_AFTER if an existing row was replaced; INSERT otherwise + */ + @Override + public SizeChangeInfo add(RowData row, long timestamp) throws Exception { + final RowDataKey key = toKey(row); + final Tuple2 highSqnAndSize = highestSqnStateAndSize.value(); + final Long highSqn = highSqnAndSize == null ? null : highSqnAndSize.f0; + final long oldSize = highSqnAndSize == null ? 0 : highSqnAndSize.f1; + final Long rowSqn = rowToSqnState.get(key); + final boolean append = rowSqn == null; + final boolean existed = highSqn != null; + + final Long oldSqn = append ? null : rowSqn; + final long newSqn = append ? (existed ? highSqn + 1 : 0) : oldSqn; + final long newSize = existed ? (append ? oldSize + 1 : oldSize) : 1; + + timestamp = timeSelector.getTimestamp(timestamp); + + sqnToNodeState.put( + newSqn, + append + ? new Node(row, newSqn, highSqn, null, null, timestamp) + : sqnToNodeState.get(oldSqn).withRow(row, timestamp)); + highestSqnStateAndSize.update(Tuple2.of(newSqn, newSize)); + if (append) { + rowToSqnState.put(key, newSqn); + if (existed) { + sqnToNodeState.put(highSqn, sqnToNodeState.get(highSqn).withNext(newSqn)); + } + } + return new SizeChangeInfo(oldSize, newSize); + } + + @Override + public SizeChangeInfo append(RowData row, long timestamp) throws Exception { + final RowDataKey key = toKey(row); + final Tuple2 highSqnAndSize = highestSqnStateAndSize.value(); + final Long highSqn = highSqnAndSize == null ? null : highSqnAndSize.f0; + final long oldSize = highSqnAndSize == null ? 0 : highSqnAndSize.f1; + final boolean existed = highSqn != null; + final long newSqn = (existed ? highSqn + 1 : 0); + final Node newNode = + new Node( + row, + newSqn, + highSqn, /*next*/ + null, /*nextForRecord*/ + null, + timeSelector.getTimestamp(timestamp)); + final long newSize = oldSize + 1; + + Long rowSqn = existed ? rowToSqnState.get(key) : null; + if (rowSqn != null) { + sqnToNodeState.put(rowSqn, sqnToNodeState.get(rowSqn).withNextForRecord(newSqn)); + } + rowToSqnState.put(key, newSqn); + highestSqnStateAndSize.update(Tuple2.of(newSqn, newSize)); + sqnToNodeState.put(newSqn, newNode); + if (existed) { + sqnToNodeState.put(highSqn, sqnToNodeState.get(highSqn).withNext(newSqn)); + } + return new SizeChangeInfo(oldSize, newSize); + } + + @Override + public Tuple3, SizeChangeInfo> remove(RowData row) + throws Exception { + final RowDataKey key = toKey(row); + final Long rowSqn = rowToSqnState.get(key); + final Tuple2 highSqnStateAndSize = highestSqnStateAndSize.value(); + final long oldSize = highSqnStateAndSize == null ? 0L : highSqnStateAndSize.f1; + if (rowSqn == null) { + return toRemovalResult(NOTHING_REMOVED, null, oldSize); + } + final Node node = sqnToNodeState.get(rowSqn); + + final Node prev = removeNode(node, key, highSqnStateAndSize); + + if (node.isHighestSqn()) { + if (prev == null) { + return toRemovalResult(ALL_REMOVED, row, oldSize); + } else { + return toRemovalResult(REMOVED_LAST_ADDED, prev.row, oldSize); + } + } else { + return toRemovalResult(REMOVED_OTHER, null, oldSize); + } + } + + @Override + public void clear() { + clearCache(); + sqnToNodeState.clear(); + highestSqnStateAndSize.clear(); + rowToSqnState.clear(); + } + + @Override + public void loadCache() {} + + @Override + public void clearCache() {} + + private Node removeNode(Node node, RowDataKey key, Tuple2 highSqnStateAndSize) + throws Exception { + + if (node.isLowestSqn() && node.isHighestSqn()) { + // fast track: if last record for PK then cleanup everything and return + clear(); + return null; + } + + sqnToNodeState.remove(node.getSqn()); + highestSqnStateAndSize.update( + Tuple2.of( + node.isHighestSqn() ? node.prevSqn : highSqnStateAndSize.f0, + highSqnStateAndSize.f1 - 1)); + if (node.isLastForRecord()) { + rowToSqnState.remove(key); + } else { + rowToSqnState.put(key, node.nextSqnForRecord); + } + // link prev node to next + Node prev = null; + if (node.hasPrev()) { + prev = sqnToNodeState.get(node.prevSqn).withNext(node.nextSqn); + sqnToNodeState.put(node.prevSqn, prev); + } + // link next node to prev + if (node.hasNext()) { + sqnToNodeState.put( + node.nextSqn, sqnToNodeState.get(node.nextSqn).withPrev(node.prevSqn)); + } + return prev; + } + + @Override + public Iterator> iterator() throws Exception { + // this can be implemented more efficiently + // however, the expected use case is to migrate all the values either to or from the memory + // state backend, so loading all into memory seems fine + List list = new ArrayList<>(); + for (Node node : sqnToNodeState.values()) { + list.add(node); + } + list.sort(Comparator.comparingLong(Node::getSqn)); + return list.stream().map(node -> Tuple2.of(node.row, node.timestamp)).iterator(); + } + + @Override + public boolean isEmpty() throws IOException { + return highestSqnStateAndSize.value() == null; + } + + private RowDataKey toKey(RowData row0) { + return RowDataKey.toKey(keyExtractor.apply(row0), keyEqualiser, keyHashFunction); + } + + private static Tuple3, SizeChangeInfo> toRemovalResult( + RemovalResultType type, @Nullable RowData row, long oldSize) { + return Tuple3.of( + type, + Optional.ofNullable(row), + new SizeChangeInfo(oldSize, type == NOTHING_REMOVED ? oldSize : oldSize - 1)); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/Node.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/Node.java new file mode 100644 index 0000000000000..7a066eeb94cf4 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/Node.java @@ -0,0 +1,124 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.table.data.RowData; + +import java.util.Objects; + +class Node { + final RowData row; + private final long sqn; + final Long prevSqn; + final Long nextSqn; + final Long nextSqnForRecord; + final Long timestamp; // for future TTL support + + Node(RowData row, long sqn, Long prevSqn, Long nextSqn, Long nextSqnForRecord, Long timestamp) { + this.row = row; + this.sqn = sqn; + this.prevSqn = prevSqn; + this.nextSqn = nextSqn; + this.nextSqnForRecord = nextSqnForRecord; + this.timestamp = timestamp; + } + + public boolean isLastForRecord() { + return nextSqnForRecord == null; + } + + public boolean isLowestSqn() { + return !hasPrev(); + } + + public boolean isHighestSqn() { + return !hasNext(); + } + + public boolean hasPrev() { + return prevSqn != null; + } + + public boolean hasNext() { + return nextSqn != null; + } + + public Node withNextForRecord(Long nextSeqNoForRecord) { + return new Node(row, sqn, prevSqn, nextSqn, nextSeqNoForRecord, timestamp); + } + + public Node withNext(Long nextSeqNo) { + return new Node(row, sqn, prevSqn, nextSeqNo, nextSqnForRecord, timestamp); + } + + public Node withPrev(Long prevSeqNo) { + return new Node(row, sqn, prevSeqNo, nextSqn, nextSqnForRecord, timestamp); + } + + public Node withRow(RowData row, long timestamp) { + return new Node(row, sqn, prevSqn, nextSqn, nextSqnForRecord, timestamp); + } + + public RowData getRow() { + return row; + } + + public long getSqn() { + return sqn; + } + + public Long getPrevSqn() { + return prevSqn; + } + + public Long getNextSqn() { + return nextSqn; + } + + public Long getNextSqnForRecord() { + return nextSqnForRecord; + } + + public Long getTimestamp() { + return timestamp; + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof Node)) { + return false; + } + Node node = (Node) o; + // do not compare row data since: + // 1. the type might be different after deserialization, e.g. GenericRowData vs + // BinaryRowData + // 2. proper comparison requires (generated) equalizer + // 3. equals is only used in tests (as opposed to RowDataKey) + return sqn == node.sqn + && Objects.equals(prevSqn, node.prevSqn) + && Objects.equals(nextSqn, node.nextSqn) + && Objects.equals(nextSqnForRecord, node.nextSqnForRecord); + } + + @Override + public int hashCode() { + // rowData is ignored - see equals + return Objects.hash(sqn, prevSqn, nextSqn, nextSqnForRecord); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/NodeSerializer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/NodeSerializer.java new file mode 100644 index 0000000000000..73c9fb49cad62 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/NodeSerializer.java @@ -0,0 +1,210 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.api.common.typeutils.CompositeSerializer; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.java.typeutils.runtime.NullableSerializer; +import org.apache.flink.table.data.RowData; + +import java.util.ArrayList; +import java.util.List; + +/** {@link TypeSerializer} for {@link Node}. */ +@SuppressWarnings("NullableProblems") +public class NodeSerializer extends CompositeSerializer { + + private static final LongSerializer LONG_SERIALIZER = LongSerializer.INSTANCE; + private static final TypeSerializer NULLABLE_LONG_SERIALIZER = + NullableSerializer.wrap(LONG_SERIALIZER, true); + + public NodeSerializer(TypeSerializer serializer) { + this(null, NodeField.getFieldSerializers(serializer)); + } + + protected NodeSerializer( + PrecomputedParameters precomputed, TypeSerializer[] originalSerializers) { + //noinspection unchecked + super( + PrecomputedParameters.precompute( + true, true, (TypeSerializer[]) originalSerializers), + originalSerializers); + } + + private NodeSerializer(TypeSerializer[] nestedSerializers) { + this(null, nestedSerializers); + } + + @Override + public Node createInstance(Object... values) { + return new Node( + NodeField.ROW.get(values), + NodeField.SEQ_NO.get(values), + NodeField.PREV_SEQ_NO.get(values), + NodeField.NEXT_SEQ_NO.get(values), + NodeField.NEXT_SEQ_NO_FOR_RECORD.get(values), + NodeField.TIMESTAMP.get(values)); + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + protected Object getField(Node node, int index) { + return NodeField.get(node, index); + } + + @Override + protected CompositeSerializer createSerializerInstance( + PrecomputedParameters precomputed, TypeSerializer... originalSerializers) { + return new NodeSerializer(precomputed, originalSerializers); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new NodeSerializerSnapshot(this); + } + + @Override + protected void setField(Node value, int index, Object fieldValue) { + throw new UnsupportedOperationException(); + } + + private enum NodeField { + ROW { + @Override + Object get(Node node) { + return node.getRow(); + } + + @Override + public TypeSerializer getSerializer(TypeSerializer serializer) { + return serializer; + } + }, + SEQ_NO { + @Override + Object get(Node node) { + return node.getSqn(); + } + + @Override + public TypeSerializer getSerializer(TypeSerializer serializer) { + return LONG_SERIALIZER; + } + }, + PREV_SEQ_NO { + @Override + Object get(Node node) { + return node.getPrevSqn(); + } + + @Override + public TypeSerializer getSerializer(TypeSerializer serializer) { + return NULLABLE_LONG_SERIALIZER; + } + }, + NEXT_SEQ_NO { + @Override + Object get(Node node) { + return node.getNextSqn(); + } + + @Override + public TypeSerializer getSerializer(TypeSerializer serializer) { + return NULLABLE_LONG_SERIALIZER; + } + }, + NEXT_SEQ_NO_FOR_RECORD { + @Override + Object get(Node node) { + return node.getNextSqnForRecord(); + } + + @Override + public TypeSerializer getSerializer(TypeSerializer serializer) { + return NULLABLE_LONG_SERIALIZER; + } + }, + TIMESTAMP { + @Override + Object get(Node node) { + return node.getTimestamp(); + } + + @Override + public TypeSerializer getSerializer(TypeSerializer serializer) { + return LONG_SERIALIZER; + } + }; + + private static TypeSerializer[] getFieldSerializers(TypeSerializer serializer) { + List> result = new ArrayList<>(); + for (NodeField field : values()) { + result.add(field.getSerializer(serializer)); + } + return result.toArray(new TypeSerializer[0]); + } + + public abstract TypeSerializer getSerializer(TypeSerializer serializer); + + abstract Object get(Node node); + + T get(Object... values) { + //noinspection unchecked + return (T) values[ordinal()]; + } + + public static Object get(Node node, int field) { + return values()[field].get(node); + } + } + + /** {@link TypeSerializerSnapshot} of {@link NodeSerializerSnapshot}. */ + public static class NodeSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + @SuppressWarnings("unused") + public NodeSerializerSnapshot() {} + + NodeSerializerSnapshot(NodeSerializer nodeSerializer) { + super(nodeSerializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return 0; + } + + @Override + protected TypeSerializer[] getNestedSerializers(NodeSerializer outerSerializer) { + return outerSerializer.fieldSerializers; + } + + @Override + protected NodeSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + return new NodeSerializer(nestedSerializers); + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetState.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetState.java new file mode 100644 index 0000000000000..57c1ec0c83cd6 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetState.java @@ -0,0 +1,173 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.table.data.RowData; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Optional; + +/** + * This class represents an interface for managing an ordered multi-set state in Apache Flink. It + * provides methods to add, append, and remove elements while maintaining insertion order. + * + *

The state supports two types of semantics for adding elements: + * + *

    + *
  • Normal Set Semantics: Replaces an existing matching element with the new one. + *
  • Multi-Set Semantics: Appends the new element, allowing duplicates. + *
+ * + *

Removal operations are supported with different result types, indicating the outcome of the + * removal process, such as whether all elements were removed, the last added element was removed, + * or no elements were removed. + * + * @param The type of elements stored in the state. + */ +@Internal +@Experimental +public interface OrderedMultiSetState { + + /** + * Add the given element using a normal (non-multi) set semantics: if a matching element exists + * already, replace it (the timestamp is updated). + */ + SizeChangeInfo add(T element, long timestamp) throws Exception; + + /** Add the given element using a multi-set semantics, i.e. append. */ + SizeChangeInfo append(T element, long timestamp) throws Exception; + + /** Get iterator over all remaining elements and their timestamps, in order of insertion. */ + Iterator> iterator() throws Exception; + + /** Tells whether any state exists (in the given key context). */ + boolean isEmpty() throws IOException; + + /** + * Remove the given element. If there are multiple instances of the same element, remove the + * first one in insertion order. + */ + Tuple3, SizeChangeInfo> remove(T element) throws Exception; + + /** Clear the state (in the current key context). */ + void clear(); + + /** Load cache. */ + void loadCache() throws IOException; + + /** Clear caches. */ + void clearCache(); + + /** Removal Result Type. */ + enum RemovalResultType { + /** + * Nothing was removed (e.g. as a result of TTL or not matching key), the result will not + * contain any elements. + */ + NOTHING_REMOVED, + /** All elements were removed. The result will contain the last removed element. */ + ALL_REMOVED, + /** + * The most recently added element was removed. The result will contain the element added + * before it. + */ + REMOVED_LAST_ADDED, + /** + * An element was removed, it was not the most recently added, there are more elements. The + * result will not contain any elements + */ + REMOVED_OTHER + } + + enum Strategy { + VALUE_STATE, + MAP_STATE, + ADAPTIVE + } + + /** + * Represents the change in size of a multi-set before and after an operation. + * + *

This class is used to track the size of the multi-set state before and after a + * modification, such as adding or removing elements. + * + *

Fields: + * + *

    + *
  • {@code sizeBefore}: The size of the multi-set before the operation. + *
  • {@code sizeAfter}: The size of the multi-set after the operation. + *
+ * + *

This class is immutable and provides a simple way to encapsulate size change information. + */ + class SizeChangeInfo { + public final long sizeBefore; + public final long sizeAfter; + + public SizeChangeInfo(long sizeBefore, long sizeAfter) { + this.sizeBefore = sizeBefore; + this.sizeAfter = sizeAfter; + } + + public boolean wasEmpty() { + return sizeBefore == 0; + } + + public boolean isEmpty() { + return sizeAfter == 0; + } + + @Override + public String toString() { + return "SizeChangeInfo{" + + "sizeBefore=" + + sizeBefore + + ", sizeAfter=" + + sizeAfter + + '}'; + } + } + + static OrderedMultiSetState create( + OrderedMultiSetStateParameters parameters, + RuntimeContext ctx, + String backendTypeIdentifier) { + switch (parameters.stateSettings.getStrategy()) { + case MAP_STATE: + return LinkedMultiSetState.create(parameters, ctx); + case VALUE_STATE: + return ValueStateMultiSetState.create(parameters, ctx); + case ADAPTIVE: + return AdaptiveOrderedMultiSetState.create( + parameters.stateSettings, + backendTypeIdentifier, + ValueStateMultiSetState.create(parameters, ctx), + LinkedMultiSetState.create(parameters, ctx)); + default: + throw new UnsupportedOperationException( + parameters.stateSettings.getStrategy().name()); + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetStateParameters.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetStateParameters.java new file mode 100644 index 0000000000000..f3d97abc9152b --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetStateParameters.java @@ -0,0 +1,52 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedHashFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; + +import java.util.function.Function; + +/** {@link OrderedMultiSetState} parameters. */ +public class OrderedMultiSetStateParameters { + + final StateSettings stateSettings; + final TypeSerializer keySerializer; + final GeneratedRecordEqualiser generatedKeyEqualiser; + final GeneratedHashFunction generatedKeyHashFunction; + final TypeSerializer recordSerializer; + final Function keyExtractor; + + public OrderedMultiSetStateParameters( + TypeSerializer keySerializer, + GeneratedRecordEqualiser generatedKeyEqualiser, + GeneratedHashFunction generatedKeyHashFunction, + TypeSerializer recordSerializer, + Function keyExtractor, + StateSettings stateSettings) { + this.keySerializer = keySerializer; + this.generatedKeyEqualiser = generatedKeyEqualiser; + this.generatedKeyHashFunction = generatedKeyHashFunction; + this.recordSerializer = recordSerializer; + this.keyExtractor = keyExtractor; + this.stateSettings = stateSettings; + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKey.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKey.java new file mode 100644 index 0000000000000..dd80f3402a115 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKey.java @@ -0,0 +1,81 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.HashFunction; +import org.apache.flink.table.runtime.generated.RecordEqualiser; +import org.apache.flink.types.RowKind; + +import static org.apache.flink.types.RowKind.INSERT; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This class wraps keys of type {@link RowData} for the following purposes: + * + *

    + *
  1. Fix the {@link RowKind} to be the same in all keys. + *
  2. Project the fields in case of upsert key. + *
  3. Fix {@link Object#equals(Object)} and hashCode for heap state backend. + *
  4. Potentially fix mutability for heap state backend (by copying using serializer) + *
+ */ +@Internal +public class RowDataKey { + private final RecordEqualiser equaliser; + private final HashFunction hashFunction; + final RowData rowData; + + RowDataKey(RecordEqualiser equaliser, HashFunction hashFunction) { + this.equaliser = checkNotNull(equaliser); + this.hashFunction = checkNotNull(hashFunction); + this.rowData = null; + } + + public RowDataKey(RowData rowData, RecordEqualiser equaliser, HashFunction hashFunction) { + this.equaliser = checkNotNull(equaliser); + this.hashFunction = checkNotNull(hashFunction); + this.rowData = checkNotNull(rowData); + } + + public static RowDataKey toKeyNotProjected( + RowData row, RecordEqualiser equaliser, HashFunction hasher) { + return toKey(row, equaliser, hasher); + } + + public static RowDataKey toKey(RowData row, RecordEqualiser equaliser, HashFunction hasher) { + row.setRowKind(INSERT); + return new RowDataKey(row, equaliser, hasher); + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof RowDataKey)) { + return false; + } + RowDataKey other = (RowDataKey) o; + return equaliser.equals(rowData, other.rowData); + } + + @Override + public int hashCode() { + return hashFunction.hashCode(rowData); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKeySerializer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKeySerializer.java new file mode 100644 index 0000000000000..064d9de582019 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKeySerializer.java @@ -0,0 +1,136 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedHashFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.generated.HashFunction; +import org.apache.flink.table.runtime.generated.RecordEqualiser; + +import java.io.IOException; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** {@link TypeSerializer} for {@link RowDataKey}. */ +@Internal +public class RowDataKeySerializer extends TypeSerializer { + final TypeSerializer serializer; + final GeneratedRecordEqualiser equaliser; // used to snapshot + final GeneratedHashFunction hashFunction; // used to snapshot + final RecordEqualiser equalizerInstance; // passed to restored keys + final HashFunction hashFunctionInstance; // passed to restored keys + + public RowDataKeySerializer( + TypeSerializer serializer, + RecordEqualiser equalizerInstance, + HashFunction hashFunctionInstance, + GeneratedRecordEqualiser equaliser, + GeneratedHashFunction hashFunction) { + this.serializer = checkNotNull(serializer); + this.equalizerInstance = checkNotNull(equalizerInstance); + this.hashFunctionInstance = checkNotNull(hashFunctionInstance); + this.equaliser = checkNotNull(equaliser); + this.hashFunction = checkNotNull(hashFunction); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new RowDataKeySerializer( + serializer.duplicate(), + equalizerInstance, + hashFunctionInstance, + equaliser, + hashFunction); + } + + @Override + public RowDataKey createInstance() { + return new RowDataKey(equalizerInstance, hashFunctionInstance); + } + + @Override + public RowDataKey copy(RowDataKey from) { + return RowDataKey.toKeyNotProjected( + serializer.copy(from.rowData), equalizerInstance, hashFunctionInstance); + } + + @Override + public RowDataKey copy(RowDataKey from, RowDataKey reuse) { + return copy(from); + } + + @Override + public int getLength() { + return serializer.getLength(); + } + + @Override + public void serialize(RowDataKey record, DataOutputView target) throws IOException { + serializer.serialize(record.rowData, target); + } + + @Override + public RowDataKey deserialize(DataInputView source) throws IOException { + return RowDataKey.toKeyNotProjected( + serializer.deserialize(source), equalizerInstance, hashFunctionInstance); + } + + @Override + public RowDataKey deserialize(RowDataKey reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serializer.copy(source, target); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof RowDataKeySerializer) { + RowDataKeySerializer other = (RowDataKeySerializer) obj; + return serializer.equals(other.serializer) + && equalizerInstance.equals(other.equalizerInstance) + && hashFunctionInstance.equals(other.hashFunctionInstance); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(serializer, equalizerInstance, hashFunctionInstance); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new RowDataKeySerializerSnapshot(this); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKeySerializerSnapshot.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKeySerializerSnapshot.java new file mode 100644 index 0000000000000..42a848d2aa6e2 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKeySerializerSnapshot.java @@ -0,0 +1,153 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedHashFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.generated.HashFunction; +import org.apache.flink.table.runtime.generated.RecordEqualiser; +import org.apache.flink.util.InstantiationUtil; + +import java.io.IOException; + +import static org.apache.flink.api.common.typeutils.TypeSerializerSnapshot.writeVersionedSnapshot; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** {@link TypeSerializerSnapshot} of {@link RowDataKeySerializer}. */ +public class RowDataKeySerializerSnapshot implements TypeSerializerSnapshot { + + private RowDataKeySerializer serializer; + private TypeSerializerSnapshot restoredRowDataSerializerSnapshot; + + @SuppressWarnings("unused") + public RowDataKeySerializerSnapshot() { + // this constructor is used when restoring from a checkpoint/savepoint. + } + + public RowDataKeySerializerSnapshot(RowDataKeySerializer serializer) { + this.serializer = checkNotNull(serializer); + } + + @Override + public int getCurrentVersion() { + return 0; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + store(serializer.equaliser, out); + store(serializer.hashFunction, out); + writeVersionedSnapshot(out, serializer.serializer.snapshotConfiguration()); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + checkArgument(readVersion == 0, "Unexpected version: " + readVersion); + + GeneratedRecordEqualiser equaliser = restore(in, userCodeClassLoader); + GeneratedHashFunction hashFunction = restore(in, userCodeClassLoader); + + restoredRowDataSerializerSnapshot = + TypeSerializerSnapshot.readVersionedSnapshot(in, userCodeClassLoader); + + serializer = + new RowDataKeySerializer( + restoredRowDataSerializerSnapshot.restoreSerializer(), + equaliser.newInstance(userCodeClassLoader), + hashFunction.newInstance(userCodeClassLoader), + equaliser, + hashFunction); + } + + private static void store(Object object, DataOutputView out) throws IOException { + byte[] bytes = InstantiationUtil.serializeObject(object); + out.writeInt(bytes.length); + out.write(bytes); + } + + private T restore(DataInputView in, ClassLoader classLoader) throws IOException { + int len = in.readInt(); + byte[] bytes = new byte[len]; + in.read(bytes); + try { + return InstantiationUtil.deserializeObject(bytes, classLoader); // here + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + } + + @Override + public TypeSerializer restoreSerializer() { + return serializer; + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof RowDataKeySerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + RowDataKeySerializerSnapshot old = (RowDataKeySerializerSnapshot) oldSerializerSnapshot; + + TypeSerializerSchemaCompatibility compatibility = + old.restoredRowDataSerializerSnapshot.resolveSchemaCompatibility( + old.serializer.serializer.snapshotConfiguration()); + + return mapToOuterCompatibility( + compatibility, + serializer.equalizerInstance, + serializer.hashFunctionInstance, + serializer.equaliser, + serializer.hashFunction); + } + + private static TypeSerializerSchemaCompatibility mapToOuterCompatibility( + TypeSerializerSchemaCompatibility rowDataCmp, + RecordEqualiser equaliserInstance, + HashFunction hashFunctionInstance, + GeneratedRecordEqualiser equaliser, + GeneratedHashFunction hashFunction) { + if (rowDataCmp.isCompatibleAsIs()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } else if (rowDataCmp.isCompatibleAfterMigration()) { + return TypeSerializerSchemaCompatibility.compatibleAfterMigration(); + } else if (rowDataCmp.isCompatibleWithReconfiguredSerializer()) { + return TypeSerializerSchemaCompatibility.compatibleWithReconfiguredSerializer( + new RowDataKeySerializer( + rowDataCmp.getReconfiguredSerializer(), + equaliserInstance, + hashFunctionInstance, + equaliser, + hashFunction)); + } else if (rowDataCmp.isIncompatible()) { + return TypeSerializerSchemaCompatibility.incompatible(); + } else { + throw new UnsupportedOperationException("Unknown compatibility mode: " + rowDataCmp); + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/StateSettings.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/StateSettings.java new file mode 100644 index 0000000000000..10f9c16ca003b --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/StateSettings.java @@ -0,0 +1,129 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.api.common.state.StateTtlConfig; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.table.api.config.ExecutionConfigOptions; + +import javax.annotation.Nullable; + +import java.util.Optional; + +import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_HIGH; +import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_LOW; +import static org.apache.flink.util.Preconditions.checkArgument; + +public class StateSettings { + private final OrderedMultiSetState.Strategy strategy; + private final @Nullable Long adaptiveHighThresholdOverride; + private final @Nullable Long adaptiveLowThresholdOverride; + private final StateTtlConfig ttlConfig; + private final TimeDomain ttlTimeDomain; // overrides the one from ttlConfig + + private StateSettings( + TimeDomain ttlTimeDomain, + OrderedMultiSetState.Strategy strategy, + @Nullable Long adaptiveHighThresholdOverride, + @Nullable Long adaptiveLowThresholdOverride, + StateTtlConfig ttlConfig) { + checkArgument( + !ttlConfig.isEnabled(), + "TTL is not supported"); // https://issues.apache.org/jira/browse/FLINK-38463 + this.ttlTimeDomain = ttlTimeDomain; + this.strategy = strategy; + this.adaptiveHighThresholdOverride = adaptiveHighThresholdOverride; + this.adaptiveLowThresholdOverride = adaptiveLowThresholdOverride; + this.ttlConfig = ttlConfig; + } + + public static StateSettings defaults(TimeDomain ttlTimeDomain, StateTtlConfig ttlConfig) { + return forValue(ttlTimeDomain, ttlConfig); + } + + public static StateSettings forMap(TimeDomain ttlTimeDomain, StateTtlConfig ttlConfig) { + return new StateSettings( + ttlTimeDomain, OrderedMultiSetState.Strategy.MAP_STATE, null, null, ttlConfig); + } + + public static StateSettings forValue(TimeDomain ttlTimeDomain, StateTtlConfig ttl) { + return new StateSettings( + ttlTimeDomain, OrderedMultiSetState.Strategy.VALUE_STATE, null, null, ttl); + } + + public static StateSettings adaptive( + TimeDomain ttlTimeDomain, + @Nullable Long adaptiveHighThresholdOverride, + @Nullable Long adaptiveLowThresholdOverride, + StateTtlConfig ttl) { + return new StateSettings( + ttlTimeDomain, + OrderedMultiSetState.Strategy.ADAPTIVE, + adaptiveHighThresholdOverride, + adaptiveLowThresholdOverride, + ttl); + } + + // todo: move this out to sink? + public static StateSettings forStrategy( + ExecutionConfigOptions.SinkUpsertMaterializeStrategy sinkUpsertMaterializeStrategy, + ReadableConfig config, + StateTtlConfig ttlConfig) { + switch (sinkUpsertMaterializeStrategy) { + case MAP: + return forMap(TimeDomain.PROCESSING_TIME, ttlConfig); + case VALUE: + return forValue(TimeDomain.PROCESSING_TIME, ttlConfig); + case ADAPTIVE: + return adaptive( + TimeDomain.PROCESSING_TIME, + config.getOptional( + TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_HIGH) + .orElse(null), + config.getOptional( + TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_LOW) + .orElse(null), + ttlConfig); + default: + throw new IllegalStateException( + "Unknown upsertMaterializeStrategy: " + sinkUpsertMaterializeStrategy); + } + } + + public TimeDomain getTtlTimeDomain() { + return ttlTimeDomain; + } + + public OrderedMultiSetState.Strategy getStrategy() { + return strategy; + } + + public Optional getAdaptiveHighThresholdOverride() { + return Optional.ofNullable(adaptiveHighThresholdOverride); + } + + public Optional getAdaptiveLowThresholdOverride() { + return Optional.ofNullable(adaptiveLowThresholdOverride); + } + + public StateTtlConfig getTtlConfig() { + return ttlConfig; + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/TimeSelector.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/TimeSelector.java new file mode 100644 index 0000000000000..71016c5ef36b4 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/TimeSelector.java @@ -0,0 +1,39 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.util.clock.SystemClock; + +@FunctionalInterface +public interface TimeSelector { + + long getTimestamp(long elementTimestamp); + + static TimeSelector getTimeDomain(TimeDomain timeDomain) { + switch (timeDomain) { + case EVENT_TIME: + return elementTimestamp -> elementTimestamp; + case PROCESSING_TIME: + return elementTimestamp -> SystemClock.getInstance().absoluteTimeMillis(); + default: + throw new IllegalStateException("unknown time domain: " + timeDomain); + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/ValueStateMultiSetState.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/ValueStateMultiSetState.java new file mode 100644 index 0000000000000..080b1ea6179f0 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/ValueStateMultiSetState.java @@ -0,0 +1,208 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.RecordEqualiser; +import org.apache.flink.types.RowKind; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.function.Function; + +/** + * Simple yet powerful implementation of {@link OrderedMultiSetState} based on plain {@code + * ValueState}. + */ +class ValueStateMultiSetState implements OrderedMultiSetState { + + private final ValueState>> valuesState; + private final RecordEqualiser keyEqualiser; + private final Function keyExtractor; + private final TimeSelector timeSelector; + private List> cache; + + ValueStateMultiSetState( + ValueState>> valuesState, + RecordEqualiser keyEqualiser, + Function keyExtractor, + TimeSelector timeSelector) { + this.valuesState = valuesState; + this.keyEqualiser = keyEqualiser; + this.keyExtractor = keyExtractor; + this.timeSelector = timeSelector; + } + + public static OrderedMultiSetState create( + OrderedMultiSetStateParameters p, RuntimeContext ctx) { + //noinspection rawtypes,unchecked + return new ValueStateMultiSetState( + ctx.getState( + new ValueStateDescriptor<>( + "list", + new ListSerializer<>( + new TupleSerializer( + Tuple2.class, + new TypeSerializer[] { + p.recordSerializer, LongSerializer.INSTANCE + })))), + p.generatedKeyEqualiser.newInstance(ctx.getUserCodeClassLoader()), + p.keyExtractor, + TimeSelector.getTimeDomain(p.stateSettings.getTtlTimeDomain())); + } + + @Override + public SizeChangeInfo add(RowData row, long ts) throws Exception { + normalizeRowKind(row); + final Tuple2 toAdd = Tuple2.of(row, timeSelector.getTimestamp(ts)); + final RowData key = asKey(row); + final List> list = maybeReadState(); + final int oldSize = list.size(); + + int idx = Integer.MIN_VALUE; + int i = 0; + for (Tuple2 t : list) { + if (keyEqualiser.equals(asKey(t.f0), key)) { + idx = i; + break; + } + } + if (idx < 0) { + list.add(toAdd); + } else { + list.set(idx, toAdd); + } + valuesState.update(list); + return new SizeChangeInfo(oldSize, list.size()); + } + + @Override + public SizeChangeInfo append(RowData row, long timestamp) throws Exception { + normalizeRowKind(row); + List> values = maybeReadState(); + long sizeBefore = values.size(); + values.add(Tuple2.of(row, timeSelector.getTimestamp(timestamp))); + long sizeAfter = values.size(); + valuesState.update(values); + return new SizeChangeInfo(sizeBefore, sizeAfter); + } + + @Override + public Iterator> iterator() throws Exception { + return maybeReadState().iterator(); + } + + @Override + public Tuple3, SizeChangeInfo> remove(RowData row) + throws Exception { + normalizeRowKind(row); + final RowData key = asKey(row); + final List> list = maybeReadState(); + final int oldSize = list.size(); + + int dropIdx = Integer.MIN_VALUE; + RowData last = null; + int i = 0; + for (Tuple2 t : list) { + if (keyEqualiser.equals(key, asKey(t.f0))) { + dropIdx = i; + break; + } else { + last = t.f0; + } + i++; + } + if (dropIdx >= 0) { + list.remove(dropIdx); + valuesState.update(list); + } + return toRemovalResult(new SizeChangeInfo(oldSize, list.size()), dropIdx, row, last); + } + + @Override + public void loadCache() throws IOException { + cache = readState(); + } + + @Override + public void clearCache() { + cache = null; + } + + private List> maybeReadState() throws IOException { + if (cache != null) { + return cache; + } + return readState(); + } + + private List> readState() throws IOException { + List> value = valuesState.value(); + if (value == null) { + value = new ArrayList<>(); + } + return value; + } + + @Override + public void clear() { + clearCache(); + valuesState.clear(); + } + + @Override + public boolean isEmpty() throws IOException { + List> list = cache == null ? valuesState.value() : cache; + return list != null && list.isEmpty(); + } + + private RowData asKey(RowData row) { + return keyExtractor.apply(row); + } + + private static void normalizeRowKind(RowData row) { + row.setRowKind(RowKind.INSERT); + } + + private static Tuple3, SizeChangeInfo> toRemovalResult( + SizeChangeInfo sizeChangeInfo, int dropIdx, RowData row, RowData last) { + if (sizeChangeInfo.wasEmpty()) { + return Tuple3.of(RemovalResultType.NOTHING_REMOVED, Optional.empty(), sizeChangeInfo); + } else if (sizeChangeInfo.isEmpty()) { + return Tuple3.of(RemovalResultType.ALL_REMOVED, Optional.of(row), sizeChangeInfo); + } else if (dropIdx + 1 == sizeChangeInfo.sizeBefore) { + return Tuple3.of( + RemovalResultType.REMOVED_LAST_ADDED, Optional.of(last), sizeChangeInfo); + } else { + return Tuple3.of(RemovalResultType.REMOVED_OTHER, Optional.empty(), sizeChangeInfo); + } + } +} diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerMigrationTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerMigrationTest.java new file mode 100644 index 0000000000000..2dd4a4add8b15 --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerMigrationTest.java @@ -0,0 +1,222 @@ +/* + * 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.sink; + +import org.apache.flink.FlinkVersion; +import org.apache.flink.api.common.state.StateTtlConfig; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OperatorSnapshotUtil; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.orderedmultisetstate.StateSettings; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.util.MigrationTest; +import org.apache.flink.types.RowKind; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +import static org.apache.flink.FlinkVersion.current; +import static org.apache.flink.streaming.util.OperatorSnapshotUtil.getResourceFilename; +import static org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializerTest.ASSERTOR; +import static org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializerTest.EQUALISER; +import static org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializerTest.GENERATED_HASH_FUNCTION; +import static org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializerTest.GENERATED_UPSERT_HASH_FUNCTION; +import static org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializerTest.LOGICAL_TYPES; +import static org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializerTest.TTL_CONFIG; +import static org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializerTest.UPSERT_KEY; +import static org.apache.flink.table.runtime.operators.sink.SinkUpsertMaterializerTest.UPSERT_KEY_EQUALISER; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.deleteRecord; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.insertRecord; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.rowOfKind; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.updateAfterRecord; + +/** Test for {@link SinkUpsertMaterializer} migration. */ +@RunWith(Parameterized.class) +public class SinkUpsertMaterializerMigrationTest implements MigrationTest { + + private static final String FOLDER_NAME = "sink-upsert-materializer"; + + @Parameterized.Parameter(0) + @SuppressWarnings({"ClassEscapesDefinedScope", "DefaultAnnotationParam"}) + public SinkOperationMode migrateFrom; + + @Parameterized.Parameter(1) + @SuppressWarnings("ClassEscapesDefinedScope") + public SinkOperationMode migrateTo; + + @Parameterized.Parameters(name = "{0} -> {1}") + public static List parameters() { + List result = new ArrayList<>(); + Set versions = FlinkVersion.rangeOf(FlinkVersion.v2_2, FlinkVersion.v2_2); + for (FlinkVersion fromVersion : versions) { + for (SumStateBackend backend : SumStateBackend.values()) { + for (SumVersion sumVersion : SumVersion.values()) { + result.add( + new Object[] { + new SinkOperationMode(fromVersion, backend, sumVersion), + new SinkOperationMode(current(), backend, sumVersion) + }); + } + } + } + return result; + } + + @Test + public void testMigration() throws Exception { + String path = getResourceFilename(FOLDER_NAME + "/" + getFileName(migrateFrom)); + try (OneInputStreamOperatorTestHarness harness = + createHarness(migrateTo, path)) { + testCorrectnessAfterSnapshot(harness); + } + } + + private OneInputStreamOperatorTestHarness createHarness( + SinkOperationMode mode, String snapshotPath) throws Exception { + int[] inputUpsertKey = {UPSERT_KEY}; + StateTtlConfig ttlConfig = mode.sumVersion.reconfigureTtl(TTL_CONFIG); + OneInputStreamOperator materializer; + switch (mode.sumVersion) { + case V1: + materializer = + SinkUpsertMaterializer.create( + ttlConfig, + RowType.of(LOGICAL_TYPES), + EQUALISER, + UPSERT_KEY_EQUALISER, + inputUpsertKey); + break; + case V2: + materializer = + SinkUpsertMaterializerV2.create( + RowType.of(LOGICAL_TYPES), + EQUALISER, + UPSERT_KEY_EQUALISER, + GENERATED_HASH_FUNCTION, + GENERATED_UPSERT_HASH_FUNCTION, + inputUpsertKey, + StateSettings.defaults(TimeDomain.PROCESSING_TIME, ttlConfig)); + break; + default: + throw new IllegalArgumentException(mode.sumVersion.name()); + } + KeyedOneInputStreamOperatorTestHarness harness = + SinkUpsertMaterializerTest.createHarness( + materializer, mode.stateBackend, LOGICAL_TYPES); + harness.setup(new RowDataSerializer(LOGICAL_TYPES)); + if (snapshotPath != null) { + harness.initializeState(snapshotPath); + } + harness.open(); + harness.setStateTtlProcessingTime(1); + return harness; + } + + private void testCorrectnessBeforeSnapshot( + OneInputStreamOperatorTestHarness testHarness) throws Exception { + + testHarness.processElement(insertRecord(1L, 1, "a1")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 1L, 1, "a1")); + + testHarness.processElement(updateAfterRecord(1L, 1, "a11")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 1L, 1, "a11")); + + testHarness.processElement(insertRecord(3L, 1, "a3")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 3L, 1, "a3")); + } + + private void testCorrectnessAfterSnapshot( + OneInputStreamOperatorTestHarness testHarness) throws Exception { + testHarness.processElement(deleteRecord(1L, 1, "a111")); + ASSERTOR.shouldEmitNothing(testHarness); + + testHarness.processElement(deleteRecord(3L, 1, "a33")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, 3L, 1, "a33")); + + testHarness.processElement(insertRecord(4L, 1, "a4")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 4L, 1, "a4")); + + testHarness.setStateTtlProcessingTime(1002); + testHarness.processElement(deleteRecord(4L, 1, "a4")); + if (migrateTo.sumVersion.isTtlSupported()) { + ASSERTOR.shouldEmitNothing(testHarness); + } else { + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, 4L, 1, "a4")); + } + } + + private static String getFileName(SinkOperationMode mode) { + return String.format( + "migration-flink-%s-%s-%s-snapshot", + mode.version, mode.stateBackend, mode.sumVersion); + } + + @SnapshotsGenerator + public void writeSnapshot(FlinkVersion version) throws Exception { + for (SumStateBackend stateBackend : SumStateBackend.values()) { + for (SumVersion sumVersion : SumVersion.values()) { + SinkOperationMode mode = new SinkOperationMode(version, stateBackend, sumVersion); + try (OneInputStreamOperatorTestHarness harness = + createHarness(mode, null)) { + testCorrectnessBeforeSnapshot(harness); + Path parent = Paths.get("src/test/resources", FOLDER_NAME); + Files.createDirectories(parent); + OperatorSnapshotUtil.writeStateHandle( + harness.snapshot(1L, 1L), parent.resolve(getFileName(mode)).toString()); + } + } + } + } + + public static void main(String... s) throws Exception { + // Run this to manually generate snapshot files for migration tests + // set working directory to flink-table/flink-table-runtime/ + new SinkUpsertMaterializerMigrationTest().writeSnapshot(current()); + } + + private static class SinkOperationMode { + private final FlinkVersion version; + private final SumStateBackend stateBackend; + private final SumVersion sumVersion; + + private SinkOperationMode( + FlinkVersion version, SumStateBackend stateBackend, SumVersion sumVersion) { + this.version = version; + this.stateBackend = stateBackend; + this.sumVersion = sumVersion; + } + + @Override + public String toString() { + return String.format("flink=%s, state=%s, sum=%s}", version, stateBackend, sumVersion); + } + } +} diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerRescalingTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerRescalingTest.java new file mode 100644 index 0000000000000..cadf6ac7520e5 --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerRescalingTest.java @@ -0,0 +1,418 @@ +/* + * 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.sink; + +import org.apache.flink.api.common.state.StateTtlConfig; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.checkpoint.CheckpointType; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.SavepointType; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedHashFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.generated.HashFunction; +import org.apache.flink.table.runtime.generated.RecordEqualiser; +import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; +import org.apache.flink.table.runtime.orderedmultisetstate.StateSettings; +import org.apache.flink.table.runtime.util.RowDataHarnessAssertor; +import org.apache.flink.table.runtime.util.StateConfigUtil; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.utils.HandwrittenSelectorUtil; +import org.apache.flink.types.RowKind; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.function.ToIntFunction; +import java.util.stream.Collectors; + +import static org.apache.flink.table.runtime.util.StreamRecordUtils.deleteRecord; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.insertRecord; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.rowOfKind; + +/** Rescaling and migration unit tests for {@link SinkUpsertMaterializer}. */ +@RunWith(Parameterized.class) +public class SinkUpsertMaterializerRescalingTest { + + @Parameter public SumVersion sumVersion; + + @Parameter(1) + public SumStateBackend backend; + + @Parameterized.Parameters(name = "sumVersion={0}, stateBackend={1}") + public static Object[][] generateTestParameters() { + List result = new ArrayList<>(); + for (SumVersion sumVersion : SumVersion.values()) { + for (SumStateBackend backend : SumStateBackend.values()) { + result.add(new Object[] {sumVersion, backend}); + } + } + return result.toArray(new Object[0][]); + } + + @Test + public void testScaleUpThenDown() throws Exception { + testRescaleFromToFrom(10, 2, 3, backend, backend, sumVersion); + } + + @Test + public void testScaleDownThenUp() throws Exception { + testRescaleFromToFrom(10, 3, 2, backend, backend, sumVersion); + } + + @Test + public void testRecovery() throws Exception { + testRescaleFromToFrom(1, 1, 1, backend, backend, sumVersion); + } + + @Test + public void testForwardAndBackwardMigration() throws Exception { + testRescaleFromToFrom(7, 3, 3, backend, getOtherBackend(backend), sumVersion); + } + + @Test + public void testScaleUpThenDownWithMigration() throws Exception { + testRescaleFromToFrom(7, 1, 5, backend, getOtherBackend(backend), sumVersion); + } + + @Test + public void testScaleDownThenUpWithMigration() throws Exception { + testRescaleFromToFrom(7, 5, 1, backend, getOtherBackend(SumStateBackend.HEAP), sumVersion); + } + + private SumStateBackend getOtherBackend(SumStateBackend backend) { + return backend == SumStateBackend.HEAP ? SumStateBackend.ROCKSDB : SumStateBackend.HEAP; + } + + @SuppressWarnings("unchecked") + private void testRescaleFromToFrom( + final int maxParallelism, + final int fromParallelism, + final int toParallelism, + final SumStateBackend fromBackend, + final SumStateBackend toBackend, + final SumVersion sumVersion) + throws Exception { + + int[] currentParallelismRef = new int[] {fromParallelism}; + + boolean useSavepoint = fromBackend != toBackend; + + OneInputStreamOperator[] materializers = + new OneInputStreamOperator[maxParallelism]; + KeyedOneInputStreamOperatorTestHarness[] harnesses = + new KeyedOneInputStreamOperatorTestHarness[maxParallelism]; + + final ToIntFunction> combinedHarnesses = + (r) -> { + try { + int subtaskIndex = + KeyGroupRangeAssignment.assignKeyToParallelOperator( + KEY_SELECTOR.getKey(r.getValue()), + maxParallelism, + currentParallelismRef[0]); + + harnesses[subtaskIndex].processElement(r); + return subtaskIndex; + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + + initHarnessesAndMaterializers( + harnesses, + materializers, + fromBackend, + maxParallelism, + fromParallelism, + null, + sumVersion); + + int idx = combinedHarnesses.applyAsInt(insertRecord(1L, 1, "a1")); + ASSERTOR.shouldEmit(harnesses[idx], rowOfKind(RowKind.INSERT, 1L, 1, "a1")); + + idx = combinedHarnesses.applyAsInt(insertRecord(2L, 1, "a2")); + ASSERTOR.shouldEmit(harnesses[idx], rowOfKind(RowKind.UPDATE_AFTER, 2L, 1, "a2")); + + List subtaskStates = + snapshotHarnesses(harnesses, fromParallelism, 1L, useSavepoint); + + currentParallelismRef[0] = toParallelism; + initHarnessesAndMaterializers( + harnesses, + materializers, + toBackend, + maxParallelism, + toParallelism, + subtaskStates, + sumVersion); + + idx = combinedHarnesses.applyAsInt(insertRecord(3L, 1, "a3")); + ASSERTOR.shouldEmit(harnesses[idx], rowOfKind(RowKind.UPDATE_AFTER, 3L, 1, "a3")); + + idx = combinedHarnesses.applyAsInt(insertRecord(4L, 1, "a4")); + ASSERTOR.shouldEmit(harnesses[idx], rowOfKind(RowKind.UPDATE_AFTER, 4L, 1, "a4")); + + subtaskStates = snapshotHarnesses(harnesses, toParallelism, 2L, useSavepoint); + + currentParallelismRef[0] = fromParallelism; + initHarnessesAndMaterializers( + harnesses, + materializers, + fromBackend, + maxParallelism, + fromParallelism, + subtaskStates, + sumVersion); + + idx = combinedHarnesses.applyAsInt(deleteRecord(4L, 1, "a4")); + ASSERTOR.shouldEmit(harnesses[idx], rowOfKind(RowKind.UPDATE_AFTER, 3L, 1, "a3")); + + idx = combinedHarnesses.applyAsInt(deleteRecord(2L, 1, "a2")); + ASSERTOR.shouldEmitNothing(harnesses[idx]); + + idx = combinedHarnesses.applyAsInt(deleteRecord(3L, 1, "a3")); + ASSERTOR.shouldEmit(harnesses[idx], rowOfKind(RowKind.UPDATE_AFTER, 1L, 1, "a1")); + + idx = combinedHarnesses.applyAsInt(deleteRecord(1L, 1, "a1")); + ASSERTOR.shouldEmit(harnesses[idx], rowOfKind(RowKind.DELETE, 1L, 1, "a1")); + + idx = combinedHarnesses.applyAsInt(insertRecord(4L, 1, "a4")); + ASSERTOR.shouldEmit(harnesses[idx], rowOfKind(RowKind.INSERT, 4L, 1, "a4")); + + Arrays.stream(harnesses) + .filter(Objects::nonNull) + .forEach(h -> h.setStateTtlProcessingTime(1002)); + + idx = combinedHarnesses.applyAsInt(deleteRecord(4L, 1, "a4")); + if (sumVersion.isTtlSupported()) { + ASSERTOR.shouldEmitNothing(harnesses[idx]); + } else { + ASSERTOR.shouldEmit(harnesses[idx], rowOfKind(RowKind.DELETE, 4L, 1, "a4")); + } + + Arrays.stream(harnesses) + .filter(Objects::nonNull) + .forEach( + h -> { + try { + h.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + private void initHarnessesAndMaterializers( + KeyedOneInputStreamOperatorTestHarness[] harnesses, + OneInputStreamOperator[] materializers, + SumStateBackend backend, + int maxParallelism, + int parallelism, + @Nullable List subtaskStates, + SumVersion sumVersion) + throws Exception { + for (int i = 0; i < parallelism; ++i) { + switch (sumVersion) { + case V1: + materializers[i] = + SinkUpsertMaterializer.create( + TTL_CONFIG, + RowType.of(LOGICAL_TYPES), + EQUALISER, + UPSERT_KEY_EQUALISER, + null); + break; + case V2: + materializers[i] = + SinkUpsertMaterializerV2.create( + RowType.of(LOGICAL_TYPES), + EQUALISER, + UPSERT_KEY_EQUALISER, + HASH_FUNCTION, + UPSERT_KEY_HASH_FUNCTION, + null, + StateSettings.defaults( + TimeDomain.PROCESSING_TIME, + sumVersion.reconfigureTtl(TTL_CONFIG))); + break; + default: + throw new IllegalArgumentException("unknown version: " + sumVersion); + } + harnesses[i] = + new KeyedOneInputStreamOperatorTestHarness<>( + materializers[i], + KEY_SELECTOR, + KEY_SELECTOR.getProducedType(), + maxParallelism, + parallelism, + i); + + harnesses[i].setStateBackend(backend.create(false)); + + if (subtaskStates != null) { + OperatorSubtaskState operatorSubtaskState = + AbstractStreamOperatorTestHarness.repackageState( + subtaskStates.toArray(new OperatorSubtaskState[0])); + + harnesses[i].initializeState( + AbstractStreamOperatorTestHarness.repartitionOperatorState( + operatorSubtaskState, + maxParallelism, + subtaskStates.size(), + parallelism, + i)); + } + + harnesses[i].open(); + harnesses[i].setStateTtlProcessingTime(1); + } + } + + private List snapshotHarnesses( + KeyedOneInputStreamOperatorTestHarness[] harnesses, + int parallelism, + long checkpointId, + boolean useSavepoint) { + return Arrays.stream(harnesses, 0, parallelism) + .map( + h -> { + try { + return h.snapshotWithLocalState( + checkpointId, + 0L, + useSavepoint + ? SavepointType.savepoint( + SavepointFormatType.CANONICAL) + : CheckpointType.CHECKPOINT) + .getJobManagerOwnedState(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + } + + /** Test equalizer for records. */ + protected static class TestRecordEqualiser implements RecordEqualiser, HashFunction { + @Override + public boolean equals(RowData row1, RowData row2) { + return row1.getRowKind() == row2.getRowKind() + && row1.getLong(0) == row2.getLong(0) + && row1.getInt(1) == row2.getInt(1) + && row1.getString(2).equals(row2.getString(2)); + } + + @Override + public int hashCode(Object data) { + RowData rd = (RowData) data; + return Objects.hash(rd.getRowKind(), rd.getLong(0), rd.getInt(1), rd.getString(2)); + } + } + + /** Test equalizer for upsert keys. */ + protected static class TestUpsertKeyEqualiser implements RecordEqualiser, HashFunction { + @Override + public boolean equals(RowData row1, RowData row2) { + return row1.getRowKind() == row2.getRowKind() && row1.getLong(0) == row2.getLong(0); + } + + @Override + public int hashCode(Object data) { + RowData rd = (RowData) data; + return Objects.hash(rd.getRowKind(), rd.getLong(0)); + } + } + + private static class MyGeneratedRecordEqualiser extends GeneratedRecordEqualiser { + + public MyGeneratedRecordEqualiser() { + super("", "", new Object[0]); + } + + @Override + public RecordEqualiser newInstance(ClassLoader classLoader) { + return new TestRecordEqualiser(); + } + } + + private static class MyGeneratedHashFunction extends GeneratedHashFunction { + + public MyGeneratedHashFunction() { + super("", "", new Object[0], new Configuration()); + } + + @Override + public HashFunction newInstance(ClassLoader classLoader) { + return new TestRecordEqualiser(); + } + } + + private static final StateTtlConfig TTL_CONFIG = StateConfigUtil.createTtlConfig(1000); + + private static final LogicalType[] LOGICAL_TYPES = + new LogicalType[] {new BigIntType(), new IntType(), new VarCharType()}; + + private static final RowDataKeySelector KEY_SELECTOR = + HandwrittenSelectorUtil.getRowDataSelector(new int[] {1}, LOGICAL_TYPES); + + private static final RowDataHarnessAssertor ASSERTOR = + new RowDataHarnessAssertor(LOGICAL_TYPES); + + private static final GeneratedRecordEqualiser EQUALISER = new MyGeneratedRecordEqualiser(); + + private static final GeneratedHashFunction HASH_FUNCTION = new MyGeneratedHashFunction(); + + private static final GeneratedRecordEqualiser UPSERT_KEY_EQUALISER = + new GeneratedRecordEqualiser("", "", new Object[0]) { + + @Override + public RecordEqualiser newInstance(ClassLoader classLoader) { + return new TestUpsertKeyEqualiser(); + } + }; + + private static final GeneratedHashFunction UPSERT_KEY_HASH_FUNCTION = + new GeneratedHashFunction("", "", new Object[0], new Configuration()) { + + @Override + public HashFunction newInstance(ClassLoader classLoader) { + return new TestUpsertKeyEqualiser(); + } + }; +} diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerTest.java index 322ac9e1cebe7..af5163f332f48 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerTest.java @@ -19,40 +19,81 @@ package org.apache.flink.table.runtime.operators.sink; import org.apache.flink.api.common.state.StateTtlConfig; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedHashFunction; import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.generated.HashFunction; import org.apache.flink.table.runtime.generated.RecordEqualiser; -import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.runtime.orderedmultisetstate.StateSettings; import org.apache.flink.table.runtime.util.RowDataHarnessAssertor; import org.apache.flink.table.runtime.util.StateConfigUtil; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.utils.HandwrittenSelectorUtil; import org.apache.flink.types.RowKind; -import org.junit.jupiter.api.Test; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +import static org.apache.flink.api.java.tuple.Tuple2.of; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.binaryRecord; import static org.apache.flink.table.runtime.util.StreamRecordUtils.deleteRecord; import static org.apache.flink.table.runtime.util.StreamRecordUtils.insertRecord; import static org.apache.flink.table.runtime.util.StreamRecordUtils.rowOfKind; import static org.apache.flink.table.runtime.util.StreamRecordUtils.updateAfterRecord; +import static org.junit.Assert.assertEquals; /** Test for {@link SinkUpsertMaterializer}. */ -class SinkUpsertMaterializerTest { +@RunWith(Parameterized.class) +public class SinkUpsertMaterializerTest { + + static final int UPSERT_KEY = 0; + + @Parameter public SumVersion sumVersion; - private final StateTtlConfig ttlConfig = StateConfigUtil.createTtlConfig(1000); - private final LogicalType[] types = + @Parameter(1) + public SumStateBackend stateBackend; + + @Parameterized.Parameters(name = "sumVersion={0}, stateBackend={1}") + public static Object[][] generateTestParameters() { + List result = new ArrayList<>(); + for (SumVersion sumVersion : SumVersion.values()) { + for (SumStateBackend backend : SumStateBackend.values()) { + result.add(new Object[] {sumVersion, backend}); + } + } + return result.toArray(new Object[0][]); + } + + static final StateTtlConfig TTL_CONFIG = StateConfigUtil.createTtlConfig(1000); + static final LogicalType[] LOGICAL_TYPES = new LogicalType[] {new BigIntType(), new IntType(), new VarCharType()}; - private final RowDataSerializer serializer = new RowDataSerializer(types); - private final RowDataKeySelector keySelector = - HandwrittenSelectorUtil.getRowDataSelector(new int[] {1}, types); - private final RowDataHarnessAssertor assertor = new RowDataHarnessAssertor(types); + static final RowDataHarnessAssertor ASSERTOR = new RowDataHarnessAssertor(LOGICAL_TYPES); - private final GeneratedRecordEqualiser equaliser = + static final GeneratedRecordEqualiser EQUALISER = new GeneratedRecordEqualiser("", "", new Object[0]) { @Override @@ -61,7 +102,15 @@ public RecordEqualiser newInstance(ClassLoader classLoader) { } }; - private final GeneratedRecordEqualiser upsertKeyEqualiser = + static final GeneratedHashFunction GENERATED_HASH_FUNCTION = + new GeneratedHashFunction("", "", new Object[0], new Configuration()) { + @Override + public HashFunction newInstance(ClassLoader classLoader) { + return new TestRecordEqualiser(); + } + }; + + static final GeneratedRecordEqualiser UPSERT_KEY_EQUALISER = new GeneratedRecordEqualiser("", "", new Object[0]) { @Override @@ -70,88 +119,257 @@ public RecordEqualiser newInstance(ClassLoader classLoader) { } }; + static final GeneratedHashFunction GENERATED_UPSERT_HASH_FUNCTION = + new GeneratedHashFunction("", "", new Object[0], new Configuration()) { + @Override + public HashFunction newInstance(ClassLoader classLoader) { + return new TestUpsertKeyEqualiser(); + } + }; + + /** + * If the composite serializer in {@link SinkUpsertMaterializer} works on projected fields then + * it might use the wrong serializer, e.g. the {@link VarCharType} instead of the {@link + * IntType}. That might cause {@link ArrayIndexOutOfBoundsException} because string serializer + * expects the first number to be the length of the string. + */ @Test - void test() throws Exception { - SinkUpsertMaterializer materializer = - new SinkUpsertMaterializer( - ttlConfig, serializer, equaliser, upsertKeyEqualiser, null); + public void testUpsertKeySerializerFailure() throws Exception { + LogicalType[] types = new LogicalType[] {new VarCharType(), new IntType()}; + // project int field, while in the original record it's string + + OneInputStreamOperator materializer = createOperator(types, 1); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + createHarness(materializer, stateBackend, types)) { + testHarness.open(); + RowDataHarnessAssertor assertor = new RowDataHarnessAssertor(types); + // -1 is not a valid string length + testHarness.processElement(binaryRecord(RowKind.INSERT, "any string", -1)); + assertor.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, "any string", -1)); + + // 999 as a string length is too long + testHarness.processElement(binaryRecord(RowKind.INSERT, "any string", 999)); + assertor.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, "any string", 999)); + } + } + + @Test + public void testUpsertKeySerializerSilentCorruption() throws Exception { + LogicalType[] types = + new LogicalType[] {new VarCharType(), new BigIntType(), new IntType()}; + // project int field, while in the original record it's string + + OneInputStreamOperator materializer = createOperator(types, 1); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + createHarness(materializer, stateBackend, types)) { + testHarness.open(); + RowDataHarnessAssertor assertor = new RowDataHarnessAssertor(types); + + // this might serialize upsert key as 32-character string potentially including "97" + testHarness.processElement(binaryRecord(RowKind.INSERT, "any string", 32L, 97)); + assertor.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, "any string", 32L, 97)); + + // but here it might include "98" which would result in no output and test failure + testHarness.processElement(binaryRecord(RowKind.DELETE, "any string", 32L, 98)); + assertor.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, "any string", 32L, 98)); + } + } + + @Test + public void testUpsertEqualizer() throws Exception { + LogicalType[] types = new LogicalType[] {new IntType(), new BigIntType()}; + + OneInputStreamOperator materializer = createOperator(types, 1); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + createHarness(materializer, stateBackend, types)) { + testHarness.open(); + RowDataHarnessAssertor assertor = new RowDataHarnessAssertor(types); + + // upsert key is 33, 0 is unused + testHarness.processElement(binaryRecord(RowKind.INSERT, 0, 33L)); + assertor.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 0, 33L)); + + // upsert key 33 - should remove AND clear the state involving upsert equalizer + // equalizer might fail if it's used on un-projected records + testHarness.processElement(binaryRecord(RowKind.DELETE, 1, 33L)); + assertor.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, 1, 33L)); + } + } + + @Test + public void testNoUpsertKeyFlow() throws Exception { KeyedOneInputStreamOperatorTestHarness testHarness = - new KeyedOneInputStreamOperatorTestHarness<>( - materializer, keySelector, keySelector.getProducedType()); + createHarness(createOperatorWithoutUpsertKey()); testHarness.open(); testHarness.setStateTtlProcessingTime(1); testHarness.processElement(insertRecord(1L, 1, "a1")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 1L, 1, "a1")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 1L, 1, "a1")); testHarness.processElement(insertRecord(2L, 1, "a2")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 2L, 1, "a2")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 2L, 1, "a2")); testHarness.processElement(insertRecord(3L, 1, "a3")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 3L, 1, "a3")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 3L, 1, "a3")); testHarness.processElement(deleteRecord(2L, 1, "a2")); - assertor.shouldEmitNothing(testHarness); + ASSERTOR.shouldEmitNothing(testHarness); testHarness.processElement(deleteRecord(3L, 1, "a3")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 1L, 1, "a1")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 1L, 1, "a1")); testHarness.processElement(deleteRecord(1L, 1, "a1")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, 1L, 1, "a1")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, 1L, 1, "a1")); testHarness.processElement(insertRecord(4L, 1, "a4")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 4L, 1, "a4")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 4L, 1, "a4")); testHarness.setStateTtlProcessingTime(1002); testHarness.processElement(deleteRecord(4L, 1, "a4")); - assertor.shouldEmitNothing(testHarness); + if (sumVersion.isTtlSupported()) { + ASSERTOR.shouldEmitNothing(testHarness); + } else { + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, 4L, 1, "a4")); + } testHarness.close(); } @Test - void testInputHasUpsertKeyWithNonDeterministicColumn() throws Exception { - SinkUpsertMaterializer materializer = - new SinkUpsertMaterializer( - ttlConfig, serializer, equaliser, upsertKeyEqualiser, new int[] {0}); + public void testInputHasUpsertKeyWithNonDeterministicColumn() throws Exception { + OneInputStreamOperator materializer = + createOperator(LOGICAL_TYPES, UPSERT_KEY); KeyedOneInputStreamOperatorTestHarness testHarness = - new KeyedOneInputStreamOperatorTestHarness<>( - materializer, keySelector, keySelector.getProducedType()); + createHarness(materializer); testHarness.open(); testHarness.setStateTtlProcessingTime(1); testHarness.processElement(insertRecord(1L, 1, "a1")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 1L, 1, "a1")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 1L, 1, "a1")); testHarness.processElement(updateAfterRecord(1L, 1, "a11")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 1L, 1, "a11")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 1L, 1, "a11")); testHarness.processElement(insertRecord(3L, 1, "a3")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 3L, 1, "a3")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.UPDATE_AFTER, 3L, 1, "a3")); testHarness.processElement(deleteRecord(1L, 1, "a111")); - assertor.shouldEmitNothing(testHarness); + ASSERTOR.shouldEmitNothing(testHarness); testHarness.processElement(deleteRecord(3L, 1, "a33")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, 3L, 1, "a33")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, 3L, 1, "a33")); testHarness.processElement(insertRecord(4L, 1, "a4")); - assertor.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 4L, 1, "a4")); + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.INSERT, 4L, 1, "a4")); testHarness.setStateTtlProcessingTime(1002); testHarness.processElement(deleteRecord(4L, 1, "a4")); - assertor.shouldEmitNothing(testHarness); + if (sumVersion.isTtlSupported()) { + ASSERTOR.shouldEmitNothing(testHarness); + } else { + ASSERTOR.shouldEmit(testHarness, rowOfKind(RowKind.DELETE, 4L, 1, "a4")); + } testHarness.close(); } - private static class TestRecordEqualiser implements RecordEqualiser { + @Test + public void testRetractionWithoutUpsertKey() throws Exception { + testRetractions((int[]) null); + } + + @Test + public void testRetractionWithUpsertKey() throws Exception { + testRetractions(UPSERT_KEY); + } + + public void testRetractions(int... upsertKey) throws Exception { + testThreeElementProcessing( + "retract first - should emit nothing until empty - then delete", + upsertKey, + of(deleteRecord(1L, 1, "a1"), null), + of(deleteRecord(2L, 1, "a2"), null), + of(deleteRecord(3L, 1, "a3"), rowOfKind(RowKind.DELETE, 3L, 1, "a3"))); + testThreeElementProcessing( + "retract middle - should emit nothing until empty - then delete", + upsertKey, + of(deleteRecord(2L, 1, "a2"), null), + of(deleteRecord(1L, 1, "a1"), null), + of(deleteRecord(3L, 1, "a3"), rowOfKind(RowKind.DELETE, 3L, 1, "a3"))); + testThreeElementProcessing( + "retract last - should emit penultimate until empty - then delete", + upsertKey, + of(deleteRecord(3L, 1, "a3"), rowOfKind(RowKind.UPDATE_AFTER, 2L, 1, "a2")), + of(deleteRecord(2L, 1, "a2"), rowOfKind(RowKind.UPDATE_AFTER, 1L, 1, "a1")), + of(deleteRecord(1L, 1, "a1"), rowOfKind(RowKind.DELETE, 1L, 1, "a1"))); + testThreeElementProcessing( + "retract in arbitrary order: 1,3,2", + upsertKey, + of(deleteRecord(1L, 1, "a1"), null), + of(deleteRecord(3L, 1, "a3"), rowOfKind(RowKind.UPDATE_AFTER, 2L, 1, "a2")), + of(deleteRecord(2L, 1, "a2"), rowOfKind(RowKind.DELETE, 2L, 1, "a2"))); + testThreeElementProcessing( + "retract in arbitrary order: 2,3,1", + upsertKey, + of(deleteRecord(2L, 1, "a2"), null), + of(deleteRecord(3L, 1, "a3"), rowOfKind(RowKind.UPDATE_AFTER, 1L, 1, "a1")), + of(deleteRecord(1L, 1, "a1"), rowOfKind(RowKind.DELETE, 1L, 1, "a1"))); + testThreeElementProcessing( + "retract in arbitrary order: 3,1,2", + upsertKey, + of(deleteRecord(3L, 1, "a3"), rowOfKind(RowKind.UPDATE_AFTER, 2L, 1, "a2")), + of(deleteRecord(1L, 1, "a1"), null), + of(deleteRecord(2L, 1, "a2"), rowOfKind(RowKind.DELETE, 2L, 1, "a2"))); + } + + // boilerplate for common test case of processing starting with three elements + @SafeVarargs + private void testThreeElementProcessing( + String description, + int[] upsertKey, + Tuple2, RowData>... inputAndOutput) + throws Exception { + @SuppressWarnings("rawtypes") + Tuple2[] merged = new Tuple2[inputAndOutput.length + 3]; + merged[0] = of(insertRecord(1L, 1, "a1"), rowOfKind(RowKind.INSERT, 1L, 1, "a1")); + merged[1] = of(insertRecord(2L, 1, "a2"), rowOfKind(RowKind.UPDATE_AFTER, 2L, 1, "a2")); + merged[2] = of(insertRecord(3L, 1, "a3"), rowOfKind(RowKind.UPDATE_AFTER, 3L, 1, "a3")); + System.arraycopy(inputAndOutput, 0, merged, 3, inputAndOutput.length); + testElementProcessing(description, upsertKey, merged); + } + + @SafeVarargs + private void testElementProcessing( + String description, + int[] upsertKey, + Tuple2, RowData>... inputAndOutput) + throws Exception { + OneInputStreamOperator materializer = + createOperator(LOGICAL_TYPES, upsertKey); + KeyedOneInputStreamOperatorTestHarness testHarness = + createHarness(materializer); + + testHarness.open(); + + for (Tuple2, RowData> el0 : inputAndOutput) { + testHarness.processElement(el0.f0); + if (el0.f1 == null) { + ASSERTOR.shouldEmitNothing(testHarness); + } else { + ASSERTOR.shouldEmit(testHarness, description, el0.f1); + } + } + + testHarness.close(); + } + + private static class TestRecordEqualiser implements RecordEqualiser, HashFunction { @Override public boolean equals(RowData row1, RowData row2) { return row1.getRowKind() == row2.getRowKind() @@ -159,12 +377,147 @@ public boolean equals(RowData row1, RowData row2) { && row1.getInt(1) == row2.getInt(1) && row1.getString(2).equals(row2.getString(2)); } + + @Override + public int hashCode(Object data) { + RowData rd = (RowData) data; + return Objects.hash(rd.getRowKind(), rd.getLong(0), rd.getInt(1), rd.getString(2)); + } } - private static class TestUpsertKeyEqualiser implements RecordEqualiser { + private static class TestUpsertKeyEqualiser implements RecordEqualiser, HashFunction { @Override public boolean equals(RowData row1, RowData row2) { - return row1.getRowKind() == row2.getRowKind() && row1.getLong(0) == row2.getLong(0); + return row1.getRowKind() == row2.getRowKind() + && row1.getLong(UPSERT_KEY) == row2.getLong(UPSERT_KEY); + } + + @Override + public int hashCode(Object data) { + RowData rd = (RowData) data; + return Objects.hash(rd.getRowKind(), rd.getLong(UPSERT_KEY)); + } + } + + static KeyedOneInputStreamOperatorTestHarness createHarness( + OneInputStreamOperator materializer, + SumStateBackend backend, + LogicalType[] types) + throws Exception { + KeyedOneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>( + materializer, + HandwrittenSelectorUtil.getRowDataSelector(new int[] {1}, types), + HandwrittenSelectorUtil.getRowDataSelector(new int[] {1}, types) + .getProducedType()); + testHarness.setStateBackend(backend.create(true)); + return testHarness; + } + + @Test + public void testEmptyUpsertKey() throws Exception { + testRecovery(createOperator(LOGICAL_TYPES), createOperatorWithoutUpsertKey()); + testRecovery(createOperatorWithoutUpsertKey(), createOperator(LOGICAL_TYPES)); + } + + private void testRecovery( + OneInputStreamOperator from, + OneInputStreamOperator to) + throws Exception { + OperatorSubtaskState snapshot; + try (KeyedOneInputStreamOperatorTestHarness testHarness = + createHarness(from)) { + testHarness.open(); + snapshot = testHarness.snapshot(1L, 1L); + } + try (KeyedOneInputStreamOperatorTestHarness testHarness = + createHarness(to)) { + testHarness.initializeState(snapshot); + testHarness.open(); + } + } + + private KeyedOneInputStreamOperatorTestHarness createHarness( + OneInputStreamOperator m2) throws Exception { + return createHarness(m2, stateBackend, LOGICAL_TYPES); + } + + @Test + public void testVersionStateGrowth() throws Exception { + int dop = 2; + int numIterations = 10; + OperatorSnapshotFinalizer[] snapshots = new OperatorSnapshotFinalizer[dop]; + long[] prevStateSizes = new long[dop]; + for (int i = 0; i < numIterations; i++) { + for (int subtask = 0; subtask < dop; subtask++) { + snapshots[subtask] = initAndSnapshot(snapshots[subtask], i); + long currentStateSize = + snapshots[subtask] + .getJobManagerOwnedState() + .getManagedOperatorState() + .stream() + .mapToLong(StateObject::getStateSize) + .sum(); + if (i > 0) { + assertEquals(prevStateSizes[subtask], currentStateSize); + } + prevStateSizes[subtask] = currentStateSize; + } + List union = + Arrays.stream(snapshots) + .flatMap( + s -> + s + .getJobManagerOwnedState() + .getManagedOperatorState() + .stream()) + .collect(Collectors.toList()); + for (int j = 0; j < dop; j++) { + snapshots[j] = + new OperatorSnapshotFinalizer( + snapshots[j].getJobManagerOwnedState().toBuilder() + .setManagedOperatorState(new StateObjectCollection<>(union)) + .build(), + snapshots[j].getTaskLocalState()); + } + } + } + + private OperatorSnapshotFinalizer initAndSnapshot( + OperatorSnapshotFinalizer from, int newCheckpointID) throws Exception { + try (OneInputStreamOperatorTestHarness harness = + createHarness( + createOperator(LOGICAL_TYPES, UPSERT_KEY), stateBackend, LOGICAL_TYPES)) { + if (from != null) { + harness.initializeState(from.getJobManagerOwnedState()); + } + harness.open(); + return harness.snapshotWithLocalState(newCheckpointID, newCheckpointID); + } + } + + private OneInputStreamOperator createOperatorWithoutUpsertKey() { + return createOperator(LOGICAL_TYPES, (int[]) null); + } + + private OneInputStreamOperator createOperator( + LogicalType[] types, int... upsertKey) { + switch (sumVersion) { + case V1: + return SinkUpsertMaterializer.create( + TTL_CONFIG, RowType.of(types), EQUALISER, UPSERT_KEY_EQUALISER, upsertKey); + case V2: + return SinkUpsertMaterializerV2.create( + RowType.of(types), + EQUALISER, + UPSERT_KEY_EQUALISER, + GENERATED_HASH_FUNCTION, + GENERATED_UPSERT_HASH_FUNCTION, + upsertKey, + StateSettings.defaults( + TimeDomain.PROCESSING_TIME, sumVersion.reconfigureTtl(TTL_CONFIG))); + default: + throw new IllegalArgumentException(); } } } diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SumStateBackend.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SumStateBackend.java new file mode 100644 index 0000000000000..05024ff8889cf --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SumStateBackend.java @@ -0,0 +1,41 @@ +/* + * 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.sink; + +import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; + +/** StateBackend to test SinkUpsertMaterializer with. */ +public enum SumStateBackend { + HEAP { + + public StateBackend create(boolean incrementalIfSupported) { + return new HashMapStateBackend(); + } + }, + ROCKSDB { + + public StateBackend create(boolean incrementalIfSupported) { + return new EmbeddedRocksDBStateBackend(incrementalIfSupported); + } + }; + + public abstract StateBackend create(boolean incrementalIfSupported); +} diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SumVersion.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SumVersion.java new file mode 100644 index 0000000000000..74a9d52463b9e --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SumVersion.java @@ -0,0 +1,44 @@ +/* + * 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.sink; + +import org.apache.flink.api.common.state.StateTtlConfig; + +/** Version of SinkUpsertMaterializer to test. */ +public enum SumVersion { + V1 { + @Override + public boolean isTtlSupported() { + return true; + } + }, + V2 { + @Override + public boolean isTtlSupported() { + // todo: add support for TTL and remove checking related code + return false; + } + }; + + public abstract boolean isTtlSupported(); + + StateTtlConfig reconfigureTtl(StateTtlConfig ttlConfig) { + return isTtlSupported() ? ttlConfig : StateTtlConfig.DISABLED; + } +} diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/NodeSerializerTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/NodeSerializerTest.java new file mode 100644 index 0000000000000..a902ebc059cd9 --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/NodeSerializerTest.java @@ -0,0 +1,53 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.runtime.util.StreamRecordUtils; +import org.apache.flink.table.types.logical.IntType; + +/** Test for {@link RowDataKeySerializer}. */ +public class NodeSerializerTest extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return new NodeSerializer(new RowDataSerializer(new IntType())); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return Node.class; + } + + @Override + protected Node[] getTestData() { + return new Node[] { + new Node(StreamRecordUtils.row(1), 1L, null, 2L, 2L, 1L), + new Node(StreamRecordUtils.row(2), 2L, 1L, 3L, 3L, 2L), + new Node(StreamRecordUtils.row(3), 3L, 2L, null, null, 3L), + }; + } +} diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKeySerializerTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKeySerializerTest.java new file mode 100644 index 0000000000000..54e91238d695c --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/RowDataKeySerializerTest.java @@ -0,0 +1,104 @@ +/* + * 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.orderedmultisetstate; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedHashFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.generated.HashFunction; +import org.apache.flink.table.runtime.generated.RecordEqualiser; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.runtime.util.StreamRecordUtils; +import org.apache.flink.table.types.logical.IntType; + +import java.util.Objects; + +/** Test for {@link RowDataKeySerializer}. */ +public class RowDataKeySerializerTest extends SerializerTestBase { + + private final TestRecordEqualiser equaliser = new TestRecordEqualiser(); + + @Override + protected TypeSerializer createSerializer() { + return new RowDataKeySerializer( + new RowDataSerializer(new IntType()), + equaliser, + equaliser, + EQUALISER, + HASH_FUNCTION); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return RowDataKey.class; + } + + @Override + protected RowDataKey[] getTestData() { + return new RowDataKey[] {new RowDataKey(StreamRecordUtils.row(123), equaliser, equaliser)}; + } + + static final GeneratedRecordEqualiser EQUALISER = + new GeneratedRecordEqualiser("", "", new Object[0]) { + + @Override + public RecordEqualiser newInstance(ClassLoader classLoader) { + return new TestRecordEqualiser(); + } + }; + + static final GeneratedHashFunction HASH_FUNCTION = + new GeneratedHashFunction("", "", new Object[0], new Configuration()) { + @Override + public HashFunction newInstance(ClassLoader classLoader) { + return new TestRecordEqualiser(); + } + }; + + private static class TestRecordEqualiser implements RecordEqualiser, HashFunction { + @Override + public boolean equals(RowData row1, RowData row2) { + return row1.getRowKind() == row2.getRowKind() && row1.getInt(0) == row2.getInt(0); + } + + @Override + public int hashCode(Object data) { + RowData rd = (RowData) data; + return Objects.hash(rd.getRowKind(), rd.getInt(0)); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof TestRecordEqualiser; + } + + @Override + public int hashCode() { + return 0; + } + } +} diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/util/RowDataHarnessAssertor.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/util/RowDataHarnessAssertor.java index d7fbef9f16293..b4b074f724913 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/util/RowDataHarnessAssertor.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/util/RowDataHarnessAssertor.java @@ -61,6 +61,14 @@ public void shouldEmit( assertThat(getEmittedRows(harness)).containsExactly(expected); } + /** Assert the test harness should emit records exactly same as the expected records. */ + public void shouldEmit( + AbstractStreamOperatorTestHarness harness, + String description, + RowData... expected) { + assertThat(getEmittedRows(harness)).describedAs(description).containsExactly(expected); + } + /** Assert the test harness should emit all records regardless of the order. */ public void shouldEmitAll( AbstractStreamOperatorTestHarness harness, RowData... expected) { diff --git a/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-HEAP-V1-snapshot b/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-HEAP-V1-snapshot new file mode 100644 index 0000000000000..be4fd71e72aca Binary files /dev/null and b/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-HEAP-V1-snapshot differ diff --git a/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-HEAP-V2-snapshot b/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-HEAP-V2-snapshot new file mode 100644 index 0000000000000..956b64af2d1d9 Binary files /dev/null and b/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-HEAP-V2-snapshot differ diff --git a/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-ROCKSDB-V1-snapshot b/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-ROCKSDB-V1-snapshot new file mode 100644 index 0000000000000..1832c7a50ebf7 Binary files /dev/null and b/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-ROCKSDB-V1-snapshot differ diff --git a/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-ROCKSDB-V2-snapshot b/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-ROCKSDB-V2-snapshot new file mode 100644 index 0000000000000..33308346686ad Binary files /dev/null and b/flink-table/flink-table-runtime/src/test/resources/sink-upsert-materializer/migration-flink-2.2-ROCKSDB-V2-snapshot differ diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml index b679676b6be77..7b24c8e6e4b30 100644 --- a/flink-test-utils-parent/flink-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-test-utils/pom.xml @@ -42,6 +42,13 @@ under the License. compile + + org.apache.flink + flink-table-api-java + ${project.version} + compile + + org.apache.flink flink-test-utils-junit diff --git a/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeSerializerTestCoverageTest.java b/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeSerializerTestCoverageTest.java index 691460a4012d9..b595a953fbad1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeSerializerTestCoverageTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeSerializerTestCoverageTest.java @@ -74,6 +74,8 @@ import org.apache.flink.table.dataview.NullAwareMapSerializer; import org.apache.flink.table.dataview.NullSerializer; import org.apache.flink.table.runtime.operators.window.CountWindow; +import org.apache.flink.table.runtime.orderedmultisetstate.NodeSerializer; +import org.apache.flink.table.runtime.orderedmultisetstate.RowDataKeySerializer; import org.apache.flink.table.runtime.typeutils.ArrayDataSerializer; import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer; import org.apache.flink.table.runtime.typeutils.DecimalDataSerializer; @@ -255,6 +257,9 @@ public void testTypeSerializerTestCoverage() { // KeyAndValueSerializer shouldn't be used to serialize data to state and // doesn't need to ensure upgrade compatibility. "org.apache.flink.streaming.api.operators.sortpartition.KeyAndValueSerializer", + // todo: implement these tests? + RowDataKeySerializer.class.getName(), + NodeSerializer.class.getName(), SetSerializer.class.getName()); // check if a test exists for each type serializer