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-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..075dce3e178c1 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/AdaptiveOrderedMultiSetState.java @@ -0,0 +1,197 @@ +/* + * 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.table.runtime.orderedmultisetstate.linked.LinkedMultiSetState; +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( + OrderedMultiSetStateConfig orderedMultiSetStateConfig, + String backendTypeIdentifier, + OrderedMultiSetState smallState, + OrderedMultiSetState largeState) { + return new AdaptiveOrderedMultiSetState( + smallState, + largeState, + orderedMultiSetStateConfig + .getAdaptiveHighThresholdOverride() + .orElse( + isHeap(backendTypeIdentifier) + ? ADAPTIVE_HEAP_HIGH_THRESHOLD + : ADAPTIVE_ROCKSDB_HIGH_THRESHOLD), + orderedMultiSetStateConfig + .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/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..9b650348057cb --- /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 org.apache.flink.table.runtime.orderedmultisetstate.linked.LinkedMultiSetState; + +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( + OrderedMultiSetStateContext parameters, + RuntimeContext ctx, + String backendTypeIdentifier) { + switch (parameters.config.getStrategy()) { + case MAP_STATE: + return LinkedMultiSetState.create(parameters, ctx); + case VALUE_STATE: + return ValueStateMultiSetState.create(parameters, ctx); + case ADAPTIVE: + return AdaptiveOrderedMultiSetState.create( + parameters.config, + backendTypeIdentifier, + ValueStateMultiSetState.create(parameters, ctx), + LinkedMultiSetState.create(parameters, ctx)); + default: + throw new UnsupportedOperationException(parameters.config.getStrategy().name()); + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetStateConfig.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetStateConfig.java new file mode 100644 index 0000000000000..183ab3bd80d31 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetStateConfig.java @@ -0,0 +1,118 @@ +/* + * 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.streaming.api.TimeDomain; + +import javax.annotation.Nullable; + +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** Configuration for {@link OrderedMultiSetState}. */ +public class OrderedMultiSetStateConfig { + + private final OrderedMultiSetState.Strategy strategy; + private final @Nullable Long adaptiveHighThresholdOverride; + private final @Nullable Long adaptiveLowThresholdOverride; + private final StateTtlConfig ttlConfig; + private final TimeSelector ttlTimeSelector; + + public OrderedMultiSetStateConfig( + OrderedMultiSetState.Strategy strategy, + @Nullable Long adaptiveHighThresholdOverride, + @Nullable Long adaptiveLowThresholdOverride, + StateTtlConfig ttlConfig, + TimeDomain ttlTimeDomain) { + this( + strategy, + adaptiveHighThresholdOverride, + adaptiveLowThresholdOverride, + ttlConfig, + TimeSelector.getTimeDomain(ttlTimeDomain)); + } + + public OrderedMultiSetStateConfig( + OrderedMultiSetState.Strategy strategy, + @Nullable Long adaptiveHighThresholdOverride, + @Nullable Long adaptiveLowThresholdOverride, + StateTtlConfig ttlConfig, + TimeSelector ttlTimeSelector) { + checkArgument( + !ttlConfig.isEnabled(), + "TTL is not supported"); // https://issues.apache.org/jira/browse/FLINK-38463 + this.strategy = strategy; + this.adaptiveHighThresholdOverride = adaptiveHighThresholdOverride; + this.adaptiveLowThresholdOverride = adaptiveLowThresholdOverride; + this.ttlConfig = ttlConfig; + this.ttlTimeSelector = ttlTimeSelector; + } + + public static OrderedMultiSetStateConfig defaults( + TimeDomain ttlTimeDomain, StateTtlConfig ttlConfig) { + return forValue(ttlTimeDomain, ttlConfig); + } + + public static OrderedMultiSetStateConfig forMap( + TimeDomain ttlTimeDomain, StateTtlConfig ttlConfig) { + return new OrderedMultiSetStateConfig( + OrderedMultiSetState.Strategy.MAP_STATE, null, null, ttlConfig, ttlTimeDomain); + } + + public static OrderedMultiSetStateConfig forValue( + TimeDomain ttlTimeDomain, StateTtlConfig ttl) { + return new OrderedMultiSetStateConfig( + OrderedMultiSetState.Strategy.VALUE_STATE, null, null, ttl, ttlTimeDomain); + } + + public static OrderedMultiSetStateConfig adaptive( + TimeDomain ttlTimeDomain, + @Nullable Long adaptiveHighThresholdOverride, + @Nullable Long adaptiveLowThresholdOverride, + StateTtlConfig ttl) { + return new OrderedMultiSetStateConfig( + OrderedMultiSetState.Strategy.ADAPTIVE, + adaptiveHighThresholdOverride, + adaptiveLowThresholdOverride, + ttl, + ttlTimeDomain); + } + + public TimeSelector getTimeSelector() { + return ttlTimeSelector; + } + + 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/OrderedMultiSetStateContext.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetStateContext.java new file mode 100644 index 0000000000000..a8e1a1cd21adb --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/OrderedMultiSetStateContext.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} (creation) context. */ +public class OrderedMultiSetStateContext { + + public final OrderedMultiSetStateConfig config; + public final TypeSerializer keySerializer; + public final GeneratedRecordEqualiser generatedKeyEqualiser; + public final GeneratedHashFunction generatedKeyHashFunction; + public final TypeSerializer recordSerializer; + public final Function keyExtractor; + + public OrderedMultiSetStateContext( + TypeSerializer keySerializer, + GeneratedRecordEqualiser generatedKeyEqualiser, + GeneratedHashFunction generatedKeyHashFunction, + TypeSerializer recordSerializer, + Function keyExtractor, + OrderedMultiSetStateConfig config) { + this.keySerializer = keySerializer; + this.generatedKeyEqualiser = generatedKeyEqualiser; + this.generatedKeyHashFunction = generatedKeyHashFunction; + this.recordSerializer = recordSerializer; + this.keyExtractor = keyExtractor; + this.config = config; + } +} 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..b86ec3ffa524d --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/TimeSelector.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.orderedmultisetstate; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.util.clock.SystemClock; + +@Internal +@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..e4710715018ad --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/ValueStateMultiSetState.java @@ -0,0 +1,207 @@ +/* + * 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 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( + OrderedMultiSetStateContext 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, + p.config.getTimeSelector()); + } + + @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/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/LinkedMultiSetState.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/LinkedMultiSetState.java new file mode 100644 index 0000000000000..a4524983a1283 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/LinkedMultiSetState.java @@ -0,0 +1,337 @@ +/* + * 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.linked; + +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 org.apache.flink.table.runtime.orderedmultisetstate.OrderedMultiSetState; +import org.apache.flink.table.runtime.orderedmultisetstate.OrderedMultiSetStateContext; +import org.apache.flink.table.runtime.orderedmultisetstate.TimeSelector; + +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 + private final ValueState> highestSqnAndSizeState; + + private final RecordEqualiser keyEqualiser; + private final HashFunction keyHashFunction; + private final Function keyExtractor; + private final TimeSelector timeSelector; + + private LinkedMultiSetState( + MapState rowToSqnState, + MapState sqnToNodeState, + ValueState> highestSqnAndSizeState, + RecordEqualiser keyEqualiser, + HashFunction keyHashFunction, + Function keyExtractor, + TimeSelector timeSelector) { + this.rowToSqnState = checkNotNull(rowToSqnState); + this.sqnToNodeState = checkNotNull(sqnToNodeState); + this.highestSqnAndSizeState = checkNotNull(highestSqnAndSizeState); + this.keyEqualiser = checkNotNull(keyEqualiser); + this.keyHashFunction = checkNotNull(keyHashFunction); + this.keyExtractor = keyExtractor; + this.timeSelector = timeSelector; + } + + public static OrderedMultiSetState create( + OrderedMultiSetStateContext 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, + p.config.getTimeSelector()); + } + + /** + * 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 = highestSqnAndSizeState.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)); + highestSqnAndSizeState.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 = highestSqnAndSizeState.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); + highestSqnAndSizeState.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 = highestSqnAndSizeState.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(); + highestSqnAndSizeState.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()); + highestSqnAndSizeState.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 highestSqnAndSizeState.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/linked/Node.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/Node.java new file mode 100644 index 0000000000000..f3efea7e9d8f7 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/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.linked; + +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/linked/NodeSerializer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/NodeSerializer.java new file mode 100644 index 0000000000000..1f98e4b10e223 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/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.linked; + +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/linked/RowDataKey.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/RowDataKey.java new file mode 100644 index 0000000000000..ffa7f64dc2ab3 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/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.linked; + +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 +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/linked/RowDataKeySerializer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/RowDataKeySerializer.java new file mode 100644 index 0000000000000..f421764c8b953 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/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.linked; + +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/linked/RowDataKeySerializerSnapshot.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/RowDataKeySerializerSnapshot.java new file mode 100644 index 0000000000000..98f786ab9b28e --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/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.linked; + +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/test/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/NodeSerializerTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/NodeSerializerTest.java new file mode 100644 index 0000000000000..29c12c08b0d95 --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/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.linked; + +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/linked/RowDataKeySerializerTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/RowDataKeySerializerTest.java new file mode 100644 index 0000000000000..82d2be761e7f7 --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/orderedmultisetstate/linked/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.linked; + +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-tests/pom.xml b/flink-tests/pom.xml index dbfee497e1393..e80e2a2efb74f 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -186,6 +186,14 @@ under the License. test + + org.apache.flink + flink-table-runtime + ${project.version} + test-jar + test + + org.apache.flink flink-runtime 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..216cb77710506 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.linked.NodeSerializer; +import org.apache.flink.table.runtime.orderedmultisetstate.linked.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,8 @@ 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", + RowDataKeySerializer.class.getName(), + NodeSerializer.class.getName(), SetSerializer.class.getName()); // check if a test exists for each type serializer