pnowojski commented on code in PR #27071: URL: https://github.com/apache/flink/pull/27071#discussion_r2399289800
########## 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 Review Comment: 🤔 aren't those two mutually exclusive? Class can either be `Internal` and not intended to be used outside of the Flink's repo, or `Experimental`/`PublicEvolving`/`Public` depending on the stability of the said api ########## 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<RowData> { + private static final Logger LOG = LoggerFactory.getLogger(AdaptiveOrderedMultiSetState.class); + + private final OrderedMultiSetState<RowData> smallState; + private final OrderedMultiSetState<RowData> largeState; + private final long switchToLargeThreshold; + private final long switchToSmallThreshold; + + AdaptiveOrderedMultiSetState( + OrderedMultiSetState<RowData> smallState, + OrderedMultiSetState<RowData> 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<Tuple2<RowData, Long>> 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<RemovalResultType, Optional<RowData>, 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> T execute( + FunctionWithException<OrderedMultiSetState<RowData>, T, Exception> stateOp, + Function<T, SizeChangeInfo> 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<RowData> currentState = isUsingLarge ? largeState : smallState; + OrderedMultiSetState<RowData> 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<RowData> state) throws IOException { + state.loadCache(); Review Comment: nit: I would move this `loadCache` call out of here, to the top of the `execute` method. ########## 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<List>}. + */ +class ValueStateMultiSetState implements OrderedMultiSetState<RowData> { + + private final ValueState<List<Tuple2<RowData, Long>>> valuesState; + private final RecordEqualiser keyEqualiser; + private final Function<RowData, RowData> keyExtractor; + private final TimeSelector timeSelector; + private List<Tuple2<RowData, Long>> cache; + + ValueStateMultiSetState( + ValueState<List<Tuple2<RowData, Long>>> valuesState, + RecordEqualiser keyEqualiser, + Function<RowData, RowData> keyExtractor, + TimeSelector timeSelector) { + this.valuesState = valuesState; + this.keyEqualiser = keyEqualiser; + this.keyExtractor = keyExtractor; + this.timeSelector = timeSelector; + } + + public static OrderedMultiSetState<RowData> 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<RowData, Long> toAdd = Tuple2.of(row, timeSelector.getTimestamp(ts)); + final RowData key = asKey(row); + final List<Tuple2<RowData, Long>> list = maybeReadState(); + final int oldSize = list.size(); + + int idx = Integer.MIN_VALUE; + int i = 0; + for (Tuple2<RowData, Long> 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); Review Comment: Would it make sense to cache writes? Or `SinkUpsertMaterializer` doesn't do more than a single write per input record? ########## 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<List>}. + */ +class ValueStateMultiSetState implements OrderedMultiSetState<RowData> { Review Comment: Did you benchmark the current `SinkUpsertMaterializer` vs new one using `ValueStateMultiSetState`? ########## 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<RowData> { + private static final Logger LOG = LoggerFactory.getLogger(AdaptiveOrderedMultiSetState.class); + + private final OrderedMultiSetState<RowData> smallState; + private final OrderedMultiSetState<RowData> largeState; Review Comment: nit: Code would be a bit more self-documenting if you used here: ``` private final ValueStateMultiSetState<RowData> smallState; private final LinkedMultiSetState<RowData> largeState; ``` And it would also fit the Java doc description then. So either change the types to concrete classes, or rephrase the java doc? -- 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]
