pnowojski commented on code in PR #27071:
URL: https://github.com/apache/flink/pull/27071#discussion_r2402510234


##########
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).
+ *
+ * <p>Key features of this state implementation:
+ *
+ * <ul>
+ *   <li>Maintains insertion order of elements using a doubly-linked list 
structure.
+ *   <li>Supports both normal set semantics (replacing existing elements) and 
multi-set semantics
+ *       (allowing duplicates).
+ *   <li>Efficiently tracks the highest sequence number and links between 
elements for fast
+ *       traversal and updates.
+ *   <li>Provides methods to add, append, and remove elements with appropriate 
handling of state
+ *       transitions.
+ * </ul>
+ *
+ * <p>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.
+ *
+ * <p>Usage:
+ *
+ * <ul>
+ *   <li>Use the {@link #add(RowData, long)} method to add an element, 
replacing any existing
+ *       matching element.
+ *   <li>Use the {@link #append(RowData, long)} method to append an element, 
allowing duplicates.
+ *   <li>Use the {@link #remove(RowData)} method to remove an element, with 
detailed removal result
+ *       types.
+ * </ul>
+ *
+ * @see OrderedMultiSetState
+ * @see org.apache.flink.api.common.state.MapState
+ * @see org.apache.flink.api.common.state.ValueState
+ */
+@Internal
+public class LinkedMultiSetState implements OrderedMultiSetState<RowData> {
+
+    // maps rows to SQNs (single SQN per RowData in case of upsert key; last 
SQN otherwise)
+    private final MapState<RowDataKey, Long> rowToSqnState;
+    // maps SQNs to Nodes, which comprise a doubly-linked list
+    private final MapState<Long, Node> sqnToNodeState;
+    // highest sequence number; also latest emitted downstream
+    private final ValueState<Tuple2<Long, Long>> highestSqnAndSizeState;
+
+    private final RecordEqualiser keyEqualiser;
+    private final HashFunction keyHashFunction;
+    private final Function<RowData, RowData> keyExtractor;
+    private final TimeSelector timeSelector;
+
+    private LinkedMultiSetState(
+            MapState<RowDataKey, Long> rowToSqnState,
+            MapState<Long, Node> sqnToNodeState,
+            ValueState<Tuple2<Long, Long>> highestSqnAndSizeState,
+            RecordEqualiser keyEqualiser,
+            HashFunction keyHashFunction,
+            Function<RowData, RowData> 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<RowData> create(
+            OrderedMultiSetStateContext p, RuntimeContext ctx) {
+
+        RecordEqualiser keyEqualiser =
+                
p.generatedKeyEqualiser.newInstance(ctx.getUserCodeClassLoader());
+        HashFunction keyHashFunction =
+                
p.generatedKeyHashFunction.newInstance(ctx.getUserCodeClassLoader());
+
+        MapState<RowDataKey, Long> rowToSqnState =
+                ctx.getMapState(
+                        new MapStateDescriptor<>(
+                                "rowToSqnState",
+                                new RowDataKeySerializer(
+                                        p.keySerializer,
+                                        keyEqualiser,
+                                        keyHashFunction,
+                                        p.generatedKeyEqualiser,
+                                        p.generatedKeyHashFunction),
+                                LongSerializer.INSTANCE));
+        MapState<Long, Node> sqnToNodeState =
+                ctx.getMapState(
+                        new MapStateDescriptor<>(
+                                "sqnToNodeState",
+                                LongSerializer.INSTANCE,
+                                new NodeSerializer(p.recordSerializer)));
+
+        //noinspection rawtypes,unchecked
+        ValueState<Tuple2<Long, Long>> highestSqnState =
+                ctx.getState(
+                        new ValueStateDescriptor<Tuple2<Long, Long>>(
+                                "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<Long, Long> 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;

Review Comment:
   Does `!append` mean the newly added record overwritten previous record? Via 
upsert-key?
   
   Those three lines above are not very readable, maybe they are lacking some 
comments? Maybe it would be better to express it via
   ```
   if (append) { 
     if (existed) {
       // comment explaining
       oldSqn = ...
       newSqn = ...
       newSize = ...
     } else {
       // comment explaining
       ...
   } else {
   ...
   ```
   
   or 
   ```
   if (append && existed) {
       // comment explaining
       oldSqn = ...
       newSqn = ...
       newSize = ...
   else if (...) {
   ...
   ```
   ?



##########
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).
+ *
+ * <p>Key features of this state implementation:
+ *
+ * <ul>
+ *   <li>Maintains insertion order of elements using a doubly-linked list 
structure.
+ *   <li>Supports both normal set semantics (replacing existing elements) and 
multi-set semantics
+ *       (allowing duplicates).
+ *   <li>Efficiently tracks the highest sequence number and links between 
elements for fast
+ *       traversal and updates.
+ *   <li>Provides methods to add, append, and remove elements with appropriate 
handling of state
+ *       transitions.
+ * </ul>
+ *
+ * <p>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.
+ *
+ * <p>Usage:
+ *
+ * <ul>
+ *   <li>Use the {@link #add(RowData, long)} method to add an element, 
replacing any existing
+ *       matching element.
+ *   <li>Use the {@link #append(RowData, long)} method to append an element, 
allowing duplicates.
+ *   <li>Use the {@link #remove(RowData)} method to remove an element, with 
detailed removal result
+ *       types.
+ * </ul>
+ *
+ * @see OrderedMultiSetState
+ * @see org.apache.flink.api.common.state.MapState
+ * @see org.apache.flink.api.common.state.ValueState
+ */
+@Internal
+public class LinkedMultiSetState implements OrderedMultiSetState<RowData> {
+
+    // maps rows to SQNs (single SQN per RowData in case of upsert key; last 
SQN otherwise)
+    private final MapState<RowDataKey, Long> rowToSqnState;
+    // maps SQNs to Nodes, which comprise a doubly-linked list
+    private final MapState<Long, Node> sqnToNodeState;
+    // highest sequence number; also latest emitted downstream
+    private final ValueState<Tuple2<Long, Long>> highestSqnAndSizeState;
+
+    private final RecordEqualiser keyEqualiser;
+    private final HashFunction keyHashFunction;
+    private final Function<RowData, RowData> keyExtractor;
+    private final TimeSelector timeSelector;
+
+    private LinkedMultiSetState(
+            MapState<RowDataKey, Long> rowToSqnState,
+            MapState<Long, Node> sqnToNodeState,
+            ValueState<Tuple2<Long, Long>> highestSqnAndSizeState,
+            RecordEqualiser keyEqualiser,
+            HashFunction keyHashFunction,
+            Function<RowData, RowData> 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<RowData> create(
+            OrderedMultiSetStateContext p, RuntimeContext ctx) {
+
+        RecordEqualiser keyEqualiser =
+                
p.generatedKeyEqualiser.newInstance(ctx.getUserCodeClassLoader());
+        HashFunction keyHashFunction =
+                
p.generatedKeyHashFunction.newInstance(ctx.getUserCodeClassLoader());
+
+        MapState<RowDataKey, Long> rowToSqnState =
+                ctx.getMapState(
+                        new MapStateDescriptor<>(
+                                "rowToSqnState",
+                                new RowDataKeySerializer(
+                                        p.keySerializer,
+                                        keyEqualiser,
+                                        keyHashFunction,
+                                        p.generatedKeyEqualiser,
+                                        p.generatedKeyHashFunction),
+                                LongSerializer.INSTANCE));
+        MapState<Long, Node> sqnToNodeState =
+                ctx.getMapState(
+                        new MapStateDescriptor<>(
+                                "sqnToNodeState",
+                                LongSerializer.INSTANCE,
+                                new NodeSerializer(p.recordSerializer)));
+
+        //noinspection rawtypes,unchecked
+        ValueState<Tuple2<Long, Long>> highestSqnState =
+                ctx.getState(
+                        new ValueStateDescriptor<Tuple2<Long, Long>>(
+                                "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

Review Comment:
   outdated doc?



##########
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).
+ *
+ * <p>Key features of this state implementation:
+ *
+ * <ul>
+ *   <li>Maintains insertion order of elements using a doubly-linked list 
structure.
+ *   <li>Supports both normal set semantics (replacing existing elements) and 
multi-set semantics
+ *       (allowing duplicates).
+ *   <li>Efficiently tracks the highest sequence number and links between 
elements for fast
+ *       traversal and updates.
+ *   <li>Provides methods to add, append, and remove elements with appropriate 
handling of state
+ *       transitions.
+ * </ul>
+ *
+ * <p>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.
+ *
+ * <p>Usage:
+ *
+ * <ul>
+ *   <li>Use the {@link #add(RowData, long)} method to add an element, 
replacing any existing
+ *       matching element.
+ *   <li>Use the {@link #append(RowData, long)} method to append an element, 
allowing duplicates.
+ *   <li>Use the {@link #remove(RowData)} method to remove an element, with 
detailed removal result
+ *       types.
+ * </ul>
+ *
+ * @see OrderedMultiSetState
+ * @see org.apache.flink.api.common.state.MapState
+ * @see org.apache.flink.api.common.state.ValueState
+ */
+@Internal
+public class LinkedMultiSetState implements OrderedMultiSetState<RowData> {

Review Comment:
   I think we should have test coverage for this quite complex class. Ditto for 
`Value` and `Adaptive` versions, so probably one test suite to be executed on 
different implementations with adaptive configured in a couple of different 
ways?
   
   Plus some dedicated tests for switchover in `Adaptive` version?



##########
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.
+ *
+ * <p>The state supports two types of semantics for adding elements:
+ *
+ * <ul>
+ *   <li><b>Normal Set Semantics:</b> Replaces an existing matching element 
with the new one.
+ *   <li><b>Multi-Set Semantics:</b> Appends the new element, allowing 
duplicates.
+ * </ul>
+ *
+ * <p>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 <T> The type of elements stored in the state.
+ */
+@Internal
+@Experimental
+public interface OrderedMultiSetState<T> {
+
+    /**
+     * 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;

Review Comment:
   Why do we need both? 🤔 



##########
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.
+ *
+ * <p>The state supports two types of semantics for adding elements:
+ *
+ * <ul>
+ *   <li><b>Normal Set Semantics:</b> Replaces an existing matching element 
with the new one.
+ *   <li><b>Multi-Set Semantics:</b> Appends the new element, allowing 
duplicates.
+ * </ul>
+ *
+ * <p>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 <T> The type of elements stored in the state.
+ */
+@Internal
+@Experimental
+public interface OrderedMultiSetState<T> {

Review Comment:
   Maybe this should be actually named `SequencedMultiSetState`? Take a look at 
[SequencedSet](https://docs.oracle.com/en/java/javase/22/docs/api/java.base/java/util/SequencedSet.html)
 and 
[SequencedCollection](https://docs.oracle.com/en/java/javase/22/docs/api/java.base/java/util/SequencedCollection.html)?
   



##########
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).
+ *
+ * <p>Key features of this state implementation:
+ *
+ * <ul>
+ *   <li>Maintains insertion order of elements using a doubly-linked list 
structure.
+ *   <li>Supports both normal set semantics (replacing existing elements) and 
multi-set semantics
+ *       (allowing duplicates).
+ *   <li>Efficiently tracks the highest sequence number and links between 
elements for fast
+ *       traversal and updates.
+ *   <li>Provides methods to add, append, and remove elements with appropriate 
handling of state
+ *       transitions.
+ * </ul>
+ *
+ * <p>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.
+ *
+ * <p>Usage:
+ *
+ * <ul>
+ *   <li>Use the {@link #add(RowData, long)} method to add an element, 
replacing any existing
+ *       matching element.
+ *   <li>Use the {@link #append(RowData, long)} method to append an element, 
allowing duplicates.
+ *   <li>Use the {@link #remove(RowData)} method to remove an element, with 
detailed removal result
+ *       types.
+ * </ul>
+ *
+ * @see OrderedMultiSetState
+ * @see org.apache.flink.api.common.state.MapState
+ * @see org.apache.flink.api.common.state.ValueState
+ */
+@Internal
+public class LinkedMultiSetState implements OrderedMultiSetState<RowData> {
+
+    // maps rows to SQNs (single SQN per RowData in case of upsert key; last 
SQN otherwise)
+    private final MapState<RowDataKey, Long> rowToSqnState;
+    // maps SQNs to Nodes, which comprise a doubly-linked list
+    private final MapState<Long, Node> sqnToNodeState;
+    // highest sequence number; also latest emitted downstream
+    private final ValueState<Tuple2<Long, Long>> highestSqnAndSizeState;
+
+    private final RecordEqualiser keyEqualiser;
+    private final HashFunction keyHashFunction;
+    private final Function<RowData, RowData> keyExtractor;
+    private final TimeSelector timeSelector;
+
+    private LinkedMultiSetState(
+            MapState<RowDataKey, Long> rowToSqnState,
+            MapState<Long, Node> sqnToNodeState,
+            ValueState<Tuple2<Long, Long>> highestSqnAndSizeState,
+            RecordEqualiser keyEqualiser,
+            HashFunction keyHashFunction,
+            Function<RowData, RowData> 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<RowData> create(
+            OrderedMultiSetStateContext p, RuntimeContext ctx) {
+
+        RecordEqualiser keyEqualiser =
+                
p.generatedKeyEqualiser.newInstance(ctx.getUserCodeClassLoader());
+        HashFunction keyHashFunction =
+                
p.generatedKeyHashFunction.newInstance(ctx.getUserCodeClassLoader());
+
+        MapState<RowDataKey, Long> rowToSqnState =
+                ctx.getMapState(
+                        new MapStateDescriptor<>(
+                                "rowToSqnState",
+                                new RowDataKeySerializer(
+                                        p.keySerializer,
+                                        keyEqualiser,
+                                        keyHashFunction,
+                                        p.generatedKeyEqualiser,
+                                        p.generatedKeyHashFunction),
+                                LongSerializer.INSTANCE));
+        MapState<Long, Node> sqnToNodeState =
+                ctx.getMapState(
+                        new MapStateDescriptor<>(
+                                "sqnToNodeState",
+                                LongSerializer.INSTANCE,
+                                new NodeSerializer(p.recordSerializer)));
+
+        //noinspection rawtypes,unchecked
+        ValueState<Tuple2<Long, Long>> highestSqnState =
+                ctx.getState(
+                        new ValueStateDescriptor<Tuple2<Long, Long>>(
+                                "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<Long, Long> 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;

Review Comment:
   what does it mean `append` and `existed`? Either rename or add a comment?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to