1996fanrui commented on code in PR #27070:
URL: https://github.com/apache/flink/pull/27070#discussion_r2437603495


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/SinkUpsertMaterializerV2.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.operators.sink;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.runtime.generated.GeneratedHashFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import 
org.apache.flink.table.runtime.sequencedmultisetstate.SequencedMultiSetState;
+import 
org.apache.flink.table.runtime.sequencedmultisetstate.SequencedMultiSetState.StateChangeInfo;
+import 
org.apache.flink.table.runtime.sequencedmultisetstate.SequencedMultiSetStateConfig;
+import 
org.apache.flink.table.runtime.sequencedmultisetstate.SequencedMultiSetStateContext;
+import org.apache.flink.table.runtime.typeutils.InternalSerializers;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.typeutils.RowTypeUtils;
+import org.apache.flink.types.RowKind;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An operator that maintains incoming records in state corresponding to the 
upsert keys and
+ * generates an upsert view for the downstream operator.
+ *
+ * <ul>
+ *   <li>Adds an insertion to state and emits it with updated {@link RowKind}.
+ *   <li>Applies a deletion to state.
+ *   <li>Emits a deletion with updated {@link RowKind} iff affects the last 
record or the state is
+ *       empty afterward. A deletion to an already updated record is swallowed.
+ * </ul>
+ */
+@Internal
+public class SinkUpsertMaterializerV2 extends TableStreamOperator<RowData>
+        implements OneInputStreamOperator<RowData, RowData> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SinkUpsertMaterializerV2.class);
+
+    private final SequencedMultiSetStateContext stateParameters;
+
+    // Buffer of emitted insertions on which deletions will be applied first.
+    // The row kind might be +I or +U and will be ignored when applying the 
deletion.
+    private transient TimestampedCollector<RowData> collector;
+
+    private transient SequencedMultiSetState<RowData> orderedMultiSetState;
+    private final boolean hasUpsertKey;
+
+    public SinkUpsertMaterializerV2(
+            boolean hasUpsertKey, SequencedMultiSetStateContext 
stateParameters) {
+        this.hasUpsertKey = hasUpsertKey;
+        this.stateParameters = stateParameters;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+        orderedMultiSetState =
+                SequencedMultiSetState.create(
+                        stateParameters,
+                        getRuntimeContext(),
+                        getKeyedStateStore().getBackendTypeIdentifier());
+        collector = new TimestampedCollector<>(output);
+        LOG.info("Opened {} with upsert key: {}", 
this.getClass().getSimpleName(), hasUpsertKey);
+    }
+
+    @SuppressWarnings("OptionalGetWithoutIsPresent")
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception 
{
+        final RowData row = element.getValue();
+        final long timestamp = element.getTimestamp();
+
+        switch (row.getRowKind()) {
+            case INSERT:
+            case UPDATE_AFTER:
+                if (hasUpsertKey) {
+                    collect(row, orderedMultiSetState.add(row, 
timestamp).wasEmpty());
+                } else {
+                    collect(row, orderedMultiSetState.append(row, 
timestamp).wasEmpty());
+                }
+                break;
+
+            case UPDATE_BEFORE:
+            case DELETE:
+                StateChangeInfo<RowData> removalResult = 
orderedMultiSetState.remove(row);
+                switch (removalResult.getChangeType()) {
+                    case REMOVAL_OTHER:
+                        // do nothing;
+                        break;
+                    case REMOVAL_NOT_FOUND:
+                        LOG.warn("Not found record to retract"); // not 
logging the record due for
+                        // security
+                        break;
+                    case REMOVAL_ALL:
+                        collect(removalResult.getPayload().get(), 
RowKind.DELETE);
+                        break;
+                    case REMOVAL_LAST_ADDED:
+                        collect(removalResult.getPayload().get(), 
RowKind.UPDATE_AFTER);
+                        break;
+                    default:
+                        throw new IllegalArgumentException(
+                                "Unexpected removal result type: " + 
removalResult.getChangeType());
+                }
+        }
+    }
+
+    private void collect(RowData row, boolean notExisted) {
+        collect(row, getRowKind(notExisted));
+    }
+
+    private RowKind getRowKind(boolean notExisted) {
+        return notExisted ? RowKind.INSERT : RowKind.UPDATE_AFTER;
+    }
+
+    private void collect(RowData row, RowKind withKind) {
+        RowKind orig = row.getRowKind();
+        row.setRowKind(withKind);
+        collector.collect(row);
+        row.setRowKind(orig);
+    }
+
+    public static SinkUpsertMaterializerV2 create(
+            RowType physicalRowType,
+            GeneratedRecordEqualiser rowEqualiser,
+            GeneratedRecordEqualiser upsertKeyEqualiser,
+            GeneratedHashFunction rowHashFunction,
+            GeneratedHashFunction upsertKeyHashFunction,
+            int[] inputUpsertKey,
+            SequencedMultiSetStateConfig stateSettings) {
+
+        boolean hasUpsertKey = inputUpsertKey != null && inputUpsertKey.length 
> 0;
+
+        return new SinkUpsertMaterializerV2(
+                hasUpsertKey,
+                new SequencedMultiSetStateContext(
+                        checkNotNull(
+                                hasUpsertKey
+                                        ? InternalSerializers.create(
+                                                RowTypeUtils.projectRowType(
+                                                        physicalRowType, 
inputUpsertKey))
+                                        : 
InternalSerializers.create(physicalRowType)),
+                        checkNotNull(hasUpsertKey ? upsertKeyEqualiser : 
rowEqualiser),
+                        checkNotNull(hasUpsertKey ? upsertKeyHashFunction : 
rowHashFunction),
+                        InternalSerializers.create(physicalRowType),
+                        row ->
+                                hasUpsertKey
+                                        ? ProjectedRowData.from(inputUpsertKey)
+                                                .withNullAtNonProjected(true)
+                                                .replaceRow(row)
+                                        : row,

Review Comment:
   > withNullAtNonProjected will create a new ProjectedRowData object, so it 
created 2 ProjectedRowData objects for each row.
   
   If you are worry about the risk of the second point, how about updating the 
first point in this PR? The first point should be low risk.
   
   >  Also, when running benchmark, I didn't see the impact of this code path.
   > I could create a follow-up ticket to test it and to benchmark the impact, 
WDYT?
   
   Thanks for the inputs, if it is a low impact, it is not needed to create a 
ticket unless we found performance is bad in the future. Do not wanna involve 
unneeded efforts for you.



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java:
##########
@@ -159,6 +159,68 @@ public class ExecutionConfigOptions {
                                                     + "or force 
materialization(FORCE).")
                                     .build());
 
+    @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING)
+    public static final ConfigOption<Long>
+            TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_LOW =
+                    
key("table.exec.sink.upsert-materialize-strategy.adaptive.threshold.low")
+                            .longType()
+                            .noDefaultValue()
+                            .withDescription(
+                                    Description.builder()
+                                            .text(
+                                                    "When using 
strategy=ADAPTIVE, defines the number of entries per key when the 
implementation is changed from MAP to VALUE. "
+                                                            + "If not 
specified, Flink uses state-backend specific defaults (300 for hashmap state 
backend and 40 for RocksDB and the rest).")
+                                            .linebreak()
+                                            .text("The option takes effect 
during job (re)starting")
+                                            .linebreak()
+                                            .build());
+
+    @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING)
+    public static final ConfigOption<Long>
+            TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_HIGH =
+                    
key("table.exec.sink.upsert-materialize-strategy.adaptive.threshold.high")
+                            .longType()
+                            .noDefaultValue()
+                            .withDescription(
+                                    Description.builder()
+                                            .text(
+                                                    "When using 
strategy=ADAPTIVE, defines the number of entries per key when the 
implementation is changed from VALUE to MAP. "
+                                                            + "If not 
specified, Flink uses state-backend specific defaults (400 for hashmap state 
backend and 50 for RocksDB and the rest).")
+                                            .linebreak()
+                                            .text("The option takes effect 
during job (re)starting")
+                                            .linebreak()
+                                            .build());
+
+    @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING)
+    public static final ConfigOption<SinkUpsertMaterializeStrategy>
+            TABLE_EXEC_SINK_UPSERT_MATERIALIZE_STRATEGY =
+                    key("table.exec.sink.upsert-materialize-strategy.type")
+                            .enumType(SinkUpsertMaterializeStrategy.class)
+                            .defaultValue(SinkUpsertMaterializeStrategy.LEGACY)
+                            .withDescription(
+                                    Description.builder()
+                                            .text(
+                                                    "Which strategy of 
SinkUpsertMaterializer to use. Supported strategies:")
+                                            .linebreak()
+                                            .text(
+                                                    "LEGACY: Simple 
implementation based on ValueState<List> (the original implementation).")
+                                            .linebreak()
+                                            .text(
+                                                    "MAP: 
OrderedMultiSetState-based implementation based on a combination of several 
MapState maintaining ordering and fast lookup properties.")
+                                            .linebreak()
+                                            .text(
+                                                    "VALUE: Similar to LEGACY, 
but compatible with MAP and therefore allows to switch to ADAPTIVE.")
+                                            .linebreak()
+                                            .text(
+                                                    "ADAPTIVE: Alternate 
between MAP and VALUE depending on the number of entries for the given key 
starting with VALUE and switching to MAP upon reaching threshold.high value 
(and back to VALUE, when reaching low).")
+                                            .linebreak()
+                                            .text("The default is LEGACY")

Review Comment:
   nit: this line is not needed.
   
   In flink configuration page, there is a separate column to show the default 
value for each config option.



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java:
##########
@@ -159,6 +159,66 @@ public class ExecutionConfigOptions {
                                                     + "or force 
materialization(FORCE).")
                                     .build());
 
+    @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING)
+    public static final ConfigOption<Long>
+            TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_LOW =
+                    
key("table.exec.sink.upsert-materialize.adaptive.threshold.low")
+                            .longType()
+                            .noDefaultValue()
+                            .withDescription(
+                                    Description.builder()
+                                            .text(
+                                                    "When using 
strategy=ADAPTIVE, defines the number of entries per key when the 
implementation is changed from MAP to VALUE. If not specified, Flink uses 
state-backend specific defaults.")
+                                            .linebreak()
+                                            .text("The option takes effect 
during job (re)starting")
+                                            .linebreak()
+                                            .build());
+
+    @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING)
+    public static final ConfigOption<Long>
+            TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_HIGH =
+                    
key("table.exec.sink.upsert-materialize.adaptive.threshold.high")
+                            .longType()
+                            .noDefaultValue()
+                            .withDescription(
+                                    Description.builder()
+                                            .text(
+                                                    "When using 
strategy=ADAPTIVE, defines the number of entries per key when the 
implementation is changed from VALUE to MAP. If not specified, Flink uses 
state-backend specific defaults.")
+                                            .linebreak()
+                                            .text("The option takes effect 
during job (re)starting")

Review Comment:
   IIUC, the purpose is to explain these 2 config options can be updated by 
dynamic configuration. I think it's useful to let users are aware of it.
   
   My concern or question is whether it is a good way to mention here. Lots of 
config options do not mention it even if `they take effect during job 
(re)starting`. In other words, they support dynamic configuration.
   
   Or a better way is adding a separate column to indicate each config option 
support or does not support dynamic configuration. (yes or no)
   
   WDYT? Of course, it is out of scope of this PR. we could discuss it later.



-- 
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